From 792358f5a34f4c262fa36e80ab5582fc7fd4fdd0 Mon Sep 17 00:00:00 2001 From: Slach Date: Wed, 30 Aug 2023 10:32:39 +0400 Subject: [PATCH 0001/1018] add status_info to prometheus settings Signed-off-by: Slach --- .../settings.md | 48 ++----------------- .../settings.md | 15 ++++-- 2 files changed, 15 insertions(+), 48 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 55ee15a09d9..77a12a15656 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1816,6 +1816,7 @@ Settings: - `metrics` – Flag that sets to expose metrics from the [system.metrics](../../operations/system-tables/metrics.md#system_tables-metrics) table. - `events` – Flag that sets to expose metrics from the [system.events](../../operations/system-tables/events.md#system_tables-events) table. - `asynchronous_metrics` – Flag that sets to expose current metrics values from the [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) table. +- `status_info` - Flag that sets to expose data from different component from CH, ex: Dictionaries status **Example** @@ -1831,6 +1832,7 @@ Settings: true true true + true @@ -2542,48 +2544,4 @@ Default value: `4194304`. ## total_memory_tracker_sample_probability {#total-memory-tracker-sample-probability} -Allows to collect random allocations and deallocations and writes them in the [system.trace_log](../../operations/system-tables/trace_log.md) system table with `trace_type` equal to a `MemorySample` with the specified probability. The probability is for every allocation or deallocations, regardless of the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds the untracked memory limit (default value is `4` MiB). It can be lowered if [total_memory_profiler_step](#total-memory-profiler-step) is lowered. You can set `total_memory_profiler_step` equal to `1` for extra fine-grained sampling. - -Possible values: - -- Positive integer. -- 0 — Writing of random allocations and deallocations in the `system.trace_log` system table is disabled. - -Default value: `0`. - -## compiled_expression_cache_size {#compiled-expression-cache-size} - -Sets the cache size (in bytes) for [compiled expressions](../../operations/caches.md). - -Possible values: - -- Positive integer. - -Default value: `134217728`. - -## compiled_expression_cache_elements_size {#compiled_expression_cache_elements_size} - -Sets the cache size (in elements) for [compiled expressions](../../operations/caches.md). - -Possible values: - -- Positive integer. - -Default value: `10000`. - -## display_secrets_in_show_and_select {#display_secrets_in_show_and_select} - -Enables or disables showing secrets in `SHOW` and `SELECT` queries for tables, databases, -table functions, and dictionaries. - -User wishing to see secrets must also have -[`format_display_secrets_in_show_and_select` format setting](../settings/formats#format_display_secrets_in_show_and_select) -turned on and a -[`displaySecretsInShowAndSelect`](../../sql-reference/statements/grant#grant-display-secrets) privilege. - -Possible values: - -- 0 — Disabled. -- 1 — Enabled. - -Default value: 0. +Allows to collect random allocations and deallocations and writes them in the [system.trace_log](../../operations/system-tables/trace_log.md) system table with `trace_type` equal to a `MemorySample` with the specified probability. The probability is for every allocation or deallocations, regardless of the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds the untracked memory limit (default value is `4` MiB). It can be lowered if [total_memory_profiler_step](#total-memory-profiler-step) is lowered. You can set `total_memory \ No newline at end of file diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 7b026244624..de87944ead2 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -1212,17 +1212,26 @@ ClickHouse использует потоки из глобального пул - `metrics` – флаг для экспорта текущих значений метрик из таблицы [system.metrics](../system-tables/metrics.md#system_tables-metrics). - `events` – флаг для экспорта текущих значений метрик из таблицы [system.events](../system-tables/events.md#system_tables-events). - `asynchronous_metrics` – флаг для экспорта текущих значений значения метрик из таблицы [system.asynchronous_metrics](../system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics). +- `status_info` - флаг для экспорта данных из различных компонентов, например: статус словарей **Пример** ``` xml - + + 0.0.0.0 + 8123 + 9000 + + /metrics - 8001 + 9363 true true true + true + + ``` ## query_log {#server_configuration_parameters-query-log} @@ -1922,4 +1931,4 @@ ClickHouse использует ZooKeeper для хранения метадан - Положительное целое число. -Значение по умолчанию: `10000`. +Значение по умолчанию: `100 \ No newline at end of file From d347bec531837af8fd7396603a15fb51c8fc73b6 Mon Sep 17 00:00:00 2001 From: Vinay Suryadevara <126499167+vinay92-ch@users.noreply.github.com> Date: Thu, 14 Dec 2023 16:09:14 -0800 Subject: [PATCH 0002/1018] [WIP] Add support for Workload identity credentials in AzureBlobStorage --- .../AzureBlobStorage/AzureBlobStorageAuth.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp index 6075b385a6c..c091cc152d8 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp @@ -108,6 +108,12 @@ template std::unique_ptr getAzureBlobStorageClientWithAuth( const String & url, const String & container_name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix) { + if (config.has(config_prefix + ".use_workload_identity_for_azure")) + { + auto workload_identity_credential = std::make_shared(); + return std::make_unique(url, workload_identity_credential); + } + if (config.has(config_prefix + ".connection_string")) { String connection_str = config.getString(config_prefix + ".connection_string"); From 4fb7fe1d18438ade7cb4b6dc5cf6de44469abfe1 Mon Sep 17 00:00:00 2001 From: vinay92-ch Date: Sat, 16 Dec 2023 00:35:10 +0000 Subject: [PATCH 0003/1018] Update Azure contrib --- contrib/azure | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/azure b/contrib/azure index 352ff0a61cb..d94ae337c8f 160000 --- a/contrib/azure +++ b/contrib/azure @@ -1 +1 @@ -Subproject commit 352ff0a61cb319ac1cc38c4058443ddf70147530 +Subproject commit d94ae337c8ffbf74e99d412ac55e38f2190490f5 From 2d32e5d15ff9b8fbc4e96534b31e147ad541fb97 Mon Sep 17 00:00:00 2001 From: vinay92-ch Date: Mon, 18 Dec 2023 22:38:46 +0000 Subject: [PATCH 0004/1018] Update azure package --- contrib/azure | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/azure b/contrib/azure index d94ae337c8f..3310cc011ff 160000 --- a/contrib/azure +++ b/contrib/azure @@ -1 +1 @@ -Subproject commit d94ae337c8ffbf74e99d412ac55e38f2190490f5 +Subproject commit 3310cc011ffbbf15274a0ad0d90a1772354094f5 From 462aeadb6ea550904a0a5544bd1d45531954f892 Mon Sep 17 00:00:00 2001 From: vinay92-ch Date: Mon, 18 Dec 2023 23:07:29 +0000 Subject: [PATCH 0005/1018] Style check --- .../ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp index c091cc152d8..00afaa83189 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp @@ -113,7 +113,7 @@ std::unique_ptr getAzureBlobStorageClientWithAuth( auto workload_identity_credential = std::make_shared(); return std::make_unique(url, workload_identity_credential); } - + if (config.has(config_prefix + ".connection_string")) { String connection_str = config.getString(config_prefix + ".connection_string"); From fb17d0d16939434be0990b18cfdd1fecc6227bb1 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 23 Jan 2024 11:31:12 +0000 Subject: [PATCH 0006/1018] Introduce EmbeddedRocksDBBulkSink No mem-table: an SST file is built from chunk, then import to rocksdb. Also add RocksDBSettings as table level settings for StorageEmbeddedRocksDB Signed-off-by: Duc Canh Le --- src/Interpreters/IKeyValueEntity.h | 1 + .../RocksDB/EmbeddedRocksDBBulkSink.cpp | 149 ++++++++++++++++++ .../RocksDB/EmbeddedRocksDBBulkSink.h | 56 +++++++ src/Storages/RocksDB/RocksDBSettings.cpp | 41 +++++ src/Storages/RocksDB/RocksDBSettings.h | 39 +++++ .../RocksDB/StorageEmbeddedRocksDB.cpp | 42 +++-- src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 11 +- .../02956_rocksdb_bulk_sink.reference | 1 + .../0_stateless/02956_rocksdb_bulk_sink.sql | 3 + 9 files changed, 326 insertions(+), 17 deletions(-) create mode 100644 src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp create mode 100644 src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h create mode 100644 src/Storages/RocksDB/RocksDBSettings.cpp create mode 100644 src/Storages/RocksDB/RocksDBSettings.h create mode 100644 tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference create mode 100644 tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql diff --git a/src/Interpreters/IKeyValueEntity.h b/src/Interpreters/IKeyValueEntity.h index d1ceda57f0e..76f652ac2d0 100644 --- a/src/Interpreters/IKeyValueEntity.h +++ b/src/Interpreters/IKeyValueEntity.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB { diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp new file mode 100644 index 00000000000..1c5d48fe62d --- /dev/null +++ b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp @@ -0,0 +1,149 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ROCKSDB_ERROR; +} + +static const IColumn::Permutation & getAscendingPermutation(const IColumn & column, IColumn::Permutation & perm) +{ + column.getPermutation(IColumn::PermutationSortDirection::Ascending, IColumn::PermutationSortStability::Stable, 0, 1, perm); + return perm; +} + +static rocksdb::Status buildSSTFile(const String & path, const ColumnString & keys, const ColumnString & values, const std::optional & perm_ = {}) +{ + IColumn::Permutation calculated_perm; + const IColumn::Permutation & perm = perm_ ? *perm_ : getAscendingPermutation(keys, calculated_perm); + + rocksdb::SstFileWriter sst_file_writer(rocksdb::EnvOptions{}, rocksdb::Options{}); + auto status = sst_file_writer.Open(path); + if (!status.ok()) + return status; + + auto rows = perm.size(); + WriteBufferFromOwnString wb_value; + for (size_t i = 0; i < rows; ++i) + { + auto row = perm[i]; + + status = sst_file_writer.Put(keys.getDataAt(row).toView(), values.getDataAt(row).toView()); + + /// There could be duplicated keys in chunk, thus Put may give IsInvalidArgument. This is ok, as we're certain that + /// keys are sorted in ascending order. + if (!status.ok() && !status.IsInvalidArgument()) + return status; + } + sst_file_writer.Finish(); + return rocksdb::Status::OK(); +} + +EmbeddedRocksDBBulkSink::EmbeddedRocksDBBulkSink( + ContextPtr context_, StorageEmbeddedRocksDB & storage_, const StorageMetadataPtr & metadata_snapshot_) + : SinkToStorage(metadata_snapshot_->getSampleBlock()), WithContext(context_), storage(storage_), metadata_snapshot(metadata_snapshot_) +{ + for (const auto & elem : getHeader()) + { + if (elem.name == storage.primary_key) + break; + ++primary_key_pos; + } + serializations = getHeader().getSerializations(); + /// If max_insert_threads > 1 we may have multiple EmbeddedRocksDBBulkSink and getContext()->getCurrentQueryId() is not guarantee to + /// to have a distinct path + insert_directory_queue = fs::path(storage.getDataPaths()[0]) / (getContext()->getCurrentQueryId() + "_" + getRandomASCIIString(8)); + fs::create_directory(insert_directory_queue); + + // serialized_key_column = ColumnString::create(); + // serialized_value_column = ColumnString::create(); + // writer_key = std::make_unique>(serialized_key_column->getChars()); + // writer_value = std::make_unique>(serialized_value_column->getChars()); +} + +EmbeddedRocksDBBulkSink::~EmbeddedRocksDBBulkSink() +{ + if (fs::exists(insert_directory_queue)) + fs::remove_all(insert_directory_queue); +} + +void EmbeddedRocksDBBulkSink::consume(Chunk chunk) +{ + auto rows = chunk.getNumRows(); + const auto columns = chunk.detachColumns(); + + auto serialized_key_column = ColumnString::create(); + auto serialized_value_column = ColumnString::create(); + { + auto & serialized_key_data = serialized_key_column->getChars(); + auto & serialized_key_offsets = serialized_key_column->getOffsets(); + auto & serialized_value_data = serialized_value_column->getChars(); + auto & serialized_value_offsets = serialized_value_column->getOffsets(); + serialized_key_offsets.reserve(rows); + serialized_value_offsets.reserve(rows); + // serialized_key_offsets.clear(); + // serialized_value_offsets.clear(); + // serialized_key_data.clear(); + // serialized_value_data.clear(); + WriteBufferFromVector writer_key(serialized_key_data); + WriteBufferFromVector writer_value(serialized_value_data); + for (size_t i = 0; i < rows; ++i) + { + for (size_t idx = 0; idx < columns.size(); ++idx) + serializations[idx]->serializeBinary(*columns[idx], i, idx == primary_key_pos ? writer_key : writer_value, {}); + writeChar('\0', writer_key); + writeChar('\0', writer_value); + serialized_key_offsets.emplace_back(writer_key.count()); + serialized_value_offsets.emplace_back(writer_value.count()); + } + writer_key.finalize(); + writer_value.finalize(); + } + + auto path = getTemporarySSTFilePath(); + if (auto status = buildSSTFile(path, *serialized_key_column, *serialized_value_column); !status.ok()) + throw Exception(ErrorCodes::ROCKSDB_ERROR, "RocksDB write error: {}", status.ToString()); + + rocksdb::IngestExternalFileOptions ingest_options; + ingest_options.move_files = true; /// The temporary file is on the same disk, so move (or hardlink) file will be faster than copy + if (auto status = storage.rocksdb_ptr->IngestExternalFile({path}, rocksdb::IngestExternalFileOptions()); !status.ok()) + throw Exception(ErrorCodes::ROCKSDB_ERROR, "RocksDB write error: {}", status.ToString()); + + if (fs::exists(path)) + fs::remove(path); +} + +String EmbeddedRocksDBBulkSink::getTemporarySSTFilePath() +{ + return fs::path(insert_directory_queue) / (toString(file_counter++) + ".sst"); +} + +} diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h new file mode 100644 index 00000000000..312ad4bd93c --- /dev/null +++ b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h @@ -0,0 +1,56 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +class StorageEmbeddedRocksDB; +class EmbeddedRocksDBBulkSink; +struct StorageInMemoryMetadata; +using StorageMetadataPtr = std::shared_ptr; + +/// Optimized for bulk importing into StorageEmbeddedRocksDB: +/// 1. No mem-table: an SST file is built from chunk, then import to rocksdb +/// 2. Overlap compute and IO: one thread prepare rocksdb data from chunk, and another thread to write the data to SST file +class EmbeddedRocksDBBulkSink : public SinkToStorage, public WithContext +{ +public: + EmbeddedRocksDBBulkSink( + ContextPtr context_, + StorageEmbeddedRocksDB & storage_, + const StorageMetadataPtr & metadata_snapshot_); + + ~EmbeddedRocksDBBulkSink() override; + + void consume(Chunk chunk) override; + String getName() const override { return "EmbeddedRocksDBBulkSink"; } + +private: + + String getTemporarySSTFilePath(); + + std::atomic_size_t file_counter = 0; + StorageEmbeddedRocksDB & storage; + StorageMetadataPtr metadata_snapshot; + size_t primary_key_pos = 0; + Serializations serializations; + String insert_directory_queue; + + // /// Columns to hold key-value pairs, reused for all `consume` calls + // /// to reduce memory re-allocations + // ColumnString::MutablePtr serialized_key_column; + // ColumnString::MutablePtr serialized_value_column; +}; + +} diff --git a/src/Storages/RocksDB/RocksDBSettings.cpp b/src/Storages/RocksDB/RocksDBSettings.cpp new file mode 100644 index 00000000000..7de2077eb47 --- /dev/null +++ b/src/Storages/RocksDB/RocksDBSettings.cpp @@ -0,0 +1,41 @@ +#include "RocksDBSettings.h" +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_SETTING; +} + +IMPLEMENT_SETTINGS_TRAITS(RockDBSettingsTraits, LIST_OF_ROCKSDB_SETTINGS) + + +void RocksDBSettings::loadFromQuery(ASTStorage & storage_def, ContextPtr /*context*/) +{ + if (storage_def.settings) + { + try + { + auto changes = storage_def.settings->changes; + applyChanges(changes); + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::UNKNOWN_SETTING) + e.addMessage("for storage " + storage_def.engine->name); + throw; + } + } +} + +std::vector RocksDBSettings::getAllRegisteredNames() const +{ + std::vector all_settings; + for (const auto & setting_field : all()) + all_settings.push_back(setting_field.getName()); + return all_settings; +} +} diff --git a/src/Storages/RocksDB/RocksDBSettings.h b/src/Storages/RocksDB/RocksDBSettings.h new file mode 100644 index 00000000000..f3d16bc4b3e --- /dev/null +++ b/src/Storages/RocksDB/RocksDBSettings.h @@ -0,0 +1,39 @@ +#pragma once + +#include +#include +#include +#include +#include + + +namespace Poco::Util +{ +class AbstractConfiguration; +} + + +namespace DB +{ +class ASTStorage; +struct Settings; + + +/** StorageEmbeddedRocksdb table settings + */ + +#define ROCKSDB_SETTINGS(M, ALIAS) \ + M(Bool, optimize_for_bulk_insert, true, "Table is optimized for bulk insertions (insert pipeline will create SST files and import to rocksdb database instead of writing " \ + "to memtables)", 0) + +#define LIST_OF_ROCKSDB_SETTINGS(M, ALIAS) ROCKSDB_SETTINGS(M, ALIAS) + +DECLARE_SETTINGS_TRAITS(RockDBSettingsTraits, LIST_OF_ROCKSDB_SETTINGS) + +struct RocksDBSettings : public BaseSettings, public IHints<2> +{ + void loadFromQuery(ASTStorage & storage_def, ContextPtr context); + std::vector getAllRegisteredNames() const override; +}; + +} diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 4ead714c740..3bfeb561408 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -1,6 +1,5 @@ #include #include -#include #include #include @@ -28,8 +27,14 @@ #include #include #include +#include +#include +#include #include +#include +#include +#include #include #include #include @@ -174,6 +179,7 @@ StorageEmbeddedRocksDB::StorageEmbeddedRocksDB(const StorageID & table_id_, const StorageInMemoryMetadata & metadata_, bool attach, ContextPtr context_, + RocksDBSettings settings_, const String & primary_key_, Int32 ttl_, String rocksdb_dir_, @@ -186,6 +192,7 @@ StorageEmbeddedRocksDB::StorageEmbeddedRocksDB(const StorageID & table_id_, , read_only(read_only_) { setInMemoryMetadata(metadata_); + setSettings(std::move(settings_)); if (rocksdb_dir.empty()) { rocksdb_dir = context_->getPath() + relative_data_path_; @@ -234,22 +241,20 @@ void StorageEmbeddedRocksDB::mutate(const MutationCommands & commands, ContextPt if (commands.front().type == MutationCommand::Type::DELETE) { - MutationsInterpreter::Settings settings(true); - settings.return_all_columns = true; - settings.return_mutated_rows = true; + MutationsInterpreter::Settings mutation_settings(true); + mutation_settings.return_all_columns = true; + mutation_settings.return_mutated_rows = true; auto interpreter = std::make_unique( storage_ptr, metadata_snapshot, commands, context_, - settings); + mutation_settings); auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute()); PullingPipelineExecutor executor(pipeline); - auto sink = std::make_shared(*this, metadata_snapshot); - auto header = interpreter->getUpdatedHeader(); auto primary_key_pos = header.getPositionByName(primary_key); @@ -285,16 +290,16 @@ void StorageEmbeddedRocksDB::mutate(const MutationCommands & commands, ContextPt if (commands.front().column_to_update_expression.contains(primary_key)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Primary key cannot be updated (cannot update column {})", primary_key); - MutationsInterpreter::Settings settings(true); - settings.return_all_columns = true; - settings.return_mutated_rows = true; + MutationsInterpreter::Settings mutation_settings(true); + mutation_settings.return_all_columns = true; + mutation_settings.return_mutated_rows = true; auto interpreter = std::make_unique( storage_ptr, metadata_snapshot, commands, context_, - settings); + mutation_settings); auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute()); PullingPipelineExecutor executor(pipeline); @@ -350,7 +355,6 @@ void StorageEmbeddedRocksDB::initDB() rocksdb::Options base; base.create_if_missing = true; - base.compression = rocksdb::CompressionType::kZSTD; base.statistics = rocksdb::CreateDBStatistics(); /// It is too verbose by default, and in fact we don't care about rocksdb logs at all. base.info_log_level = rocksdb::ERROR_LEVEL; @@ -590,8 +594,11 @@ void ReadFromEmbeddedRocksDB::applyFilters() } SinkToStoragePtr StorageEmbeddedRocksDB::write( - const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/, bool /*async_insert*/) + const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context, bool /*async_insert*/) { + if (getSettings().optimize_for_bulk_insert) + return std::make_shared(query_context, *this, metadata_snapshot); + return std::make_shared(*this, metadata_snapshot); } @@ -630,7 +637,9 @@ static StoragePtr create(const StorageFactory::Arguments & args) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "StorageEmbeddedRocksDB must require one column in primary key"); } - return std::make_shared(args.table_id, args.relative_data_path, metadata, args.attach, args.getContext(), primary_key_names[0], ttl, std::move(rocksdb_dir), read_only); + RocksDBSettings settings; + settings.loadFromQuery(*args.storage_def, args.getContext()); + return std::make_shared(args.table_id, args.relative_data_path, metadata, args.attach, args.getContext(), std::move(settings), primary_key_names[0], ttl, std::move(rocksdb_dir), read_only); } std::shared_ptr StorageEmbeddedRocksDB::getRocksDBStatistics() const @@ -721,9 +730,9 @@ Chunk StorageEmbeddedRocksDB::getBySerializedKeys( return Chunk(std::move(columns), num_rows); } -std::optional StorageEmbeddedRocksDB::totalRows(const Settings & settings) const +std::optional StorageEmbeddedRocksDB::totalRows(const Settings & query_settings) const { - if (!settings.optimize_trivial_approximate_count_query) + if (!query_settings.optimize_trivial_approximate_count_query) return {}; std::shared_lock lock(rocksdb_ptr_mx); if (!rocksdb_ptr) @@ -748,6 +757,7 @@ std::optional StorageEmbeddedRocksDB::totalBytes(const Settings & /*sett void registerStorageEmbeddedRocksDB(StorageFactory & factory) { StorageFactory::StorageFeatures features{ + .supports_settings = true, .supports_sort_order = true, .supports_ttl = true, .supports_parallel_insert = true, diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index b09dfca7338..fff7ed3644f 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -2,10 +2,12 @@ #include #include -#include #include #include +#include #include +#include +#include namespace rocksdb @@ -27,6 +29,7 @@ class Context; class StorageEmbeddedRocksDB final : public IStorage, public IKeyValueEntity, WithContext { friend class EmbeddedRocksDBSink; + friend class EmbeddedRocksDBBulkSink; friend class ReadFromEmbeddedRocksDB; public: StorageEmbeddedRocksDB(const StorageID & table_id_, @@ -34,6 +37,7 @@ public: const StorageInMemoryMetadata & metadata, bool attach, ContextPtr context_, + RocksDBSettings settings_, const String & primary_key_, Int32 ttl_ = 0, String rocksdb_dir_ = "", @@ -97,7 +101,12 @@ public: std::optional totalBytes(const Settings & settings) const override; + const RocksDBSettings & getSettings() const { return settings; } + + void setSettings(RocksDBSettings settings_) { settings = std::move(settings_); } + private: + RocksDBSettings settings; const String primary_key; using RocksDBPtr = std::unique_ptr; RocksDBPtr rocksdb_ptr; diff --git a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference new file mode 100644 index 00000000000..83b33d238da --- /dev/null +++ b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference @@ -0,0 +1 @@ +1000 diff --git a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql new file mode 100644 index 00000000000..cfb97b049bf --- /dev/null +++ b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql @@ -0,0 +1,3 @@ +CREATE TABLE rocksdb_worm (key UInt64, value UInt64) ENGINE = EmbeddedRocksDB() PRIMARY KEY key SETTINGS optimize_for_bulk_insert = 1; +INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers_mt(1000) SETTINGS max_insert_threads = 2; +SELECT count() FROM rocksdb_worm; From 8218657cbc5bc997c391fd42de2d9fb7cc66fec7 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 25 Jan 2024 10:14:43 +0100 Subject: [PATCH 0007/1018] Add optimize_for_bulk_insert to embedded-rocksdb.md --- .../engines/table-engines/integrations/embedded-rocksdb.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md index 44febe78c77..c880ad7253c 100644 --- a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md +++ b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md @@ -17,6 +17,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], ... ) ENGINE = EmbeddedRocksDB([ttl, rocksdb_dir, read_only]) PRIMARY KEY(primary_key_name) +[ SETTINGS optimize_for_bulk_insert = (0|1)] ``` Engine parameters: @@ -29,6 +30,10 @@ Engine parameters: - columns other than the primary key will be serialized in binary as `rocksdb` value in corresponding order. - queries with key `equals` or `in` filtering will be optimized to multi keys lookup from `rocksdb`. +Engine settings: + +- `optimize_for_bulk_insert` – Table is optimized for bulk insertions (insert pipeline will create SST files and import to rocksdb database instead of writing to memtables). + Example: ``` sql From cce998e875fa7097a8d49d1193b684d52fc9a00d Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Thu, 25 Jan 2024 14:15:01 +0000 Subject: [PATCH 0008/1018] clean up and add some comments Signed-off-by: Duc Canh Le --- .../RocksDB/EmbeddedRocksDBBulkSink.cpp | 20 +++++++++---------- .../RocksDB/EmbeddedRocksDBBulkSink.h | 8 ++------ 2 files changed, 12 insertions(+), 16 deletions(-) diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp index 1c5d48fe62d..d25bfc50b22 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp @@ -42,6 +42,7 @@ static const IColumn::Permutation & getAscendingPermutation(const IColumn & colu static rocksdb::Status buildSSTFile(const String & path, const ColumnString & keys, const ColumnString & values, const std::optional & perm_ = {}) { + /// rocksdb::SstFileWriter requires keys to be sorted in ascending order IColumn::Permutation calculated_perm; const IColumn::Permutation & perm = perm_ ? *perm_ : getAscendingPermutation(keys, calculated_perm); @@ -63,6 +64,7 @@ static rocksdb::Status buildSSTFile(const String & path, const ColumnString & ke if (!status.ok() && !status.IsInvalidArgument()) return status; } + sst_file_writer.Finish(); return rocksdb::Status::OK(); } @@ -80,13 +82,8 @@ EmbeddedRocksDBBulkSink::EmbeddedRocksDBBulkSink( serializations = getHeader().getSerializations(); /// If max_insert_threads > 1 we may have multiple EmbeddedRocksDBBulkSink and getContext()->getCurrentQueryId() is not guarantee to /// to have a distinct path - insert_directory_queue = fs::path(storage.getDataPaths()[0]) / (getContext()->getCurrentQueryId() + "_" + getRandomASCIIString(8)); + insert_directory_queue = fs::path(storage.getDataPaths()[0]) / (getContext()->getCurrentQueryId() + "-" + getRandomASCIIString(8)); fs::create_directory(insert_directory_queue); - - // serialized_key_column = ColumnString::create(); - // serialized_value_column = ColumnString::create(); - // writer_key = std::make_unique>(serialized_key_column->getChars()); - // writer_value = std::make_unique>(serialized_value_column->getChars()); } EmbeddedRocksDBBulkSink::~EmbeddedRocksDBBulkSink() @@ -100,21 +97,21 @@ void EmbeddedRocksDBBulkSink::consume(Chunk chunk) auto rows = chunk.getNumRows(); const auto columns = chunk.detachColumns(); + /// Convert chunk to rocksdb key-value pairs auto serialized_key_column = ColumnString::create(); auto serialized_value_column = ColumnString::create(); + { auto & serialized_key_data = serialized_key_column->getChars(); auto & serialized_key_offsets = serialized_key_column->getOffsets(); auto & serialized_value_data = serialized_value_column->getChars(); auto & serialized_value_offsets = serialized_value_column->getOffsets(); + serialized_key_offsets.reserve(rows); serialized_value_offsets.reserve(rows); - // serialized_key_offsets.clear(); - // serialized_value_offsets.clear(); - // serialized_key_data.clear(); - // serialized_value_data.clear(); WriteBufferFromVector writer_key(serialized_key_data); WriteBufferFromVector writer_value(serialized_value_data); + for (size_t i = 0; i < rows; ++i) { for (size_t idx = 0; idx < columns.size(); ++idx) @@ -124,14 +121,17 @@ void EmbeddedRocksDBBulkSink::consume(Chunk chunk) serialized_key_offsets.emplace_back(writer_key.count()); serialized_value_offsets.emplace_back(writer_value.count()); } + writer_key.finalize(); writer_value.finalize(); } + /// Build SST file from key-value pairs auto path = getTemporarySSTFilePath(); if (auto status = buildSSTFile(path, *serialized_key_column, *serialized_value_column); !status.ok()) throw Exception(ErrorCodes::ROCKSDB_ERROR, "RocksDB write error: {}", status.ToString()); + /// Ingest the SST file rocksdb::IngestExternalFileOptions ingest_options; ingest_options.move_files = true; /// The temporary file is on the same disk, so move (or hardlink) file will be faster than copy if (auto status = storage.rocksdb_ptr->IngestExternalFile({path}, rocksdb::IngestExternalFileOptions()); !status.ok()) diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h index 312ad4bd93c..ed6a8068683 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h +++ b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h @@ -34,10 +34,11 @@ public: ~EmbeddedRocksDBBulkSink() override; void consume(Chunk chunk) override; + String getName() const override { return "EmbeddedRocksDBBulkSink"; } private: - + /// Get a unique path to write temporary SST file String getTemporarySSTFilePath(); std::atomic_size_t file_counter = 0; @@ -46,11 +47,6 @@ private: size_t primary_key_pos = 0; Serializations serializations; String insert_directory_queue; - - // /// Columns to hold key-value pairs, reused for all `consume` calls - // /// to reduce memory re-allocations - // ColumnString::MutablePtr serialized_key_column; - // ColumnString::MutablePtr serialized_value_column; }; } From 84efec443be2d05ba16d136533e6a3fd9ce598be Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Fri, 26 Jan 2024 01:28:26 +0000 Subject: [PATCH 0009/1018] fix document spell check Signed-off-by: Duc Canh Le --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 73b7a081797..49acc6b8bfb 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1858,6 +1858,8 @@ mdadm meanZTest meanztest mebibytes +memtable +memtables mergeable mergetree messageID From b99fd655323de3da1ad5f96247261f8db17b6ff4 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Fri, 26 Jan 2024 04:58:56 +0000 Subject: [PATCH 0010/1018] no rocksdb in fast test Signed-off-by: Duc Canh Le --- tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql index cfb97b049bf..69879a5e6b1 100644 --- a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql +++ b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql @@ -1,3 +1,4 @@ +-- Tags: no-fasttest CREATE TABLE rocksdb_worm (key UInt64, value UInt64) ENGINE = EmbeddedRocksDB() PRIMARY KEY key SETTINGS optimize_for_bulk_insert = 1; INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers_mt(1000) SETTINGS max_insert_threads = 2; SELECT count() FROM rocksdb_worm; From ff9a9e51564b6272b78d76d2ca87e51502fa4ea1 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Fri, 26 Jan 2024 13:25:56 +0000 Subject: [PATCH 0011/1018] fix tests Signed-off-by: Duc Canh Le --- tests/queries/0_stateless/01686_rocksdb.sql | 2 +- tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01686_rocksdb.sql b/tests/queries/0_stateless/01686_rocksdb.sql index f3177ce140e..3ff218bf398 100644 --- a/tests/queries/0_stateless/01686_rocksdb.sql +++ b/tests/queries/0_stateless/01686_rocksdb.sql @@ -4,7 +4,7 @@ DROP TABLE IF EXISTS 01686_test; -CREATE TABLE 01686_test (key UInt64, value String) Engine=EmbeddedRocksDB PRIMARY KEY(key); +CREATE TABLE 01686_test (key UInt64, value String) Engine=EmbeddedRocksDB PRIMARY KEY(key) SETTINGS optimize_for_bulk_insert = 0; SELECT value FROM system.rocksdb WHERE database = currentDatabase() and table = '01686_test' and name = 'number.keys.written'; INSERT INTO 01686_test SELECT number, format('Hello, world ({})', toString(number)) FROM numbers(10000); diff --git a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql index 69879a5e6b1..01a014ddfe1 100644 --- a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql +++ b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql @@ -1,4 +1,6 @@ --- Tags: no-fasttest +-- Tags: no-ordinary-database, no-fasttest +-- Tag no-ordinary-database: Sometimes cannot lock file most likely due to concurrent or adjacent tests, but we don't care how it works in Ordinary database +-- Tag no-fasttest: In fasttest, ENABLE_LIBRARIES=0, so rocksdb engine is not enabled by default CREATE TABLE rocksdb_worm (key UInt64, value UInt64) ENGINE = EmbeddedRocksDB() PRIMARY KEY key SETTINGS optimize_for_bulk_insert = 1; INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers_mt(1000) SETTINGS max_insert_threads = 2; SELECT count() FROM rocksdb_worm; From 00b1ac53bd93c775b48cd2c2554ae3192af5e037 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 29 Jan 2024 11:26:38 +0800 Subject: [PATCH 0012/1018] allow float32/float64/array/map/tuple as map key --- src/DataTypes/DataTypeMap.cpp | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/src/DataTypes/DataTypeMap.cpp b/src/DataTypes/DataTypeMap.cpp index 1f246af74d3..d81b42e531c 100644 --- a/src/DataTypes/DataTypeMap.cpp +++ b/src/DataTypes/DataTypeMap.cpp @@ -124,11 +124,10 @@ bool DataTypeMap::checkKeyType(DataTypePtr key_type) if (!isStringOrFixedString(*(low_cardinality_data_type.getDictionaryType()))) return false; } - else if (!key_type->isValueRepresentedByInteger() - && !isStringOrFixedString(*key_type) - && !WhichDataType(key_type).isNothing() - && !WhichDataType(key_type).isIPv6() - && !WhichDataType(key_type).isUUID()) + else if ( + !key_type->isValueRepresentedByNumber() && !isStringOrFixedString(*key_type) && !WhichDataType(key_type).isNothing() + && !WhichDataType(key_type).isIPv6() && !WhichDataType(key_type).isUUID() && !isMap(key_type) && !isArray(key_type) + && !isTuple(key_type)) { return false; } From 2cd0d51b6947f0b7c4f62a8626ed164417cfe8a3 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 30 Jan 2024 02:22:07 +0000 Subject: [PATCH 0013/1018] hotfix rocksdb bulk sink fails with user defined query id Signed-off-by: Duc Canh Le --- src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp index d25bfc50b22..ad54970df5f 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -81,8 +82,10 @@ EmbeddedRocksDBBulkSink::EmbeddedRocksDBBulkSink( } serializations = getHeader().getSerializations(); /// If max_insert_threads > 1 we may have multiple EmbeddedRocksDBBulkSink and getContext()->getCurrentQueryId() is not guarantee to - /// to have a distinct path - insert_directory_queue = fs::path(storage.getDataPaths()[0]) / (getContext()->getCurrentQueryId() + "-" + getRandomASCIIString(8)); + /// to have a distinct path. Also we cannot use query id as directory name here, because it could be defined by user and not suitable + /// for directory name + auto base_directory_name = sipHash128String(getContext()->getCurrentQueryId()); + insert_directory_queue = fs::path(storage.getDataPaths()[0]) / (base_directory_name + "-" + getRandomASCIIString(8)); fs::create_directory(insert_directory_queue); } From 9f0ac21f5e1855b726aa8851ea00e2a0be1b79c2 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 30 Jan 2024 17:15:50 +0800 Subject: [PATCH 0014/1018] fix ut 02169_map_functions --- tests/queries/0_stateless/02169_map_functions.reference | 1 + tests/queries/0_stateless/02169_map_functions.sql | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02169_map_functions.reference b/tests/queries/0_stateless/02169_map_functions.reference index 10746a70f06..998619ef828 100644 --- a/tests/queries/0_stateless/02169_map_functions.reference +++ b/tests/queries/0_stateless/02169_map_functions.reference @@ -104,3 +104,4 @@ {1:1,3:3,5:5,0:0,2:4,4:16,6:36} {1:1,3:3,5:5,7:7,0:0,2:4,4:16,6:36} {1:1,3:3,5:5,7:7,0:0,2:4,4:16,6:36,8:64} +{(1,0):(1,0),(2,0):(2,0)} diff --git a/tests/queries/0_stateless/02169_map_functions.sql b/tests/queries/0_stateless/02169_map_functions.sql index febaf2bd9d0..e9d9007b272 100644 --- a/tests/queries/0_stateless/02169_map_functions.sql +++ b/tests/queries/0_stateless/02169_map_functions.sql @@ -49,7 +49,7 @@ SELECT DISTINCT mapUpdate(m1, m2) FROM numbers (100000); SELECT mapApply(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT mapApply((x, y) -> (x), map(1, 0, 2, 0)); -- { serverError BAD_ARGUMENTS } SELECT mapApply((x, y) -> ('x'), map(1, 0, 2, 0)); -- { serverError BAD_ARGUMENTS } -SELECT mapApply((x) -> (x, x), map(1, 0, 2, 0)); -- { serverError BAD_ARGUMENTS } +SELECT mapApply((x) -> (x, x), map(1, 0, 2, 0)); SELECT mapApply((x, y) -> (x, 1, 2), map(1, 0, 2, 0)); -- { serverError BAD_ARGUMENTS } SELECT mapApply((x, y) -> (x, x + 1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT mapApply(map(1, 0, 2, 0), (x, y) -> (x, x + 1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } From ee46be6883d89d1af0be488a3c92aa48b85452f8 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 30 Jan 2024 17:18:57 +0800 Subject: [PATCH 0015/1018] fix ut 01651_map_functions --- tests/queries/0_stateless/01651_map_functions.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01651_map_functions.sql b/tests/queries/0_stateless/01651_map_functions.sql index 5942bf8b2c2..261d966c4b8 100644 --- a/tests/queries/0_stateless/01651_map_functions.sql +++ b/tests/queries/0_stateless/01651_map_functions.sql @@ -38,7 +38,7 @@ select mapFromArrays('aa', [4, 5]); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } select mapFromArrays(['aa', 'bb'], 5); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } select mapFromArrays(['aa', 'bb'], [4, 5], [6, 7]); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } select mapFromArrays(['aa', 'bb'], [4, 5, 6]); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } -select mapFromArrays([[1,2], [3,4]], [4, 5, 6]); -- { serverError BAD_ARGUMENTS } +select mapFromArrays([[1,2], [3,4]], [4, 5, 6]); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } select mapFromArrays(['aa', 'bb'], map('a', 4, 'b', 5)); select mapFromArrays(['aa', 'bb'], materialize(map('a', 4, 'b', 5))) from numbers(2); From 639f210dfd49684090a6502541b06171e7fd5e30 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 30 Jan 2024 17:21:06 +0800 Subject: [PATCH 0016/1018] fix ut 01720_type_map_and_casts --- tests/queries/0_stateless/01720_type_map_and_casts.sql | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01720_type_map_and_casts.sql b/tests/queries/0_stateless/01720_type_map_and_casts.sql index f03773d9084..d090d0e5b66 100644 --- a/tests/queries/0_stateless/01720_type_map_and_casts.sql +++ b/tests/queries/0_stateless/01720_type_map_and_casts.sql @@ -79,6 +79,10 @@ SELECT m[toUInt64(0)], m[toInt64(0)], m[toUInt8(0)], m[toUInt16(0)] FROM table_m DROP TABLE IF EXISTS table_map_with_key_integer; -CREATE TABLE table_map_with_key_integer (m Map(Float32, String)) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 36} +CREATE TABLE table_map_with_key_integer (m Map(Float32, String)) ENGINE = MergeTree() ORDER BY tuple(); +DROP TABLE IF EXISTS table_map_with_key_integer; + +CREATE TABLE table_map_with_key_integer (m Map(Array(UInt32), String)) ENGINE = MergeTree() ORDER BY tuple(); +DROP TABLE IF EXISTS table_map_with_key_integer; + CREATE TABLE table_map_with_key_integer (m Map(Nullable(String), String)) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 36} -CREATE TABLE table_map_with_key_integer (m Map(Array(UInt32), String)) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 36} From fce0cca924ded39196d338bd57416340ac75aa84 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 30 Jan 2024 08:37:41 +0000 Subject: [PATCH 0017/1018] hotfix: handling empty chunk Signed-off-by: Duc Canh Le --- src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp index ad54970df5f..309229f2931 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp @@ -98,6 +98,10 @@ EmbeddedRocksDBBulkSink::~EmbeddedRocksDBBulkSink() void EmbeddedRocksDBBulkSink::consume(Chunk chunk) { auto rows = chunk.getNumRows(); + + if (rows == 0) /// TODO: squashing if rows are too small + return; + const auto columns = chunk.detachColumns(); /// Convert chunk to rocksdb key-value pairs From ffa7784a3cc89f0fc9d55a9f67dd1a5105ce430d Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 31 Jan 2024 02:11:17 +0000 Subject: [PATCH 0018/1018] rocksdb: enforce batch size for bulk insert Signed-off-by: Duc Canh Le --- .../RocksDB/EmbeddedRocksDBBulkSink.cpp | 113 +++++++++++++++--- .../RocksDB/EmbeddedRocksDBBulkSink.h | 19 ++- src/Storages/RocksDB/RocksDBSettings.h | 4 +- 3 files changed, 115 insertions(+), 21 deletions(-) diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp index 309229f2931..0654a1dd33d 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp @@ -41,6 +41,7 @@ static const IColumn::Permutation & getAscendingPermutation(const IColumn & colu return perm; } +/// Build SST file from key-value pairs static rocksdb::Status buildSSTFile(const String & path, const ColumnString & keys, const ColumnString & values, const std::optional & perm_ = {}) { /// rocksdb::SstFileWriter requires keys to be sorted in ascending order @@ -80,7 +81,10 @@ EmbeddedRocksDBBulkSink::EmbeddedRocksDBBulkSink( break; ++primary_key_pos; } + serializations = getHeader().getSerializations(); + min_block_size_rows = std::max(storage.getSettings().bulk_insert_block_size, getContext()->getSettingsRef().min_insert_block_size_rows); + /// If max_insert_threads > 1 we may have multiple EmbeddedRocksDBBulkSink and getContext()->getCurrentQueryId() is not guarantee to /// to have a distinct path. Also we cannot use query id as directory name here, because it could be defined by user and not suitable /// for directory name @@ -95,16 +99,59 @@ EmbeddedRocksDBBulkSink::~EmbeddedRocksDBBulkSink() fs::remove_all(insert_directory_queue); } -void EmbeddedRocksDBBulkSink::consume(Chunk chunk) +std::vector EmbeddedRocksDBBulkSink::squash(Chunk chunk) { - auto rows = chunk.getNumRows(); + /// End of input stream + if (chunk.getNumRows() == 0) + { + if (chunks.empty()) + return {}; + std::vector to_return; + std::swap(to_return, chunks); + return to_return; + } - if (rows == 0) /// TODO: squashing if rows are too small - return; + /// Just read block is already enough. + if (isEnoughSize(chunk)) + { + /// If no accumulated data, return just read block. + if (chunks.empty()) + { + chunks.emplace_back(std::move(chunk)); + return {}; + } - const auto columns = chunk.detachColumns(); + /// Return accumulated data (maybe it has small size) and place new block to accumulated data. + std::vector to_return; + std::swap(to_return, chunks); + chunks.emplace_back(std::move(chunk)); + return to_return; + } - /// Convert chunk to rocksdb key-value pairs + /// Accumulated block is already enough. + if (isEnoughSize(chunks)) + { + /// Return accumulated data and place new block to accumulated data. + std::vector to_return; + std::swap(to_return, chunks); + chunks.emplace_back(std::move(chunk)); + return to_return; + } + + chunks.emplace_back(std::move(chunk)); + if (isEnoughSize(chunks)) + { + std::vector to_return; + std::swap(to_return, chunks); + return to_return; + } + + /// Squashed block is not ready. + return {}; +} + +std::pair EmbeddedRocksDBBulkSink::serializeChunks(const std::vector & input_chunks) const +{ auto serialized_key_column = ColumnString::create(); auto serialized_value_column = ColumnString::create(); @@ -113,27 +160,39 @@ void EmbeddedRocksDBBulkSink::consume(Chunk chunk) auto & serialized_key_offsets = serialized_key_column->getOffsets(); auto & serialized_value_data = serialized_value_column->getChars(); auto & serialized_value_offsets = serialized_value_column->getOffsets(); - - serialized_key_offsets.reserve(rows); - serialized_value_offsets.reserve(rows); WriteBufferFromVector writer_key(serialized_key_data); WriteBufferFromVector writer_value(serialized_value_data); - for (size_t i = 0; i < rows; ++i) + for (const auto & chunk : input_chunks) { - for (size_t idx = 0; idx < columns.size(); ++idx) - serializations[idx]->serializeBinary(*columns[idx], i, idx == primary_key_pos ? writer_key : writer_value, {}); - writeChar('\0', writer_key); - writeChar('\0', writer_value); - serialized_key_offsets.emplace_back(writer_key.count()); - serialized_value_offsets.emplace_back(writer_value.count()); + const auto & columns = chunk.getColumns(); + auto rows = chunk.getNumRows(); + for (size_t i = 0; i < rows; ++i) + { + for (size_t idx = 0; idx < columns.size(); ++idx) + serializations[idx]->serializeBinary(*columns[idx], i, idx == primary_key_pos ? writer_key : writer_value, {}); + writeChar('\0', writer_key); + writeChar('\0', writer_value); + serialized_key_offsets.emplace_back(writer_key.count()); + serialized_value_offsets.emplace_back(writer_value.count()); + } } writer_key.finalize(); writer_value.finalize(); } - /// Build SST file from key-value pairs + return {std::move(serialized_key_column), std::move(serialized_value_column)}; +} + +void EmbeddedRocksDBBulkSink::consume(Chunk chunk_) +{ + std::vector to_written = squash(std::move(chunk_)); + + if (to_written.empty()) + return; + + auto [serialized_key_column, serialized_value_column] = serializeChunks(to_written); auto path = getTemporarySSTFilePath(); if (auto status = buildSSTFile(path, *serialized_key_column, *serialized_value_column); !status.ok()) throw Exception(ErrorCodes::ROCKSDB_ERROR, "RocksDB write error: {}", status.ToString()); @@ -148,9 +207,29 @@ void EmbeddedRocksDBBulkSink::consume(Chunk chunk) fs::remove(path); } +void EmbeddedRocksDBBulkSink::onFinish() +{ + /// If there is any data left, write it. + if (!chunks.empty()) + consume({}); +} + + String EmbeddedRocksDBBulkSink::getTemporarySSTFilePath() { return fs::path(insert_directory_queue) / (toString(file_counter++) + ".sst"); } +bool EmbeddedRocksDBBulkSink::isEnoughSize(const std::vector & input_chunks) const +{ + size_t total_rows = 0; + for (const auto & chunk : input_chunks) + total_rows += chunk.getNumRows(); + return total_rows >= min_block_size_rows; +} + +bool EmbeddedRocksDBBulkSink::isEnoughSize(const Chunk & chunk) const +{ + return chunk.getNumRows() >= min_block_size_rows; +} } diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h index ed6a8068683..fe28576a4a3 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h +++ b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h @@ -9,7 +9,7 @@ #include #include #include -#include +#include namespace DB @@ -35,17 +35,32 @@ public: void consume(Chunk chunk) override; + void onFinish() override; + String getName() const override { return "EmbeddedRocksDBBulkSink"; } private: /// Get a unique path to write temporary SST file String getTemporarySSTFilePath(); - std::atomic_size_t file_counter = 0; + /// Squash chunks to a minimum size + std::vector squash(Chunk chunk); + bool isEnoughSize(const std::vector & input_chunks) const; + bool isEnoughSize(const Chunk & chunk) const; + /// Serialize chunks to rocksdb key-value pairs + std::pair serializeChunks(const std::vector & input_chunks) const; + StorageEmbeddedRocksDB & storage; StorageMetadataPtr metadata_snapshot; size_t primary_key_pos = 0; Serializations serializations; + + /// For squashing chunks + std::vector chunks; + size_t min_block_size_rows = 0; + + /// For writing SST files + std::atomic_size_t file_counter = 0; String insert_directory_queue; }; diff --git a/src/Storages/RocksDB/RocksDBSettings.h b/src/Storages/RocksDB/RocksDBSettings.h index f3d16bc4b3e..ccfa8449187 100644 --- a/src/Storages/RocksDB/RocksDBSettings.h +++ b/src/Storages/RocksDB/RocksDBSettings.h @@ -23,8 +23,8 @@ struct Settings; */ #define ROCKSDB_SETTINGS(M, ALIAS) \ - M(Bool, optimize_for_bulk_insert, true, "Table is optimized for bulk insertions (insert pipeline will create SST files and import to rocksdb database instead of writing " \ - "to memtables)", 0) + M(Bool, optimize_for_bulk_insert, true, "Table is optimized for bulk insertions (insert pipeline will create SST files and import to rocksdb database instead of writing to memtables)", 0) \ + M(UInt64, bulk_insert_block_size, DEFAULT_INSERT_BLOCK_SIZE, "Size of block for bulk insert, if it's smaller than query setting min_insert_block_size_rows then it will be overriden by min_insert_block_size_rows", 0) \ #define LIST_OF_ROCKSDB_SETTINGS(M, ALIAS) ROCKSDB_SETTINGS(M, ALIAS) From 3b8b5f7403fba7b090c67838d9b34722160750ac Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 31 Jan 2024 15:17:15 +0000 Subject: [PATCH 0019/1018] fix typo Signed-off-by: Duc Canh Le --- src/Storages/RocksDB/RocksDBSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/RocksDB/RocksDBSettings.h b/src/Storages/RocksDB/RocksDBSettings.h index ccfa8449187..1b168c56d89 100644 --- a/src/Storages/RocksDB/RocksDBSettings.h +++ b/src/Storages/RocksDB/RocksDBSettings.h @@ -24,7 +24,7 @@ struct Settings; #define ROCKSDB_SETTINGS(M, ALIAS) \ M(Bool, optimize_for_bulk_insert, true, "Table is optimized for bulk insertions (insert pipeline will create SST files and import to rocksdb database instead of writing to memtables)", 0) \ - M(UInt64, bulk_insert_block_size, DEFAULT_INSERT_BLOCK_SIZE, "Size of block for bulk insert, if it's smaller than query setting min_insert_block_size_rows then it will be overriden by min_insert_block_size_rows", 0) \ + M(UInt64, bulk_insert_block_size, DEFAULT_INSERT_BLOCK_SIZE, "Size of block for bulk insert, if it's smaller than query setting min_insert_block_size_rows then it will be overridden by min_insert_block_size_rows", 0) \ #define LIST_OF_ROCKSDB_SETTINGS(M, ALIAS) ROCKSDB_SETTINGS(M, ALIAS) From d8175451db35db8a17a52e016c6b4c4496890eb9 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Fri, 2 Feb 2024 13:50:50 +0800 Subject: [PATCH 0020/1018] Apply suggestions from code review Co-authored-by: vdimir --- src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp | 16 +++++++++++----- .../0_stateless/02956_rocksdb_bulk_sink.sql | 2 +- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp index 0654a1dd33d..6d9c320684d 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp @@ -54,7 +54,6 @@ static rocksdb::Status buildSSTFile(const String & path, const ColumnString & ke return status; auto rows = perm.size(); - WriteBufferFromOwnString wb_value; for (size_t i = 0; i < rows; ++i) { auto row = perm[i]; @@ -95,8 +94,15 @@ EmbeddedRocksDBBulkSink::EmbeddedRocksDBBulkSink( EmbeddedRocksDBBulkSink::~EmbeddedRocksDBBulkSink() { - if (fs::exists(insert_directory_queue)) - fs::remove_all(insert_directory_queue); + try + { + if (fs::exists(insert_directory_queue)) + fs::remove_all(insert_directory_queue); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } } std::vector EmbeddedRocksDBBulkSink::squash(Chunk chunk) @@ -193,14 +199,14 @@ void EmbeddedRocksDBBulkSink::consume(Chunk chunk_) return; auto [serialized_key_column, serialized_value_column] = serializeChunks(to_written); - auto path = getTemporarySSTFilePath(); + auto sst_file_path = getTemporarySSTFilePath(); if (auto status = buildSSTFile(path, *serialized_key_column, *serialized_value_column); !status.ok()) throw Exception(ErrorCodes::ROCKSDB_ERROR, "RocksDB write error: {}", status.ToString()); /// Ingest the SST file rocksdb::IngestExternalFileOptions ingest_options; ingest_options.move_files = true; /// The temporary file is on the same disk, so move (or hardlink) file will be faster than copy - if (auto status = storage.rocksdb_ptr->IngestExternalFile({path}, rocksdb::IngestExternalFileOptions()); !status.ok()) + if (auto status = storage.rocksdb_ptr->IngestExternalFile({path}, ingest_options); !status.ok()) throw Exception(ErrorCodes::ROCKSDB_ERROR, "RocksDB write error: {}", status.ToString()); if (fs::exists(path)) diff --git a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql index 01a014ddfe1..d685afadf81 100644 --- a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql +++ b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql @@ -1,4 +1,4 @@ --- Tags: no-ordinary-database, no-fasttest +-- Tags: no-ordinary-database, use-rocksdb -- Tag no-ordinary-database: Sometimes cannot lock file most likely due to concurrent or adjacent tests, but we don't care how it works in Ordinary database -- Tag no-fasttest: In fasttest, ENABLE_LIBRARIES=0, so rocksdb engine is not enabled by default CREATE TABLE rocksdb_worm (key UInt64, value UInt64) ENGINE = EmbeddedRocksDB() PRIMARY KEY key SETTINGS optimize_for_bulk_insert = 1; From 7f21995f116fba230789925a608da1cf7bf79c95 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 5 Feb 2024 12:20:33 +0800 Subject: [PATCH 0021/1018] add uts --- .../array/FunctionsMapMiscellaneous.cpp | 32 +++++++++++++++++-- .../0_stateless/01651_map_functions.reference | 10 ++++++ .../0_stateless/01651_map_functions.sql | 11 +++++++ .../0_stateless/02169_map_functions.reference | 20 ++++++++++++ .../0_stateless/02169_map_functions.sql | 24 ++++++++++++++ 5 files changed, 94 insertions(+), 3 deletions(-) diff --git a/src/Functions/array/FunctionsMapMiscellaneous.cpp b/src/Functions/array/FunctionsMapMiscellaneous.cpp index 157f2fa8a26..ed3f609d550 100644 --- a/src/Functions/array/FunctionsMapMiscellaneous.cpp +++ b/src/Functions/array/FunctionsMapMiscellaneous.cpp @@ -182,11 +182,37 @@ struct MapToNestedAdapter : public MapAdapterBase -struct MapToSubcolumnAdapter : public MapAdapterBase, Name> +struct MapToSubcolumnAdapter { static_assert(position <= 1); - using MapAdapterBase::extractNestedTypes; - using MapAdapterBase::extractNestedTypesAndColumns; + + static void extractNestedTypes(DataTypes & types) + { + if (types.empty()) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be at least 1", + Name::name, + types.size()); + + DataTypes new_types = {types[0]}; + MapAdapterBase::extractNestedTypes(new_types); + types[0] = new_types[0]; + } + + static void extractNestedTypesAndColumns(ColumnsWithTypeAndName & arguments) + { + if (arguments.empty()) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be at least 1", + Name::name, + arguments.size()); + + ColumnsWithTypeAndName new_arguments = {arguments[0]}; + MapAdapterBase::extractNestedTypesAndColumns(new_arguments); + arguments[0] = new_arguments[0]; + } static DataTypePtr extractNestedType(const DataTypeMap & type_map) { diff --git a/tests/queries/0_stateless/01651_map_functions.reference b/tests/queries/0_stateless/01651_map_functions.reference index 60f1b6e0d0c..0dafb841e0c 100644 --- a/tests/queries/0_stateless/01651_map_functions.reference +++ b/tests/queries/0_stateless/01651_map_functions.reference @@ -24,6 +24,16 @@ {'1002':'2004','1002':'3006','1002':'4008'} {'aa':4,'bb':5} ['aa','bb'] [4,5] {'aa':4,'bb':5} 1 0 +{0.1:4,0.2:5} [0.1,0.2] [4,5] +{0.1:4,0.2:5} 1 0 +{0.1:4,0.2:5} [0.1,0.2] [4,5] +{0.1:4,0.2:5} 1 0 +{[1,2]:4,[3,4]:5} [[1,2],[3,4]] [4,5] +{[1,2]:4,[3,4]:5} 1 0 +{{1:2}:4,{3:4}:5} [{1:2},{3:4}] [4,5] +{{1:2}:4,{3:4}:5} 1 0 +{(1,2):4,(3,4):5} [(1,2),(3,4)] [4,5] +{(1,2):4,(3,4):5} 1 0 {0:0} 1 {0:0} 0 {'aa':4,'bb':5} diff --git a/tests/queries/0_stateless/01651_map_functions.sql b/tests/queries/0_stateless/01651_map_functions.sql index 261d966c4b8..5a54d22ac97 100644 --- a/tests/queries/0_stateless/01651_map_functions.sql +++ b/tests/queries/0_stateless/01651_map_functions.sql @@ -28,6 +28,17 @@ drop table if exists table_map; select map( 'aa', 4, 'bb' , 5) as m, mapKeys(m), mapValues(m); select map( 'aa', 4, 'bb' , 5) as m, mapContains(m, 'aa'), mapContains(m, 'k'); +select map(0.1::Float32, 4, 0.2::Float32, 5) as m, mapKeys(m), mapValues(m); +select map(0.1::Float32, 4, 0.2::Float32, 5) as m, mapContains(m, 0.1::Float32), mapContains(m, 0.3::Float32); +select map(0.1::Float64, 4, 0.2::Float64, 5) as m, mapKeys(m), mapValues(m); +select map(0.1::Float64, 4, 0.2::Float64, 5) as m, mapContains(m, 0.1::Float64), mapContains(m, 0.3::Float64); +select map(array(1,2), 4, array(3,4), 5) as m, mapKeys(m), mapValues(m); +select map(array(1,2), 4, array(3,4), 5) as m, mapContains(m, array(1,2)), mapContains(m, array(1,3)); +select map(map(1,2), 4, map(3,4), 5) as m, mapKeys(m), mapValues(m); +select map(map(1,2), 4, map(3,4), 5) as m, mapContains(m, map(1,2)), mapContains(m, map(1,3)); +select map(tuple(1,2), 4, tuple(3,4), 5) as m, mapKeys(m), mapValues(m); +select map(tuple(1,2), 4, tuple(3,4), 5) as m, mapContains(m, tuple(1,2)), mapContains(m, tuple(1,3)); + select map(0, 0) as m, mapContains(m, number % 2) from numbers(2); select mapFromArrays(['aa', 'bb'], [4, 5]); diff --git a/tests/queries/0_stateless/02169_map_functions.reference b/tests/queries/0_stateless/02169_map_functions.reference index 998619ef828..6100de6abaf 100644 --- a/tests/queries/0_stateless/02169_map_functions.reference +++ b/tests/queries/0_stateless/02169_map_functions.reference @@ -26,6 +26,21 @@ {} {} {} +{0.1:4} +{0.1:4} +{[1,2]:4} +{{1:2}:4} +{(1,2):4} +1 +1 +1 +1 +1 +{0.1:4,0.2:5} +{0.1:4,0.2:5} +{[1,2]:4,[3,4]:5} +{{1:2}:4,{3:4}:5} +{(1,2):4,(3,4):5} {'key3':100,'key2':101,'key4':102,'key5':500,'key6':600} {'key3':101,'key2':102,'key4':103,'key5':500,'key6':600} {'key3':102,'key2':103,'key4':104,'key5':500,'key6':600} @@ -42,6 +57,11 @@ {'key1':1113,'key2':2226,'key5':500,'key6':600} {'key5':500,'key6':600} {'key5':500,'key6':600} +{0.1:4,0.2:5} +{0.1:4,0.2:5} +{[1,2]:4,[3,4]:5} +{{1:2}:4,{3:4}:5} +{(1,2):4,(3,4):5} 1 1 1 diff --git a/tests/queries/0_stateless/02169_map_functions.sql b/tests/queries/0_stateless/02169_map_functions.sql index e9d9007b272..854ce214924 100644 --- a/tests/queries/0_stateless/02169_map_functions.sql +++ b/tests/queries/0_stateless/02169_map_functions.sql @@ -9,11 +9,35 @@ SELECT mapApply((k, v) -> (k, v + 1), col) FROM table_map ORDER BY id; SELECT mapFilter((k, v) -> 0, col) from table_map; SELECT mapApply((k, v) -> tuple(v + 9223372036854775806), col) FROM table_map; -- { serverError BAD_ARGUMENTS } +SELECT mapFilter((k, v) -> k = 0.1::Float32, map(0.1::Float32, 4, 0.2::Float32, 5)); +SELECT mapFilter((k, v) -> k = 0.1::Float64, map(0.1::Float64, 4, 0.2::Float64, 5)); +SELECT mapFilter((k, v) -> k = array(1,2), map(array(1,2), 4, array(3,4), 5)); +SELECT mapFilter((k, v) -> k = map(1,2), map(map(1,2), 4, map(3,4), 5)); +SELECT mapFilter((k, v) -> k = tuple(1,2), map(tuple(1,2), 4, tuple(3,4), 5)); + +SELECT mapExists((k, v) -> k = 0.1::Float32, map(0.1::Float32, 4, 0.2::Float32, 5)); +SELECT mapExists((k, v) -> k = 0.1::Float64, map(0.1::Float64, 4, 0.2::Float64, 5)); +SELECT mapExists((k, v) -> k = array(1,2), map(array(1,2), 4, array(3,4), 5)); +SELECT mapExists((k, v) -> k = map(1,2), map(map(1,2), 4, map(3,4), 5)); +SELECT mapExists((k, v) -> k = tuple(1,2), map(tuple(1,2), 4, tuple(3,4), 5)); + +SELECT mapSort((k, v) -> k, map(0.1::Float32, 4, 0.2::Float32, 5)); +SELECT mapSort((k, v) -> k, map(0.1::Float64, 4, 0.2::Float64, 5)); +SELECT mapSort((k, v) -> k, map(array(1,2), 4, array(3,4), 5)); +SELECT mapSort((k, v) -> k, map(map(1,2), 4, map(3,4), 5)); +SELECT mapSort((k, v) -> k, map(tuple(1,2), 4, tuple(3,4), 5)); + SELECT mapConcat(col, map('key5', 500), map('key6', 600)) FROM table_map ORDER BY id; SELECT mapConcat(col, materialize(map('key5', 500)), map('key6', 600)) FROM table_map ORDER BY id; SELECT concat(map('key5', 500), map('key6', 600)); SELECT map('key5', 500) || map('key6', 600); +SELECT mapConcat(map(0.1::Float32, 4), map(0.2::Float32, 5)); +SELECT mapConcat(map(0.1::Float64, 4), map(0.2::Float64, 5)); +SELECT mapConcat(map(array(1,2), 4), map(array(3,4), 5)); +SELECT mapConcat(map(map(1,2), 4), map(map(3,4), 5)); +SELECT mapConcat(map(tuple(1,2), 4), map(tuple(3,4), 5)); + SELECT mapExists((k, v) -> k LIKE '%3', col) FROM table_map ORDER BY id; SELECT mapExists((k, v) -> k LIKE '%2' AND v < 1000, col) FROM table_map ORDER BY id; From 91d681693c0c2b022f548467adfccd29d4bf365c Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 14 Feb 2024 10:19:06 +0100 Subject: [PATCH 0022/1018] add failing test and Form format skeleton code --- src/Formats/registerFormats.cpp | 4 + .../Formats/Impl/FormInputFormat.cpp | 123 ++++++++++++++++++ src/Processors/Formats/Impl/FormInputFormat.h | 34 +++++ .../queries/0_stateless/02984_form_format.sh | 14 ++ 4 files changed, 175 insertions(+) create mode 100644 src/Processors/Formats/Impl/FormInputFormat.cpp create mode 100644 src/Processors/Formats/Impl/FormInputFormat.h create mode 100755 tests/queries/0_stateless/02984_form_format.sh diff --git a/src/Formats/registerFormats.cpp b/src/Formats/registerFormats.cpp index cc9cf380693..1f851da850a 100644 --- a/src/Formats/registerFormats.cpp +++ b/src/Formats/registerFormats.cpp @@ -76,6 +76,7 @@ void registerInputFormatCustomSeparated(FormatFactory & factory); void registerOutputFormatCustomSeparated(FormatFactory & factory); void registerInputFormatCapnProto(FormatFactory & factory); void registerOutputFormatCapnProto(FormatFactory & factory); +void registerInputFormatForm(FormatFactory & factory); /// Output only (presentational) formats. @@ -148,6 +149,7 @@ void registerParquetMetadataSchemaReader(FormatFactory & factory); void registerDWARFSchemaReader(FormatFactory & factory); void registerOneSchemaReader(FormatFactory & factory); void registerNpySchemaReader(FormatFactory & factory); +void registerFormSchemaReader(FormatFactory & factory); void registerFileExtensions(FormatFactory & factory); @@ -212,6 +214,7 @@ void registerFormats() registerOutputFormatRawBLOB(factory); registerInputFormatCustomSeparated(factory); registerOutputFormatCustomSeparated(factory); + registerInputFormatForm(factory); registerInputFormatORC(factory); registerOutputFormatORC(factory); @@ -291,6 +294,7 @@ void registerFormats() registerDWARFSchemaReader(factory); registerOneSchemaReader(factory); registerNpySchemaReader(factory); + registerFormSchemaReader(factory); } } diff --git a/src/Processors/Formats/Impl/FormInputFormat.cpp b/src/Processors/Formats/Impl/FormInputFormat.cpp new file mode 100644 index 00000000000..3ac7317cfc3 --- /dev/null +++ b/src/Processors/Formats/Impl/FormInputFormat.cpp @@ -0,0 +1,123 @@ +#include +#include +#include +#include "Core/NamesAndTypes.h" +#include "Formats/EscapingRuleUtils.h" +#include "Formats/FormatSettings.h" +#include "Processors/Formats/IRowInputFormat.h" +#include "base/find_symbols.h" +#include + +namespace DB +{ + +FormInputFormat::FormInputFormat(ReadBuffer & in_, Block header_, Params params_, const FormatSettings & format_settings_) + : IRowInputFormat(std::move(header_), in_, params_), format_settings(format_settings_) +{ + +} + +void FormInputFormat::readPrefix() +{ + skipBOMIfExists(*in); +} + +static bool readName(ReadBuffer & buf, StringRef & ref, String & tmp) +{ + tmp.clear(); + + while (!buf.eof()) + { + const char * next_pos = find_first_symbols<'='>(buf.position(), buf.buffer().end()); + + bool have_value = *next_pos == '='; + if (next_pos == buf.buffer().end()) + { + tmp.append(buf.position(), next_pos - buf.position()); + buf.position() = buf.buffer().end(); + buf.next(); + continue; + } + + // names occur before = + if (*next_pos == '=') + { + ref = StringRef(buf.position(), next_pos - buf.position()); + + } + + // data occurs before & + if (*next_pos == '&') + { + + } + return have_value; + } + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected end of stream while reading key name from Form format"); +} + +bool FormInputFormat::readRow(MutableColumns & columns, RowReadExtension &) +{ + size_t num_columns = columns.size(); + if (!num_columns){} + return false; +} + +FormSchemaReader::FormSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) + : IRowWithNamesSchemaReader(in_, format_settings_,getDefaultDataTypeForEscapingRule(FormatSettings::EscapingRule::Escaped)) +{ +} + +NamesAndTypesList FormSchemaReader::readRowAndGetNamesAndDataTypes(bool & eof) +{ + if(in.eof()) + { + eof = true; + return {}; + } + + NamesAndTypesList names_and_types; + StringRef name_ref; + String name_buf; + String value; + do { + bool has_value = readName(in, name_ref, name_buf); + String name = String(name_ref); + if (has_value) + { + readEscapedString(value,in); + names_and_types.emplace_back(std::move(name), tryInferDataTypeByEscapingRule(value, format_settings, FormatSettings::EscapingRule::Escaped)); + } + else + { + + } + + } + while (checkChar('=',in)); + return names_and_types; +} + +void registerInputFormatForm(FormatFactory & factory) +{ + factory.registerInputFormat("Form", []( + ReadBuffer & buf, + const Block & sample, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, std::move(params),settings); + }); +} + +void registerFormSchemaReader(FormatFactory & factory) +{ + factory.registerSchemaReader("Form", [](ReadBuffer & buffer, const FormatSettings & settings) + { + return std::make_shared(buffer, settings); + }); +} + +} + + diff --git a/src/Processors/Formats/Impl/FormInputFormat.h b/src/Processors/Formats/Impl/FormInputFormat.h new file mode 100644 index 00000000000..fcf04146e66 --- /dev/null +++ b/src/Processors/Formats/Impl/FormInputFormat.h @@ -0,0 +1,34 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class ReadBuffer; + +class FormInputFormat final : public IRowInputFormat +{ +public: + FormInputFormat(ReadBuffer & in_, Block header_, Params params_, const FormatSettings & format_settings_); + String getName() const override { return "FormInputFormat"; } + +private: + void readPrefix() override; + bool readRow(MutableColumns & columns, RowReadExtension & extra) override; + + const FormatSettings format_settings; +}; + +class FormSchemaReader : public IRowWithNamesSchemaReader +{ +public: + FormSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_); +private: + NamesAndTypesList readRowAndGetNamesAndDataTypes(bool & eof) override; +}; + +} + diff --git a/tests/queries/0_stateless/02984_form_format.sh b/tests/queries/0_stateless/02984_form_format.sh new file mode 100755 index 00000000000..4182ce7ee62 --- /dev/null +++ b/tests/queries/0_stateless/02984_form_format.sh @@ -0,0 +1,14 @@ +#!/bin/bash +# Tags: no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# Test setup +USER_FILES_PATH=$($CLICKHOUSE_CLIENT --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +FORM_DATA=${USER_FILES_PATH:?}/data.tmp +touch $FORM_DATA +echo -ne 'c.e=ls7xfkpm&c.tti.m=raf&rt.start=navigation&rt.bmr=390%2C11%2C10&rt.tstart=1707076768666&rt.bstart=1707076769091&rt.blstart=1707076769056&rt.end=1707076769078&t_resp=296&t_page=116&t_done=412&t_other=boomerang%7C6%2Cboomr_fb%7C425%2Cboomr_ld%7C390%2Cboomr_lat%7C35&rt.tt=2685&rt.obo=0&pt.fcp=407&nt_nav_st=1707076768666&nt_dns_st=1707076768683&nt_dns_end=1707076768684&nt_con_st=1707076768684&nt_con_end=1707076768850&nt_req_st=1707076768850&nt_res_st=1707076768962&nt_res_end=1707076768962&nt_domloading=1707076769040&nt_domint=1707076769066&nt_domcontloaded_st=1707076769067&nt_domcontloaded_end=1707076769068&nt_domcomp=1707076769069&nt_load_st=1707076769069&nt_load_end=1707076769078&nt_unload_st=1707076769040&nt_unload_end=1707076769041&nt_ssl_st=1707076768788&nt_enc_size=3209&nt_dec_size=10093&nt_trn_size=3940&nt_protocol=h2&nt_red_cnt=0&nt_nav_type=1&restiming=%7B%22https%3A%2F%2Fwww.basicrum.com%2F%22%3A%7B%22publications%2F%22%3A%226%2C88%2C88%2C54%2C54%2C3e%2Ci%2Ci%2Ch*12h5%2Ckb%2C5b8%22%2C%22assets%2Fjs%2F%22%3A%7B%22just-the-docs.js%22%3A%223am%2Ce%2Ce*12pc%2C_%2C8oj*20%22%2C%22boomerang-1.737.60.cutting-edge.min.js%22%3A%222au%2Cb%2Ca*1pu3%2C_%2C1m19*21*42%22%2C%22vendor%2Flunr.min.js%22%3A%223am%2Cd%2C8*16t2%2C_%2Cfym*20%22%7D%7D%7D&u=https%3A%2F%2Fwww.basicrum.com%2Fpublications%2F&r=https%3A%2F%2Fwww.basicrum.com%2Fcost-analyses%2F&v=1.737.60&sv=14&sm=p&rt.si=dd0c542f-7adf-4310-830a-6c0a3d157c90-s8cjr1&rt.ss=1707075325294&rt.sl=4&vis.st=visible&ua.plt=Linux%20x86_64&ua.vnd=&pid=8fftz949&n=1&c.t.fps=07*4*65*j*61&c.t.busy=2*4*0034&c.tti.vr=408&c.tti=408&c.b=2&c.f=60&c.f.d=2511&c.f.m=1&c.f.s=ls7xfl1h&dom.res=5&dom.doms=1&mem.lsln=0&mem.ssln=0&mem.lssz=2&mem.sssz=2&scr.xy=1920x1200&scr.bpp=24%2F24&scr.orn=0%2Flandscape-primary&cpu.cnc=16&dom.ln=114&dom.sz=10438&dom.ck=157&dom.img=0&dom.script=6&dom.script.ext=3&dom.iframe=0&dom.link=4&dom.link.css=1&sb=1' > $FORM_DATA + +$CLICKHOUSE_CLIENT --query="CREATE TABLE form_table ENGINE = MergeEngine() ORDER BY tuple() AS SELECT * FROM file('${FORM_DATA}','Form')" \ No newline at end of file From 6b4d42689caf04e45ca889eb04fc51096dffb6a9 Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 14 Feb 2024 17:03:11 +0100 Subject: [PATCH 0023/1018] add SchemaReader for Form format --- src/IO/ReadHelpers.cpp | 6 ++++ src/IO/ReadHelpers.h | 2 ++ .../Formats/Impl/FormInputFormat.cpp | 33 +++++++++++-------- src/Processors/Formats/Impl/FormInputFormat.h | 3 ++ 4 files changed, 31 insertions(+), 13 deletions(-) diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index bcfe5fd5230..90f158d945e 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -243,6 +243,12 @@ void readStringUntilWhitespace(String & s, ReadBuffer & buf) readStringUntilWhitespaceInto(s, buf); } +void readStringUntilAmpersand(String & s, ReadBuffer & buf) +{ + s.clear(); + readStringUntilCharsInto<'&'>(s, buf); +} + template void readNullTerminated>(PODArray & s, ReadBuffer & buf); template void readNullTerminated(String & s, ReadBuffer & buf); diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 49530f4787a..63dabe528cb 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -607,6 +607,8 @@ void readEscapedStringUntilEOL(String & s, ReadBuffer & buf); /// Only 0x20 as whitespace character void readStringUntilWhitespace(String & s, ReadBuffer & buf); +void readStringUntilAmpersand(String & s, ReadBuffer & buf); + /** Read string in CSV format. * Parsing rules: diff --git a/src/Processors/Formats/Impl/FormInputFormat.cpp b/src/Processors/Formats/Impl/FormInputFormat.cpp index 3ac7317cfc3..d9db431931c 100644 --- a/src/Processors/Formats/Impl/FormInputFormat.cpp +++ b/src/Processors/Formats/Impl/FormInputFormat.cpp @@ -9,7 +9,7 @@ #include namespace DB -{ +{ FormInputFormat::FormInputFormat(ReadBuffer & in_, Block header_, Params params_, const FormatSettings & format_settings_) : IRowInputFormat(std::move(header_), in_, params_), format_settings(format_settings_) @@ -22,13 +22,19 @@ void FormInputFormat::readPrefix() skipBOMIfExists(*in); } +/** Read the field name in the `Form` format. + * Return true if field name is followed by an equal sign, + * otherwise (field with no value) return false. + * The reference to the field name is written to `ref`. + * Temporary buffer `tmp` is used to copy the field name to it. + */ static bool readName(ReadBuffer & buf, StringRef & ref, String & tmp) { tmp.clear(); while (!buf.eof()) { - const char * next_pos = find_first_symbols<'='>(buf.position(), buf.buffer().end()); + const char * next_pos = find_first_symbols<'=','&'>(buf.position(), buf.buffer().end()); bool have_value = *next_pos == '='; if (next_pos == buf.buffer().end()) @@ -43,14 +49,9 @@ static bool readName(ReadBuffer & buf, StringRef & ref, String & tmp) if (*next_pos == '=') { ref = StringRef(buf.position(), next_pos - buf.position()); - + buf.position() += next_pos + have_value - buf.position(); } - // data occurs before & - if (*next_pos == '&') - { - - } return have_value; } throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected end of stream while reading key name from Form format"); @@ -58,8 +59,15 @@ static bool readName(ReadBuffer & buf, StringRef & ref, String & tmp) bool FormInputFormat::readRow(MutableColumns & columns, RowReadExtension &) { + + if (in->eof()) + return false; + size_t num_columns = columns.size(); - if (!num_columns){} + + read_columns.assign(num_columns, false); + seen_columns.assign(num_columns, false); + return false; } @@ -85,16 +93,15 @@ NamesAndTypesList FormSchemaReader::readRowAndGetNamesAndDataTypes(bool & eof) String name = String(name_ref); if (has_value) { - readEscapedString(value,in); + readStringUntilAmpersand(value,in); names_and_types.emplace_back(std::move(name), tryInferDataTypeByEscapingRule(value, format_settings, FormatSettings::EscapingRule::Escaped)); } else { - + throw Exception(ErrorCodes::INCORRECT_DATA, "Found field without value while parsing Form format: {}", name_ref.toString()); } - } - while (checkChar('=',in)); + while (checkChar('&',in)); return names_and_types; } diff --git a/src/Processors/Formats/Impl/FormInputFormat.h b/src/Processors/Formats/Impl/FormInputFormat.h index fcf04146e66..32047741a8c 100644 --- a/src/Processors/Formats/Impl/FormInputFormat.h +++ b/src/Processors/Formats/Impl/FormInputFormat.h @@ -20,6 +20,9 @@ private: bool readRow(MutableColumns & columns, RowReadExtension & extra) override; const FormatSettings format_settings; + String name_buf; + std::vector read_columns; + std::vector seen_columns; }; class FormSchemaReader : public IRowWithNamesSchemaReader From 4697194817e0f398ebd5ed125c73eff5502800fa Mon Sep 17 00:00:00 2001 From: Blargian Date: Fri, 16 Feb 2024 17:11:30 +0100 Subject: [PATCH 0024/1018] Form format reads in data but doesn't escape URL sequence correctly yet --- .../Formats/Impl/FormInputFormat.cpp | 61 ++++++++++++++++++- src/Processors/Formats/Impl/FormInputFormat.h | 5 ++ 2 files changed, 63 insertions(+), 3 deletions(-) diff --git a/src/Processors/Formats/Impl/FormInputFormat.cpp b/src/Processors/Formats/Impl/FormInputFormat.cpp index d9db431931c..c0bf0e8dc02 100644 --- a/src/Processors/Formats/Impl/FormInputFormat.cpp +++ b/src/Processors/Formats/Impl/FormInputFormat.cpp @@ -2,6 +2,8 @@ #include #include #include "Core/NamesAndTypes.h" +#include "Core/QueryProcessingStage.h" +#include "DataTypes/IDataType.h" #include "Formats/EscapingRuleUtils.h" #include "Formats/FormatSettings.h" #include "Processors/Formats/IRowInputFormat.h" @@ -11,10 +13,16 @@ namespace DB { +enum +{ + INVALID_INDEX = size_t(-1), +}; + FormInputFormat::FormInputFormat(ReadBuffer & in_, Block header_, Params params_, const FormatSettings & format_settings_) : IRowInputFormat(std::move(header_), in_, params_), format_settings(format_settings_) { - + const auto & header = getPort().getHeader(); + name_map = header.getNamesToIndexesMap(); } void FormInputFormat::readPrefix() @@ -22,6 +30,11 @@ void FormInputFormat::readPrefix() skipBOMIfExists(*in); } +const String & FormInputFormat::columnName(size_t i) const +{ + return getPort().getHeader().getByPosition(i).name; +} + /** Read the field name in the `Form` format. * Return true if field name is followed by an equal sign, * otherwise (field with no value) return false. @@ -57,6 +70,24 @@ static bool readName(ReadBuffer & buf, StringRef & ref, String & tmp) throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected end of stream while reading key name from Form format"); } +void FormInputFormat::readField(size_t index, MutableColumns & columns) +{ + if (seen_columns[index]) + throw Exception(ErrorCodes::INCORRECT_DATA, "Duplicate field found while parsing Form format: {}", columnName(index)); + + seen_columns[index] = true; + const auto & serialization = serializations[index]; + String str; + readStringUntilAmpersand(str,*in); + + if (!in->eof()) + ++in->position(); /// skip & + + ReadBufferFromString buf(str); + serialization->deserializeTextRaw(*columns[index], buf, format_settings); + read_columns[index] = true; +} + bool FormInputFormat::readRow(MutableColumns & columns, RowReadExtension &) { @@ -64,11 +95,35 @@ bool FormInputFormat::readRow(MutableColumns & columns, RowReadExtension &) return false; size_t num_columns = columns.size(); - read_columns.assign(num_columns, false); seen_columns.assign(num_columns, false); + + for (size_t i = 0; i < num_columns; i++) + { + if(in->eof()) + break; + + StringRef name_ref; + bool has_value = readName(*in, name_ref, name_buf); + const auto it = name_map.find(String(name_ref)); + + if (has_value) + { + size_t column_index; + if (it != name_map.end()) + column_index = it->second; + else + column_index = INVALID_INDEX; + + if (column_index == INVALID_INDEX) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: illegal value of column_index"); + + readField(column_index, columns); + } + + } - return false; + return true; } FormSchemaReader::FormSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) diff --git a/src/Processors/Formats/Impl/FormInputFormat.h b/src/Processors/Formats/Impl/FormInputFormat.h index 32047741a8c..0124646524b 100644 --- a/src/Processors/Formats/Impl/FormInputFormat.h +++ b/src/Processors/Formats/Impl/FormInputFormat.h @@ -18,11 +18,16 @@ public: private: void readPrefix() override; bool readRow(MutableColumns & columns, RowReadExtension & extra) override; + void readField(size_t index, MutableColumns & columns); + const String & columnName(size_t i) const; const FormatSettings format_settings; String name_buf; std::vector read_columns; std::vector seen_columns; + + /// Hash table matches field name to position in the block + Block::NameMap name_map; }; class FormSchemaReader : public IRowWithNamesSchemaReader From 28733a859a85a6f646d107fba93c4a69cdffb170 Mon Sep 17 00:00:00 2001 From: Blargian Date: Sat, 17 Feb 2024 15:45:56 +0100 Subject: [PATCH 0025/1018] finish implementing Format format and check that test query succeeds --- .../Formats/Impl/FormInputFormat.cpp | 22 +++-- src/Processors/Formats/Impl/FormInputFormat.h | 2 +- .../0_stateless/02984_form_format.reference | 84 +++++++++++++++++++ .../queries/0_stateless/02984_form_format.sh | 14 +++- 4 files changed, 109 insertions(+), 13 deletions(-) create mode 100644 tests/queries/0_stateless/02984_form_format.reference diff --git a/src/Processors/Formats/Impl/FormInputFormat.cpp b/src/Processors/Formats/Impl/FormInputFormat.cpp index c0bf0e8dc02..6f1bccf00e9 100644 --- a/src/Processors/Formats/Impl/FormInputFormat.cpp +++ b/src/Processors/Formats/Impl/FormInputFormat.cpp @@ -58,7 +58,7 @@ static bool readName(ReadBuffer & buf, StringRef & ref, String & tmp) continue; } - // names occur before = + /// Column names (keys) occur before '=' if (*next_pos == '=') { ref = StringRef(buf.position(), next_pos - buf.position()); @@ -75,15 +75,17 @@ void FormInputFormat::readField(size_t index, MutableColumns & columns) if (seen_columns[index]) throw Exception(ErrorCodes::INCORRECT_DATA, "Duplicate field found while parsing Form format: {}", columnName(index)); - seen_columns[index] = true; + seen_columns[index] = read_columns[index] = true; const auto & serialization = serializations[index]; - String str; - readStringUntilAmpersand(str,*in); + String encoded_str, decoded_str; + readStringUntilAmpersand(encoded_str,*in); + Poco::URI::decode(encoded_str, decoded_str); + /// skip '&' before next key value pair if (!in->eof()) - ++in->position(); /// skip & + ++in->position(); - ReadBufferFromString buf(str); + ReadBufferFromString buf(decoded_str); serialization->deserializeTextRaw(*columns[index], buf, format_settings); read_columns[index] = true; } @@ -102,7 +104,7 @@ bool FormInputFormat::readRow(MutableColumns & columns, RowReadExtension &) { if(in->eof()) break; - + StringRef name_ref; bool has_value = readName(*in, name_ref, name_buf); const auto it = name_map.find(String(name_ref)); @@ -120,9 +122,13 @@ bool FormInputFormat::readRow(MutableColumns & columns, RowReadExtension &) readField(column_index, columns); } + else + { + throw Exception(ErrorCodes::INCORRECT_DATA, "Found field without value while parsing TSKV format: {}", name_ref.toString()); + } } - + return true; } diff --git a/src/Processors/Formats/Impl/FormInputFormat.h b/src/Processors/Formats/Impl/FormInputFormat.h index 0124646524b..e56e60747b9 100644 --- a/src/Processors/Formats/Impl/FormInputFormat.h +++ b/src/Processors/Formats/Impl/FormInputFormat.h @@ -20,7 +20,7 @@ private: bool readRow(MutableColumns & columns, RowReadExtension & extra) override; void readField(size_t index, MutableColumns & columns); const String & columnName(size_t i) const; - + const FormatSettings format_settings; String name_buf; std::vector read_columns; diff --git a/tests/queries/0_stateless/02984_form_format.reference b/tests/queries/0_stateless/02984_form_format.reference new file mode 100644 index 00000000000..361f0985b74 --- /dev/null +++ b/tests/queries/0_stateless/02984_form_format.reference @@ -0,0 +1,84 @@ +Row 1: +────── +c.e: ls7xfkpm +c.tti.m: raf +rt.start: navigation +rt.bmr: 390,11,10 +rt.tstart: 1707076768666 +rt.bstart: 1707076769091 +rt.blstart: 1707076769056 +rt.end: 1707076769078 +t_resp: 296 +t_page: 116 +t_done: 412 +t_other: boomerang|6,boomr_fb|425,boomr_ld|390,boomr_lat|35 +rt.tt: 2685 +rt.obo: 0 +pt.fcp: 407 +nt_nav_st: 1707076768666 +nt_dns_st: 1707076768683 +nt_dns_end: 1707076768684 +nt_con_st: 1707076768684 +nt_con_end: 1707076768850 +nt_req_st: 1707076768850 +nt_res_st: 1707076768962 +nt_res_end: 1707076768962 +nt_domloading: 1707076769040 +nt_domint: 1707076769066 +nt_domcontloaded_st: 1707076769067 +nt_domcontloaded_end: 1707076769068 +nt_domcomp: 1707076769069 +nt_load_st: 1707076769069 +nt_load_end: 1707076769078 +nt_unload_st: 1707076769040 +nt_unload_end: 1707076769041 +nt_ssl_st: 1707076768788 +nt_enc_size: 3209 +nt_dec_size: 10093 +nt_trn_size: 3940 +nt_protocol: h2 +nt_red_cnt: 0 +nt_nav_type: 1 +restiming: {"https://www.basicrum.com/":{"publications/":"6,88,88,54,54,3e,i,i,h*12h5,kb,5b8","assets/js/":{"just-the-docs.js":"3am,e,e*12pc,_,8oj*20","boomerang-1.737.60.cutting-edge.min.js":"2au,b,a*1pu3,_,1m19*21*42","vendor/lunr.min.js":"3am,d,8*16t2,_,fym*20"}}} +u: https://www.basicrum.com/publications/ +r: https://www.basicrum.com/cost-analyses/ +v: 1.737.60 +sv: 14 +sm: p +rt.si: dd0c542f-7adf-4310-830a-6c0a3d157c90-s8cjr1 +rt.ss: 1707075325294 +rt.sl: 4 +vis.st: visible +ua.plt: Linux x86_64 +ua.vnd: +pid: 8fftz949 +n: 1 +c.t.fps: 07*4*65*j*61 +c.t.busy: 2*4*0034 +c.tti.vr: 408 +c.tti: 408 +c.b: 2 +c.f: 60 +c.f.d: 2511 +c.f.m: 1 +c.f.s: ls7xfl1h +dom.res: 5 +dom.doms: 1 +mem.lsln: 0 +mem.ssln: 0 +mem.lssz: 2 +mem.sssz: 2 +scr.xy: 1920x1200 +scr.bpp: 24/24 +scr.orn: 0/landscape-primary +cpu.cnc: 16 +dom.ln: 114 +dom.sz: 10438 +dom.ck: 157 +dom.img: 0 +dom.script: 6 +dom.script.ext: 3 +dom.iframe: 0 +dom.link: 4 +dom.link.css: 1 +sb: 1 diff --git a/tests/queries/0_stateless/02984_form_format.sh b/tests/queries/0_stateless/02984_form_format.sh index 4182ce7ee62..21553f387af 100755 --- a/tests/queries/0_stateless/02984_form_format.sh +++ b/tests/queries/0_stateless/02984_form_format.sh @@ -6,9 +6,15 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh # Test setup -USER_FILES_PATH=$($CLICKHOUSE_CLIENT --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') -FORM_DATA=${USER_FILES_PATH:?}/data.tmp +# USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +USER_FILES_PATH="/home/shaun/Desktop/ClickHouse/programs/server/user_files" +FORM_DATA="${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/data.tmp" +mkdir -p ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ touch $FORM_DATA -echo -ne 'c.e=ls7xfkpm&c.tti.m=raf&rt.start=navigation&rt.bmr=390%2C11%2C10&rt.tstart=1707076768666&rt.bstart=1707076769091&rt.blstart=1707076769056&rt.end=1707076769078&t_resp=296&t_page=116&t_done=412&t_other=boomerang%7C6%2Cboomr_fb%7C425%2Cboomr_ld%7C390%2Cboomr_lat%7C35&rt.tt=2685&rt.obo=0&pt.fcp=407&nt_nav_st=1707076768666&nt_dns_st=1707076768683&nt_dns_end=1707076768684&nt_con_st=1707076768684&nt_con_end=1707076768850&nt_req_st=1707076768850&nt_res_st=1707076768962&nt_res_end=1707076768962&nt_domloading=1707076769040&nt_domint=1707076769066&nt_domcontloaded_st=1707076769067&nt_domcontloaded_end=1707076769068&nt_domcomp=1707076769069&nt_load_st=1707076769069&nt_load_end=1707076769078&nt_unload_st=1707076769040&nt_unload_end=1707076769041&nt_ssl_st=1707076768788&nt_enc_size=3209&nt_dec_size=10093&nt_trn_size=3940&nt_protocol=h2&nt_red_cnt=0&nt_nav_type=1&restiming=%7B%22https%3A%2F%2Fwww.basicrum.com%2F%22%3A%7B%22publications%2F%22%3A%226%2C88%2C88%2C54%2C54%2C3e%2Ci%2Ci%2Ch*12h5%2Ckb%2C5b8%22%2C%22assets%2Fjs%2F%22%3A%7B%22just-the-docs.js%22%3A%223am%2Ce%2Ce*12pc%2C_%2C8oj*20%22%2C%22boomerang-1.737.60.cutting-edge.min.js%22%3A%222au%2Cb%2Ca*1pu3%2C_%2C1m19*21*42%22%2C%22vendor%2Flunr.min.js%22%3A%223am%2Cd%2C8*16t2%2C_%2Cfym*20%22%7D%7D%7D&u=https%3A%2F%2Fwww.basicrum.com%2Fpublications%2F&r=https%3A%2F%2Fwww.basicrum.com%2Fcost-analyses%2F&v=1.737.60&sv=14&sm=p&rt.si=dd0c542f-7adf-4310-830a-6c0a3d157c90-s8cjr1&rt.ss=1707075325294&rt.sl=4&vis.st=visible&ua.plt=Linux%20x86_64&ua.vnd=&pid=8fftz949&n=1&c.t.fps=07*4*65*j*61&c.t.busy=2*4*0034&c.tti.vr=408&c.tti=408&c.b=2&c.f=60&c.f.d=2511&c.f.m=1&c.f.s=ls7xfl1h&dom.res=5&dom.doms=1&mem.lsln=0&mem.ssln=0&mem.lssz=2&mem.sssz=2&scr.xy=1920x1200&scr.bpp=24%2F24&scr.orn=0%2Flandscape-primary&cpu.cnc=16&dom.ln=114&dom.sz=10438&dom.ck=157&dom.img=0&dom.script=6&dom.script.ext=3&dom.iframe=0&dom.link=4&dom.link.css=1&sb=1' > $FORM_DATA +echo -ne "c.e=ls7xfkpm&c.tti.m=raf&rt.start=navigation&rt.bmr=390%2C11%2C10&rt.tstart=1707076768666&rt.bstart=1707076769091&rt.blstart=1707076769056&rt.end=1707076769078&t_resp=296&t_page=116&t_done=412&t_other=boomerang%7C6%2Cboomr_fb%7C425%2Cboomr_ld%7C390%2Cboomr_lat%7C35&rt.tt=2685&rt.obo=0&pt.fcp=407&nt_nav_st=1707076768666&nt_dns_st=1707076768683&nt_dns_end=1707076768684&nt_con_st=1707076768684&nt_con_end=1707076768850&nt_req_st=1707076768850&nt_res_st=1707076768962&nt_res_end=1707076768962&nt_domloading=1707076769040&nt_domint=1707076769066&nt_domcontloaded_st=1707076769067&nt_domcontloaded_end=1707076769068&nt_domcomp=1707076769069&nt_load_st=1707076769069&nt_load_end=1707076769078&nt_unload_st=1707076769040&nt_unload_end=1707076769041&nt_ssl_st=1707076768788&nt_enc_size=3209&nt_dec_size=10093&nt_trn_size=3940&nt_protocol=h2&nt_red_cnt=0&nt_nav_type=1&restiming=%7B%22https%3A%2F%2Fwww.basicrum.com%2F%22%3A%7B%22publications%2F%22%3A%226%2C88%2C88%2C54%2C54%2C3e%2Ci%2Ci%2Ch*12h5%2Ckb%2C5b8%22%2C%22assets%2Fjs%2F%22%3A%7B%22just-the-docs.js%22%3A%223am%2Ce%2Ce*12pc%2C_%2C8oj*20%22%2C%22boomerang-1.737.60.cutting-edge.min.js%22%3A%222au%2Cb%2Ca*1pu3%2C_%2C1m19*21*42%22%2C%22vendor%2Flunr.min.js%22%3A%223am%2Cd%2C8*16t2%2C_%2Cfym*20%22%7D%7D%7D&u=https%3A%2F%2Fwww.basicrum.com%2Fpublications%2F&r=https%3A%2F%2Fwww.basicrum.com%2Fcost-analyses%2F&v=1.737.60&sv=14&sm=p&rt.si=dd0c542f-7adf-4310-830a-6c0a3d157c90-s8cjr1&rt.ss=1707075325294&rt.sl=4&vis.st=visible&ua.plt=Linux%20x86_64&ua.vnd=&pid=8fftz949&n=1&c.t.fps=07*4*65*j*61&c.t.busy=2*4*0034&c.tti.vr=408&c.tti=408&c.b=2&c.f=60&c.f.d=2511&c.f.m=1&c.f.s=ls7xfl1h&dom.res=5&dom.doms=1&mem.lsln=0&mem.ssln=0&mem.lssz=2&mem.sssz=2&scr.xy=1920x1200&scr.bpp=24%2F24&scr.orn=0%2Flandscape-primary&cpu.cnc=16&dom.ln=114&dom.sz=10438&dom.ck=157&dom.img=0&dom.script=6&dom.script.ext=3&dom.iframe=0&dom.link=4&dom.link.css=1&sb=1" > $FORM_DATA -$CLICKHOUSE_CLIENT --query="CREATE TABLE form_table ENGINE = MergeEngine() ORDER BY tuple() AS SELECT * FROM file('${FORM_DATA}','Form')" \ No newline at end of file +# Insert Form data from clickhouse-client +$CLICKHOUSE_CLIENT --query "SELECT * FROM file('$FORM_DATA',Form) FORMAT Vertical" + +# Test teardown +rm -r ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME} \ No newline at end of file From d2fb1e4ff8fbfcab1771c94c6befa400a230ee9b Mon Sep 17 00:00:00 2001 From: Blargian Date: Mon, 19 Feb 2024 07:58:22 +0100 Subject: [PATCH 0026/1018] Add FormSchemaReader --- docs/en/interfaces/formats.md | 8 + src/IO/ReadHelpers.cpp | 6 + src/IO/ReadHelpers.h | 1 + .../Formats/Impl/FormInputFormat.cpp | 129 +++++++++------- src/Processors/Formats/Impl/FormInputFormat.h | 6 + .../queries/0_stateless/02984_form_format.sh | 139 +++++++++++++++++- 6 files changed, 232 insertions(+), 57 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index a11c3e5ef19..3bbd9a112c0 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -90,6 +90,7 @@ The supported formats are: | [MySQLDump](#mysqldump) | ✔ | ✗ | | [DWARF](#dwarf) | ✔ | ✗ | | [Markdown](#markdown) | ✗ | ✔ | +| [Form](#form) | ✔ | ✗ | You can control some format processing parameters with the ClickHouse settings. For more information read the [Settings](/docs/en/operations/settings/settings-formats.md) section. @@ -2839,3 +2840,10 @@ FORMAT Markdown ``` Markdown table will be generated automatically and can be used on markdown-enabled platforms, like Github. This format is used only for output. + +## Form {#form} + +The Form format can be used to read or write a single record in the application/x-www-form-urlencoded format in which data is formatted `key1=value1&key2=value2` + +```sql +``` \ No newline at end of file diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index 90f158d945e..ff75d3bce31 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -249,6 +249,12 @@ void readStringUntilAmpersand(String & s, ReadBuffer & buf) readStringUntilCharsInto<'&'>(s, buf); } +void readStringUntilEquals(String & s, ReadBuffer & buf) +{ + s.clear(); + readStringUntilCharsInto<'='>(s, buf); +} + template void readNullTerminated>(PODArray & s, ReadBuffer & buf); template void readNullTerminated(String & s, ReadBuffer & buf); diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 63dabe528cb..28330171c94 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -608,6 +608,7 @@ void readEscapedStringUntilEOL(String & s, ReadBuffer & buf); void readStringUntilWhitespace(String & s, ReadBuffer & buf); void readStringUntilAmpersand(String & s, ReadBuffer & buf); +void readStringUntilEquals(String & s, ReadBuffer & buf); /** Read string in CSV format. diff --git a/src/Processors/Formats/Impl/FormInputFormat.cpp b/src/Processors/Formats/Impl/FormInputFormat.cpp index 6f1bccf00e9..6408da52ebc 100644 --- a/src/Processors/Formats/Impl/FormInputFormat.cpp +++ b/src/Processors/Formats/Impl/FormInputFormat.cpp @@ -2,27 +2,57 @@ #include #include #include "Core/NamesAndTypes.h" -#include "Core/QueryProcessingStage.h" #include "DataTypes/IDataType.h" #include "Formats/EscapingRuleUtils.h" #include "Formats/FormatSettings.h" +#include "Formats/SchemaInferenceUtils.h" +#include "IO/ReadBufferFromString.h" #include "Processors/Formats/IRowInputFormat.h" #include "base/find_symbols.h" #include +#include namespace DB { enum { - INVALID_INDEX = size_t(-1), + UNKNOWN_FIELD = size_t(-1), + NESTED_FIELD = size_t(-2) }; +/** + * Recursively check if column_name contains '.' in name + * and split into separate columns if it does + */ +void FormInputFormat::checkAndSplitIfNested(const StringRef column_name) +{ + while(true) + { + const auto split = Nested::splitName(column_name.toView()); + if (!split.second.empty()) + { + const StringRef table_name(column_name.data, split.first.size()); + name_map[table_name] = NESTED_FIELD; + const StringRef next_table_name(String(split.second).c_str(), split.second.size()); + checkAndSplitIfNested(next_table_name); + } + break; + } +} + FormInputFormat::FormInputFormat(ReadBuffer & in_, Block header_, Params params_, const FormatSettings & format_settings_) : IRowInputFormat(std::move(header_), in_, params_), format_settings(format_settings_) { const auto & header = getPort().getHeader(); name_map = header.getNamesToIndexesMap(); + + /// not sure if this needs to be on a setting or not? + for (size_t i=0; i != header.columns(); ++i) + { + const StringRef column_name = header.getByPosition(i).name; + checkAndSplitIfNested(column_name); + } } void FormInputFormat::readPrefix() @@ -41,7 +71,7 @@ const String & FormInputFormat::columnName(size_t i) const * The reference to the field name is written to `ref`. * Temporary buffer `tmp` is used to copy the field name to it. */ -static bool readName(ReadBuffer & buf, StringRef & ref, String & tmp) +StringRef readName(ReadBuffer & buf, StringRef & ref, String & tmp) { tmp.clear(); @@ -49,7 +79,6 @@ static bool readName(ReadBuffer & buf, StringRef & ref, String & tmp) { const char * next_pos = find_first_symbols<'=','&'>(buf.position(), buf.buffer().end()); - bool have_value = *next_pos == '='; if (next_pos == buf.buffer().end()) { tmp.append(buf.position(), next_pos - buf.position()); @@ -62,10 +91,10 @@ static bool readName(ReadBuffer & buf, StringRef & ref, String & tmp) if (*next_pos == '=') { ref = StringRef(buf.position(), next_pos - buf.position()); - buf.position() += next_pos + have_value - buf.position(); + buf.position() += next_pos + 1 - buf.position(); } - return have_value; + return ref; } throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected end of stream while reading key name from Form format"); } @@ -90,43 +119,36 @@ void FormInputFormat::readField(size_t index, MutableColumns & columns) read_columns[index] = true; } + +String readFieldName(ReadBuffer & in) +{ + String field; + readStringUntilEquals(field, in); + assertChar('=', in); + return field; +} + +inline size_t FormInputFormat::columnIndex(StringRef name) +{ + const auto it = name_map.find(name); + if (it != name_map.end()) + { + return it->second; + } + else + return UNKNOWN_FIELD; +} + bool FormInputFormat::readRow(MutableColumns & columns, RowReadExtension &) { - - if (in->eof()) - return false; - size_t num_columns = columns.size(); + read_columns.assign(num_columns, false); seen_columns.assign(num_columns, false); - for (size_t i = 0; i < num_columns; i++) + for (size_t index = 0; index < num_columns; ++index) { - if(in->eof()) - break; - - StringRef name_ref; - bool has_value = readName(*in, name_ref, name_buf); - const auto it = name_map.find(String(name_ref)); - - if (has_value) - { - size_t column_index; - if (it != name_map.end()) - column_index = it->second; - else - column_index = INVALID_INDEX; - - if (column_index == INVALID_INDEX) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: illegal value of column_index"); - - readField(column_index, columns); - } - else - { - throw Exception(ErrorCodes::INCORRECT_DATA, "Found field without value while parsing TSKV format: {}", name_ref.toString()); - } - + readField(1, columns); } return true; @@ -137,6 +159,21 @@ FormSchemaReader::FormSchemaReader(ReadBuffer & in_, const FormatSettings & form { } +NamesAndTypesList readRowAndGetNamesAndDataTypesForFormRow(ReadBuffer & in, const FormatSettings & settings) +{ + NamesAndTypesList names_and_types; + String field, value; + do + { + auto name = readFieldName(in); + readStringUntilAmpersand(value,in); + auto type = tryInferDataTypeByEscapingRule(value, settings, FormatSettings::EscapingRule::Escaped); + names_and_types.emplace_back(name, type); + } + while(checkChar('&',in)); + return names_and_types; +} + NamesAndTypesList FormSchemaReader::readRowAndGetNamesAndDataTypes(bool & eof) { if(in.eof()) @@ -145,25 +182,7 @@ NamesAndTypesList FormSchemaReader::readRowAndGetNamesAndDataTypes(bool & eof) return {}; } - NamesAndTypesList names_and_types; - StringRef name_ref; - String name_buf; - String value; - do { - bool has_value = readName(in, name_ref, name_buf); - String name = String(name_ref); - if (has_value) - { - readStringUntilAmpersand(value,in); - names_and_types.emplace_back(std::move(name), tryInferDataTypeByEscapingRule(value, format_settings, FormatSettings::EscapingRule::Escaped)); - } - else - { - throw Exception(ErrorCodes::INCORRECT_DATA, "Found field without value while parsing Form format: {}", name_ref.toString()); - } - } - while (checkChar('&',in)); - return names_and_types; + return readRowAndGetNamesAndDataTypesForFormRow(in, format_settings); } void registerInputFormatForm(FormatFactory & factory) diff --git a/src/Processors/Formats/Impl/FormInputFormat.h b/src/Processors/Formats/Impl/FormInputFormat.h index e56e60747b9..af13b6540d3 100644 --- a/src/Processors/Formats/Impl/FormInputFormat.h +++ b/src/Processors/Formats/Impl/FormInputFormat.h @@ -19,7 +19,12 @@ private: void readPrefix() override; bool readRow(MutableColumns & columns, RowReadExtension & extra) override; void readField(size_t index, MutableColumns & columns); + String readFieldName(ReadBuffer & in); const String & columnName(size_t i) const; + size_t columnIndex(StringRef name); + + /// recursively split names separated by '.' into own columns + void checkAndSplitIfNested(StringRef column_name); const FormatSettings format_settings; String name_buf; @@ -36,6 +41,7 @@ public: FormSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_); private: NamesAndTypesList readRowAndGetNamesAndDataTypes(bool & eof) override; + NamesAndTypesList readRowAndGetNamesAndDataTypesForForm(ReadBuffer & in, const FormatSettings & settings); }; } diff --git a/tests/queries/0_stateless/02984_form_format.sh b/tests/queries/0_stateless/02984_form_format.sh index 21553f387af..32e372dfd81 100755 --- a/tests/queries/0_stateless/02984_form_format.sh +++ b/tests/queries/0_stateless/02984_form_format.sh @@ -10,11 +10,146 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) USER_FILES_PATH="/home/shaun/Desktop/ClickHouse/programs/server/user_files" FORM_DATA="${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/data.tmp" mkdir -p ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ -touch $FORM_DATA +touch $FORM_DATA + echo -ne "c.e=ls7xfkpm&c.tti.m=raf&rt.start=navigation&rt.bmr=390%2C11%2C10&rt.tstart=1707076768666&rt.bstart=1707076769091&rt.blstart=1707076769056&rt.end=1707076769078&t_resp=296&t_page=116&t_done=412&t_other=boomerang%7C6%2Cboomr_fb%7C425%2Cboomr_ld%7C390%2Cboomr_lat%7C35&rt.tt=2685&rt.obo=0&pt.fcp=407&nt_nav_st=1707076768666&nt_dns_st=1707076768683&nt_dns_end=1707076768684&nt_con_st=1707076768684&nt_con_end=1707076768850&nt_req_st=1707076768850&nt_res_st=1707076768962&nt_res_end=1707076768962&nt_domloading=1707076769040&nt_domint=1707076769066&nt_domcontloaded_st=1707076769067&nt_domcontloaded_end=1707076769068&nt_domcomp=1707076769069&nt_load_st=1707076769069&nt_load_end=1707076769078&nt_unload_st=1707076769040&nt_unload_end=1707076769041&nt_ssl_st=1707076768788&nt_enc_size=3209&nt_dec_size=10093&nt_trn_size=3940&nt_protocol=h2&nt_red_cnt=0&nt_nav_type=1&restiming=%7B%22https%3A%2F%2Fwww.basicrum.com%2F%22%3A%7B%22publications%2F%22%3A%226%2C88%2C88%2C54%2C54%2C3e%2Ci%2Ci%2Ch*12h5%2Ckb%2C5b8%22%2C%22assets%2Fjs%2F%22%3A%7B%22just-the-docs.js%22%3A%223am%2Ce%2Ce*12pc%2C_%2C8oj*20%22%2C%22boomerang-1.737.60.cutting-edge.min.js%22%3A%222au%2Cb%2Ca*1pu3%2C_%2C1m19*21*42%22%2C%22vendor%2Flunr.min.js%22%3A%223am%2Cd%2C8*16t2%2C_%2Cfym*20%22%7D%7D%7D&u=https%3A%2F%2Fwww.basicrum.com%2Fpublications%2F&r=https%3A%2F%2Fwww.basicrum.com%2Fcost-analyses%2F&v=1.737.60&sv=14&sm=p&rt.si=dd0c542f-7adf-4310-830a-6c0a3d157c90-s8cjr1&rt.ss=1707075325294&rt.sl=4&vis.st=visible&ua.plt=Linux%20x86_64&ua.vnd=&pid=8fftz949&n=1&c.t.fps=07*4*65*j*61&c.t.busy=2*4*0034&c.tti.vr=408&c.tti=408&c.b=2&c.f=60&c.f.d=2511&c.f.m=1&c.f.s=ls7xfl1h&dom.res=5&dom.doms=1&mem.lsln=0&mem.ssln=0&mem.lssz=2&mem.sssz=2&scr.xy=1920x1200&scr.bpp=24%2F24&scr.orn=0%2Flandscape-primary&cpu.cnc=16&dom.ln=114&dom.sz=10438&dom.ck=157&dom.img=0&dom.script=6&dom.script.ext=3&dom.iframe=0&dom.link=4&dom.link.css=1&sb=1" > $FORM_DATA +$CLICKHOUSE_CLIENT -nm --query " +DROP TABLE IF EXISTS form_data; +CREATE TABLE IF NOT EXISTS form_data +( +c Nested +( + t Nested + ( + fps Nullable(String), + busy Nullable(String) + ), + tti Nested + ( + tti_value Nullable(Int64), + vr Nullable(Int64), + m Nullable(String) + ), + f Nested + ( + f_value Nullable(Int64), + d Nullable(Int64), + m Nullable(Int64), + s Nullable(String) + ), + e Nullable(String), + b Nullable(Int64) +), +rt Nested +( + start Nullable(String), + bmr Nullable(String), + tstart Nullable(Int64), + bstart Nullable(Int64), + blstart Nullable(Int64), + end Nullable(Int64), + tt Nullable(Int64), + obo Nullable(Int64), + si Nullable(String), + ss Nullable(Int64), + sl Nullable(Int64) +), +pt Nested +( + fcp Nullable(Int64) +), +vis Nested +( + st Nullable(String) +), +ua Nested +( + plt Nullable(String), + vnd Nullable(String) +), +dom Nested +( + res Nullable(Int64), + doms Nullable(Int64), + ln Nullable(Int64), + sz Nullable(Int64), + ck Nullable(Int64), + img Nullable(Int64), + iframe Nullable(Int64), + link Nested + ( + link_value Nullable(Int64), + css Nullable(Int64) + ), + script Nested + ( + script_value Nullable(Int64), + ext Nullable(Int64) + ) +), +mem Nested +( + lsln Nullable(Int64), + ssln Nullable(Int64), + lssz Nullable(Int64), + sssz Nullable(Int64) +), +scr Nested +( + xy Nullable(String), + bpp Nullable(String), + orn Nullable(String) +), +cpc Nested +( + cnc Nullable(Int64) +), +t_resp Nullable(Int64), +t_page Nullable(Int64), +t_done Nullable(Int64), +t_other Nullable(String), +nt_nav_st Nullable(Int64), +nt_dns_st Nullable(Int64), +nt_dns_end Nullable(Int64), +nt_con_st Nullable(Int64), +nt_con_end Nullable(Int64), +nt_req_st Nullable(Int64), +nt_res_st Nullable(Int64), +nt_res_end Nullable(Int64), +nt_domloading Nullable(Int64), +nt_domint Nullable(Int64), +nt_domcontloaded_st Nullable(Int64), +nt_domcontloaded_end Nullable(Int64), +nt_domcomp Nullable(Int64), +nt_load_st Nullable(Int64), +nt_load_end Nullable(Int64), +nt_unload_st Nullable(Int64) , +nt_unload_end Nullable(Int64), +nt_ssl_st Nullable(Int64), +nt_enc_size Nullable(Int64), +nt_dec_size Nullable(Int64), +nt_trn_size Nullable(Int64), +nt_protocol Nullable(String), +nt_red_cnt Nullable(Int64), +nt_nav_type Nullable(Int64), +restiming Nullable(String), +u Nullable(String), +r Nullable(String), +v Nullable(String), +sv Nullable(Int64), +sm Nullable(String), +sb Nullable(Int64), +pid Nullable(String), +n Nullable(Int64) +) +ENGINE = TinyLog; +" + +#${CLICKHOUSE_CURL} http://localhost:8123/?query=INSERT%20INTO%20form%5Fdata%20FORMAT%20Form --data-binary @- < $FORM_DATA + # Insert Form data from clickhouse-client -$CLICKHOUSE_CLIENT --query "SELECT * FROM file('$FORM_DATA',Form) FORMAT Vertical" +#$CLICKHOUSE_CLIENT --query "SELECT * FROM file('$FORM_DATA',Form) FORMAT Vertical" # Test teardown rm -r ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME} \ No newline at end of file From 2a20c2e7bbf03a3f0b746205ec77fab63ad51961 Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 20 Feb 2024 19:03:38 +0100 Subject: [PATCH 0027/1018] refactor to FormInputFormat --- .../Formats/Impl/FormInputFormat.cpp | 160 +++++++++++++----- src/Processors/Formats/Impl/FormInputFormat.h | 25 ++- .../queries/0_stateless/02984_form_format.sh | 12 +- 3 files changed, 140 insertions(+), 57 deletions(-) diff --git a/src/Processors/Formats/Impl/FormInputFormat.cpp b/src/Processors/Formats/Impl/FormInputFormat.cpp index 6408da52ebc..140d436eec2 100644 --- a/src/Processors/Formats/Impl/FormInputFormat.cpp +++ b/src/Processors/Formats/Impl/FormInputFormat.cpp @@ -5,15 +5,25 @@ #include "DataTypes/IDataType.h" #include "Formats/EscapingRuleUtils.h" #include "Formats/FormatSettings.h" -#include "Formats/SchemaInferenceUtils.h" -#include "IO/ReadBufferFromString.h" +#include +#include #include "Processors/Formats/IRowInputFormat.h" #include "base/find_symbols.h" #include #include namespace DB -{ +{ + +namespace ErrorCodes +{ + extern const int CANNOT_READ_ALL_DATA; + extern const int INCORRECT_DATA; + extern const int LOGICAL_ERROR; +} + +namespace +{ enum { @@ -21,24 +31,6 @@ enum NESTED_FIELD = size_t(-2) }; -/** - * Recursively check if column_name contains '.' in name - * and split into separate columns if it does - */ -void FormInputFormat::checkAndSplitIfNested(const StringRef column_name) -{ - while(true) - { - const auto split = Nested::splitName(column_name.toView()); - if (!split.second.empty()) - { - const StringRef table_name(column_name.data, split.first.size()); - name_map[table_name] = NESTED_FIELD; - const StringRef next_table_name(String(split.second).c_str(), split.second.size()); - checkAndSplitIfNested(next_table_name); - } - break; - } } FormInputFormat::FormInputFormat(ReadBuffer & in_, Block header_, Params params_, const FormatSettings & format_settings_) @@ -47,11 +39,15 @@ FormInputFormat::FormInputFormat(ReadBuffer & in_, Block header_, Params params_ const auto & header = getPort().getHeader(); name_map = header.getNamesToIndexesMap(); - /// not sure if this needs to be on a setting or not? - for (size_t i=0; i != header.columns(); ++i) + for (size_t i = 0; i != header.columns(); ++i) { const StringRef column_name = header.getByPosition(i).name; - checkAndSplitIfNested(column_name); + const auto split = Nested::splitName(column_name.toView()); + if (!split.second.empty()) + { + const StringRef table_name(column_name.data, split.first.size()); + name_map[table_name] = NESTED_FIELD; + } } } @@ -65,12 +61,6 @@ const String & FormInputFormat::columnName(size_t i) const return getPort().getHeader().getByPosition(i).name; } -/** Read the field name in the `Form` format. - * Return true if field name is followed by an equal sign, - * otherwise (field with no value) return false. - * The reference to the field name is written to `ref`. - * Temporary buffer `tmp` is used to copy the field name to it. - */ StringRef readName(ReadBuffer & buf, StringRef & ref, String & tmp) { tmp.clear(); @@ -87,7 +77,6 @@ StringRef readName(ReadBuffer & buf, StringRef & ref, String & tmp) continue; } - /// Column names (keys) occur before '=' if (*next_pos == '=') { ref = StringRef(buf.position(), next_pos - buf.position()); @@ -105,29 +94,41 @@ void FormInputFormat::readField(size_t index, MutableColumns & columns) throw Exception(ErrorCodes::INCORRECT_DATA, "Duplicate field found while parsing Form format: {}", columnName(index)); seen_columns[index] = read_columns[index] = true; + const auto & type = getPort().getHeader().getByPosition(index).type; const auto & serialization = serializations[index]; + String encoded_str, decoded_str; readStringUntilAmpersand(encoded_str,*in); Poco::URI::decode(encoded_str, decoded_str); - - /// skip '&' before next key value pair - if (!in->eof()) - ++in->position(); - - ReadBufferFromString buf(decoded_str); - serialization->deserializeTextRaw(*columns[index], buf, format_settings); - read_columns[index] = true; + ReadBufferFromString buf(decoded_str); + + if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) + read_columns[index] = SerializationNullable::deserializeNullAsDefaultOrNestedTextRaw(*columns[index], buf, format_settings, serialization); + else + serialization->deserializeTextRaw(*columns[index], buf, format_settings); } -String readFieldName(ReadBuffer & in) +String readFieldName(ReadBuffer & buf) { String field; - readStringUntilEquals(field, in); - assertChar('=', in); + readStringUntilEquals(field, buf); + assertChar('=', buf); return field; } +void FormInputFormat::skipUnknownFormField(StringRef name_ref) +{ + if (!format_settings.skip_unknown_fields) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unknown field found while parsing Form format: {}", name_ref.toString()); + + /// read name and value but do nothing with them + readFieldName(*in); + String value; + readStringUntilAmpersand(value,*in); +} + + inline size_t FormInputFormat::columnIndex(StringRef name) { const auto it = name_map.find(name); @@ -139,21 +140,86 @@ inline size_t FormInputFormat::columnIndex(StringRef name) return UNKNOWN_FIELD; } -bool FormInputFormat::readRow(MutableColumns & columns, RowReadExtension &) + +void FormInputFormat::readFormData(MutableColumns & columns) { + size_t index = 0; + while (index < columns.size()) + { + if (in->eof()) + break; + + StringRef name_ref = readFieldName(*in); + const size_t column_index = columnIndex(name_ref); + + if (ssize_t(column_index) < 0) + { + /// copy name_ref to temporary string as name_ref may + /// point directly to the input buffer + + current_column_name.assign(name_ref.data, name_ref.size); + name_ref = StringRef(current_column_name); + + if (column_index == UNKNOWN_FIELD) + skipUnknownFormField(name_ref); + else if (column_index == NESTED_FIELD) + readNestedFormData(name_ref.toString(), columns); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: illegal value of column_index"); + } + else + { + readField(column_index, columns); + } + ++index; + } +} + +void FormInputFormat::readNestedFormData(const String & name, MutableColumns & columns) +{ + current_column_name = name; + current_column_name.push_back('.'); + nested_prefix_length = current_column_name.size(); + readFormData(columns); + nested_prefix_length = 0; +} + +bool FormInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext) +{ + if (in->eof()) + return false; + size_t num_columns = columns.size(); read_columns.assign(num_columns, false); seen_columns.assign(num_columns, false); - for (size_t index = 0; index < num_columns; ++index) - { - readField(1, columns); - } + readFormData(columns); + + const auto & header = getPort().getHeader(); + /// Non-visited columns get filled with default values + for (size_t i = 0; i < num_columns; ++i) + if(!seen_columns[i]) + header.getByPosition(i).type->insertDefaultInto(*columns[i]); + + /// Return info about defaults set. + /// If defaults_for_omitted_fields is set to 0, then we leave already inserted defaults. + if (format_settings.defaults_for_omitted_fields) + ext.read_columns = read_columns; + else + ext.read_columns.assign(read_columns.size(), true); return true; } +void FormInputFormat::resetParser() +{ + IRowInputFormat::resetParser(); + nested_prefix_length = 0; + read_columns.clear(); + seen_columns.clear(); +} + FormSchemaReader::FormSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) : IRowWithNamesSchemaReader(in_, format_settings_,getDefaultDataTypeForEscapingRule(FormatSettings::EscapingRule::Escaped)) { diff --git a/src/Processors/Formats/Impl/FormInputFormat.h b/src/Processors/Formats/Impl/FormInputFormat.h index af13b6540d3..58fe43e7eef 100644 --- a/src/Processors/Formats/Impl/FormInputFormat.h +++ b/src/Processors/Formats/Impl/FormInputFormat.h @@ -14,25 +14,38 @@ class FormInputFormat final : public IRowInputFormat public: FormInputFormat(ReadBuffer & in_, Block header_, Params params_, const FormatSettings & format_settings_); String getName() const override { return "FormInputFormat"; } + void resetParser() override; private: void readPrefix() override; - bool readRow(MutableColumns & columns, RowReadExtension & extra) override; + bool readRow(MutableColumns & columns, RowReadExtension & ext) override; + void readFormData(MutableColumns & columns); + void readNestedFormData(const String & name, MutableColumns & columns); void readField(size_t index, MutableColumns & columns); - String readFieldName(ReadBuffer & in); + void skipUnknownFormField(StringRef name_ref); const String & columnName(size_t i) const; size_t columnIndex(StringRef name); /// recursively split names separated by '.' into own columns void checkAndSplitIfNested(StringRef column_name); - const FormatSettings format_settings; String name_buf; - std::vector read_columns; - std::vector seen_columns; + + /// holds common prefix of nested column names + String current_column_name; + + /// holds length of common prefix of nested column names + /// eg: given 'n.a', 'n.b' -> 'n.a' and 'n.b' are nested + /// column names and 'n.' is the common prefix. + size_t nested_prefix_length = 0; /// Hash table matches field name to position in the block Block::NameMap name_map; + +protected: + const FormatSettings format_settings; + std::vector read_columns; + std::vector seen_columns; }; class FormSchemaReader : public IRowWithNamesSchemaReader @@ -44,5 +57,7 @@ private: NamesAndTypesList readRowAndGetNamesAndDataTypesForForm(ReadBuffer & in, const FormatSettings & settings); }; +String readFieldName(ReadBuffer & buf); + } diff --git a/tests/queries/0_stateless/02984_form_format.sh b/tests/queries/0_stateless/02984_form_format.sh index 32e372dfd81..8ff86116fee 100755 --- a/tests/queries/0_stateless/02984_form_format.sh +++ b/tests/queries/0_stateless/02984_form_format.sh @@ -8,11 +8,12 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # Test setup # USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') USER_FILES_PATH="/home/shaun/Desktop/ClickHouse/programs/server/user_files" -FORM_DATA="${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/data.tmp" +FILE_NAME="data.tmp" +FORM_DATA="${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/${FILE_NAME}" mkdir -p ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ touch $FORM_DATA -echo -ne "c.e=ls7xfkpm&c.tti.m=raf&rt.start=navigation&rt.bmr=390%2C11%2C10&rt.tstart=1707076768666&rt.bstart=1707076769091&rt.blstart=1707076769056&rt.end=1707076769078&t_resp=296&t_page=116&t_done=412&t_other=boomerang%7C6%2Cboomr_fb%7C425%2Cboomr_ld%7C390%2Cboomr_lat%7C35&rt.tt=2685&rt.obo=0&pt.fcp=407&nt_nav_st=1707076768666&nt_dns_st=1707076768683&nt_dns_end=1707076768684&nt_con_st=1707076768684&nt_con_end=1707076768850&nt_req_st=1707076768850&nt_res_st=1707076768962&nt_res_end=1707076768962&nt_domloading=1707076769040&nt_domint=1707076769066&nt_domcontloaded_st=1707076769067&nt_domcontloaded_end=1707076769068&nt_domcomp=1707076769069&nt_load_st=1707076769069&nt_load_end=1707076769078&nt_unload_st=1707076769040&nt_unload_end=1707076769041&nt_ssl_st=1707076768788&nt_enc_size=3209&nt_dec_size=10093&nt_trn_size=3940&nt_protocol=h2&nt_red_cnt=0&nt_nav_type=1&restiming=%7B%22https%3A%2F%2Fwww.basicrum.com%2F%22%3A%7B%22publications%2F%22%3A%226%2C88%2C88%2C54%2C54%2C3e%2Ci%2Ci%2Ch*12h5%2Ckb%2C5b8%22%2C%22assets%2Fjs%2F%22%3A%7B%22just-the-docs.js%22%3A%223am%2Ce%2Ce*12pc%2C_%2C8oj*20%22%2C%22boomerang-1.737.60.cutting-edge.min.js%22%3A%222au%2Cb%2Ca*1pu3%2C_%2C1m19*21*42%22%2C%22vendor%2Flunr.min.js%22%3A%223am%2Cd%2C8*16t2%2C_%2Cfym*20%22%7D%7D%7D&u=https%3A%2F%2Fwww.basicrum.com%2Fpublications%2F&r=https%3A%2F%2Fwww.basicrum.com%2Fcost-analyses%2F&v=1.737.60&sv=14&sm=p&rt.si=dd0c542f-7adf-4310-830a-6c0a3d157c90-s8cjr1&rt.ss=1707075325294&rt.sl=4&vis.st=visible&ua.plt=Linux%20x86_64&ua.vnd=&pid=8fftz949&n=1&c.t.fps=07*4*65*j*61&c.t.busy=2*4*0034&c.tti.vr=408&c.tti=408&c.b=2&c.f=60&c.f.d=2511&c.f.m=1&c.f.s=ls7xfl1h&dom.res=5&dom.doms=1&mem.lsln=0&mem.ssln=0&mem.lssz=2&mem.sssz=2&scr.xy=1920x1200&scr.bpp=24%2F24&scr.orn=0%2Flandscape-primary&cpu.cnc=16&dom.ln=114&dom.sz=10438&dom.ck=157&dom.img=0&dom.script=6&dom.script.ext=3&dom.iframe=0&dom.link=4&dom.link.css=1&sb=1" > $FORM_DATA +echo -ne 'c.e=ls7xfkpm&c.tti.m=raf&rt.start=navigation&rt.bmr=390%2C11%2C10&rt.tstart=1707076768666&rt.bstart=1707076769091&rt.blstart=1707076769056&rt.end=1707076769078&t_resp=296&t_page=116&t_done=412&t_other=boomerang%7C6%2Cboomr_fb%7C425%2Cboomr_ld%7C390%2Cboomr_lat%7C35&rt.tt=2685&rt.obo=0&pt.fcp=407&nt_nav_st=1707076768666&nt_dns_st=1707076768683&nt_dns_end=1707076768684&nt_con_st=1707076768684&nt_con_end=1707076768850&nt_req_st=1707076768850&nt_res_st=1707076768962&nt_res_end=1707076768962&nt_domloading=1707076769040&nt_domint=1707076769066&nt_domcontloaded_st=1707076769067&nt_domcontloaded_end=1707076769068&nt_domcomp=1707076769069&nt_load_st=1707076769069&nt_load_end=1707076769078&nt_unload_st=1707076769040&nt_unload_end=1707076769041&nt_ssl_st=1707076768788&nt_enc_size=3209&nt_dec_size=10093&nt_trn_size=3940&nt_protocol=h2&nt_red_cnt=0&nt_nav_type=1&restiming=%7B%22https%3A%2F%2Fwww.basicrum.com%2F%22%3A%7B%22publications%2F%22%3A%226%2C88%2C88%2C54%2C54%2C3e%2Ci%2Ci%2Ch*12h5%2Ckb%2C5b8%22%2C%22assets%2Fjs%2F%22%3A%7B%22just-the-docs.js%22%3A%223am%2Ce%2Ce*12pc%2C_%2C8oj*20%22%2C%22boomerang-1.737.60.cutting-edge.min.js%22%3A%222au%2Cb%2Ca*1pu3%2C_%2C1m19*21*42%22%2C%22vendor%2Flunr.min.js%22%3A%223am%2Cd%2C8*16t2%2C_%2Cfym*20%22%7D%7D%7D&u=https%3A%2F%2Fwww.basicrum.com%2Fpublications%2F&r=https%3A%2F%2Fwww.basicrum.com%2Fcost-analyses%2F&v=1.737.60&sv=14&sm=p&rt.si=dd0c542f-7adf-4310-830a-6c0a3d157c90-s8cjr1&rt.ss=1707075325294&rt.sl=4&vis.st=visible&ua.plt=Linux%20x86_64&ua.vnd=&pid=8fftz949&n=1&c.t.fps=07*4*65*j*61&c.t.busy=2*4*0034&c.tti.vr=408&c.tti=408&c.b=2&c.f=60&c.f.d=2511&c.f.m=1&c.f.s=ls7xfl1h&dom.res=5&dom.doms=1&mem.lsln=0&mem.ssln=0&mem.lssz=2&mem.sssz=2&scr.xy=1920x1200&scr.bpp=24%2F24&scr.orn=0%2Flandscape-primary&cpu.cnc=16&dom.ln=114&dom.sz=10438&dom.ck=157&dom.img=0&dom.script=6&dom.script.ext=3&dom.iframe=0&dom.link=4&dom.link.css=1&sb=1' > $FORM_DATA $CLICKHOUSE_CLIENT -nm --query " DROP TABLE IF EXISTS form_data; @@ -146,10 +147,11 @@ n Nullable(Int64) ENGINE = TinyLog; " -#${CLICKHOUSE_CURL} http://localhost:8123/?query=INSERT%20INTO%20form%5Fdata%20FORMAT%20Form --data-binary @- < $FORM_DATA - # Insert Form data from clickhouse-client -#$CLICKHOUSE_CLIENT --query "SELECT * FROM file('$FORM_DATA',Form) FORMAT Vertical" +$CLICKHOUSE_CLIENT --query "INSERT INTO default.form_data FORMAT Form" < $FORM_DATA +$CLICKHOUSE_CLIENT --query "SELECT * FROM default.form_data FORMAT Vertical" + +#${CLICKHOUSE_CURL} http://localhost:8123/?query=INSERT%20INTO%20form%5Fdata%20FORMAT%20Form --data-binary @- < $FORM_DATA # Test teardown rm -r ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME} \ No newline at end of file From 4f893870c3b7907f65da980d11cf3f4564bbe0db Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 20 Feb 2024 21:27:39 +0100 Subject: [PATCH 0028/1018] remove nested logic in constructor --- .../Formats/Impl/FormInputFormat.cpp | 28 +++++++++++++------ 1 file changed, 19 insertions(+), 9 deletions(-) diff --git a/src/Processors/Formats/Impl/FormInputFormat.cpp b/src/Processors/Formats/Impl/FormInputFormat.cpp index 140d436eec2..4d1562b63b7 100644 --- a/src/Processors/Formats/Impl/FormInputFormat.cpp +++ b/src/Processors/Formats/Impl/FormInputFormat.cpp @@ -39,16 +39,21 @@ FormInputFormat::FormInputFormat(ReadBuffer & in_, Block header_, Params params_ const auto & header = getPort().getHeader(); name_map = header.getNamesToIndexesMap(); - for (size_t i = 0; i != header.columns(); ++i) + // we don't want this as the data should be flattened + if (format_settings_.import_nested_json) { - const StringRef column_name = header.getByPosition(i).name; - const auto split = Nested::splitName(column_name.toView()); - if (!split.second.empty()) + for (size_t i = 0; i != header.columns(); ++i) { - const StringRef table_name(column_name.data, split.first.size()); - name_map[table_name] = NESTED_FIELD; + const StringRef column_name = header.getByPosition(i).name; + const auto split = Nested::splitName(column_name.toView()); + if (!split.second.empty()) + { + const StringRef table_name(column_name.data, split.first.size()); + name_map[table_name] = NESTED_FIELD; + } } } + } void FormInputFormat::readPrefix() @@ -99,6 +104,7 @@ void FormInputFormat::readField(size_t index, MutableColumns & columns) String encoded_str, decoded_str; readStringUntilAmpersand(encoded_str,*in); + assertChar('&',*in); Poco::URI::decode(encoded_str, decoded_str); ReadBufferFromString buf(decoded_str); @@ -123,9 +129,13 @@ void FormInputFormat::skipUnknownFormField(StringRef name_ref) throw Exception(ErrorCodes::INCORRECT_DATA, "Unknown field found while parsing Form format: {}", name_ref.toString()); /// read name and value but do nothing with them - readFieldName(*in); - String value; - readStringUntilAmpersand(value,*in); + if(!in->eof()) + { + readFieldName(*in); + String value; + readStringUntilAmpersand(value,*in); + } + } From 87fb8d187be0b7ca6b5fdce6dd3ab6b63d8ed060 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 22 Feb 2024 13:57:38 +0100 Subject: [PATCH 0029/1018] Change back how receive_timeout is handled for INSERTs Right now the receive_timeout for INSERT works as a timeout for receiving data block, however this is not very convenient, since sometimes server may not send data for quite some time (i.e. due to in order aggregation), Ping packets is there for a reason (also Progress and ProfileEvents as well, though the purpose is different). Initially this special handling of receive_timeout had been added in 6a5ef9be832820b2482c257626f794a43f1f980a ("dbms: fixed error with hanging INSERTs [#METR-16514]"), but the behaviour was different, since that time the receivePacket() was outside loop, only poll() was there, and that was the workaround for poll() timeout (which does not triggers the socket timeout). But in fabd7193bd687ee4b10ca826303399ff35e3d3dd ("Code cleanups and improvements"), receivePacket() had been moved into the loop, and so this changed the behaviour of the timeout to current one. Though all of this will not help for INSERT queries anyway, since there are no Ping packets for them. Yet. Signed-off-by: Azat Khuzhin --- src/Server/TCPHandler.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 833f8ecc818..44b5a4adcca 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -795,6 +795,8 @@ bool TCPHandler::readDataNext() /// We accept and process data. read_ok = receivePacket(); + /// Reset the timeout on Ping packet (NOTE: there is no Ping for INSERT queries yet). + watch.restart(); break; } From 4d57aaebb1057d717e8df3830131ace76e5e2fe6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 22 Feb 2024 16:56:33 +0100 Subject: [PATCH 0030/1018] Make rabbitmq reject broken messages --- src/Storages/RabbitMQ/RabbitMQConsumer.cpp | 12 +++ src/Storages/RabbitMQ/RabbitMQConsumer.h | 1 + src/Storages/RabbitMQ/RabbitMQSource.cpp | 20 +++- .../integration/test_storage_rabbitmq/test.py | 95 +++++++++++++++++++ 4 files changed, 125 insertions(+), 3 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQConsumer.cpp b/src/Storages/RabbitMQ/RabbitMQConsumer.cpp index 28dc239ae37..ed16175b987 100644 --- a/src/Storages/RabbitMQ/RabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/RabbitMQConsumer.cpp @@ -106,6 +106,18 @@ bool RabbitMQConsumer::ackMessages(const CommitInfo & commit_info) if (commit_info.channel_id != channel_id) return false; + for (const auto & delivery_tag : commit_info.failed_delivery_tags) + { + if (consumer_channel->reject(delivery_tag)) + LOG_TRACE( + log, "Consumer rejected message with deliveryTag {} on channel {}", + delivery_tag, channel_id); + else + LOG_WARNING( + log, "Failed to reject message with deliveryTag {} on channel {}", + delivery_tag, channel_id); + } + /// Duplicate ack? if (commit_info.delivery_tag > last_commited_delivery_tag && consumer_channel->ack(commit_info.delivery_tag, AMQP::multiple)) diff --git a/src/Storages/RabbitMQ/RabbitMQConsumer.h b/src/Storages/RabbitMQ/RabbitMQConsumer.h index 9dad175dda3..470cb0a2458 100644 --- a/src/Storages/RabbitMQ/RabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/RabbitMQConsumer.h @@ -39,6 +39,7 @@ public: { UInt64 delivery_tag = 0; String channel_id; + std::vector failed_delivery_tags; }; struct MessageData diff --git a/src/Storages/RabbitMQ/RabbitMQSource.cpp b/src/Storages/RabbitMQ/RabbitMQSource.cpp index 72196e7dd3c..801aec1d0a9 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSource.cpp @@ -168,7 +168,9 @@ Chunk RabbitMQSource::generateImpl() StreamingFormatExecutor executor(non_virtual_header, input_format, on_error); - RabbitMQConsumer::CommitInfo current_commit_info; + /// Channel id will not change during read. + commit_info.channel_id = consumer->getChannelID(); + while (true) { exception_message.reset(); @@ -176,8 +178,12 @@ Chunk RabbitMQSource::generateImpl() if (consumer->hasPendingMessages()) { + /// A buffer containing a single RabbitMQ message. if (auto buf = consumer->consume()) + { new_rows = executor.execute(*buf); + chassert(new_rows == 1); + } } if (new_rows) @@ -185,6 +191,16 @@ Chunk RabbitMQSource::generateImpl() const auto exchange_name = storage.getExchange(); const auto & message = consumer->currentMessage(); + if (exception_message.has_value()) + { + commit_info.failed_delivery_tags.push_back(message.delivery_tag); + } + else + { + chassert(commit_info.delivery_tag < message.delivery_tag); + commit_info.delivery_tag = message.delivery_tag; + } + for (size_t i = 0; i < new_rows; ++i) { virtual_columns[0]->insert(exchange_name); @@ -209,7 +225,6 @@ Chunk RabbitMQSource::generateImpl() } total_rows += new_rows; - current_commit_info = {message.delivery_tag, message.channel_id}; } else if (total_rows == 0) { @@ -251,7 +266,6 @@ Chunk RabbitMQSource::generateImpl() for (auto & column : virtual_columns) result_columns.push_back(std::move(column)); - commit_info = current_commit_info; return Chunk(std::move(result_columns), total_rows); } diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 280ce230921..e41a7091551 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -3648,3 +3648,98 @@ def test_rabbitmq_nack_failed_insert(rabbitmq_cluster): """ ) connection.close() + + +def test_rabbitmq_reject_broken_messages(rabbitmq_cluster): + credentials = pika.PlainCredentials("root", "clickhouse") + parameters = pika.ConnectionParameters( + rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, "/", credentials + ) + connection = pika.BlockingConnection(parameters) + channel = connection.channel() + + deadletter_exchange = "deadletter_exchange_handle_error_mode_stream" + deadletter_queue = "deadletter_queue_handle_error_mode_stream" + channel.exchange_declare(exchange=deadletter_exchange) + + result = channel.queue_declare(queue=deadletter_queue) + channel.queue_bind( + exchange=deadletter_exchange, routing_key="", queue=deadletter_queue + ) + + instance.query( + f""" + DROP TABLE IF EXISTS test.rabbitmq; + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.data; + DROP TABLE IF EXISTS test.errors; + DROP TABLE IF EXISTS test.errors_view; + + CREATE TABLE test.rabbit (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = '{rabbitmq_cluster.rabbitmq_host}:5672', + rabbitmq_exchange_name = 'select', + rabbitmq_commit_on_select = 1, + rabbitmq_format = 'JSONEachRow', + rabbitmq_row_delimiter = '\\n', + rabbitmq_handle_error_mode = 'stream', + rabbitmq_queue_settings_list='x-dead-letter-exchange={deadletter_exchange}'; + + + CREATE TABLE test.errors (error Nullable(String), broken_message Nullable(String)) + ENGINE = MergeTree() + ORDER BY tuple(); + + CREATE MATERIALIZED VIEW test.errors_view TO test.errors AS + SELECT _error as error, _raw_message as broken_message FROM test.rabbit where not isNull(_error); + + CREATE TABLE test.data (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + + CREATE MATERIALIZED VIEW test.view TO test.data AS + SELECT key, value FROM test.rabbit; + """ + ) + + messages = [] + num_rows = 50 + for i in range(num_rows): + if i % 2 == 0: + messages.append(json.dumps({"key": i, "value": i})) + else: + messages.append("Broken message " + str(i)) + + for message in messages: + channel.basic_publish(exchange="select", routing_key="", body=message) + + time.sleep(1) + + attempt = 0 + rows = 0 + while attempt < 500: + rows = int(instance.query("SELECT count() FROM test.data")) + if rows == num_rows: + break + attempt += 1 + + assert rows == num_rows + + dead_letters = [] + + def on_dead_letter(channel, method, properties, body): + dead_letters.append(body) + if len(dead_letters) == num_rows / 2: + channel.stop_consuming() + + channel.basic_consume(deadletter_queue, on_dead_letter) + channel.start_consuming() + + assert len(dead_letters) == num_rows / 2 + + i = 1 + for letter in dead_letters: + assert f"Broken message {i}" in str(letter) + i += 2 + + connection.close() From a1465ab2bf2f53ca043417a52a66bc3b4481ab77 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 19 Feb 2024 19:33:06 +0100 Subject: [PATCH 0031/1018] Enable all access control improvements by default (even without config.xml) Signed-off-by: Azat Khuzhin --- src/Access/AccessControl.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index d02af01126a..26a32336b2d 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -280,11 +280,11 @@ void AccessControl::setUpFromMainConfig(const Poco::Util::AbstractConfiguration /// Optional improvements in access control system. /// The default values are false because we need to be compatible with earlier access configurations - setEnabledUsersWithoutRowPoliciesCanReadRows(config_.getBool("access_control_improvements.users_without_row_policies_can_read_rows", false)); - setOnClusterQueriesRequireClusterGrant(config_.getBool("access_control_improvements.on_cluster_queries_require_cluster_grant", false)); - setSelectFromSystemDatabaseRequiresGrant(config_.getBool("access_control_improvements.select_from_system_db_requires_grant", false)); - setSelectFromInformationSchemaRequiresGrant(config_.getBool("access_control_improvements.select_from_information_schema_requires_grant", false)); - setSettingsConstraintsReplacePrevious(config_.getBool("access_control_improvements.settings_constraints_replace_previous", false)); + setEnabledUsersWithoutRowPoliciesCanReadRows(config_.getBool("access_control_improvements.users_without_row_policies_can_read_rows", true)); + setOnClusterQueriesRequireClusterGrant(config_.getBool("access_control_improvements.on_cluster_queries_require_cluster_grant", true)); + setSelectFromSystemDatabaseRequiresGrant(config_.getBool("access_control_improvements.select_from_system_db_requires_grant", true)); + setSelectFromInformationSchemaRequiresGrant(config_.getBool("access_control_improvements.select_from_information_schema_requires_grant", true)); + setSettingsConstraintsReplacePrevious(config_.getBool("access_control_improvements.settings_constraints_replace_previous", true)); addStoragesFromMainConfig(config_, config_path_, get_zookeeper_function_); From ef4a9e8ff99848a5c9e9f7336af24eb5ec259c18 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 22 Feb 2024 18:05:02 +0100 Subject: [PATCH 0032/1018] Disable access controls explicitly in tests Signed-off-by: Azat Khuzhin --- .../config.d/disable_access_control_improvements.xml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_disabled_access_control_improvements/configs/config.d/disable_access_control_improvements.xml b/tests/integration/test_disabled_access_control_improvements/configs/config.d/disable_access_control_improvements.xml index a335c7f8a1f..cd2025b8a86 100644 --- a/tests/integration/test_disabled_access_control_improvements/configs/config.d/disable_access_control_improvements.xml +++ b/tests/integration/test_disabled_access_control_improvements/configs/config.d/disable_access_control_improvements.xml @@ -1,8 +1,8 @@ - - - - + false + false + false + false From de16def0685a5db6c6c9afe32bd1530eb258537b Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 22 Feb 2024 18:48:06 +0100 Subject: [PATCH 0033/1018] Update test.py --- tests/integration/test_storage_rabbitmq/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index e41a7091551..7c755853653 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -3722,6 +3722,7 @@ def test_rabbitmq_reject_broken_messages(rabbitmq_cluster): if rows == num_rows: break attempt += 1 + time.sleep(1) assert rows == num_rows From 6e35190cb152cc60c6c1ba650b49395d98a44dee Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 22 Feb 2024 20:30:55 +0100 Subject: [PATCH 0034/1018] Update RabbitMQSource.cpp --- src/Storages/RabbitMQ/RabbitMQSource.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/RabbitMQ/RabbitMQSource.cpp b/src/Storages/RabbitMQ/RabbitMQSource.cpp index 801aec1d0a9..8681169df11 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSource.cpp @@ -182,7 +182,6 @@ Chunk RabbitMQSource::generateImpl() if (auto buf = consumer->consume()) { new_rows = executor.execute(*buf); - chassert(new_rows == 1); } } From ff5c444bb978c4fa1e93b536418b23c216502017 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 26 Feb 2024 12:28:36 +0800 Subject: [PATCH 0035/1018] Fix assertion --- src/Storages/RabbitMQ/RabbitMQSource.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/RabbitMQ/RabbitMQSource.cpp b/src/Storages/RabbitMQ/RabbitMQSource.cpp index 8681169df11..75c02d5fa23 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSource.cpp @@ -196,7 +196,7 @@ Chunk RabbitMQSource::generateImpl() } else { - chassert(commit_info.delivery_tag < message.delivery_tag); + chassert(!commit_info.delivery_tag || commit_info.delivery_tag < message.delivery_tag); commit_info.delivery_tag = message.delivery_tag; } From b902c60043ef3a630a11be1a8f6c76ba1cc217a6 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 27 Feb 2024 18:38:37 +0800 Subject: [PATCH 0036/1018] allow arbitrary map type --- src/DataTypes/DataTypeMap.cpp | 16 +--------------- 1 file changed, 1 insertion(+), 15 deletions(-) diff --git a/src/DataTypes/DataTypeMap.cpp b/src/DataTypes/DataTypeMap.cpp index e6d59954288..d9f70e1659d 100644 --- a/src/DataTypes/DataTypeMap.cpp +++ b/src/DataTypes/DataTypeMap.cpp @@ -118,21 +118,7 @@ bool DataTypeMap::equals(const IDataType & rhs) const bool DataTypeMap::checkKeyType(DataTypePtr key_type) { - if (key_type->getTypeId() == TypeIndex::LowCardinality) - { - const auto & low_cardinality_data_type = assert_cast(*key_type); - if (!isStringOrFixedString(*(low_cardinality_data_type.getDictionaryType()))) - return false; - } - else if ( - !key_type->isValueRepresentedByNumber() && !isStringOrFixedString(*key_type) && !WhichDataType(key_type).isNothing() - && !WhichDataType(key_type).isIPv6() && !WhichDataType(key_type).isUUID() && !isMap(key_type) && !isArray(key_type) - && !isTuple(key_type)) - { - return false; - } - - return true; + return !isNullableOrLowCardinalityNullable(key_type); } DataTypePtr DataTypeMap::getNestedTypeWithUnnamedTuple() const From 48a2934b15d1d518f4b29a58fb24205e2511065e Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 27 Feb 2024 19:47:32 +0800 Subject: [PATCH 0037/1018] adjust docs --- docs/en/sql-reference/data-types/map.md | 2 +- docs/en/sql-reference/functions/tuple-map-functions.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/data-types/map.md b/docs/en/sql-reference/data-types/map.md index e0c8b98f9f8..131f8058288 100644 --- a/docs/en/sql-reference/data-types/map.md +++ b/docs/en/sql-reference/data-types/map.md @@ -10,7 +10,7 @@ sidebar_label: Map(key, value) **Parameters** -- `key` — The key part of the pair. [String](../../sql-reference/data-types/string.md), [Integer](../../sql-reference/data-types/int-uint.md), [LowCardinality](../../sql-reference/data-types/lowcardinality.md), [FixedString](../../sql-reference/data-types/fixedstring.md), [UUID](../../sql-reference/data-types/uuid.md), [Date](../../sql-reference/data-types/date.md), [DateTime](../../sql-reference/data-types/datetime.md), [Date32](../../sql-reference/data-types/date32.md), [Enum](../../sql-reference/data-types/enum.md). +- `key` — The key part of the pair. Arbitrary type, except [Nullable](../../sql-reference/data-types/nullable.md) and [LowCardinality](../../sql-reference/data-types/lowcardinality.md) nested with [Nullable](../../sql-reference/data-types/nullable.md) types. - `value` — The value part of the pair. Arbitrary type, including [Map](../../sql-reference/data-types/map.md) and [Array](../../sql-reference/data-types/array.md). To get the value from an `a Map('key', 'value')` column, use `a['key']` syntax. This lookup works now with a linear complexity. diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index 8283de95994..377283bc006 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -16,7 +16,7 @@ map(key1, value1[, key2, value2, ...]) **Arguments** -- `key` — The key part of the pair. [String](../../sql-reference/data-types/string.md), [Integer](../../sql-reference/data-types/int-uint.md), [LowCardinality](../../sql-reference/data-types/lowcardinality.md), [FixedString](../../sql-reference/data-types/fixedstring.md), [UUID](../../sql-reference/data-types/uuid.md), [Date](../../sql-reference/data-types/date.md), [DateTime](../../sql-reference/data-types/datetime.md), [Date32](../../sql-reference/data-types/date32.md), [Enum](../../sql-reference/data-types/enum.md). +- `key` — The key part of the pair. Arbitrary type, except [Nullable](../../sql-reference/data-types/nullable.md) and [LowCardinality](../../sql-reference/data-types/lowcardinality.md) nested with [Nullable](../../sql-reference/data-types/nullable.md). - `value` — The value part of the pair. Arbitrary type, including [Map](../../sql-reference/data-types/map.md) and [Array](../../sql-reference/data-types/array.md). **Returned value** From 4ac32668bd7053d3c094020e75bd1ae5e7187fe5 Mon Sep 17 00:00:00 2001 From: Maksim Alekseev Date: Tue, 27 Feb 2024 21:48:24 +0500 Subject: [PATCH 0038/1018] Added possibility to compress in CROSS JOIN --- src/Core/Block.cpp | 17 +++++++++++++++++ src/Core/Block.h | 4 ++++ src/Core/Settings.h | 1 + src/Interpreters/HashJoin.cpp | 11 ++++++++--- src/Interpreters/TableJoin.cpp | 1 + src/Interpreters/TableJoin.h | 3 +++ 6 files changed, 34 insertions(+), 3 deletions(-) diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index a7d5b0a869f..ae769855c6b 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -7,6 +7,7 @@ #include #include +#include "Columns/IColumn.h" #include #include @@ -605,6 +606,22 @@ Block Block::shrinkToFit() const return cloneWithColumns(new_columns); } +Block Block::compress() const +{ + Columns new_columns(data.size(), nullptr); + for (size_t i = 0; i < data.size(); ++i) + new_columns[i] = data[i].column->compress(); + return cloneWithColumns(new_columns); +} + +Block Block::decompress() const +{ + Columns new_columns(data.size(), nullptr); + for (size_t i = 0; i < data.size(); ++i) + new_columns[i] = data[i].column->decompress(); + return cloneWithColumns(new_columns); +} + const ColumnsWithTypeAndName & Block::getColumnsWithTypeAndName() const { diff --git a/src/Core/Block.h b/src/Core/Block.h index 1a4f8c2e446..b990b3b8775 100644 --- a/src/Core/Block.h +++ b/src/Core/Block.h @@ -152,6 +152,10 @@ public: /** See IColumn::shrinkToFit() */ Block shrinkToFit() const; + Block compress() const; + + Block decompress() const; + void clear(); void swap(Block & other) noexcept; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e0b3ca39899..9f47507ae1a 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -432,6 +432,7 @@ class IColumn; M(OverflowMode, join_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ M(Bool, join_any_take_last_row, false, "When disabled (default) ANY JOIN will take the first found row for a key. When enabled, it will take the last row seen if there are multiple rows for the same key.", IMPORTANT) \ M(JoinAlgorithm, join_algorithm, JoinAlgorithm::DEFAULT, "Specify join algorithm.", 0) \ + M(Bool, cross_join_compress_blocks, false, "When enabled will compress blocks in CROSS JOIN", 0) \ M(UInt64, default_max_bytes_in_join, 1000000000, "Maximum size of right-side table if limit is required but max_bytes_in_join is not set.", 0) \ M(UInt64, partial_merge_join_left_table_buffer_bytes, 0, "If not 0 group left table blocks in bigger ones for left-side table in partial merge join. It uses up to 2x of specified memory per joining thread.", 0) \ M(UInt64, partial_merge_join_rows_in_right_blocks, 65536, "Split right-hand joining data in blocks of specified size. It's a portion of data indexed by min-max values and possibly unloaded on disk.", 0) \ diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 33dc178ca00..f5484049c33 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -824,9 +824,12 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) if (storage_join_lock) throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "addBlockToJoin called when HashJoin locked to prevent updates"); - data->blocks_allocated_size += block_to_save.allocatedBytes(); - assertBlocksHaveEqualStructure(data->sample_block, block_to_save, "joined block"); + + if (kind == JoinKind::Cross && table_join->crossJoinCompressBlocks()) + block_to_save = block_to_save.compress(); + + data->blocks_allocated_size += block_to_save.allocatedBytes(); data->blocks.emplace_back(std::move(block_to_save)); Block * stored_block = &data->blocks.back(); @@ -1804,12 +1807,14 @@ void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) for (size_t left_row = start_left_row; left_row < rows_left; ++left_row) { size_t block_number = 0; - for (const Block & block_right : data->blocks) + for (const Block & compressed_block_right : data->blocks) { ++block_number; if (block_number < start_right_block) continue; + auto block_right = compressed_block_right.decompress(); + size_t rows_right = block_right.rows(); rows_added += rows_right; diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index e9fa224df11..8b2078c2cbe 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -107,6 +107,7 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_) : size_limits(SizeLimits{settings.max_rows_in_join, settings.max_bytes_in_join, settings.join_overflow_mode}) , default_max_bytes(settings.default_max_bytes_in_join) , join_use_nulls(settings.join_use_nulls) + , cross_join_compress_blocks(settings.cross_join_compress_blocks) , max_joined_block_rows(settings.max_joined_block_size_rows) , join_algorithm(settings.join_algorithm) , partial_merge_join_rows_in_right_blocks(settings.partial_merge_join_rows_in_right_blocks) diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 75e2342d1e9..1acba19f4f0 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -139,6 +139,7 @@ private: SizeLimits size_limits; const size_t default_max_bytes = 0; const bool join_use_nulls = false; + const bool cross_join_compress_blocks = false; const size_t max_joined_block_rows = 0; std::vector join_algorithm; const size_t partial_merge_join_rows_in_right_blocks = 0; @@ -269,6 +270,8 @@ public: bool joinUseNulls() const { return join_use_nulls; } + bool crossJoinCompressBlocks() const { return cross_join_compress_blocks; } + bool forceNullableRight() const { return join_use_nulls && isLeftOrFull(kind()); From ee772aaf0f7a05fafb3fa286419148211362b0d6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 27 Feb 2024 23:17:11 +0100 Subject: [PATCH 0039/1018] Use clang-18 in CI --- .github/workflows/nightly.yml | 4 +- docker/packager/README.md | 8 ++-- docker/packager/packager | 24 +++++------ docker/test/fuzzer/run-fuzzer.sh | 2 +- docker/test/keeper-jepsen/run.sh | 2 +- docker/test/server-jepsen/run.sh | 2 +- docker/test/sqltest/run.sh | 2 +- docker/test/util/Dockerfile | 2 +- docs/en/development/build-cross-osx.md | 6 +-- docs/en/development/build-cross-riscv.md | 2 +- docs/en/development/build.md | 6 +-- docs/en/development/continuous-integration.md | 4 +- docs/en/development/developer-instruction.md | 2 +- tests/ci/ci_config.py | 42 +++++++++---------- 14 files changed, 54 insertions(+), 54 deletions(-) diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index 93ac2be19b4..4f7915acd28 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -51,8 +51,8 @@ jobs: SONAR_SCANNER_VERSION: 4.8.0.2856 SONAR_SERVER_URL: "https://sonarcloud.io" BUILD_WRAPPER_OUT_DIR: build_wrapper_output_directory # Directory where build-wrapper output will be placed - CC: clang-17 - CXX: clang++-17 + CC: clang-18 + CXX: clang++-18 steps: - name: Check out repository code uses: ClickHouse/checkout@v1 diff --git a/docker/packager/README.md b/docker/packager/README.md index e0b7f38ea58..efc79f2e131 100644 --- a/docker/packager/README.md +++ b/docker/packager/README.md @@ -3,10 +3,10 @@ compilers and build settings. Correctly configured Docker daemon is single depen Usage: -Build deb package with `clang-17` in `debug` mode: +Build deb package with `clang-18` in `debug` mode: ``` $ mkdir deb/test_output -$ ./packager --output-dir deb/test_output/ --package-type deb --compiler=clang-17 --debug-build +$ ./packager --output-dir deb/test_output/ --package-type deb --compiler=clang-18 --debug-build $ ls -l deb/test_output -rw-r--r-- 1 root root 3730 clickhouse-client_22.2.2+debug_all.deb -rw-r--r-- 1 root root 84221888 clickhouse-common-static_22.2.2+debug_amd64.deb @@ -17,11 +17,11 @@ $ ls -l deb/test_output ``` -Build ClickHouse binary with `clang-17` and `address` sanitizer in `relwithdebuginfo` +Build ClickHouse binary with `clang-18` and `address` sanitizer in `relwithdebuginfo` mode: ``` $ mkdir $HOME/some_clickhouse -$ ./packager --output-dir=$HOME/some_clickhouse --package-type binary --compiler=clang-17 --sanitizer=address +$ ./packager --output-dir=$HOME/some_clickhouse --package-type binary --compiler=clang-18 --sanitizer=address $ ls -l $HOME/some_clickhouse -rwxr-xr-x 1 root root 787061952 clickhouse lrwxrwxrwx 1 root root 10 clickhouse-benchmark -> clickhouse diff --git a/docker/packager/packager b/docker/packager/packager index ca0ae8358f3..43391af8660 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -403,19 +403,19 @@ def parse_args() -> argparse.Namespace: parser.add_argument( "--compiler", choices=( - "clang-17", - "clang-17-darwin", - "clang-17-darwin-aarch64", - "clang-17-aarch64", - "clang-17-aarch64-v80compat", - "clang-17-ppc64le", - "clang-17-riscv64", - "clang-17-s390x", - "clang-17-amd64-compat", - "clang-17-amd64-musl", - "clang-17-freebsd", + "clang-18", + "clang-18-darwin", + "clang-18-darwin-aarch64", + "clang-18-aarch64", + "clang-18-aarch64-v80compat", + "clang-18-ppc64le", + "clang-18-riscv64", + "clang-18-s390x", + "clang-18-amd64-compat", + "clang-18-amd64-musl", + "clang-18-freebsd", ), - default="clang-17", + default="clang-18", help="a compiler to use", ) parser.add_argument( diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index b4376fe2409..9e950668b00 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -17,7 +17,7 @@ stage=${stage:-} script_dir="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" echo "$script_dir" repo_dir=ch -BINARY_TO_DOWNLOAD=${BINARY_TO_DOWNLOAD:="clang-17_debug_none_unsplitted_disable_False_binary"} +BINARY_TO_DOWNLOAD=${BINARY_TO_DOWNLOAD:="clang-18_debug_none_unsplitted_disable_False_binary"} BINARY_URL_TO_DOWNLOAD=${BINARY_URL_TO_DOWNLOAD:="https://clickhouse-builds.s3.amazonaws.com/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/$BINARY_TO_DOWNLOAD/clickhouse"} function git_clone_with_retry diff --git a/docker/test/keeper-jepsen/run.sh b/docker/test/keeper-jepsen/run.sh index 576a0f0ef8e..444f3cd0de7 100644 --- a/docker/test/keeper-jepsen/run.sh +++ b/docker/test/keeper-jepsen/run.sh @@ -2,7 +2,7 @@ set -euo pipefail -CLICKHOUSE_PACKAGE=${CLICKHOUSE_PACKAGE:="https://clickhouse-builds.s3.amazonaws.com/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/clang-17_relwithdebuginfo_none_unsplitted_disable_False_binary/clickhouse"} +CLICKHOUSE_PACKAGE=${CLICKHOUSE_PACKAGE:="https://clickhouse-builds.s3.amazonaws.com/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/clang-18_relwithdebuginfo_none_unsplitted_disable_False_binary/clickhouse"} CLICKHOUSE_REPO_PATH=${CLICKHOUSE_REPO_PATH:=""} diff --git a/docker/test/server-jepsen/run.sh b/docker/test/server-jepsen/run.sh index 81e442e65b6..6ea9c03b954 100644 --- a/docker/test/server-jepsen/run.sh +++ b/docker/test/server-jepsen/run.sh @@ -2,7 +2,7 @@ set -euo pipefail -CLICKHOUSE_PACKAGE=${CLICKHOUSE_PACKAGE:="https://clickhouse-builds.s3.amazonaws.com/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/clang-17_relwithdebuginfo_none_unsplitted_disable_False_binary/clickhouse"} +CLICKHOUSE_PACKAGE=${CLICKHOUSE_PACKAGE:="https://clickhouse-builds.s3.amazonaws.com/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/clang-18_relwithdebuginfo_none_unsplitted_disable_False_binary/clickhouse"} CLICKHOUSE_REPO_PATH=${CLICKHOUSE_REPO_PATH:=""} diff --git a/docker/test/sqltest/run.sh b/docker/test/sqltest/run.sh index 1d939805c7b..7edc1341d7d 100755 --- a/docker/test/sqltest/run.sh +++ b/docker/test/sqltest/run.sh @@ -6,7 +6,7 @@ set -e set -u set -o pipefail -BINARY_TO_DOWNLOAD=${BINARY_TO_DOWNLOAD:="clang-17_debug_none_unsplitted_disable_False_binary"} +BINARY_TO_DOWNLOAD=${BINARY_TO_DOWNLOAD:="clang-18_debug_none_unsplitted_disable_False_binary"} BINARY_URL_TO_DOWNLOAD=${BINARY_URL_TO_DOWNLOAD:="https://clickhouse-builds.s3.amazonaws.com/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/$BINARY_TO_DOWNLOAD/clickhouse"} function wget_with_retry diff --git a/docker/test/util/Dockerfile b/docker/test/util/Dockerfile index 396d5801be9..a5057f024b8 100644 --- a/docker/test/util/Dockerfile +++ b/docker/test/util/Dockerfile @@ -5,7 +5,7 @@ FROM ubuntu:22.04 ARG apt_archive="http://archive.ubuntu.com" RUN sed -i "s|http://archive.ubuntu.com|$apt_archive|g" /etc/apt/sources.list -ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=17 +ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=18 RUN apt-get update \ && apt-get install \ diff --git a/docs/en/development/build-cross-osx.md b/docs/en/development/build-cross-osx.md index eddf24448c1..66c6e2c6912 100644 --- a/docs/en/development/build-cross-osx.md +++ b/docs/en/development/build-cross-osx.md @@ -13,14 +13,14 @@ The cross-build for macOS is based on the [Build instructions](../development/bu The following sections provide a walk-through for building ClickHouse for `x86_64` macOS. If you’re targeting ARM architecture, simply substitute all occurrences of `x86_64` with `aarch64`. For example, replace `x86_64-apple-darwin` with `aarch64-apple-darwin` throughout the steps. -## Install Clang-17 +## Install clang-18 Follow the instructions from https://apt.llvm.org/ for your Ubuntu or Debian setup. For example the commands for Bionic are like: ``` bash sudo echo "deb [trusted=yes] http://apt.llvm.org/bionic/ llvm-toolchain-bionic-17 main" >> /etc/apt/sources.list -sudo apt-get install clang-17 +sudo apt-get install clang-18 ``` ## Install Cross-Compilation Toolset {#install-cross-compilation-toolset} @@ -59,7 +59,7 @@ curl -L 'https://github.com/phracker/MacOSX-SDKs/releases/download/11.3/MacOSX11 cd ClickHouse mkdir build-darwin cd build-darwin -CC=clang-17 CXX=clang++-17 cmake -DCMAKE_AR:FILEPATH=${CCTOOLS}/bin/x86_64-apple-darwin-ar -DCMAKE_INSTALL_NAME_TOOL=${CCTOOLS}/bin/x86_64-apple-darwin-install_name_tool -DCMAKE_RANLIB:FILEPATH=${CCTOOLS}/bin/x86_64-apple-darwin-ranlib -DLINKER_NAME=${CCTOOLS}/bin/x86_64-apple-darwin-ld -DCMAKE_TOOLCHAIN_FILE=cmake/darwin/toolchain-x86_64.cmake .. +CC=clang-18 CXX=clang++-18 cmake -DCMAKE_AR:FILEPATH=${CCTOOLS}/bin/x86_64-apple-darwin-ar -DCMAKE_INSTALL_NAME_TOOL=${CCTOOLS}/bin/x86_64-apple-darwin-install_name_tool -DCMAKE_RANLIB:FILEPATH=${CCTOOLS}/bin/x86_64-apple-darwin-ranlib -DLINKER_NAME=${CCTOOLS}/bin/x86_64-apple-darwin-ld -DCMAKE_TOOLCHAIN_FILE=cmake/darwin/toolchain-x86_64.cmake .. ninja ``` diff --git a/docs/en/development/build-cross-riscv.md b/docs/en/development/build-cross-riscv.md index 9ee5346f258..759d97823e2 100644 --- a/docs/en/development/build-cross-riscv.md +++ b/docs/en/development/build-cross-riscv.md @@ -23,7 +23,7 @@ sudo bash -c "$(wget -O - https://apt.llvm.org/llvm.sh)" ``` bash cd ClickHouse mkdir build-riscv64 -CC=clang-17 CXX=clang++-17 cmake . -Bbuild-riscv64 -G Ninja -DCMAKE_TOOLCHAIN_FILE=cmake/linux/toolchain-riscv64.cmake -DGLIBC_COMPATIBILITY=OFF -DENABLE_LDAP=OFF -DOPENSSL_NO_ASM=ON -DENABLE_JEMALLOC=ON -DENABLE_PARQUET=OFF -DENABLE_GRPC=OFF -DENABLE_HDFS=OFF -DENABLE_MYSQL=OFF +CC=clang-18 CXX=clang++-18 cmake . -Bbuild-riscv64 -G Ninja -DCMAKE_TOOLCHAIN_FILE=cmake/linux/toolchain-riscv64.cmake -DGLIBC_COMPATIBILITY=OFF -DENABLE_LDAP=OFF -DOPENSSL_NO_ASM=ON -DENABLE_JEMALLOC=ON -DENABLE_PARQUET=OFF -DENABLE_GRPC=OFF -DENABLE_HDFS=OFF -DENABLE_MYSQL=OFF ninja -C build-riscv64 ``` diff --git a/docs/en/development/build.md b/docs/en/development/build.md index b474c445604..e4d0fb146ef 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -67,8 +67,8 @@ to see what version you have installed before setting this environment variable. ::: ``` bash -export CC=clang-17 -export CXX=clang++-17 +export CC=clang-18 +export CXX=clang++-18 ``` ### Checkout ClickHouse Sources {#checkout-clickhouse-sources} @@ -105,7 +105,7 @@ The build requires the following components: - Git (used to checkout the sources, not needed for the build) - CMake 3.20 or newer -- Compiler: clang-17 or newer +- Compiler: clang-18 or newer - Linker: lld-17 or newer - Ninja - Yasm diff --git a/docs/en/development/continuous-integration.md b/docs/en/development/continuous-integration.md index 46a30f56f11..91253ca5e44 100644 --- a/docs/en/development/continuous-integration.md +++ b/docs/en/development/continuous-integration.md @@ -153,7 +153,7 @@ Builds ClickHouse in various configurations for use in further steps. You have t ### Report Details -- **Compiler**: `clang-17`, optionally with the name of a target platform +- **Compiler**: `clang-18`, optionally with the name of a target platform - **Build type**: `Debug` or `RelWithDebInfo` (cmake). - **Sanitizer**: `none` (without sanitizers), `address` (ASan), `memory` (MSan), `undefined` (UBSan), or `thread` (TSan). - **Status**: `success` or `fail` @@ -177,7 +177,7 @@ Performs static analysis and code style checks using `clang-tidy`. The report is There is a convenience `packager` script that runs the clang-tidy build in docker ```sh mkdir build_tidy -./docker/packager/packager --output-dir=./build_tidy --package-type=binary --compiler=clang-17 --debug-build --clang-tidy +./docker/packager/packager --output-dir=./build_tidy --package-type=binary --compiler=clang-18 --debug-build --clang-tidy ``` diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index e08096d8042..44a5af5911a 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -115,7 +115,7 @@ While inside the `build` directory, configure your build by running CMake. Befor export CC=clang CXX=clang++ cmake .. -If you installed clang using the automatic installation script above, also specify the version of clang installed in the first command, e.g. `export CC=clang-17 CXX=clang++-17`. The clang version will be in the script output. +If you installed clang using the automatic installation script above, also specify the version of clang installed in the first command, e.g. `export CC=clang-18 CXX=clang++-18`. The clang version will be in the script output. The `CC` variable specifies the compiler for C (short for C Compiler), and `CXX` variable instructs which C++ compiler is to be used for building. diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 80994f71280..4757341ce7c 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -670,63 +670,63 @@ CI_CONFIG = CIConfig( build_config={ Build.PACKAGE_RELEASE: BuildConfig( name=Build.PACKAGE_RELEASE, - compiler="clang-17", + compiler="clang-18", package_type="deb", static_binary_name="amd64", additional_pkgs=True, ), Build.PACKAGE_AARCH64: BuildConfig( name=Build.PACKAGE_AARCH64, - compiler="clang-17-aarch64", + compiler="clang-18-aarch64", package_type="deb", static_binary_name="aarch64", additional_pkgs=True, ), Build.PACKAGE_ASAN: BuildConfig( name=Build.PACKAGE_ASAN, - compiler="clang-17", + compiler="clang-18", sanitizer="address", package_type="deb", ), Build.PACKAGE_UBSAN: BuildConfig( name=Build.PACKAGE_UBSAN, - compiler="clang-17", + compiler="clang-18", sanitizer="undefined", package_type="deb", ), Build.PACKAGE_TSAN: BuildConfig( name=Build.PACKAGE_TSAN, - compiler="clang-17", + compiler="clang-18", sanitizer="thread", package_type="deb", ), Build.PACKAGE_MSAN: BuildConfig( name=Build.PACKAGE_MSAN, - compiler="clang-17", + compiler="clang-18", sanitizer="memory", package_type="deb", ), Build.PACKAGE_DEBUG: BuildConfig( name=Build.PACKAGE_DEBUG, - compiler="clang-17", + compiler="clang-18", debug_build=True, package_type="deb", sparse_checkout=True, # Check that it works with at least one build, see also update-submodules.sh ), Build.PACKAGE_RELEASE_COVERAGE: BuildConfig( name=Build.PACKAGE_RELEASE_COVERAGE, - compiler="clang-17", + compiler="clang-18", coverage=True, package_type="deb", ), Build.BINARY_RELEASE: BuildConfig( name=Build.BINARY_RELEASE, - compiler="clang-17", + compiler="clang-18", package_type="binary", ), Build.BINARY_TIDY: BuildConfig( name=Build.BINARY_TIDY, - compiler="clang-17", + compiler="clang-18", debug_build=True, package_type="binary", static_binary_name="debug-amd64", @@ -735,69 +735,69 @@ CI_CONFIG = CIConfig( ), Build.BINARY_DARWIN: BuildConfig( name=Build.BINARY_DARWIN, - compiler="clang-17-darwin", + compiler="clang-18-darwin", package_type="binary", static_binary_name="macos", ), Build.BINARY_AARCH64: BuildConfig( name=Build.BINARY_AARCH64, - compiler="clang-17-aarch64", + compiler="clang-18-aarch64", package_type="binary", ), Build.BINARY_AARCH64_V80COMPAT: BuildConfig( name=Build.BINARY_AARCH64_V80COMPAT, - compiler="clang-17-aarch64-v80compat", + compiler="clang-18-aarch64-v80compat", package_type="binary", static_binary_name="aarch64v80compat", comment="For ARMv8.1 and older", ), Build.BINARY_FREEBSD: BuildConfig( name=Build.BINARY_FREEBSD, - compiler="clang-17-freebsd", + compiler="clang-18-freebsd", package_type="binary", static_binary_name="freebsd", ), Build.BINARY_DARWIN_AARCH64: BuildConfig( name=Build.BINARY_DARWIN_AARCH64, - compiler="clang-17-darwin-aarch64", + compiler="clang-18-darwin-aarch64", package_type="binary", static_binary_name="macos-aarch64", ), Build.BINARY_PPC64LE: BuildConfig( name=Build.BINARY_PPC64LE, - compiler="clang-17-ppc64le", + compiler="clang-18-ppc64le", package_type="binary", static_binary_name="powerpc64le", ), Build.BINARY_AMD64_COMPAT: BuildConfig( name=Build.BINARY_AMD64_COMPAT, - compiler="clang-17-amd64-compat", + compiler="clang-18-amd64-compat", package_type="binary", static_binary_name="amd64compat", comment="SSE2-only build", ), Build.BINARY_AMD64_MUSL: BuildConfig( name=Build.BINARY_AMD64_MUSL, - compiler="clang-17-amd64-musl", + compiler="clang-18-amd64-musl", package_type="binary", static_binary_name="amd64musl", comment="Build with Musl", ), Build.BINARY_RISCV64: BuildConfig( name=Build.BINARY_RISCV64, - compiler="clang-17-riscv64", + compiler="clang-18-riscv64", package_type="binary", static_binary_name="riscv64", ), Build.BINARY_S390X: BuildConfig( name=Build.BINARY_S390X, - compiler="clang-17-s390x", + compiler="clang-18-s390x", package_type="binary", static_binary_name="s390x", ), Build.FUZZERS: BuildConfig( name=Build.FUZZERS, - compiler="clang-17", + compiler="clang-18", package_type="fuzzers", job_config=JobConfig(run_by_label=Labels.libFuzzer), ), From aaec92f860c92f403c2f79fb9f724102487ed903 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 3 Mar 2024 23:02:55 +0100 Subject: [PATCH 0040/1018] Address review comments --- cmake/tools.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/tools.cmake b/cmake/tools.cmake index 1ba3007b0f3..789dc76e6fa 100644 --- a/cmake/tools.cmake +++ b/cmake/tools.cmake @@ -13,7 +13,7 @@ execute_process(COMMAND ${CMAKE_CXX_COMPILER} --version OUTPUT_VARIABLE COMPILER message (STATUS "Using compiler:\n${COMPILER_SELF_IDENTIFICATION}") # Require minimum compiler versions -set (CLANG_MINIMUM_VERSION 16) +set (CLANG_MINIMUM_VERSION 17) set (XCODE_MINIMUM_VERSION 12.0) set (APPLE_CLANG_MINIMUM_VERSION 12.0.0) From 8d25f27b1901d4d4999558b987a415c4c289e2ba Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 3 Mar 2024 23:07:05 +0100 Subject: [PATCH 0041/1018] Fix UBSan report --- src/Compression/CompressionCodecDoubleDelta.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Compression/CompressionCodecDoubleDelta.cpp b/src/Compression/CompressionCodecDoubleDelta.cpp index 99089ed6770..fe18578eb2a 100644 --- a/src/Compression/CompressionCodecDoubleDelta.cpp +++ b/src/Compression/CompressionCodecDoubleDelta.cpp @@ -343,7 +343,7 @@ UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest) const auto sign = signed_dd < 0; // -1 shrinks dd down to fit into number of bits, and there can't be 0, so it is OK. - const auto abs_value = static_cast(std::abs(signed_dd) - 1); + const auto abs_value = (sign ? -static_cast(signed_dd) : static_cast(signed_dd)) - 1; const auto write_spec = getDeltaWriteSpec(signed_dd); writer.writeBits(write_spec.prefix_bits, write_spec.prefix); From f7c05edc083e811d89381789c673746cd4802082 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 4 Mar 2024 16:53:49 +0800 Subject: [PATCH 0042/1018] Debug logging --- src/Storages/RabbitMQ/RabbitMQSource.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/RabbitMQ/RabbitMQSource.cpp b/src/Storages/RabbitMQ/RabbitMQSource.cpp index 75c02d5fa23..25b8af4f431 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSource.cpp @@ -190,6 +190,9 @@ Chunk RabbitMQSource::generateImpl() const auto exchange_name = storage.getExchange(); const auto & message = consumer->currentMessage(); + LOG_TEST(log, "Pulled {} rows, message delivery tag: {} (previous delivery tag: {}, redelivered: {})", + new_rows, message.delivery_tag, commit_info.delivery_tag, message.redelivered); + if (exception_message.has_value()) { commit_info.failed_delivery_tags.push_back(message.delivery_tag); From 6c27483b12c35eb36a3599666868ae7e030d4dc2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 5 Mar 2024 11:45:52 +0800 Subject: [PATCH 0043/1018] Fix --- src/Storages/RabbitMQ/RabbitMQSource.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQSource.cpp b/src/Storages/RabbitMQ/RabbitMQSource.cpp index 25b8af4f431..a72c5ad7f27 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSource.cpp @@ -199,8 +199,8 @@ Chunk RabbitMQSource::generateImpl() } else { - chassert(!commit_info.delivery_tag || commit_info.delivery_tag < message.delivery_tag); - commit_info.delivery_tag = message.delivery_tag; + chassert(!commit_info.delivery_tag || message.redelivered || commit_info.delivery_tag < message.delivery_tag); + commit_info.delivery_tag = std::max(commit_info.delivery_tag, message.delivery_tag); } for (size_t i = 0; i < new_rows; ++i) From b3a43346b110ecbdbbc9d8c025fee09b4b35e2b6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Mar 2024 06:59:42 +0100 Subject: [PATCH 0044/1018] Revert "Fix UBSan report" This reverts commit 8d25f27b1901d4d4999558b987a415c4c289e2ba. --- src/Compression/CompressionCodecDoubleDelta.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Compression/CompressionCodecDoubleDelta.cpp b/src/Compression/CompressionCodecDoubleDelta.cpp index fe18578eb2a..99089ed6770 100644 --- a/src/Compression/CompressionCodecDoubleDelta.cpp +++ b/src/Compression/CompressionCodecDoubleDelta.cpp @@ -343,7 +343,7 @@ UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest) const auto sign = signed_dd < 0; // -1 shrinks dd down to fit into number of bits, and there can't be 0, so it is OK. - const auto abs_value = (sign ? -static_cast(signed_dd) : static_cast(signed_dd)) - 1; + const auto abs_value = static_cast(std::abs(signed_dd) - 1); const auto write_spec = getDeltaWriteSpec(signed_dd); writer.writeBits(write_spec.prefix_bits, write_spec.prefix); From 4e8a363b34380a914a326cadff5f9f796837eb70 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Mar 2024 07:32:47 +0100 Subject: [PATCH 0045/1018] Fix something entirely wrong in DoubleDelta --- src/Compression/CompressionCodecDoubleDelta.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Compression/CompressionCodecDoubleDelta.cpp b/src/Compression/CompressionCodecDoubleDelta.cpp index 99089ed6770..e6e8db4c699 100644 --- a/src/Compression/CompressionCodecDoubleDelta.cpp +++ b/src/Compression/CompressionCodecDoubleDelta.cpp @@ -343,7 +343,10 @@ UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest) const auto sign = signed_dd < 0; // -1 shrinks dd down to fit into number of bits, and there can't be 0, so it is OK. - const auto abs_value = static_cast(std::abs(signed_dd) - 1); + const auto abs_value = + signed_dd == std::numeric_limits::min() + ? (static_cast(-1) >> 1) + : static_cast(std::abs(signed_dd) - 1); const auto write_spec = getDeltaWriteSpec(signed_dd); writer.writeBits(write_spec.prefix_bits, write_spec.prefix); From 9bb32dc78fa7872cb965d087e64050c091688777 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Mar 2024 09:22:14 +0100 Subject: [PATCH 0046/1018] Loosen --- CMakeLists.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index b55e9810361..9ffb4789dc9 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -61,8 +61,8 @@ if (ENABLE_CHECK_HEAVY_BUILDS) # set CPU time limit to 1000 seconds set (RLIMIT_CPU 1000) - # -fsanitize=memory is too heavy - if (SANITIZE STREQUAL "memory") + # -fsanitize=memory and address are too heavy + if (SANITIZE) set (RLIMIT_DATA 10000000000) # 10G endif() From 4605cdc92f39e1d5ccfc7843268c7ced2c8c3f72 Mon Sep 17 00:00:00 2001 From: Blargian Date: Mon, 11 Mar 2024 18:51:46 +0100 Subject: [PATCH 0047/1018] Update test --- .../queries/0_stateless/02984_form_format.sh | 142 +----------------- 1 file changed, 3 insertions(+), 139 deletions(-) diff --git a/tests/queries/0_stateless/02984_form_format.sh b/tests/queries/0_stateless/02984_form_format.sh index 8ff86116fee..38cd7190a05 100755 --- a/tests/queries/0_stateless/02984_form_format.sh +++ b/tests/queries/0_stateless/02984_form_format.sh @@ -13,145 +13,9 @@ FORM_DATA="${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/${FILE_NAME}" mkdir -p ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ touch $FORM_DATA -echo -ne 'c.e=ls7xfkpm&c.tti.m=raf&rt.start=navigation&rt.bmr=390%2C11%2C10&rt.tstart=1707076768666&rt.bstart=1707076769091&rt.blstart=1707076769056&rt.end=1707076769078&t_resp=296&t_page=116&t_done=412&t_other=boomerang%7C6%2Cboomr_fb%7C425%2Cboomr_ld%7C390%2Cboomr_lat%7C35&rt.tt=2685&rt.obo=0&pt.fcp=407&nt_nav_st=1707076768666&nt_dns_st=1707076768683&nt_dns_end=1707076768684&nt_con_st=1707076768684&nt_con_end=1707076768850&nt_req_st=1707076768850&nt_res_st=1707076768962&nt_res_end=1707076768962&nt_domloading=1707076769040&nt_domint=1707076769066&nt_domcontloaded_st=1707076769067&nt_domcontloaded_end=1707076769068&nt_domcomp=1707076769069&nt_load_st=1707076769069&nt_load_end=1707076769078&nt_unload_st=1707076769040&nt_unload_end=1707076769041&nt_ssl_st=1707076768788&nt_enc_size=3209&nt_dec_size=10093&nt_trn_size=3940&nt_protocol=h2&nt_red_cnt=0&nt_nav_type=1&restiming=%7B%22https%3A%2F%2Fwww.basicrum.com%2F%22%3A%7B%22publications%2F%22%3A%226%2C88%2C88%2C54%2C54%2C3e%2Ci%2Ci%2Ch*12h5%2Ckb%2C5b8%22%2C%22assets%2Fjs%2F%22%3A%7B%22just-the-docs.js%22%3A%223am%2Ce%2Ce*12pc%2C_%2C8oj*20%22%2C%22boomerang-1.737.60.cutting-edge.min.js%22%3A%222au%2Cb%2Ca*1pu3%2C_%2C1m19*21*42%22%2C%22vendor%2Flunr.min.js%22%3A%223am%2Cd%2C8*16t2%2C_%2Cfym*20%22%7D%7D%7D&u=https%3A%2F%2Fwww.basicrum.com%2Fpublications%2F&r=https%3A%2F%2Fwww.basicrum.com%2Fcost-analyses%2F&v=1.737.60&sv=14&sm=p&rt.si=dd0c542f-7adf-4310-830a-6c0a3d157c90-s8cjr1&rt.ss=1707075325294&rt.sl=4&vis.st=visible&ua.plt=Linux%20x86_64&ua.vnd=&pid=8fftz949&n=1&c.t.fps=07*4*65*j*61&c.t.busy=2*4*0034&c.tti.vr=408&c.tti=408&c.b=2&c.f=60&c.f.d=2511&c.f.m=1&c.f.s=ls7xfl1h&dom.res=5&dom.doms=1&mem.lsln=0&mem.ssln=0&mem.lssz=2&mem.sssz=2&scr.xy=1920x1200&scr.bpp=24%2F24&scr.orn=0%2Flandscape-primary&cpu.cnc=16&dom.ln=114&dom.sz=10438&dom.ck=157&dom.img=0&dom.script=6&dom.script.ext=3&dom.iframe=0&dom.link=4&dom.link.css=1&sb=1' > $FORM_DATA - -$CLICKHOUSE_CLIENT -nm --query " -DROP TABLE IF EXISTS form_data; -CREATE TABLE IF NOT EXISTS form_data -( -c Nested -( - t Nested - ( - fps Nullable(String), - busy Nullable(String) - ), - tti Nested - ( - tti_value Nullable(Int64), - vr Nullable(Int64), - m Nullable(String) - ), - f Nested - ( - f_value Nullable(Int64), - d Nullable(Int64), - m Nullable(Int64), - s Nullable(String) - ), - e Nullable(String), - b Nullable(Int64) -), -rt Nested -( - start Nullable(String), - bmr Nullable(String), - tstart Nullable(Int64), - bstart Nullable(Int64), - blstart Nullable(Int64), - end Nullable(Int64), - tt Nullable(Int64), - obo Nullable(Int64), - si Nullable(String), - ss Nullable(Int64), - sl Nullable(Int64) -), -pt Nested -( - fcp Nullable(Int64) -), -vis Nested -( - st Nullable(String) -), -ua Nested -( - plt Nullable(String), - vnd Nullable(String) -), -dom Nested -( - res Nullable(Int64), - doms Nullable(Int64), - ln Nullable(Int64), - sz Nullable(Int64), - ck Nullable(Int64), - img Nullable(Int64), - iframe Nullable(Int64), - link Nested - ( - link_value Nullable(Int64), - css Nullable(Int64) - ), - script Nested - ( - script_value Nullable(Int64), - ext Nullable(Int64) - ) -), -mem Nested -( - lsln Nullable(Int64), - ssln Nullable(Int64), - lssz Nullable(Int64), - sssz Nullable(Int64) -), -scr Nested -( - xy Nullable(String), - bpp Nullable(String), - orn Nullable(String) -), -cpc Nested -( - cnc Nullable(Int64) -), -t_resp Nullable(Int64), -t_page Nullable(Int64), -t_done Nullable(Int64), -t_other Nullable(String), -nt_nav_st Nullable(Int64), -nt_dns_st Nullable(Int64), -nt_dns_end Nullable(Int64), -nt_con_st Nullable(Int64), -nt_con_end Nullable(Int64), -nt_req_st Nullable(Int64), -nt_res_st Nullable(Int64), -nt_res_end Nullable(Int64), -nt_domloading Nullable(Int64), -nt_domint Nullable(Int64), -nt_domcontloaded_st Nullable(Int64), -nt_domcontloaded_end Nullable(Int64), -nt_domcomp Nullable(Int64), -nt_load_st Nullable(Int64), -nt_load_end Nullable(Int64), -nt_unload_st Nullable(Int64) , -nt_unload_end Nullable(Int64), -nt_ssl_st Nullable(Int64), -nt_enc_size Nullable(Int64), -nt_dec_size Nullable(Int64), -nt_trn_size Nullable(Int64), -nt_protocol Nullable(String), -nt_red_cnt Nullable(Int64), -nt_nav_type Nullable(Int64), -restiming Nullable(String), -u Nullable(String), -r Nullable(String), -v Nullable(String), -sv Nullable(Int64), -sm Nullable(String), -sb Nullable(Int64), -pid Nullable(String), -n Nullable(Int64) -) -ENGINE = TinyLog; -" - -# Insert Form data from clickhouse-client -$CLICKHOUSE_CLIENT --query "INSERT INTO default.form_data FORMAT Form" < $FORM_DATA -$CLICKHOUSE_CLIENT --query "SELECT * FROM default.form_data FORMAT Vertical" - -#${CLICKHOUSE_CURL} http://localhost:8123/?query=INSERT%20INTO%20form%5Fdata%20FORMAT%20Form --data-binary @- < $FORM_DATA +# Simple tests +echo "col1=42&col2=Hello%2C%20World%21" > $FORM_DATA +$CLICKHOUSE_CLIENT -q "SELECT * from file('$FORM_DATA', Form, 'col1 UInt64, col2 String')" # Test teardown rm -r ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME} \ No newline at end of file From 2ff1368678504b3100e7c0f4dc8496e4bfdc9539 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 13 Mar 2024 09:37:17 +0100 Subject: [PATCH 0048/1018] Fix FreeBSD --- src/Common/waitForPid.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/waitForPid.cpp b/src/Common/waitForPid.cpp index 0ec10811354..894406f69b5 100644 --- a/src/Common/waitForPid.cpp +++ b/src/Common/waitForPid.cpp @@ -144,7 +144,7 @@ static PollPidResult pollPid(pid_t pid, int timeout_in_ms) return PollPidResult::FAILED; } - struct kevent event = {.ident = 0}; + struct kevent event{}; struct timespec remaining_timespec = {.tv_sec = timeout_in_ms / 1000, .tv_nsec = (timeout_in_ms % 1000) * 1000000}; int ready = HANDLE_EINTR(kevent(kq, nullptr, 0, &event, 1, &remaining_timespec)); PollPidResult result = ready < 0 ? PollPidResult::FAILED : PollPidResult::RESTART; From 480ae2cd4982fda4fa3f24e494faa8b1848eff59 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 13 Mar 2024 09:39:03 +0100 Subject: [PATCH 0049/1018] Fix clang-tidy --- cmake/clang_tidy.cmake | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cmake/clang_tidy.cmake b/cmake/clang_tidy.cmake index 4323c20463a..4c9331f6283 100644 --- a/cmake/clang_tidy.cmake +++ b/cmake/clang_tidy.cmake @@ -5,14 +5,14 @@ if (ENABLE_CLANG_TIDY) find_program (CLANG_TIDY_CACHE_PATH NAMES "clang-tidy-cache") if (CLANG_TIDY_CACHE_PATH) - find_program (_CLANG_TIDY_PATH NAMES "clang-tidy-17" "clang-tidy-16" "clang-tidy") + find_program (_CLANG_TIDY_PATH NAMES "clang-tidy-18" "clang-tidy-17" "clang-tidy-16" "clang-tidy") # Why do we use ';' here? # It's a cmake black magic: https://cmake.org/cmake/help/latest/prop_tgt/LANG_CLANG_TIDY.html#prop_tgt:%3CLANG%3E_CLANG_TIDY # The CLANG_TIDY_PATH is passed to CMAKE_CXX_CLANG_TIDY, which follows CXX_CLANG_TIDY syntax. set (CLANG_TIDY_PATH "${CLANG_TIDY_CACHE_PATH};${_CLANG_TIDY_PATH}" CACHE STRING "A combined command to run clang-tidy with caching wrapper") else () - find_program (CLANG_TIDY_PATH NAMES "clang-tidy-17" "clang-tidy-16" "clang-tidy") + find_program (CLANG_TIDY_PATH NAMES "clang-tidy-18" "clang-tidy-17" "clang-tidy-16" "clang-tidy") endif () if (CLANG_TIDY_PATH) From bb55a0ca50fe2a93829e249df1f5f342ed11b4d2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 11 Mar 2024 19:18:15 +0100 Subject: [PATCH 0050/1018] Less contention in cache (Part 4) --- src/Interpreters/Cache/EvictionCandidates.h | 10 +++ src/Interpreters/Cache/FileCache.cpp | 83 +++++++++++++++++++ src/Interpreters/Cache/FileCache.h | 8 ++ src/Interpreters/Cache/FileCacheSettings.cpp | 9 ++ src/Interpreters/Cache/FileCacheSettings.h | 4 + src/Interpreters/Cache/FileCache_fwd.h | 3 + src/Interpreters/Cache/IFileCachePriority.h | 9 ++ .../Cache/LRUFileCachePriority.cpp | 51 +++++++++--- src/Interpreters/Cache/LRUFileCachePriority.h | 14 ++++ .../Cache/SLRUFileCachePriority.cpp | 31 +++++++ .../Cache/SLRUFileCachePriority.h | 7 ++ tests/config/config.d/storage_conf.xml | 2 + 12 files changed, 221 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/Cache/EvictionCandidates.h b/src/Interpreters/Cache/EvictionCandidates.h index 7859762be09..752d26031e8 100644 --- a/src/Interpreters/Cache/EvictionCandidates.h +++ b/src/Interpreters/Cache/EvictionCandidates.h @@ -7,10 +7,20 @@ namespace DB class EvictionCandidates { public: + EvictionCandidates() = default; + EvictionCandidates(EvictionCandidates && other) noexcept + { + candidates = std::move(other.candidates); + candidates_size = std::move(other.candidates_size); + invalidated_queue_entries = std::move(other.invalidated_queue_entries); + finalize_eviction_func = std::move(other.finalize_eviction_func); + } ~EvictionCandidates(); void add(LockedKey & locked_key, const FileSegmentMetadataPtr & candidate); + void add(const EvictionCandidates & other, const CacheGuard::Lock &) { candidates.insert(other.candidates.begin(), other.candidates.end()); } + void evict(); void finalize(FileCacheQueryLimit::QueryContext * query_context, const CacheGuard::Lock & lock); diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 90508d74554..bbaf3aebbee 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -85,6 +85,9 @@ FileCache::FileCache(const std::string & cache_name, const FileCacheSettings & s , boundary_alignment(settings.boundary_alignment) , load_metadata_threads(settings.load_metadata_threads) , write_cache_per_user_directory(settings.write_cache_per_user_id_directory) + , keep_current_size_to_max_ratio(1 - settings.keep_free_space_size_ratio) + , keep_current_elements_to_max_ratio(1 - settings.keep_free_space_elements_ratio) + , keep_up_free_space_remove_batch(settings.keep_free_space_remove_batch) , log(getLogger("FileCache(" + cache_name + ")")) , metadata(settings.base_path, settings.background_download_queue_size_limit, settings.background_download_threads, write_cache_per_user_directory) { @@ -179,6 +182,10 @@ void FileCache::initialize() } metadata.startup(); + + if (keep_current_size_to_max_ratio != 1 || keep_current_elements_to_max_ratio != 1) + keep_up_free_space_ratio_task = Context::getGlobalContextInstance()->getSchedulePool().createTask(log->name(), [this] { freeSpaceRatioKeepingThreadFunc(); }); + is_initialized = true; } @@ -884,6 +891,81 @@ bool FileCache::tryReserve( return true; } +void FileCache::freeSpaceRatioKeepingThreadFunc() +{ + static constexpr auto lock_failed_reschedule_ms = 1000; + static constexpr auto space_ratio_satisfied_reschedule_ms = 5000; + static constexpr auto general_reschedule_ms = 5000; + + while (true) + { + if (shutdown) + return; + + auto lock = tryLockCache(); + if (!lock) + { + keep_up_free_space_ratio_task->scheduleAfter(lock_failed_reschedule_ms); + return; + } + + const size_t size_limit = main_priority->getSizeLimit(lock); + const size_t elements_limit = main_priority->getElementsLimit(lock); + + const size_t desired_size = std::lround(keep_current_size_to_max_ratio * size_limit); + const size_t desired_elements_num = std::lround(keep_current_elements_to_max_ratio * elements_limit); + + if ((size_limit == 0 || main_priority->getSize(lock) <= desired_size) + && (elements_limit == 0 || main_priority->getElementsCount(lock) <= desired_elements_num)) + { + /// Nothing to free - all limits are satisfied. + keep_up_free_space_ratio_task->scheduleAfter(space_ratio_satisfied_reschedule_ms); + return; + } + + try + { + FileCacheReserveStat stat; + auto eviction_candidates = main_priority->collectCandidatesForEviction( + desired_size, desired_elements_num, keep_up_free_space_remove_batch, stat, lock); + + if (shutdown) + return; + + if (eviction_candidates.size() == 0) + { + /// This case is impossible in realistic cache setup, + /// e.g. we should always be able to evict something. + keep_up_free_space_ratio_task->scheduleAfter(general_reschedule_ms); + return; + } + + LOG_TRACE(log, "Current usage {}/{} in size, {}/{} in elements count " + "(trying to keep size ration at {} and elements ratio at {}). " + "Collected {} eviction candidates, " + "skipped {} candidates while iterating", + main_priority->getSize(lock), size_limit, + main_priority->getElementsCount(lock), elements_limit, + desired_size, desired_elements_num, + eviction_candidates.size(), stat.stat.non_releasable_count); + + lock.unlock(); + eviction_candidates.evict(); + + lock.lock(); + eviction_candidates.finalize(nullptr, lock); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + keep_up_free_space_ratio_task->scheduleAfter(general_reschedule_ms); + + /// Let's catch such cases in ci, in general there should not be exceptions. + chassert(false); + } + } +} + void FileCache::iterate(IterateFunc && func, const UserID & user_id) { return metadata.iterate([&](const LockedKey & locked_key) @@ -1213,6 +1295,7 @@ void FileCache::deactivateBackgroundOperations() { shutdown.store(true); metadata.shutdown(); + keep_up_free_space_ratio_task->deactivate(); } std::vector FileCache::getFileSegmentInfos(const UserID & user_id) diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 5b665ad0271..22279eccb8b 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -18,6 +18,7 @@ #include #include #include +#include #include @@ -185,6 +186,8 @@ public: void applySettingsIfPossible(const FileCacheSettings & new_settings, FileCacheSettings & actual_settings); + void freeSpaceRatioKeepingThreadFunc(); + private: using KeyAndOffset = FileCacheKeyAndOffset; @@ -194,6 +197,11 @@ private: size_t load_metadata_threads; const bool write_cache_per_user_directory; + BackgroundSchedulePool::TaskHolder keep_up_free_space_ratio_task; + const double keep_current_size_to_max_ratio; + const double keep_current_elements_to_max_ratio; + const size_t keep_up_free_space_remove_batch; + LoggerPtr log; std::exception_ptr init_exception; diff --git a/src/Interpreters/Cache/FileCacheSettings.cpp b/src/Interpreters/Cache/FileCacheSettings.cpp index 8a48a2de68f..b7ee91a7ec2 100644 --- a/src/Interpreters/Cache/FileCacheSettings.cpp +++ b/src/Interpreters/Cache/FileCacheSettings.cpp @@ -78,6 +78,15 @@ void FileCacheSettings::loadImpl(FuncHas has, FuncGetUInt get_uint, FuncGetStrin if (has("write_cache_per_user_id_directory")) slru_size_ratio = get_uint("write_cache_per_user_id_directory"); + + if (has("keep_free_space_size_ratio")) + keep_free_space_size_ratio = get_double("keep_free_space_size_ratio"); + + if (has("keep_free_space_elements_ratio")) + keep_free_space_elements_ratio = get_double("keep_free_space_elements_ratio"); + + if (has("keep_free_space_remove_batch")) + keep_free_space_elements_ratio = get_uint("keep_free_space_remove_batch"); } void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) diff --git a/src/Interpreters/Cache/FileCacheSettings.h b/src/Interpreters/Cache/FileCacheSettings.h index 14770b3f005..1ba9f7e17cc 100644 --- a/src/Interpreters/Cache/FileCacheSettings.h +++ b/src/Interpreters/Cache/FileCacheSettings.h @@ -38,6 +38,10 @@ struct FileCacheSettings std::string cache_policy = "LRU"; double slru_size_ratio = 0.5; + double keep_free_space_size_ratio = FILECACHE_DEFAULT_FREE_SPACE_SIZE_RATIO; + double keep_free_space_elements_ratio = FILECACHE_DEFAULT_FREE_SPACE_ELEMENTS_RATIO; + size_t keep_free_space_remove_batch = FILECACHE_DEFAULT_FREE_SPACE_REMOVE_BATCH; + void loadFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); void loadFromCollection(const NamedCollection & collection); diff --git a/src/Interpreters/Cache/FileCache_fwd.h b/src/Interpreters/Cache/FileCache_fwd.h index 06261b19db7..55453b78ead 100644 --- a/src/Interpreters/Cache/FileCache_fwd.h +++ b/src/Interpreters/Cache/FileCache_fwd.h @@ -12,6 +12,9 @@ static constexpr int FILECACHE_DEFAULT_LOAD_METADATA_THREADS = 16; static constexpr int FILECACHE_DEFAULT_MAX_ELEMENTS = 10000000; static constexpr int FILECACHE_DEFAULT_HITS_THRESHOLD = 0; static constexpr size_t FILECACHE_BYPASS_THRESHOLD = 256 * 1024 * 1024; +static constexpr double FILECACHE_DEFAULT_FREE_SPACE_SIZE_RATIO = 0; /// Disabled. +static constexpr double FILECACHE_DEFAULT_FREE_SPACE_ELEMENTS_RATIO = 0; /// Disabled. +static constexpr int FILECACHE_DEFAULT_FREE_SPACE_REMOVE_BATCH = 10; class FileCache; using FileCachePtr = std::shared_ptr; diff --git a/src/Interpreters/Cache/IFileCachePriority.h b/src/Interpreters/Cache/IFileCachePriority.h index 37270cf0873..8d30e88c112 100644 --- a/src/Interpreters/Cache/IFileCachePriority.h +++ b/src/Interpreters/Cache/IFileCachePriority.h @@ -98,6 +98,7 @@ public: virtual PriorityDumpPtr dump(const CacheGuard::Lock &) = 0; + /// Collect eviction candidates sufficient to free `size` bytes. virtual bool collectCandidatesForEviction( size_t size, FileCacheReserveStat & stat, @@ -106,6 +107,14 @@ public: const UserID & user_id, const CacheGuard::Lock &) = 0; + /// Collect eviction `candidates_num` candidates for eviction. + virtual EvictionCandidates collectCandidatesForEviction( + size_t desired_size, + size_t desired_elements_count, + size_t max_candidates_to_evict, + FileCacheReserveStat & stat, + const CacheGuard::Lock &) = 0; + virtual bool modifySizeLimits(size_t max_size_, size_t max_elements_, double size_ratio_, const CacheGuard::Lock &) = 0; protected: diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index 05bbc26e602..cd122271ebe 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -230,6 +230,40 @@ bool LRUFileCachePriority::collectCandidatesForEviction( if (canFit(size, lock)) return true; + auto can_fit = [&] + { + return canFit(size, stat.stat.releasable_size, stat.stat.releasable_count, lock); + }; + iterateForEviction(res, stat, can_fit, lock); + return can_fit(); +} + +EvictionCandidates LRUFileCachePriority::collectCandidatesForEviction( + size_t desired_size, + size_t desired_elements_count, + size_t max_candidates_to_evict, + FileCacheReserveStat & stat, + const CacheGuard::Lock & lock) +{ + if (!max_candidates_to_evict) + return {}; + + EvictionCandidates res; + auto stop_condition = [&, this]() + { + return (getSize(lock) <= desired_size && getElementsCount(lock) <= desired_elements_count) + || res.size() >= max_candidates_to_evict; + }; + iterateForEviction(res, stat, stop_condition, lock); + return res; +} + +void LRUFileCachePriority::iterateForEviction( + EvictionCandidates & res, + FileCacheReserveStat & stat, + StopConditionFunc stop_condition, + const CacheGuard::Lock & lock) +{ ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictionTries); IterateFunc iterate_func = [&](LockedKey & locked_key, const FileSegmentMetadataPtr & segment_metadata) @@ -240,6 +274,7 @@ bool LRUFileCachePriority::collectCandidatesForEviction( if (segment_metadata->evicting()) { ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictionSkippedEvictingFileSegments); + stat.update(segment_metadata->size(), file_segment->getKind(), false); } else if (segment_metadata->releasable()) { @@ -248,27 +283,23 @@ bool LRUFileCachePriority::collectCandidatesForEviction( } else { - stat.update(segment_metadata->size(), file_segment->getKind(), false); ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictionSkippedFileSegments); + stat.update(segment_metadata->size(), file_segment->getKind(), false); } return IterationResult::CONTINUE; }; - auto can_fit = [&] - { - return canFit(size, stat.stat.releasable_size, stat.stat.releasable_count, lock); - }; - iterate([&](LockedKey & locked_key, const FileSegmentMetadataPtr & segment_metadata) { - return can_fit() ? IterationResult::BREAK : iterate_func(locked_key, segment_metadata); + return stop_condition() ? IterationResult::BREAK : iterate_func(locked_key, segment_metadata); }, lock); - - return can_fit(); } -LRUFileCachePriority::LRUIterator LRUFileCachePriority::move(LRUIterator & it, LRUFileCachePriority & other, const CacheGuard::Lock &) +LRUFileCachePriority::LRUIterator LRUFileCachePriority::move( + LRUIterator & it, + LRUFileCachePriority & other, + const CacheGuard::Lock &) { const auto & entry = *it.getEntry(); if (entry.size == 0) diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index d8907f678a2..3e735682dc1 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -50,6 +50,13 @@ public: const UserID & user_id, const CacheGuard::Lock &) override; + EvictionCandidates collectCandidatesForEviction( + size_t desired_size, + size_t desired_elements_count, + size_t max_candidates_to_evict, + FileCacheReserveStat & stat, + const CacheGuard::Lock &) override; + void shuffle(const CacheGuard::Lock &) override; struct LRUPriorityDump : public IPriorityDump @@ -89,6 +96,13 @@ private: using IterateFunc = std::function; void iterate(IterateFunc && func, const CacheGuard::Lock &); + using StopConditionFunc = std::function; + void iterateForEviction( + EvictionCandidates & res, + FileCacheReserveStat & stat, + StopConditionFunc stop_condition, + const CacheGuard::Lock &); + LRUIterator move(LRUIterator & it, LRUFileCachePriority & other, const CacheGuard::Lock &); LRUIterator add(EntryPtr entry, const CacheGuard::Lock &); }; diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.cpp b/src/Interpreters/Cache/SLRUFileCachePriority.cpp index 543d6a03669..316190d03e6 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/SLRUFileCachePriority.cpp @@ -157,6 +157,37 @@ bool SLRUFileCachePriority::collectCandidatesForEviction( return true; } +EvictionCandidates SLRUFileCachePriority::collectCandidatesForEviction( + size_t desired_size, + size_t desired_elements_count, + size_t max_candidates_to_evict, + FileCacheReserveStat & stat, + const CacheGuard::Lock & lock) +{ + if (!max_candidates_to_evict) + return {}; + + const auto desired_probationary_size = getRatio(desired_size, 1 - size_ratio); + const auto desired_probationary_elements_num = getRatio(desired_elements_count, 1 - size_ratio); + + auto res = probationary_queue.collectCandidatesForEviction( + desired_probationary_size, desired_probationary_elements_num, max_candidates_to_evict, stat, lock); + + chassert(res.size() <= max_candidates_to_evict); + chassert(res.size() == stat.stat.releasable_count); + + if (res.size() == max_candidates_to_evict) + return res; + + const auto desired_protected_size = getRatio(max_size, size_ratio); + const auto desired_protected_elements_num = getRatio(max_elements, size_ratio); + + auto res_add = protected_queue.collectCandidatesForEviction( + desired_protected_size, desired_protected_elements_num, max_candidates_to_evict - res.size(), stat, lock); + res.add(res_add, lock); + return res; +} + void SLRUFileCachePriority::increasePriority(SLRUIterator & iterator, const CacheGuard::Lock & lock) { /// If entry is already in protected queue, diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.h b/src/Interpreters/Cache/SLRUFileCachePriority.h index 28e61396572..eda85291cd2 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.h +++ b/src/Interpreters/Cache/SLRUFileCachePriority.h @@ -47,6 +47,13 @@ public: const UserID & user_id, const CacheGuard::Lock &) override; + EvictionCandidates collectCandidatesForEviction( + size_t desired_size, + size_t desired_elements_count, + size_t max_candidates_to_evict, + FileCacheReserveStat & stat, + const CacheGuard::Lock &) override; + void shuffle(const CacheGuard::Lock &) override; PriorityDumpPtr dump(const CacheGuard::Lock &) override; diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index 00d8cb3aea5..24e85031a3f 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -24,6 +24,8 @@ 100 LRU 0.3 + 0.1 + 0.1 cache From 23a4486ad9e98980e81c2414188b3f6d559c525e Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Mon, 11 Mar 2024 16:39:13 +0000 Subject: [PATCH 0051/1018] CI: fix sync build issue with reuse #do_not_test --- tests/ci/ci.py | 21 +++++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 9d57f161be3..cc4d0b11eef 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -140,7 +140,7 @@ class CiCache: self.s3 = s3 self.job_digests = job_digests self.cache_s3_paths = { - job_type: f"{self._S3_CACHE_PREFIX}/{job_type.value}-{self.job_digests[self._get_reference_job_name(job_type)]}/" + job_type: f"{self._S3_CACHE_PREFIX}/{job_type.value}-{self._get_digest_for_job_type(self.job_digests, job_type)}/" for job_type in self.JobType } self.s3_record_prefixes = { @@ -155,14 +155,23 @@ class CiCache: if not self._LOCAL_CACHE_PATH.exists(): self._LOCAL_CACHE_PATH.mkdir(parents=True, exist_ok=True) - def _get_reference_job_name(self, job_type: JobType) -> str: - res = Build.PACKAGE_RELEASE + def _get_digest_for_job_type( + self, job_digests: Dict[str, str], job_type: JobType + ) -> str: if job_type == self.JobType.DOCS: - res = JobNames.DOCS_CHECK + res = job_digests[JobNames.DOCS_CHECK] elif job_type == self.JobType.SRCS: - res = Build.PACKAGE_RELEASE + # any build type job has the same digest - pick up Build.PACKAGE_RELEASE or Build.PACKAGE_ASAN as a failover + # Build.PACKAGE_RELEASE may not exist in the list if we have reduced CI pipeline + if Build.PACKAGE_RELEASE in job_digests: + res = job_digests[Build.PACKAGE_RELEASE] + elif Build.PACKAGE_ASAN in job_digests: + # failover, if failover does not work - fix it! + res = job_digests[Build.PACKAGE_ASAN] + else: + assert False, "BUG, no build job in digest' list" else: - assert False + assert False, "BUG, New JobType? - please update func" return res def _get_record_file_name( From 7e101bc07fca78ede2d070d3dc84ad7400bce76c Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 13 Mar 2024 12:47:02 +0100 Subject: [PATCH 0052/1018] Simple tests passing --- .../Formats/Impl/FormInputFormat.cpp | 105 +++++------------- src/Processors/Formats/Impl/FormInputFormat.h | 9 +- .../0_stateless/02984_form_format.reference | 86 +------------- .../queries/0_stateless/02984_form_format.sh | 6 +- 4 files changed, 41 insertions(+), 165 deletions(-) diff --git a/src/Processors/Formats/Impl/FormInputFormat.cpp b/src/Processors/Formats/Impl/FormInputFormat.cpp index 4d1562b63b7..565ba66bc9f 100644 --- a/src/Processors/Formats/Impl/FormInputFormat.cpp +++ b/src/Processors/Formats/Impl/FormInputFormat.cpp @@ -19,41 +19,14 @@ namespace ErrorCodes { extern const int CANNOT_READ_ALL_DATA; extern const int INCORRECT_DATA; - extern const int LOGICAL_ERROR; -} - -namespace -{ - -enum -{ - UNKNOWN_FIELD = size_t(-1), - NESTED_FIELD = size_t(-2) -}; - } -FormInputFormat::FormInputFormat(ReadBuffer & in_, Block header_, Params params_, const FormatSettings & format_settings_) - : IRowInputFormat(std::move(header_), in_, params_), format_settings(format_settings_) +FormInputFormat::FormInputFormat(ReadBuffer & in_, Block header_, Params params_, const FormatSettings & format_settings_) : IRowInputFormat(std::move(header_), in_, params_), format_settings(format_settings_) { const auto & header = getPort().getHeader(); - name_map = header.getNamesToIndexesMap(); - - // we don't want this as the data should be flattened - if (format_settings_.import_nested_json) - { - for (size_t i = 0; i != header.columns(); ++i) - { - const StringRef column_name = header.getByPosition(i).name; - const auto split = Nested::splitName(column_name.toView()); - if (!split.second.empty()) - { - const StringRef table_name(column_name.data, split.first.size()); - name_map[table_name] = NESTED_FIELD; - } - } - } - + size_t num_columns = header.columns(); + for (size_t i = 0; i < num_columns; ++i) + name_map[header.getByPosition(i).name] = i; } void FormInputFormat::readPrefix() @@ -73,7 +46,7 @@ StringRef readName(ReadBuffer & buf, StringRef & ref, String & tmp) while (!buf.eof()) { const char * next_pos = find_first_symbols<'=','&'>(buf.position(), buf.buffer().end()); - + if (next_pos == buf.buffer().end()) { tmp.append(buf.position(), next_pos - buf.position()); @@ -99,22 +72,19 @@ void FormInputFormat::readField(size_t index, MutableColumns & columns) throw Exception(ErrorCodes::INCORRECT_DATA, "Duplicate field found while parsing Form format: {}", columnName(index)); seen_columns[index] = read_columns[index] = true; - const auto & type = getPort().getHeader().getByPosition(index).type; const auto & serialization = serializations[index]; String encoded_str, decoded_str; readStringUntilAmpersand(encoded_str,*in); - assertChar('&',*in); + + if (!in->eof()) + assertChar('&',*in); + Poco::URI::decode(encoded_str, decoded_str); ReadBufferFromString buf(decoded_str); - - if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) - read_columns[index] = SerializationNullable::deserializeNullAsDefaultOrNestedTextRaw(*columns[index], buf, format_settings, serialization); - else - serialization->deserializeTextRaw(*columns[index], buf, format_settings); + serialization->deserializeWholeText(*columns[index], buf, format_settings); } - String readFieldName(ReadBuffer & buf) { String field; @@ -129,59 +99,46 @@ void FormInputFormat::skipUnknownFormField(StringRef name_ref) throw Exception(ErrorCodes::INCORRECT_DATA, "Unknown field found while parsing Form format: {}", name_ref.toString()); /// read name and value but do nothing with them - if(!in->eof()) + if (!in->eof()) { readFieldName(*in); String value; readStringUntilAmpersand(value,*in); } - } - -inline size_t FormInputFormat::columnIndex(StringRef name) -{ - const auto it = name_map.find(name); - if (it != name_map.end()) - { - return it->second; - } - else - return UNKNOWN_FIELD; -} - - void FormInputFormat::readFormData(MutableColumns & columns) { size_t index = 0; - while (index < columns.size()) + while (true) { if (in->eof()) break; StringRef name_ref = readFieldName(*in); - const size_t column_index = columnIndex(name_ref); + auto * it = name_map.find(name_ref); - if (ssize_t(column_index) < 0) + if (!it) { - /// copy name_ref to temporary string as name_ref may - /// point directly to the input buffer + if (!format_settings.skip_unknown_fields) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unknown field found while parsing Form format: {}", name_ref.toString()); - current_column_name.assign(name_ref.data, name_ref.size); - name_ref = StringRef(current_column_name); + /// Skip the value if key is not found. + NullOutput sink; + String encoded_str; + readStringUntilAmpersand(encoded_str,*in); - if (column_index == UNKNOWN_FIELD) - skipUnknownFormField(name_ref); - else if (column_index == NESTED_FIELD) - readNestedFormData(name_ref.toString(), columns); - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: illegal value of column_index"); + if (!in->eof()) + assertChar('&',*in); + + ReadBufferFromString buf(encoded_str); + readStringInto(sink, buf); } else { - readField(column_index, columns); + index = it->getMapped(); + readField(index, columns); } - ++index; } } @@ -209,7 +166,7 @@ bool FormInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext) const auto & header = getPort().getHeader(); /// Non-visited columns get filled with default values for (size_t i = 0; i < num_columns; ++i) - if(!seen_columns[i]) + if (!seen_columns[i]) header.getByPosition(i).type->insertDefaultInto(*columns[i]); /// Return info about defaults set. @@ -218,7 +175,6 @@ bool FormInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext) ext.read_columns = read_columns; else ext.read_columns.assign(read_columns.size(), true); - return true; } @@ -246,18 +202,17 @@ NamesAndTypesList readRowAndGetNamesAndDataTypesForFormRow(ReadBuffer & in, cons auto type = tryInferDataTypeByEscapingRule(value, settings, FormatSettings::EscapingRule::Escaped); names_and_types.emplace_back(name, type); } - while(checkChar('&',in)); + while (checkChar('&',in)); return names_and_types; } NamesAndTypesList FormSchemaReader::readRowAndGetNamesAndDataTypes(bool & eof) { - if(in.eof()) + if (in.eof()) { eof = true; return {}; } - return readRowAndGetNamesAndDataTypesForFormRow(in, format_settings); } diff --git a/src/Processors/Formats/Impl/FormInputFormat.h b/src/Processors/Formats/Impl/FormInputFormat.h index 58fe43e7eef..212567a3273 100644 --- a/src/Processors/Formats/Impl/FormInputFormat.h +++ b/src/Processors/Formats/Impl/FormInputFormat.h @@ -3,8 +3,9 @@ #include #include #include +#include -namespace DB +namespace DB { class ReadBuffer; @@ -24,11 +25,10 @@ private: void readField(size_t index, MutableColumns & columns); void skipUnknownFormField(StringRef name_ref); const String & columnName(size_t i) const; - size_t columnIndex(StringRef name); /// recursively split names separated by '.' into own columns void checkAndSplitIfNested(StringRef column_name); - + String name_buf; /// holds common prefix of nested column names @@ -40,7 +40,8 @@ private: size_t nested_prefix_length = 0; /// Hash table matches field name to position in the block - Block::NameMap name_map; + using NameMap = HashMap; + NameMap name_map; protected: const FormatSettings format_settings; diff --git a/tests/queries/0_stateless/02984_form_format.reference b/tests/queries/0_stateless/02984_form_format.reference index 361f0985b74..ac8ad7a3006 100644 --- a/tests/queries/0_stateless/02984_form_format.reference +++ b/tests/queries/0_stateless/02984_form_format.reference @@ -1,84 +1,2 @@ -Row 1: -────── -c.e: ls7xfkpm -c.tti.m: raf -rt.start: navigation -rt.bmr: 390,11,10 -rt.tstart: 1707076768666 -rt.bstart: 1707076769091 -rt.blstart: 1707076769056 -rt.end: 1707076769078 -t_resp: 296 -t_page: 116 -t_done: 412 -t_other: boomerang|6,boomr_fb|425,boomr_ld|390,boomr_lat|35 -rt.tt: 2685 -rt.obo: 0 -pt.fcp: 407 -nt_nav_st: 1707076768666 -nt_dns_st: 1707076768683 -nt_dns_end: 1707076768684 -nt_con_st: 1707076768684 -nt_con_end: 1707076768850 -nt_req_st: 1707076768850 -nt_res_st: 1707076768962 -nt_res_end: 1707076768962 -nt_domloading: 1707076769040 -nt_domint: 1707076769066 -nt_domcontloaded_st: 1707076769067 -nt_domcontloaded_end: 1707076769068 -nt_domcomp: 1707076769069 -nt_load_st: 1707076769069 -nt_load_end: 1707076769078 -nt_unload_st: 1707076769040 -nt_unload_end: 1707076769041 -nt_ssl_st: 1707076768788 -nt_enc_size: 3209 -nt_dec_size: 10093 -nt_trn_size: 3940 -nt_protocol: h2 -nt_red_cnt: 0 -nt_nav_type: 1 -restiming: {"https://www.basicrum.com/":{"publications/":"6,88,88,54,54,3e,i,i,h*12h5,kb,5b8","assets/js/":{"just-the-docs.js":"3am,e,e*12pc,_,8oj*20","boomerang-1.737.60.cutting-edge.min.js":"2au,b,a*1pu3,_,1m19*21*42","vendor/lunr.min.js":"3am,d,8*16t2,_,fym*20"}}} -u: https://www.basicrum.com/publications/ -r: https://www.basicrum.com/cost-analyses/ -v: 1.737.60 -sv: 14 -sm: p -rt.si: dd0c542f-7adf-4310-830a-6c0a3d157c90-s8cjr1 -rt.ss: 1707075325294 -rt.sl: 4 -vis.st: visible -ua.plt: Linux x86_64 -ua.vnd: -pid: 8fftz949 -n: 1 -c.t.fps: 07*4*65*j*61 -c.t.busy: 2*4*0034 -c.tti.vr: 408 -c.tti: 408 -c.b: 2 -c.f: 60 -c.f.d: 2511 -c.f.m: 1 -c.f.s: ls7xfl1h -dom.res: 5 -dom.doms: 1 -mem.lsln: 0 -mem.ssln: 0 -mem.lssz: 2 -mem.sssz: 2 -scr.xy: 1920x1200 -scr.bpp: 24/24 -scr.orn: 0/landscape-primary -cpu.cnc: 16 -dom.ln: 114 -dom.sz: 10438 -dom.ck: 157 -dom.img: 0 -dom.script: 6 -dom.script.ext: 3 -dom.iframe: 0 -dom.link: 4 -dom.link.css: 1 -sb: 1 +42 Hello, World! +Hello, World! diff --git a/tests/queries/0_stateless/02984_form_format.sh b/tests/queries/0_stateless/02984_form_format.sh index 38cd7190a05..2b0742db556 100755 --- a/tests/queries/0_stateless/02984_form_format.sh +++ b/tests/queries/0_stateless/02984_form_format.sh @@ -7,15 +7,17 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # Test setup # USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') -USER_FILES_PATH="/home/shaun/Desktop/ClickHouse/programs/server/user_files" +USER_FILES_PATH="/home/shaun/Desktop/ClickHouse/user_files" FILE_NAME="data.tmp" FORM_DATA="${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/${FILE_NAME}" mkdir -p ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ touch $FORM_DATA +echo -ne "col1=42&col2=Hello%2C%20World%21" > $FORM_DATA + # Simple tests -echo "col1=42&col2=Hello%2C%20World%21" > $FORM_DATA $CLICKHOUSE_CLIENT -q "SELECT * from file('$FORM_DATA', Form, 'col1 UInt64, col2 String')" +$CLICKHOUSE_CLIENT -q "SELECT * from file('$FORM_DATA', Form, 'col2 String')" # Test teardown rm -r ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME} \ No newline at end of file From 3b43379dd22fe5b6905d04c5b9ef4bb65e29e2a7 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Thu, 14 Mar 2024 10:19:37 +0000 Subject: [PATCH 0053/1018] fix count(*) FILTER Signed-off-by: Duc Canh Le --- src/Parsers/ExpressionElementParsers.cpp | 9 +++++++++ .../0_stateless/03003_count_asterisk_filter.reference | 1 + .../queries/0_stateless/03003_count_asterisk_filter.sql | 7 +++++++ 3 files changed, 17 insertions(+) create mode 100644 tests/queries/0_stateless/03003_count_asterisk_filter.reference create mode 100644 tests/queries/0_stateless/03003_count_asterisk_filter.sql diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 486555ae86d..cb9f16d7b53 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -355,6 +355,15 @@ bool ParserFilterClause::parseImpl(Pos & pos, ASTPtr & node, Expected & expected return false; } + if (function.name == "count") + { + /// Remove child from function.arguments if it's '*' because countIf(*) is not supported. + /// See https://github.com/ClickHouse/ClickHouse/issues/61004 + std::erase_if(function.arguments->children, [](const ASTPtr & child) { + return typeid_cast(child.get()); + }); + } + function.name += "If"; function.arguments->children.push_back(condition->children[0]); return true; diff --git a/tests/queries/0_stateless/03003_count_asterisk_filter.reference b/tests/queries/0_stateless/03003_count_asterisk_filter.reference new file mode 100644 index 00000000000..0cfbf08886f --- /dev/null +++ b/tests/queries/0_stateless/03003_count_asterisk_filter.reference @@ -0,0 +1 @@ +2 diff --git a/tests/queries/0_stateless/03003_count_asterisk_filter.sql b/tests/queries/0_stateless/03003_count_asterisk_filter.sql new file mode 100644 index 00000000000..940f511e86d --- /dev/null +++ b/tests/queries/0_stateless/03003_count_asterisk_filter.sql @@ -0,0 +1,7 @@ +CREATE TABLE users (uid Int16, name String, age Int16) ENGINE=Memory; + +INSERT INTO users VALUES (1231, 'John', 33); +INSERT INTO users VALUES (6666, 'Ksenia', 48); +INSERT INTO users VALUES (8888, 'Alice', 50); + +SELECT count(*) FILTER (WHERE uid > 2000) FROM users; From ac4053e3233807f941915c5bbbaa803ff42de203 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 14 Mar 2024 14:56:12 +0300 Subject: [PATCH 0054/1018] Update 03003_count_asterisk_filter.sql --- tests/queries/0_stateless/03003_count_asterisk_filter.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03003_count_asterisk_filter.sql b/tests/queries/0_stateless/03003_count_asterisk_filter.sql index 940f511e86d..c256b58851f 100644 --- a/tests/queries/0_stateless/03003_count_asterisk_filter.sql +++ b/tests/queries/0_stateless/03003_count_asterisk_filter.sql @@ -5,3 +5,5 @@ INSERT INTO users VALUES (6666, 'Ksenia', 48); INSERT INTO users VALUES (8888, 'Alice', 50); SELECT count(*) FILTER (WHERE uid > 2000) FROM users; + +DROP TABLE users; From 6a41dbca0bbaae258ee034e271293479a793df9b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 14 Mar 2024 13:08:01 +0100 Subject: [PATCH 0055/1018] Fix Apple's Macintosh OS X 'darwin' --- src/Common/waitForPid.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/waitForPid.cpp b/src/Common/waitForPid.cpp index 894406f69b5..5fb2c62415b 100644 --- a/src/Common/waitForPid.cpp +++ b/src/Common/waitForPid.cpp @@ -132,7 +132,7 @@ static PollPidResult pollPid(pid_t pid, int timeout_in_ms) if (kq == -1) return PollPidResult::FAILED; - struct kevent change = {.ident = 0}; + struct kevent change{}; EV_SET(&change, pid, EVFILT_PROC, EV_ADD, NOTE_EXIT, 0, NULL); int event_add_result = HANDLE_EINTR(kevent(kq, &change, 1, NULL, 0, NULL)); From 53ad605aa0f6d84a67982c32fc519ee2a5adc077 Mon Sep 17 00:00:00 2001 From: Blargian Date: Fri, 15 Mar 2024 19:06:37 +0100 Subject: [PATCH 0056/1018] Add test for schema reader and remove unneeded code --- .../Formats/Impl/FormInputFormat.cpp | 22 +------------------ src/Processors/Formats/Impl/FormInputFormat.h | 8 ------- .../queries/0_stateless/02984_form_format.sh | 16 +++++++++++--- 3 files changed, 14 insertions(+), 32 deletions(-) diff --git a/src/Processors/Formats/Impl/FormInputFormat.cpp b/src/Processors/Formats/Impl/FormInputFormat.cpp index 565ba66bc9f..0a90bf1da1c 100644 --- a/src/Processors/Formats/Impl/FormInputFormat.cpp +++ b/src/Processors/Formats/Impl/FormInputFormat.cpp @@ -1,16 +1,6 @@ -#include #include -#include -#include "Core/NamesAndTypes.h" -#include "DataTypes/IDataType.h" #include "Formats/EscapingRuleUtils.h" -#include "Formats/FormatSettings.h" -#include -#include -#include "Processors/Formats/IRowInputFormat.h" -#include "base/find_symbols.h" #include -#include namespace DB { @@ -142,15 +132,6 @@ void FormInputFormat::readFormData(MutableColumns & columns) } } -void FormInputFormat::readNestedFormData(const String & name, MutableColumns & columns) -{ - current_column_name = name; - current_column_name.push_back('.'); - nested_prefix_length = current_column_name.size(); - readFormData(columns); - nested_prefix_length = 0; -} - bool FormInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext) { if (in->eof()) @@ -181,7 +162,6 @@ bool FormInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext) void FormInputFormat::resetParser() { IRowInputFormat::resetParser(); - nested_prefix_length = 0; read_columns.clear(); seen_columns.clear(); } @@ -199,7 +179,7 @@ NamesAndTypesList readRowAndGetNamesAndDataTypesForFormRow(ReadBuffer & in, cons { auto name = readFieldName(in); readStringUntilAmpersand(value,in); - auto type = tryInferDataTypeByEscapingRule(value, settings, FormatSettings::EscapingRule::Escaped); + auto type = tryInferDataTypeByEscapingRule(value, settings, FormatSettings::EscapingRule::Raw); names_and_types.emplace_back(name, type); } while (checkChar('&',in)); diff --git a/src/Processors/Formats/Impl/FormInputFormat.h b/src/Processors/Formats/Impl/FormInputFormat.h index 212567a3273..e6275545b5e 100644 --- a/src/Processors/Formats/Impl/FormInputFormat.h +++ b/src/Processors/Formats/Impl/FormInputFormat.h @@ -26,19 +26,11 @@ private: void skipUnknownFormField(StringRef name_ref); const String & columnName(size_t i) const; - /// recursively split names separated by '.' into own columns - void checkAndSplitIfNested(StringRef column_name); - String name_buf; /// holds common prefix of nested column names String current_column_name; - /// holds length of common prefix of nested column names - /// eg: given 'n.a', 'n.b' -> 'n.a' and 'n.b' are nested - /// column names and 'n.' is the common prefix. - size_t nested_prefix_length = 0; - /// Hash table matches field name to position in the block using NameMap = HashMap; NameMap name_map; diff --git a/tests/queries/0_stateless/02984_form_format.sh b/tests/queries/0_stateless/02984_form_format.sh index 2b0742db556..46c607f8fcf 100755 --- a/tests/queries/0_stateless/02984_form_format.sh +++ b/tests/queries/0_stateless/02984_form_format.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # Test setup # USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') -USER_FILES_PATH="/home/shaun/Desktop/ClickHouse/user_files" +USER_FILES_PATH="/home/shaun/Desktop/ClickHouse/programs/server/user_files" FILE_NAME="data.tmp" FORM_DATA="${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/${FILE_NAME}" mkdir -p ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ @@ -19,5 +19,15 @@ echo -ne "col1=42&col2=Hello%2C%20World%21" > $FORM_DATA $CLICKHOUSE_CLIENT -q "SELECT * from file('$FORM_DATA', Form, 'col1 UInt64, col2 String')" $CLICKHOUSE_CLIENT -q "SELECT * from file('$FORM_DATA', Form, 'col2 String')" -# Test teardown -rm -r ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME} \ No newline at end of file +rm $FORM_DATA + +touch $FORM_DATA +echo -ne "col1=42&col2=Hello%2C%20World%21&col3=%5B1%2C%202%2C%203%5D" > $FORM_DATA +$CLICKHOUSE_CLIENT -q "DESC file('$FORM_DATA', Form)" +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('$FORM_DATA', Form)" + +rm $FORM_DATA + +touch $FORM_DATA +echo -ne "c.e=ls7xfkpm&c.tti.m=raf&rt.start=navigation&rt.bmr=390%2C11%2C10&rt.tstart=1707076768666&rt.bstart=1707076769091&rt.blstart=1707076769056&rt.end=1707076769078&t_resp=296&t_page=116&t_done=412&t_other=boomerang%7C6%2Cboomr_fb%7C425%2Cboomr_ld%7C390%2Cboomr_lat%7C35&rt.tt=2685&rt.obo=0&pt.fcp=407&nt_nav_st=1707076768666&nt_dns_st=1707076768683&nt_dns_end=1707076768684&nt_con_st=1707076768684&nt_con_end=1707076768850&nt_req_st=1707076768850&nt_res_st=1707076768962&nt_res_end=1707076768962&nt_domloading=1707076769040&nt_domint=1707076769066&nt_domcontloaded_st=1707076769067&nt_domcontloaded_end=1707076769068&nt_domcomp=1707076769069&nt_load_st=1707076769069&nt_load_end=1707076769078&nt_unload_st=1707076769040&nt_unload_end=1707076769041&nt_ssl_st=1707076768788&nt_enc_size=3209&nt_dec_size=10093&nt_trn_size=3940&nt_protocol=h2&nt_red_cnt=0&nt_nav_type=1&restiming=%7B%22https%3A%2F%2Fwww.basicrum.com%2F%22%3A%7B%22publications%2F%22%3A%226%2C88%2C88%2C54%2C54%2C3e%2Ci%2Ci%2Ch*12h5%2Ckb%2C5b8%22%2C%22assets%2Fjs%2F%22%3A%7B%22just-the-docs.js%22%3A%223am%2Ce%2Ce*12pc%2C_%2C8oj*20%22%2C%22boomerang-1.737.60.cutting-edge.min.js%22%3A%222au%2Cb%2Ca*1pu3%2C_%2C1m19*21*42%22%2C%22vendor%2Flunr.min.js%22%3A%223am%2Cd%2C8*16t2%2C_%2Cfym*20%22%7D%7D%7D&u=https%3A%2F%2Fwww.basicrum.com%2Fpublications%2F&r=https%3A%2F%2Fwww.basicrum.com%2Fcost-analyses%2F&v=1.737.60&sv=14&sm=p&rt.si=dd0c542f-7adf-4310-830a-6c0a3d157c90-s8cjr1&rt.ss=1707075325294&rt.sl=4&vis.st=visible&ua.plt=Linux%20x86_64&ua.vnd=&pid=8fftz949&n=1&c.t.fps=07*4*65*j*61&c.t.busy=2*4*0034&c.tti.vr=408&c.tti=408&c.b=2&c.f=60&c.f.d=2511&c.f.m=1&c.f.s=ls7xfl1h&dom.res=5&dom.doms=1&mem.lsln=0&mem.ssln=0&mem.lssz=2&mem.sssz=2&scr.xy=1920x1200&scr.bpp=24%2F24&scr.orn=0%2Flandscape-primary&cpu.cnc=16&dom.ln=114&dom.sz=10438&dom.ck=157&dom.img=0&dom.script=6&dom.script.ext=3&dom.iframe=0&dom.link=4&dom.link.css=1&sb=1" > $FORM_DATA +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('$FORM_DATA', Form) FORMAT Vertical" \ No newline at end of file From aa06bd8a7a2b82c5df79b0942271fc7d12be2735 Mon Sep 17 00:00:00 2001 From: Blargian Date: Fri, 15 Mar 2024 21:10:37 +0100 Subject: [PATCH 0057/1018] Update test and reference file --- .../0_stateless/02984_form_format.reference | 88 +++++++++++++++++++ .../queries/0_stateless/02984_form_format.sh | 14 +-- 2 files changed, 95 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/02984_form_format.reference b/tests/queries/0_stateless/02984_form_format.reference index ac8ad7a3006..34612dbbb50 100644 --- a/tests/queries/0_stateless/02984_form_format.reference +++ b/tests/queries/0_stateless/02984_form_format.reference @@ -1,2 +1,90 @@ 42 Hello, World! Hello, World! +col1 Nullable(Int64) +col2 Nullable(String) +col3 Nullable(String) +42 Hello, World! [1, 2, 3] +Row 1: +────── +c.e: ls7xfkpm +c.tti.m: raf +rt.start: navigation +rt.bmr: 390,11,10 +rt.tstart: 1707076768666 +rt.bstart: 1707076769091 +rt.blstart: 1707076769056 +rt.end: 1707076769078 +t_resp: 296 +t_page: 116 +t_done: 412 +t_other: boomerang|6,boomr_fb|425,boomr_ld|390,boomr_lat|35 +rt.tt: 2685 +rt.obo: 0 +pt.fcp: 407 +nt_nav_st: 1707076768666 +nt_dns_st: 1707076768683 +nt_dns_end: 1707076768684 +nt_con_st: 1707076768684 +nt_con_end: 1707076768850 +nt_req_st: 1707076768850 +nt_res_st: 1707076768962 +nt_res_end: 1707076768962 +nt_domloading: 1707076769040 +nt_domint: 1707076769066 +nt_domcontloaded_st: 1707076769067 +nt_domcontloaded_end: 1707076769068 +nt_domcomp: 1707076769069 +nt_load_st: 1707076769069 +nt_load_end: 1707076769078 +nt_unload_st: 1707076769040 +nt_unload_end: 1707076769041 +nt_ssl_st: 1707076768788 +nt_enc_size: 3209 +nt_dec_size: 10093 +nt_trn_size: 3940 +nt_protocol: h2 +nt_red_cnt: 0 +nt_nav_type: 1 +restiming: {"https://www.basicrum.com/":{"publications/":"6,88,88,54,54,3e,i,i,h*12h5,kb,5b8","assets/js/":{"just-the-docs.js":"3am,e,e*12pc,_,8oj*20","boomerang-1.737.60.cutting-edge.min.js":"2au,b,a*1pu3,_,1m19*21*42","vendor/lunr.min.js":"3am,d,8*16t2,_,fym*20"}}} +u: https://www.basicrum.com/publications/ +r: https://www.basicrum.com/cost-analyses/ +v: 1.737.60 +sv: 14 +sm: p +rt.si: dd0c542f-7adf-4310-830a-6c0a3d157c90-s8cjr1 +rt.ss: 1707075325294 +rt.sl: 4 +vis.st: visible +ua.plt: Linux x86_64 +ua.vnd: +pid: 8fftz949 +n: 1 +c.t.fps: 07*4*65*j*61 +c.t.busy: 2*4*0034 +c.tti.vr: 408 +c.tti: 408 +c.b: 2 +c.f: 60 +c.f.d: 2511 +c.f.m: 1 +c.f.s: ls7xfl1h +dom.res: 5 +dom.doms: 1 +mem.lsln: 0 +mem.ssln: 0 +mem.lssz: 2 +mem.sssz: 2 +scr.xy: 1920x1200 +scr.bpp: 24/24 +scr.orn: 0/landscape-primary +cpu.cnc: 16 +dom.ln: 114 +dom.sz: 10438 +dom.ck: 157 +dom.img: 0 +dom.script: 6 +dom.script.ext: 3 +dom.iframe: 0 +dom.link: 4 +dom.link.css: 1 +sb: 1 diff --git a/tests/queries/0_stateless/02984_form_format.sh b/tests/queries/0_stateless/02984_form_format.sh index 46c607f8fcf..ce5feb60130 100755 --- a/tests/queries/0_stateless/02984_form_format.sh +++ b/tests/queries/0_stateless/02984_form_format.sh @@ -6,28 +6,28 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh # Test setup -# USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') -USER_FILES_PATH="/home/shaun/Desktop/ClickHouse/programs/server/user_files" +USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') FILE_NAME="data.tmp" FORM_DATA="${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/${FILE_NAME}" mkdir -p ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ touch $FORM_DATA -echo -ne "col1=42&col2=Hello%2C%20World%21" > $FORM_DATA - # Simple tests +echo -ne "col1=42&col2=Hello%2C%20World%21" > $FORM_DATA $CLICKHOUSE_CLIENT -q "SELECT * from file('$FORM_DATA', Form, 'col1 UInt64, col2 String')" $CLICKHOUSE_CLIENT -q "SELECT * from file('$FORM_DATA', Form, 'col2 String')" - rm $FORM_DATA +# Schema reader test touch $FORM_DATA echo -ne "col1=42&col2=Hello%2C%20World%21&col3=%5B1%2C%202%2C%203%5D" > $FORM_DATA $CLICKHOUSE_CLIENT -q "DESC file('$FORM_DATA', Form)" $CLICKHOUSE_CLIENT -q "SELECT * FROM file('$FORM_DATA', Form)" - rm $FORM_DATA +# Test with data-raw from request touch $FORM_DATA echo -ne "c.e=ls7xfkpm&c.tti.m=raf&rt.start=navigation&rt.bmr=390%2C11%2C10&rt.tstart=1707076768666&rt.bstart=1707076769091&rt.blstart=1707076769056&rt.end=1707076769078&t_resp=296&t_page=116&t_done=412&t_other=boomerang%7C6%2Cboomr_fb%7C425%2Cboomr_ld%7C390%2Cboomr_lat%7C35&rt.tt=2685&rt.obo=0&pt.fcp=407&nt_nav_st=1707076768666&nt_dns_st=1707076768683&nt_dns_end=1707076768684&nt_con_st=1707076768684&nt_con_end=1707076768850&nt_req_st=1707076768850&nt_res_st=1707076768962&nt_res_end=1707076768962&nt_domloading=1707076769040&nt_domint=1707076769066&nt_domcontloaded_st=1707076769067&nt_domcontloaded_end=1707076769068&nt_domcomp=1707076769069&nt_load_st=1707076769069&nt_load_end=1707076769078&nt_unload_st=1707076769040&nt_unload_end=1707076769041&nt_ssl_st=1707076768788&nt_enc_size=3209&nt_dec_size=10093&nt_trn_size=3940&nt_protocol=h2&nt_red_cnt=0&nt_nav_type=1&restiming=%7B%22https%3A%2F%2Fwww.basicrum.com%2F%22%3A%7B%22publications%2F%22%3A%226%2C88%2C88%2C54%2C54%2C3e%2Ci%2Ci%2Ch*12h5%2Ckb%2C5b8%22%2C%22assets%2Fjs%2F%22%3A%7B%22just-the-docs.js%22%3A%223am%2Ce%2Ce*12pc%2C_%2C8oj*20%22%2C%22boomerang-1.737.60.cutting-edge.min.js%22%3A%222au%2Cb%2Ca*1pu3%2C_%2C1m19*21*42%22%2C%22vendor%2Flunr.min.js%22%3A%223am%2Cd%2C8*16t2%2C_%2Cfym*20%22%7D%7D%7D&u=https%3A%2F%2Fwww.basicrum.com%2Fpublications%2F&r=https%3A%2F%2Fwww.basicrum.com%2Fcost-analyses%2F&v=1.737.60&sv=14&sm=p&rt.si=dd0c542f-7adf-4310-830a-6c0a3d157c90-s8cjr1&rt.ss=1707075325294&rt.sl=4&vis.st=visible&ua.plt=Linux%20x86_64&ua.vnd=&pid=8fftz949&n=1&c.t.fps=07*4*65*j*61&c.t.busy=2*4*0034&c.tti.vr=408&c.tti=408&c.b=2&c.f=60&c.f.d=2511&c.f.m=1&c.f.s=ls7xfl1h&dom.res=5&dom.doms=1&mem.lsln=0&mem.ssln=0&mem.lssz=2&mem.sssz=2&scr.xy=1920x1200&scr.bpp=24%2F24&scr.orn=0%2Flandscape-primary&cpu.cnc=16&dom.ln=114&dom.sz=10438&dom.ck=157&dom.img=0&dom.script=6&dom.script.ext=3&dom.iframe=0&dom.link=4&dom.link.css=1&sb=1" > $FORM_DATA -$CLICKHOUSE_CLIENT -q "SELECT * FROM file('$FORM_DATA', Form) FORMAT Vertical" \ No newline at end of file +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('$FORM_DATA', Form) FORMAT Vertical" + +rm $FORM_DATA \ No newline at end of file From 8a4d4641ddb70ba42104ca30dd45a2b64300a556 Mon Sep 17 00:00:00 2001 From: Blargian Date: Fri, 15 Mar 2024 21:22:02 +0100 Subject: [PATCH 0058/1018] Update documentation with an example --- docs/en/interfaces/formats.md | 21 +++++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 2131104932f..77b5f242aed 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -2848,5 +2848,26 @@ Markdown table will be generated automatically and can be used on markdown-enabl The Form format can be used to read or write a single record in the application/x-www-form-urlencoded format in which data is formatted `key1=value1&key2=value2` +Examples: + +Given a file `data.tmp` placed in the `user_files` path with some url-encoded data: + +```text +t_page=116&c.e=ls7xfkpm&c.tti.m=raf&rt.start=navigation&rt.bmr=390%2C11%2C10 +``` + ```sql +SELECT * FROM file(data.tmp, Form) FORMAT vertical; +``` + +Result: + +```text +Row 1: +────── +t_page: 116 +c.e: ls7xfkpm +c.tti.m: raf +rt.start: navigation +rt.bmr: 390,11,10 ``` \ No newline at end of file From b0131a278bd13001f1d54413b4bc781eab8882a6 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 30 Jan 2024 21:38:07 +0100 Subject: [PATCH 0059/1018] impl --- .../MergeTree/MergeTreeDataWriter.cpp | 27 +++++++------------ src/Storages/MergeTree/MergeTreeDataWriter.h | 6 +++-- 2 files changed, 13 insertions(+), 20 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index cadd94867ec..ce11a535e1c 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -618,7 +618,8 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( if (projection_block.rows()) { - auto proj_temp_part = writeProjectionPart(data, log, projection_block, projection, new_data_part.get()); + auto proj_temp_part + = writeProjectionPart(data, log, projection_block, projection, new_data_part.get(), /*merge_is_needed=*/false); new_data_part->addProjectionPart(projection.name, std::move(proj_temp_part.part)); for (auto & stream : proj_temp_part.streams) temp_part.streams.emplace_back(std::move(stream)); @@ -647,7 +648,8 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( const MergeTreeData & data, LoggerPtr log, Block block, - const ProjectionDescription & projection) + const ProjectionDescription & projection, + bool merge_is_needed) { TemporaryPart temp_part; const auto & metadata_snapshot = projection.metadata; @@ -716,7 +718,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( ProfileEvents::increment(ProfileEvents::MergeTreeDataProjectionWriterBlocksAlreadySorted); } - if (projection.type == ProjectionDescription::Type::Aggregate) + if (projection.type == ProjectionDescription::Type::Aggregate && merge_is_needed) { ProfileEventTimeIncrement watch(ProfileEvents::MergeTreeDataProjectionWriterMergingBlocksMicroseconds); @@ -756,16 +758,11 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPart( LoggerPtr log, Block block, const ProjectionDescription & projection, - IMergeTreeDataPart * parent_part) + IMergeTreeDataPart * parent_part, + bool merge_is_needed) { return writeProjectionPartImpl( - projection.name, - false /* is_temp */, - parent_part, - data, - log, - std::move(block), - projection); + projection.name, false /* is_temp */, parent_part, data, log, std::move(block), projection, merge_is_needed); } /// This is used for projection materialization process which may contain multiple stages of @@ -780,13 +777,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempProjectionPart( { auto part_name = fmt::format("{}_{}", projection.name, block_num); return writeProjectionPartImpl( - part_name, - true /* is_temp */, - parent_part, - data, - log, - std::move(block), - projection); + part_name, true /* is_temp */, parent_part, data, log, std::move(block), projection, /*merge_is_needed=*/true); } } diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.h b/src/Storages/MergeTree/MergeTreeDataWriter.h index 9dffea0a471..863c951d957 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.h +++ b/src/Storages/MergeTree/MergeTreeDataWriter.h @@ -95,7 +95,8 @@ public: LoggerPtr log, Block block, const ProjectionDescription & projection, - IMergeTreeDataPart * parent_part); + IMergeTreeDataPart * parent_part, + bool merge_is_needed); /// For mutation: MATERIALIZE PROJECTION. static TemporaryPart writeTempProjectionPart( @@ -129,7 +130,8 @@ private: const MergeTreeData & data, LoggerPtr log, Block block, - const ProjectionDescription & projection); + const ProjectionDescription & projection, + bool merge_is_needed); MergeTreeData & data; LoggerPtr log; From a2bb7d2141dd8f0f2f9739ff837fa4484d7f0af5 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Mon, 18 Mar 2024 18:22:23 +0100 Subject: [PATCH 0060/1018] Update aspell-dict.txt Add `URL-encoded` to aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index c7eb213bff2..f6fb46dbf3f 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -972,6 +972,7 @@ UNDROP UPDATEs URIs URL's +URL-encoded URLHash URLHierarchy URLPathHierarchy From f4c33b85fcd494213d8b6452dd63ff51ef56804c Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Mon, 18 Mar 2024 18:23:47 +0100 Subject: [PATCH 0061/1018] Update formats.md Change `urlencoded` to `URL-encoded` --- docs/en/interfaces/formats.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 77b5f242aed..9c94907a9d8 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -2850,7 +2850,7 @@ The Form format can be used to read or write a single record in the application/ Examples: -Given a file `data.tmp` placed in the `user_files` path with some url-encoded data: +Given a file `data.tmp` placed in the `user_files` path with some URL-encoded data: ```text t_page=116&c.e=ls7xfkpm&c.tti.m=raf&rt.start=navigation&rt.bmr=390%2C11%2C10 @@ -2870,4 +2870,4 @@ c.e: ls7xfkpm c.tti.m: raf rt.start: navigation rt.bmr: 390,11,10 -``` \ No newline at end of file +``` From a8689db92295689d111bc0cecaee8dd4e544b7bd Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 19 Mar 2024 14:45:03 +0000 Subject: [PATCH 0062/1018] try fix style Signed-off-by: Duc Canh Le --- src/Parsers/ExpressionElementParsers.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index cb9f16d7b53..27cf51db5f8 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -359,7 +359,8 @@ bool ParserFilterClause::parseImpl(Pos & pos, ASTPtr & node, Expected & expected { /// Remove child from function.arguments if it's '*' because countIf(*) is not supported. /// See https://github.com/ClickHouse/ClickHouse/issues/61004 - std::erase_if(function.arguments->children, [](const ASTPtr & child) { + std::erase_if(function.arguments->children, [] (const ASTPtr & child) + { return typeid_cast(child.get()); }); } From 02ff01f2468b36479bd40abe23138bb28a4071b6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 19 Mar 2024 16:48:42 +0100 Subject: [PATCH 0063/1018] Fix build --- src/Interpreters/Cache/EvictionCandidates.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/EvictionCandidates.h b/src/Interpreters/Cache/EvictionCandidates.h index 74872618543..b786480aa69 100644 --- a/src/Interpreters/Cache/EvictionCandidates.h +++ b/src/Interpreters/Cache/EvictionCandidates.h @@ -19,7 +19,7 @@ public: void add(const FileSegmentMetadataPtr & candidate, LockedKey & locked_key, const CachePriorityGuard::Lock &); - void add(const EvictionCandidates & other, const CacheGuard::Lock &) { candidates.insert(other.candidates.begin(), other.candidates.end()); } + void add(const EvictionCandidates & other, const CachePriorityGuard::Lock &) { candidates.insert(other.candidates.begin(), other.candidates.end()); } void evict(); From fcb34995e92f2b2a41638e9c1f376693fd86c9d1 Mon Sep 17 00:00:00 2001 From: johnnymatthews <9611008+johnnymatthews@users.noreply.github.com> Date: Thu, 21 Mar 2024 22:14:41 -0300 Subject: [PATCH 0064/1018] Adds connectionId and connection_id function docs. --- .../functions/other-functions.md | 68 +++++++++++++++++++ 1 file changed, 68 insertions(+) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index e7fca31483a..79df2e1ae27 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -3132,3 +3132,71 @@ Result: │ (616.2931945826209,108.8825,115.6175) │ └───────────────────────────────────────┘ ``` + +## connectionId + +Retrieves the connection ID of the client that submitted the current query and returns it as a UInt64 integer. + +**Syntax** + +```sql +connectionId() +``` + +**Parameters** + +None. + +**Returned value** + +Returns an integer of type UInt64. + +**Implementation details** + +This function is most useful in debugging scenarios or for internal purposes within the MySQL handler. It was created for compatibility with [MySQL's `CONNECTION_ID` function](https://dev.mysql.com/doc/refman/8.0/en/information-functions.html#function_connection-id) It is not typically used in production queries. + +**Example** + +Query: + +```sql +SELECT connectionId(); +``` + +```response +0 +``` + +## connection_id + +An alias of `connectionId`. Retrieves the connection ID of the client that submitted the current query and returns it as a UInt64 integer. + +**Syntax** + +```sql +connection_id() +``` + +**Parameters** + +None. + +**Returned value** + +Returns an integer of type UInt64. + +**Implementation details** + +This function is most useful in debugging scenarios or for internal purposes within the MySQL handler. It was created for compatibility with [MySQL's `CONNECTION_ID` function](https://dev.mysql.com/doc/refman/8.0/en/information-functions.html#function_connection-id) It is not typically used in production queries. + +**Example** + +Query: + +```sql +SELECT connection_id(); +``` + +```response +0 +``` From 51ff688f9f9d3c6d8d848130531079ca3a923ac5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=9A=D0=B8=D1=80=D0=B8=D0=BB=D0=BB=20=D0=93=D0=B0=D1=80?= =?UTF-8?q?=D0=B1=D0=B0=D1=80?= Date: Fri, 22 Mar 2024 16:04:10 +0300 Subject: [PATCH 0065/1018] Use storage policy to find convert_to_replicated flag path --- src/Databases/DatabaseOrdinary.cpp | 23 +++- src/Databases/DatabaseOrdinary.h | 2 +- .../test_modify_engine_on_restart/common.py | 10 +- .../configs/config.d/storage_policies.xml | 42 ++++++++ .../test_storage_policies.py | 102 ++++++++++++++++++ 5 files changed, 170 insertions(+), 9 deletions(-) create mode 100644 tests/integration/test_modify_engine_on_restart/configs/config.d/storage_policies.xml create mode 100644 tests/integration/test_modify_engine_on_restart/test_storage_policies.py diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 95bdcfc7dce..65d5f21bbea 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -31,6 +31,7 @@ #include #include #include +#include #include @@ -95,16 +96,21 @@ static void setReplicatedEngine(ASTCreateQuery * create_query, ContextPtr contex create_query->storage->set(create_query->storage->engine, engine->clone()); } -String DatabaseOrdinary::getConvertToReplicatedFlagPath(const String & name, bool tableStarted) +String DatabaseOrdinary::getConvertToReplicatedFlagPath(const String & name, const StoragePolicyPtr storage_policy, bool tableStarted) { fs::path data_path; + if (storage_policy->getDisks().empty()) + data_path = getContext()->getPath(); + else + data_path = storage_policy->getDisks()[0]->getPath(); + if (!tableStarted) { auto create_query = tryGetCreateTableQuery(name, getContext()); - data_path = fs::path(getContext()->getPath()) / getTableDataPath(create_query->as()); + data_path = data_path / getTableDataPath(create_query->as()); } else - data_path = fs::path(getContext()->getPath()) / getTableDataPath(name); + data_path = data_path / getTableDataPath(name); return (data_path / CONVERT_TO_REPLICATED_FLAG_NAME).string(); } @@ -120,7 +126,14 @@ void DatabaseOrdinary::convertMergeTreeToReplicatedIfNeeded(ASTPtr ast, const Qu if (!create_query->storage || !create_query->storage->engine->name.ends_with("MergeTree") || create_query->storage->engine->name.starts_with("Replicated") || create_query->storage->engine->name.starts_with("Shared")) return; - auto convert_to_replicated_flag_path = getConvertToReplicatedFlagPath(qualified_name.table, false); + /// Get table's storage policy + MergeTreeSettings default_settings = getContext()->getMergeTreeSettings(); + auto * query_settings = create_query->storage->settings->as(); + auto policy = getContext()->getStoragePolicy(default_settings.storage_policy); + if (Field * policy_setting = query_settings->changes.tryGet("storage_policy")) + policy = getContext()->getStoragePolicy(policy_setting->get()); + + auto convert_to_replicated_flag_path = getConvertToReplicatedFlagPath(qualified_name.table, policy, false); if (!fs::exists(convert_to_replicated_flag_path)) return; @@ -288,7 +301,7 @@ void DatabaseOrdinary::restoreMetadataAfterConvertingToReplicated(StoragePtr tab if (!rmt) return; - auto convert_to_replicated_flag_path = getConvertToReplicatedFlagPath(name.table, true); + auto convert_to_replicated_flag_path = getConvertToReplicatedFlagPath(name.table, table->getStoragePolicy(), true); if (!fs::exists(convert_to_replicated_flag_path)) return; diff --git a/src/Databases/DatabaseOrdinary.h b/src/Databases/DatabaseOrdinary.h index 7089540337a..9e4bd035576 100644 --- a/src/Databases/DatabaseOrdinary.h +++ b/src/Databases/DatabaseOrdinary.h @@ -85,7 +85,7 @@ protected: private: void convertMergeTreeToReplicatedIfNeeded(ASTPtr ast, const QualifiedTableName & qualified_name, const String & file_name); void restoreMetadataAfterConvertingToReplicated(StoragePtr table, const QualifiedTableName & name); - String getConvertToReplicatedFlagPath(const String & name, bool tableStarted); + String getConvertToReplicatedFlagPath(const String & name, StoragePolicyPtr storage_policy, bool tableStarted); }; } diff --git a/tests/integration/test_modify_engine_on_restart/common.py b/tests/integration/test_modify_engine_on_restart/common.py index 18b6c2dc1d7..3d8529d70ce 100644 --- a/tests/integration/test_modify_engine_on_restart/common.py +++ b/tests/integration/test_modify_engine_on_restart/common.py @@ -2,9 +2,13 @@ from helpers.cluster import ClickHouseCluster def get_table_path(node, table, database): - return node.query( - sql=f"SELECT data_paths FROM system.tables WHERE table = '{table}' and database = '{database}'" - ).strip("'[]\n") + return ( + node.query( + sql=f"SELECT data_paths FROM system.tables WHERE table = '{table}' and database = '{database}' LIMIT 1" + ) + .split(",")[0] + .strip("'[]\n") + ) def check_flags_deleted(node, database_name, tables): diff --git a/tests/integration/test_modify_engine_on_restart/configs/config.d/storage_policies.xml b/tests/integration/test_modify_engine_on_restart/configs/config.d/storage_policies.xml new file mode 100644 index 00000000000..c456b5c1f12 --- /dev/null +++ b/tests/integration/test_modify_engine_on_restart/configs/config.d/storage_policies.xml @@ -0,0 +1,42 @@ + + + + + 1024 + + + /jbod1/ + + + /jbod2/ + + + s3 + http://minio1:9001/root/data/ + minio + minio123 + + + + + + + jbod1 + jbod2 + + + + + + + s3 + + + + + + + + jbod + + diff --git a/tests/integration/test_modify_engine_on_restart/test_storage_policies.py b/tests/integration/test_modify_engine_on_restart/test_storage_policies.py new file mode 100644 index 00000000000..e49af164ed7 --- /dev/null +++ b/tests/integration/test_modify_engine_on_restart/test_storage_policies.py @@ -0,0 +1,102 @@ +import pytest +from test_modify_engine_on_restart.common import check_flags_deleted, set_convert_flags +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +ch1 = cluster.add_instance( + "ch1", + main_configs=[ + "configs/config.d/clusters.xml", + "configs/config.d/distributed_ddl.xml", + "configs/config.d/storage_policies.xml", + ], + with_zookeeper=True, + with_minio=True, + macros={"replica": "node1"}, + stay_alive=True, +) + +database_name = "modify_engine_storage_policies" + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def q(node, query): + return node.query(database=database_name, sql=query) + + +def create_tables(): + # Implicit jbod (set default in config) + q( + ch1, + "CREATE TABLE jbod_imp ( A Int64, D Date, S String ) ENGINE MergeTree() PARTITION BY toYYYYMM(D) ORDER BY A;", + ) + + # Explicit jbod + q( + ch1, + """ + CREATE TABLE jbod_exp ( A Int64, D Date, S String ) ENGINE MergeTree() PARTITION BY toYYYYMM(D) ORDER BY A + SETTINGS storage_policy='jbod'; + """, + ) + + # s3 + q( + ch1, + """ + CREATE TABLE s3 ( A Int64, D Date, S String ) ENGINE MergeTree() PARTITION BY toYYYYMM(D) ORDER BY A + SETTINGS storage_policy='s3'; + """, + ) + + # Default + q( + ch1, + """ + CREATE TABLE default ( A Int64, D Date, S String ) ENGINE MergeTree() PARTITION BY toYYYYMM(D) ORDER BY A + SETTINGS storage_policy='default'; + """, + ) + + +def check_tables(converted): + engine_prefix = "" + if converted: + engine_prefix = "Replicated" + + assert ( + q( + ch1, + f"SELECT name, engine FROM system.tables WHERE database = '{database_name}'", + ).strip() + == f"default\t{engine_prefix}MergeTree\njbod_exp\t{engine_prefix}MergeTree\njbod_imp\t{engine_prefix}MergeTree\ns3\t{engine_prefix}MergeTree" + ) + + +def test_modify_engine_on_restart(started_cluster): + ch1.query("CREATE DATABASE " + database_name) + + create_tables() + + check_tables(False) + + ch1.restart_clickhouse() + + check_tables(False) + + set_convert_flags(ch1, database_name, ["default", "jbod_exp", "jbod_imp", "s3"]) + + ch1.restart_clickhouse() + + check_flags_deleted(ch1, database_name, ["default", "jbod_exp", "jbod_imp", "s3"]) + + check_tables(True) From acb623f802e1ceb0e80038609da7781908ea8cc6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=9A=D0=B8=D1=80=D0=B8=D0=BB=D0=BB=20=D0=93=D0=B0=D1=80?= =?UTF-8?q?=D0=B1=D0=B0=D1=80?= Date: Fri, 22 Mar 2024 16:11:23 +0300 Subject: [PATCH 0066/1018] Mention problem in docs --- docs/en/engines/table-engines/mergetree-family/replication.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/replication.md b/docs/en/engines/table-engines/mergetree-family/replication.md index f70e275fd4e..257084d8691 100644 --- a/docs/en/engines/table-engines/mergetree-family/replication.md +++ b/docs/en/engines/table-engines/mergetree-family/replication.md @@ -304,10 +304,10 @@ We use the term `MergeTree` to refer to all table engines in the `MergeTree fami If you had a `MergeTree` table that was manually replicated, you can convert it to a replicated table. You might need to do this if you have already collected a large amount of data in a `MergeTree` table and now you want to enable replication. -`MergeTree` table can be automatically converted on server restart if `convert_to_replicated` flag is set at the table's data directory (`/var/lib/clickhouse/store/xxx/xxxyyyyy-yyyy-yyyy-yyyy-yyyyyyyyyyyy/` for `Atomic` database). +`MergeTree` table can be automatically converted on server restart if `convert_to_replicated` flag is set at the table's data directory (`/store/xxx/xxxyyyyy-yyyy-yyyy-yyyy-yyyyyyyyyyyy/` for `Atomic` database). Create empty `convert_to_replicated` file and the table will be loaded as replicated on next server restart. -This query can be used to get the table's data path. +This query can be used to get the table's data path. It table has many data paths, you have to use the first one. ```sql SELECT data_paths FROM system.tables WHERE table = 'table_name' AND database = 'database_name'; From 0beb74b931bc6e7f946d1d16d6a205025e89faa7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 23 Mar 2024 22:12:17 +0100 Subject: [PATCH 0067/1018] Add required changes --- src/Core/Block.cpp | 11 ++++++----- src/Core/SettingsChangesHistory.h | 1 + 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index 20c103adf65..fd924540113 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -7,7 +7,6 @@ #include #include -#include "Columns/IColumn.h" #include #include @@ -608,16 +607,18 @@ Block Block::shrinkToFit() const Block Block::compress() const { - Columns new_columns(data.size(), nullptr); - for (size_t i = 0; i < data.size(); ++i) + size_t num_columns = data.size(); + Columns new_columns(num_columns); + for (size_t i = 0; i < num_columns; ++i) new_columns[i] = data[i].column->compress(); return cloneWithColumns(new_columns); } Block Block::decompress() const { - Columns new_columns(data.size(), nullptr); - for (size_t i = 0; i < data.size(); ++i) + size_t num_columns = data.size(); + Columns new_columns(num_columns); + for (size_t i = 0; i < num_columns; ++i) new_columns[i] = data[i].column->decompress(); return cloneWithColumns(new_columns); } diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index ee56b86d9f2..759fe68505e 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -104,6 +104,7 @@ static std::map sett {"keeper_retry_initial_backoff_ms", 100, 100, "Initial backoff timeout for general keeper operations"}, {"keeper_retry_max_backoff_ms", 5000, 5000, "Max backoff timeout for general keeper operations"}, {"s3queue_allow_experimental_sharded_mode", false, false, "Enable experimental sharded mode of S3Queue table engine. It is experimental because it will be rewritten"}, + {"cross_join_compress_blocks", false, false, "A new setting."}, {"merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability", 0.0, 0.0, "For testing of `PartsSplitter` - split read ranges into intersecting and non intersecting every time you read from MergeTree with the specified probability."}, }}, {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, From ea2fd57cca95d45e4d86260f273abaca67012896 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Mar 2024 00:42:59 +0100 Subject: [PATCH 0068/1018] Set total_rows_approx for trivial queries with LIMIT from system.zeros and generateRandom --- src/Interpreters/InterpreterSelectQuery.cpp | 3 ++ .../QueryPlan/ReadFromSystemNumbersStep.cpp | 6 +-- .../QueryPlan/ReadFromSystemNumbersStep.h | 2 + src/Storages/StorageGenerateRandom.cpp | 15 ++++-- src/Storages/System/StorageSystemNumbers.cpp | 5 +- src/Storages/System/StorageSystemZeros.cpp | 13 +++-- ...rate_random_with_limit_progress_bar.expect | 49 +++++++++++++++++++ ...e_random_with_limit_progress_bar.reference | 0 ...system_zeros_and_generate_random.reference | 0 ...t_for_system_zeros_and_generate_random.sql | 9 ++++ 10 files changed, 86 insertions(+), 16 deletions(-) create mode 100755 tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect create mode 100644 tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.reference create mode 100644 tests/queries/0_stateless/03024_total_rows_approx_is_set_for_system_zeros_and_generate_random.reference create mode 100644 tests/queries/0_stateless/03024_total_rows_approx_is_set_for_system_zeros_and_generate_random.sql diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 7c87dadfce6..fa46b115979 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2501,10 +2501,13 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc max_block_size = std::max(1, max_block_limited); max_threads_execute_query = max_streams = 1; } + if (local_limits.local_limits.size_limits.max_rows != 0) { if (max_block_limited < local_limits.local_limits.size_limits.max_rows) query_info.limit = max_block_limited; + else /// Ask to read just enough rows to make the max_rows limit effective (so it has a chance to be triggered). + query_info.limit = 1 + local_limits.local_limits.size_limits.max_rows; } else { diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index 7a61d09bdd2..11371578c79 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -12,8 +12,8 @@ #include #include #include -#include "Core/Types.h" -#include "base/types.h" +#include + namespace DB { @@ -443,7 +443,6 @@ Pipe ReadFromSystemNumbersStep::makePipe() /// Build rpn of query filters KeyCondition condition(filter_actions_dag, context, column_names, key_expression); - if (condition.extractPlainRanges(ranges)) { /// Intersect ranges with table range @@ -505,7 +504,6 @@ Pipe ReadFromSystemNumbersStep::makePipe() } } - /// ranges is blank, return a source who has no data if (intersected_ranges.empty()) { diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.h b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.h index cab0686474b..bc84e31be62 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.h +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.h @@ -8,6 +8,7 @@ #include #include + namespace DB { @@ -43,4 +44,5 @@ private: UInt64 limit; std::shared_ptr storage_limits; }; + } diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index fbce6c2bb7d..cdbade51695 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -30,12 +31,9 @@ #include #include #include -#include #include -#include - namespace DB { @@ -639,7 +637,7 @@ void registerStorageGenerateRandom(StorageFactory & factory) Pipe StorageGenerateRandom::read( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & /*query_info*/, + SelectQueryInfo & query_info, ContextPtr context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, @@ -682,7 +680,14 @@ Pipe StorageGenerateRandom::read( pcg64 generate(random_seed); for (UInt64 i = 0; i < num_streams; ++i) - pipes.emplace_back(std::make_shared(max_block_size, max_array_length, max_string_length, generate(), block_header, context)); + { + auto source = std::make_shared(max_block_size, max_array_length, max_string_length, generate(), block_header, context); + + if (i == 0 && query_info.limit) + source->addTotalRowsApprox(query_info.limit); + + pipes.emplace_back(std::move(source)); + } return Pipe::unitePipes(std::move(pipes)); } diff --git a/src/Storages/System/StorageSystemNumbers.cpp b/src/Storages/System/StorageSystemNumbers.cpp index 629b11ee7f1..da700a7a4e9 100644 --- a/src/Storages/System/StorageSystemNumbers.cpp +++ b/src/Storages/System/StorageSystemNumbers.cpp @@ -1,17 +1,14 @@ #include #include -#include #include #include -#include #include #include #include -#include -#include #include + namespace DB { diff --git a/src/Storages/System/StorageSystemZeros.cpp b/src/Storages/System/StorageSystemZeros.cpp index ce2e153ea66..a48b109fbbe 100644 --- a/src/Storages/System/StorageSystemZeros.cpp +++ b/src/Storages/System/StorageSystemZeros.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -6,6 +7,7 @@ #include #include + namespace DB { @@ -93,7 +95,7 @@ StorageSystemZeros::StorageSystemZeros(const StorageID & table_id_, bool multith Pipe StorageSystemZeros::read( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo &, + SelectQueryInfo & query_info, ContextPtr /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, @@ -123,8 +125,13 @@ Pipe StorageSystemZeros::read( { auto source = std::make_shared(max_block_size, limit ? *limit : 0, state); - if (limit && i == 0) - source->addTotalRowsApprox(*limit); + if (i == 0) + { + if (limit) + source->addTotalRowsApprox(*limit); + else if (query_info.limit) + source->addTotalRowsApprox(query_info.limit); + } res.addSource(std::move(source)); } diff --git a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect new file mode 100755 index 00000000000..272dc0fdfef --- /dev/null +++ b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect @@ -0,0 +1,49 @@ +#!/usr/bin/expect -f + +set basedir [file dirname $argv0] +set basename [file tail $argv0] +if {[info exists env(CLICKHOUSE_TMP)]} { + set CLICKHOUSE_TMP $env(CLICKHOUSE_TMP) +} else { + set CLICKHOUSE_TMP "." +} +exp_internal -f $CLICKHOUSE_TMP/$basename.debuglog 0 + +log_user 0 +set timeout 60 +match_max 100000 +set stty_init "rows 25 cols 120" + +expect_after { + -i $any_spawn_id eof { exp_continue } + -i $any_spawn_id timeout { exit 1 } +} + +spawn clickhouse-local +expect ":) " + +# Trivial SELECT with LIMIT from system.zeros shows progress bar. +send "SELECT * FROM system.zeros LIMIT 1e11 FORMAT Null\r" +expect "Progress: " +expect "█" +send "\3" +expect "Query was cancelled." +expect ":) " + +send "SELECT * FROM system.zeros_mt LIMIT 1e11 FORMAT Null\r" +expect "Progress: " +expect "█" +send "\3" +expect "Query was cancelled." +expect ":) " + +# As well as from generateRandom +send "SELECT * FROM generateRandom() LIMIT 1e9 FORMAT Null\r" +expect "Progress: " +expect "█" +send "\3" +expect "Query was cancelled." +expect ":) " + +send "exit\r" +expect eof diff --git a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.reference b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03024_total_rows_approx_is_set_for_system_zeros_and_generate_random.reference b/tests/queries/0_stateless/03024_total_rows_approx_is_set_for_system_zeros_and_generate_random.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03024_total_rows_approx_is_set_for_system_zeros_and_generate_random.sql b/tests/queries/0_stateless/03024_total_rows_approx_is_set_for_system_zeros_and_generate_random.sql new file mode 100644 index 00000000000..0db09ead2cd --- /dev/null +++ b/tests/queries/0_stateless/03024_total_rows_approx_is_set_for_system_zeros_and_generate_random.sql @@ -0,0 +1,9 @@ +SET max_rows_to_read = 1e11; + +SELECT * FROM system.numbers LIMIT 1e12 FORMAT Null; -- { serverError TOO_MANY_ROWS } +SELECT * FROM system.numbers_mt LIMIT 1e12 FORMAT Null; -- { serverError TOO_MANY_ROWS } + +SELECT * FROM system.zeros LIMIT 1e12 FORMAT Null; -- { serverError TOO_MANY_ROWS } +SELECT * FROM system.zeros_mt LIMIT 1e12 FORMAT Null; -- { serverError TOO_MANY_ROWS } + +SELECT * FROM generateRandom() LIMIT 1e12 FORMAT Null; -- { serverError TOO_MANY_ROWS } From 86039802e3327c0a527135233f3589d6bcea2348 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Mar 2024 00:46:48 +0100 Subject: [PATCH 0069/1018] Set total_rows_approx for trivial queries with LIMIT from system.zeros and generateRandom --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index fa46b115979..65beef27d16 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2506,7 +2506,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc { if (max_block_limited < local_limits.local_limits.size_limits.max_rows) query_info.limit = max_block_limited; - else /// Ask to read just enough rows to make the max_rows limit effective (so it has a chance to be triggered). + else if (local_limits.local_limits.size_limits.max_rows < std::numeric_limits::max()) /// Ask to read just enough rows to make the max_rows limit effective (so it has a chance to be triggered). query_info.limit = 1 + local_limits.local_limits.size_limits.max_rows; } else From c55e45bff6212e8e4828198fd13a56bc174d2062 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Mar 2024 18:58:57 +0100 Subject: [PATCH 0070/1018] Test robustness --- ...023_zeros_generate_random_with_limit_progress_bar.expect | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect index 272dc0fdfef..de15a199132 100755 --- a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect +++ b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect @@ -23,14 +23,14 @@ spawn clickhouse-local expect ":) " # Trivial SELECT with LIMIT from system.zeros shows progress bar. -send "SELECT * FROM system.zeros LIMIT 1e11 FORMAT Null\r" +send "SELECT * FROM system.zeros LIMIT 10000000 FORMAT Null SETTINGS max_execution_speed = 1000000, timeout_before_checking_execution_speed = 0, max_block_size = 128\r" expect "Progress: " expect "█" send "\3" expect "Query was cancelled." expect ":) " -send "SELECT * FROM system.zeros_mt LIMIT 1e11 FORMAT Null\r" +send "SELECT * FROM system.zeros_mt LIMIT 10000000 FORMAT Null SETTINGS max_execution_speed = 1000000, timeout_before_checking_execution_speed = 0, max_block_size = 128\r" expect "Progress: " expect "█" send "\3" @@ -38,7 +38,7 @@ expect "Query was cancelled." expect ":) " # As well as from generateRandom -send "SELECT * FROM generateRandom() LIMIT 1e9 FORMAT Null\r" +send "SELECT * FROM generateRandom() LIMIT 10000000 FORMAT Null SETTINGS max_execution_speed = 1000000, timeout_before_checking_execution_speed = 0, max_block_size = 128\r" expect "Progress: " expect "█" send "\3" From f2e0a3be1c917b17cddfb720f24fddd8f89117d9 Mon Sep 17 00:00:00 2001 From: Kirill <71129570+kirillgarbar@users.noreply.github.com> Date: Mon, 25 Mar 2024 01:31:59 +0300 Subject: [PATCH 0071/1018] Fix typo in docs Co-authored-by: Alexey Milovidov --- docs/en/engines/table-engines/mergetree-family/replication.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/replication.md b/docs/en/engines/table-engines/mergetree-family/replication.md index 257084d8691..0fbdbec6fff 100644 --- a/docs/en/engines/table-engines/mergetree-family/replication.md +++ b/docs/en/engines/table-engines/mergetree-family/replication.md @@ -307,7 +307,7 @@ If you had a `MergeTree` table that was manually replicated, you can convert it `MergeTree` table can be automatically converted on server restart if `convert_to_replicated` flag is set at the table's data directory (`/store/xxx/xxxyyyyy-yyyy-yyyy-yyyy-yyyyyyyyyyyy/` for `Atomic` database). Create empty `convert_to_replicated` file and the table will be loaded as replicated on next server restart. -This query can be used to get the table's data path. It table has many data paths, you have to use the first one. +This query can be used to get the table's data path. If table has many data paths, you have to use the first one. ```sql SELECT data_paths FROM system.tables WHERE table = 'table_name' AND database = 'database_name'; From e54dbeeb1b4b10840d55ce8ee78c2ea3bd2e5007 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 25 Mar 2024 09:57:35 +0000 Subject: [PATCH 0072/1018] impl. for qualified asterisk, update tests Signed-off-by: Duc Canh Le --- src/Parsers/ExpressionElementParsers.cpp | 2 +- .../0_stateless/03003_count_asterisk_filter.reference | 3 +++ tests/queries/0_stateless/03003_count_asterisk_filter.sql | 8 ++++++-- 3 files changed, 10 insertions(+), 3 deletions(-) diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 27cf51db5f8..0c9aeb68e0f 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -361,7 +361,7 @@ bool ParserFilterClause::parseImpl(Pos & pos, ASTPtr & node, Expected & expected /// See https://github.com/ClickHouse/ClickHouse/issues/61004 std::erase_if(function.arguments->children, [] (const ASTPtr & child) { - return typeid_cast(child.get()); + return typeid_cast(child.get()) || typeid_cast(child.get()); }); } diff --git a/tests/queries/0_stateless/03003_count_asterisk_filter.reference b/tests/queries/0_stateless/03003_count_asterisk_filter.reference index 0cfbf08886f..be589c9ceb0 100644 --- a/tests/queries/0_stateless/03003_count_asterisk_filter.reference +++ b/tests/queries/0_stateless/03003_count_asterisk_filter.reference @@ -1 +1,4 @@ +1 +1 +2 2 diff --git a/tests/queries/0_stateless/03003_count_asterisk_filter.sql b/tests/queries/0_stateless/03003_count_asterisk_filter.sql index c256b58851f..9bd10dfae25 100644 --- a/tests/queries/0_stateless/03003_count_asterisk_filter.sql +++ b/tests/queries/0_stateless/03003_count_asterisk_filter.sql @@ -1,9 +1,13 @@ -CREATE TABLE users (uid Int16, name String, age Int16) ENGINE=Memory; +CREATE TABLE users (uid Int16, name Nullable(String), age Int16) ENGINE=Memory; INSERT INTO users VALUES (1231, 'John', 33); -INSERT INTO users VALUES (6666, 'Ksenia', 48); +INSERT INTO users VALUES (6666, Null, 48); INSERT INTO users VALUES (8888, 'Alice', 50); +SELECT count(name) FILTER (WHERE uid > 2000) FROM users; +SELECT countIf(name, uid > 2000) FROM users; + SELECT count(*) FILTER (WHERE uid > 2000) FROM users; +SELECT countIf(uid > 2000) FROM users; DROP TABLE users; From 6c56c0c89ed63c35bf524071852eba066de917da Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 25 Mar 2024 11:19:12 -0300 Subject: [PATCH 0073/1018] Do not allow table to be attached if there already is an active replica path --- src/Storages/StorageReplicatedMergeTree.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c41403e312b..8c711ffd25f 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -399,6 +399,11 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( { try { + if (LoadingStrictnessLevel::ATTACH == mode && current_zookeeper && current_zookeeper->exists(replica_path + "/is_active")) + { + throw Exception(ErrorCodes::REPLICA_ALREADY_EXISTS, "There already is an active replica with this replica path {}", replica_path); + } + if (current_zookeeper && current_zookeeper->exists(replica_path + "/host")) { /// Check it earlier if we can (we don't want incompatible version to start). From 255e4b4bda442968197148051ac507266f1708ac Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 25 Mar 2024 14:22:39 +0000 Subject: [PATCH 0074/1018] fix test --- .../queries/0_stateless/02982_perf_introspection_for_inserts.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02982_perf_introspection_for_inserts.sh b/tests/queries/0_stateless/02982_perf_introspection_for_inserts.sh index f5fb54b54d3..409bd996cbd 100755 --- a/tests/queries/0_stateless/02982_perf_introspection_for_inserts.sh +++ b/tests/queries/0_stateless/02982_perf_introspection_for_inserts.sh @@ -36,7 +36,7 @@ FROM numbers_mt(1000000); $CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS" $CLICKHOUSE_CLIENT -q """ SELECT - ProfileEvents['MergeTreeDataProjectionWriterMergingBlocksMicroseconds'] > 0, + ProfileEvents['MergeTreeDataProjectionWriterMergingBlocksMicroseconds'] = 0, ProfileEvents['MergeTreeDataProjectionWriterSortingBlocksMicroseconds'] > 0, ProfileEvents['MergeTreeDataWriterSortingBlocksMicroseconds'] > 0, ProfileEvents['MergeTreeDataWriterProjectionsCalculationMicroseconds'] > 0, From c3b3d1ae99b51269e8374e08c9506122191787ff Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 25 Mar 2024 14:53:39 -0300 Subject: [PATCH 0075/1018] add basic test --- ..._attach_table_if_active_replica_already_exists.reference | 0 ...forbid_attach_table_if_active_replica_already_exists.sql | 6 ++++++ 2 files changed, 6 insertions(+) create mode 100644 tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.reference create mode 100644 tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sql diff --git a/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.reference b/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sql b/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sql new file mode 100644 index 00000000000..5115e6cf776 --- /dev/null +++ b/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sql @@ -0,0 +1,6 @@ +create table t1 (a Int) + engine=ReplicatedMergeTree('/clickhouse/tables/default/test', 'r1') + order by tuple() SETTINGS index_granularity = 8192; +attach table t2 UUID '6c32d92e-bebf-4730-ae73-c43e5748f829' + (a Int) engine=ReplicatedMergeTree('/clickhouse/tables/default/test', 'r1') + order by tuple() SETTINGS index_granularity = 8192; -- { serverError REPLICA_ALREADY_EXISTS }; From 16e8fd0a59bf38eeeb0eb58af6dc96ed0491893b Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 25 Mar 2024 15:27:58 -0300 Subject: [PATCH 0076/1018] add {database} --- ...3_forbid_attach_table_if_active_replica_already_exists.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sql b/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sql index 5115e6cf776..e7c875d6f67 100644 --- a/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sql +++ b/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sql @@ -1,6 +1,6 @@ create table t1 (a Int) - engine=ReplicatedMergeTree('/clickhouse/tables/default/test', 'r1') + engine=ReplicatedMergeTree('/clickhouse/tables/{database}/test', 'r1') order by tuple() SETTINGS index_granularity = 8192; attach table t2 UUID '6c32d92e-bebf-4730-ae73-c43e5748f829' - (a Int) engine=ReplicatedMergeTree('/clickhouse/tables/default/test', 'r1') + (a Int) engine=ReplicatedMergeTree('/clickhouse/tables/{database}/test', 'r1') order by tuple() SETTINGS index_granularity = 8192; -- { serverError REPLICA_ALREADY_EXISTS }; From a47b407495c99ad049f04185b45c66e87f28881b Mon Sep 17 00:00:00 2001 From: peter279k Date: Tue, 26 Mar 2024 13:35:35 +0800 Subject: [PATCH 0077/1018] Fix PR #60656 for install check tests --- tests/ci/install_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/install_check.py b/tests/ci/install_check.py index 71e2198f533..54a18c7e26c 100644 --- a/tests/ci/install_check.py +++ b/tests/ci/install_check.py @@ -33,7 +33,7 @@ set -e trap "bash -ex /packages/preserve_logs.sh" ERR test_env='TEST_THE_DEFAULT_PARAMETER=15' echo "$test_env" >> /etc/default/clickhouse -systemctl start clickhouse-server +systemctl restart clickhouse-server clickhouse-client -q 'SELECT version()' grep "$test_env" /proc/$(cat /var/run/clickhouse-server/clickhouse-server.pid)/environ""" initd_test = r"""#!/bin/bash From 972af33400952abce28451f0b248ae3d0fc4579c Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 26 Mar 2024 12:30:16 +0000 Subject: [PATCH 0078/1018] Timeout injection --- src/Client/Connection.cpp | 11 +++ src/Common/FailPoint.cpp | 3 +- .../__init__.py | 0 .../configs/remote_servers.xml | 22 ++++++ .../test.py | 75 +++++++++++++++++++ 5 files changed, 110 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_parallel_replicas_failover_timeout/__init__.py create mode 100644 tests/integration/test_parallel_replicas_failover_timeout/configs/remote_servers.xml create mode 100644 tests/integration/test_parallel_replicas_failover_timeout/test.py diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 180942e6b83..057cceb68df 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -34,6 +34,7 @@ #include #include #include +#include #include #include "config.h" @@ -51,6 +52,11 @@ namespace CurrentMetrics namespace DB { +namespace FailPoints +{ + extern const char receive_timeout_on_table_status_response[]; +} + namespace ErrorCodes { extern const int NETWORK_ERROR; @@ -608,6 +614,11 @@ TablesStatusResponse Connection::getTablesStatus(const ConnectionTimeouts & time request.write(*out, server_revision); out->next(); + fiu_do_on(FailPoints::receive_timeout_on_table_status_response, { + sleepForSeconds(5); + throw NetException(ErrorCodes::SOCKET_TIMEOUT, "Injected timeout exceeded while reading from socket ({}:{})", host, port); + }); + UInt64 response_type = 0; readVarUInt(response_type, *in); diff --git a/src/Common/FailPoint.cpp b/src/Common/FailPoint.cpp index 9e551c8f2cd..369d5938ec7 100644 --- a/src/Common/FailPoint.cpp +++ b/src/Common/FailPoint.cpp @@ -52,7 +52,8 @@ static struct InitFiu REGULAR(prefetched_reader_pool_failpoint) \ PAUSEABLE_ONCE(dummy_pausable_failpoint_once) \ PAUSEABLE(dummy_pausable_failpoint) \ - ONCE(execute_query_calling_empty_set_result_func_on_exception) + ONCE(execute_query_calling_empty_set_result_func_on_exception) \ + ONCE(receive_timeout_on_table_status_response) namespace FailPoints { diff --git a/tests/integration/test_parallel_replicas_failover_timeout/__init__.py b/tests/integration/test_parallel_replicas_failover_timeout/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_parallel_replicas_failover_timeout/configs/remote_servers.xml b/tests/integration/test_parallel_replicas_failover_timeout/configs/remote_servers.xml new file mode 100644 index 00000000000..ac46202cc95 --- /dev/null +++ b/tests/integration/test_parallel_replicas_failover_timeout/configs/remote_servers.xml @@ -0,0 +1,22 @@ + + + + + true + + node1 + 9000 + + + node2 + 9000 + + + node3 + 9000 + + + + + + diff --git a/tests/integration/test_parallel_replicas_failover_timeout/test.py b/tests/integration/test_parallel_replicas_failover_timeout/test.py new file mode 100644 index 00000000000..66289fed720 --- /dev/null +++ b/tests/integration/test_parallel_replicas_failover_timeout/test.py @@ -0,0 +1,75 @@ +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance("node1", main_configs=["configs/remote_servers.xml"], with_zookeeper=True) +node2 = cluster.add_instance("node2", main_configs=["configs/remote_servers.xml"], with_zookeeper=True) +node3 = cluster.add_instance("node3", main_configs=["configs/remote_servers.xml"], with_zookeeper=True) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def create_tables(cluster, table_name): + node1.query(f"DROP TABLE IF EXISTS {table_name} SYNC") + node3.query(f"DROP TABLE IF EXISTS {table_name} SYNC") + + node1.query( + f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r1') ORDER BY (key)" + ) + node2.query( + f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r2') ORDER BY (key)" + ) + node3.query( + f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r3') ORDER BY (key)" + ) + + # populate data + node1.query( + f"INSERT INTO {table_name} SELECT number % 4, number FROM numbers(1000)" + ) + node1.query( + f"INSERT INTO {table_name} SELECT number % 4, number FROM numbers(1000, 1000)" + ) + node1.query( + f"INSERT INTO {table_name} SELECT number % 4, number FROM numbers(2000, 1000)" + ) + node1.query( + f"INSERT INTO {table_name} SELECT number % 4, number FROM numbers(3000, 1000)" + ) + node2.query(f"SYSTEM SYNC REPLICA {table_name}") + node3.query(f"SYSTEM SYNC REPLICA {table_name}") + + +def test_skip_unresponsive_replicas(start_cluster): + cluster_name = "test_1_shard_3_replicas" + table_name = "tt" + create_tables(cluster_name, table_name) + + expected_result = "" + for i in range(4): + expected_result += f"{i}\t1000\n" + + node1.query("SYSTEM ENABLE FAILPOINT receive_timeout_on_table_status_response") + + assert ( + node1.query( + f"SELECT key, count() FROM {table_name} GROUP BY key ORDER BY key", + settings={ + # "log_comment": log_comment, + "allow_experimental_parallel_reading_from_replicas": 2, + "max_parallel_replicas": 3, + "cluster_for_parallel_replicas": cluster_name, + "receive_timeout": 2, + }, + ) + == expected_result + ) From 3af0f12d68fa6cbccc248bde69b1be14961ad46e Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 26 Mar 2024 09:47:40 -0300 Subject: [PATCH 0079/1018] databse atomic --- ..._forbid_attach_table_if_active_replica_already_exists.sql | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sql b/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sql index e7c875d6f67..0be67ded71a 100644 --- a/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sql +++ b/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sql @@ -1,6 +1,7 @@ -create table t1 (a Int) +create database at engine = Atomic; +create table at.t1 (a Int) engine=ReplicatedMergeTree('/clickhouse/tables/{database}/test', 'r1') order by tuple() SETTINGS index_granularity = 8192; -attach table t2 UUID '6c32d92e-bebf-4730-ae73-c43e5748f829' +attach table at.t2 UUID '6c32d92e-bebf-4730-ae73-c43e5748f829' (a Int) engine=ReplicatedMergeTree('/clickhouse/tables/{database}/test', 'r1') order by tuple() SETTINGS index_granularity = 8192; -- { serverError REPLICA_ALREADY_EXISTS }; From bbb14955a6b44fc4c4c4bcb76105c54ab70cbe67 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 26 Mar 2024 12:48:01 +0000 Subject: [PATCH 0080/1018] Automatic style fix --- .../test_parallel_replicas_failover_timeout/test.py | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_parallel_replicas_failover_timeout/test.py b/tests/integration/test_parallel_replicas_failover_timeout/test.py index 66289fed720..6031f8cbe4a 100644 --- a/tests/integration/test_parallel_replicas_failover_timeout/test.py +++ b/tests/integration/test_parallel_replicas_failover_timeout/test.py @@ -4,9 +4,15 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance("node1", main_configs=["configs/remote_servers.xml"], with_zookeeper=True) -node2 = cluster.add_instance("node2", main_configs=["configs/remote_servers.xml"], with_zookeeper=True) -node3 = cluster.add_instance("node3", main_configs=["configs/remote_servers.xml"], with_zookeeper=True) +node1 = cluster.add_instance( + "node1", main_configs=["configs/remote_servers.xml"], with_zookeeper=True +) +node2 = cluster.add_instance( + "node2", main_configs=["configs/remote_servers.xml"], with_zookeeper=True +) +node3 = cluster.add_instance( + "node3", main_configs=["configs/remote_servers.xml"], with_zookeeper=True +) @pytest.fixture(scope="module") From f41622f9b5626c48f6e08585ce8d10b3031ed428 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 27 Mar 2024 09:52:13 -0300 Subject: [PATCH 0081/1018] fix flaky check --- ..._forbid_attach_table_if_active_replica_already_exists.sql | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sql b/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sql index 0be67ded71a..0129e7392f0 100644 --- a/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sql +++ b/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sql @@ -1,4 +1,9 @@ +drop database if exists at; create database at engine = Atomic; + +drop table if exists at.t1 sync; +drop table if exists at.t2 sync; + create table at.t1 (a Int) engine=ReplicatedMergeTree('/clickhouse/tables/{database}/test', 'r1') order by tuple() SETTINGS index_granularity = 8192; From d18aba1194cb1a48560315d420404783b418e439 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 27 Mar 2024 19:26:19 +0000 Subject: [PATCH 0082/1018] Done --- src/Client/Suggest.cpp | 19 +- src/Parsers/obfuscateQueries.cpp | 454 ++---------------- ...6_clickhouse_client_autocomplete.reference | 1 + .../01676_clickhouse_client_autocomplete.sh | 2 + 4 files changed, 51 insertions(+), 425 deletions(-) diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index 03df582de10..7bbd45ed1d8 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -30,24 +30,6 @@ namespace ErrorCodes Suggest::Suggest() { - /// Keywords may be not up to date with ClickHouse parser. - addWords({"CREATE", "DATABASE", "IF", "NOT", "EXISTS", "TEMPORARY", "TABLE", "ON", - "CLUSTER", "DEFAULT", "MATERIALIZED", "ALIAS", "ENGINE", "AS", "VIEW", "POPULATE", - "SETTINGS", "ATTACH", "DETACH", "DROP", "RENAME", "TO", "ALTER", "ADD", - "MODIFY", "CLEAR", "COLUMN", "AFTER", "COPY", "PROJECT", "PRIMARY", "KEY", - "CHECK", "PARTITION", "PART", "FREEZE", "FETCH", "FROM", "SHOW", "INTO", - "OUTFILE", "FORMAT", "TABLES", "DATABASES", "LIKE", "PROCESSLIST", "CASE", "WHEN", - "THEN", "ELSE", "END", "DESCRIBE", "DESC", "USE", "SET", "OPTIMIZE", - "FINAL", "DEDUPLICATE", "INSERT", "VALUES", "SELECT", "DISTINCT", "SAMPLE", "ARRAY", - "JOIN", "GLOBAL", "LOCAL", "ANY", "ALL", "INNER", "LEFT", "RIGHT", - "FULL", "OUTER", "CROSS", "USING", "PREWHERE", "WHERE", "GROUP", "BY", - "WITH", "TOTALS", "HAVING", "ORDER", "COLLATE", "LIMIT", "UNION", "AND", - "OR", "ASC", "IN", "KILL", "QUERY", "SYNC", "ASYNC", "TEST", - "BETWEEN", "TRUNCATE", "USER", "ROLE", "PROFILE", "QUOTA", "POLICY", "ROW", - "GRANT", "REVOKE", "OPTION", "ADMIN", "EXCEPT", "REPLACE", "IDENTIFIED", "HOST", - "NAME", "READONLY", "WRITABLE", "PERMISSIVE", "FOR", "RESTRICTIVE", "RANDOMIZED", "INTERVAL", - "LIMITS", "ONLY", "TRACKING", "IP", "REGEXP", "ILIKE", "CLEANUP", "APPEND", - "IGNORE NULLS", "RESPECT NULLS", "OVER", "PASTE"}); } static String getLoadSuggestionQuery(Int32 suggestion_limit, bool basic_suggestion) @@ -82,6 +64,7 @@ static String getLoadSuggestionQuery(Int32 suggestion_limit, bool basic_suggesti add_column("name", "data_type_families", false, {}); add_column("name", "merge_tree_settings", false, {}); add_column("name", "settings", false, {}); + add_column("keyword", "keywords", false, {}); if (!basic_suggestion) { diff --git a/src/Parsers/obfuscateQueries.cpp b/src/Parsers/obfuscateQueries.cpp index 8012dbb37c6..49af925212d 100644 --- a/src/Parsers/obfuscateQueries.cpp +++ b/src/Parsers/obfuscateQueries.cpp @@ -1,3 +1,5 @@ +#include "Parsers/CommonParsers.h" +#include #include #include @@ -10,6 +12,7 @@ #include #include #include +#include namespace DB @@ -24,413 +27,50 @@ namespace ErrorCodes namespace { -const std::unordered_set keywords +const std::unordered_set & getObfuscateKeywords() { - "!=", - "", - "%", - "*", - "+", - "-", - "->", - ".", - "/", - ":", - "::", - "<", - "<=", - "<>", - "=", - "==", - "<=>", - ">", - ">=", - "?", - "[", - "]+", - "]+|[", - "^[", - "||", - "]+$", - "ACCESS", - "ACTION", - "ADD", - "ADMIN", - "AFTER", - "ALGORITHM", - "ALIAS", - "ALL", - "ALLOWED_LATENESS", - "ALTER", - "AND", - "ANTI", - "ANY", - "APPLY", - "ARRAY", - "AS", - "ASC", - "ASCENDING", - "ASOF", - "ASSUME", - "AST", - "ASYNC", - "ATTACH", - "AUTO_INCREMENT", - "BACKUP", - "BASE_BACKUP", - "BEGIN", - "BETWEEN", - "BIDIRECTIONAL", - "BOTH", - "BY", - "CACHE", - "CACHES", - "CASCADE", - "CASE", - "CASEWITHEXPRESSION", - "CAST", - "CHANGE", - "CHANGEABLE_IN_READONLY", - "CHANGED", - "CHAR", - "CHARACTER", - "CHECK", - "CLEANUP", - "CLEAR", - "CLUSTER", - "CLUSTER_HOST_IDS", - "CLUSTERS", - "CN", - "CODEC", - "COLLATE", - "COLLECTION", - "COLUMN", - "COLUMNS", - "COMMENT", - "COMMIT", - "COMPRESSION", - "CONCAT", - "CONSTRAINT", - "CREATE", - "CROSS", - "CUBE", - "CURRENT", - "CURRENT_USER", - "DATABASE", - "DATABASES", - "DATE", - "DATE_ADD", - "DATEADD", - "DATE_DIFF", - "DATEDIFF", - "DATE_SUB", - "DATESUB", - "DAY", - "DD", - "DDL", - "DEDUPLICATE", - "DEFAULT", - "DELAY", - "DELETE", - "DESC", - "DESCENDING", - "DESCRIBE", - "DETACH", - "DETACHED", - "DICTIONARIES", - "DICTIONARY", - "DISK", - "DISTINCT", - "DIV", - "DOUBLE_SHA1_HASH", - "DROP", - "ELSE", - "EMPTY", - "ENABLED", - "END", - "ENFORCED", - "ENGINE", - "EPHEMERAL", - "EQUALS", - "ESTIMATE", - "EVENT", - "EVENTS", - "EXCEPT", - "EXCHANGE", - "EXISTS", - "EXPLAIN", - "EXPRESSION", - "EXTERNAL", - "EXTRACT", - "FALSE", - "FETCH", - "FILE", - "FILESYSTEM", - "FILL", - "FILTER", - "FINAL", - "FIRST", - "FOLLOWING", - "FOR", - "FOREIGN", - "FORMAT", - "FREEZE", - "FROM", - "FULL", - "FULLTEXT", - "FUNCTION", - "GLOBAL", - "GRANT", - "GRANTEES", - "GRANTS", - "GRANULARITY", - "GREATER", - "GREATEROREQUALS", - "GROUP", - "GROUPING", - "GROUPS", - "HASH", - "HAVING", - "HDFS", - "HH", - "HIERARCHICAL", - "HOST", - "HOUR", - "ID", - "IDENTIFIED", - "IF", - "ILIKE", - "IN", - "INDEX", - "INFILE", - "INHERIT", - "INJECTIVE", - "INNER", - "INSERT", - "INTERPOLATE", - "INTERSECT", - "INTERVAL", - "INTO", - "INVISIBLE", - "IP", - "IS", - "IS_OBJECT_ID", - "JOIN", - "KEY", - "KEYED", - "KILL", - "LAMBDA", - "LARGE", - "LAST", - "LAYOUT", - "LEADING", - "LEFT", - "LESS", - "LESSOREQUALS", - "LEVEL", - "LIFETIME", - "LIKE", - "LIMIT", - "LIMITS", - "LINEAR", - "LIST", - "LITERAL", - "LIVE", - "LOCAL", - "LTRIM", - "MATCH", - "MATERIALIZE", - "MATERIALIZED", - "MAX", - "MCS", - "MEMORY", - "MI", - "MICROSECOND", - "MILLISECOND", - "MIN", - "MINUS", - "MINUTE", - "MM", - "MOD", - "MODIFY", - "MONTH", - "MOVE", - "MS", - "MULTIIF", - "MUTATION", - "NAME", - "NAMED", - "NANOSECOND", - "NEXT", - "NO", - "NONE", - "NOT", - "NOTEQUALS", - "NOTIN", - "NS", - "NULL", - "NULLS", - "OBJECT", - "OFFSET", - "ON", - "ONLY", - "OPTIMIZE", - "OPTION", - "OR", - "ORDER", - "OUTER", - "OUTFILE", - "OVER", - "OVERRIDE", - "PART", - "PARTIAL", - "PARTITION", - "PARTITIONS", - "PART_MOVE_TO_SHARD", - "PERMANENTLY", - "PERMISSIVE", - "PIPELINE", - "PLAN", - "PLUS", - "POLICY", - "POPULATE", - "POSITION", - "PRECEDING", - "PRECISION", - "PREWHERE", - "PRIMARY", - "PRIVILEGES", - "PROCESSLIST", - "PROFILE", - "PROJECTION", - "QQ", - "QUARTER", - "QUERY", - "QUOTA", - "RANDOMIZED", - "RANGE", - "READONLY", - "REALM", - "RECOMPRESS", - "REFERENCES", - "REFRESH", - "REGEXP", - "REGEXPQUOTEMETA", - "REMOVE", - "RENAME", - "REPLACE", - "REPLACEREGEXPALL", - "REPLACEREGEXPONE", - "RESET", - "RESTORE", - "RESTRICT", - "RESTRICTIVE", - "RESUME", - "REVOKE", - "RIGHT", - "ROLE", - "ROLES", - "ROLLBACK", - "ROLLUP", - "ROW", - "ROWS", - "RTRIM", - "S3", - "SALT", - "SAMPLE", - "SECOND", - "SELECT", - "SEMI", - "SERVER", - "SET", - "SETS", - "SETTING", - "SETTINGS", - "SHA256_HASH", - "SHARD", - "SHOW", - "SIGNED", - "SIMPLE", - "SINGLEVALUEORNULL", - "SNAPSHOT", - "SOURCE", - "SPATIAL", - "SS", - "STDOUT", - "STEP", - "STORAGE", - "STRICT", - "STRICTLY_ASCENDING", - "SUBPARTITION", - "SUBPARTITIONS", - "SUBSTRING", - "SUSPEND", - "SYNC", - "SYNTAX", - "SYSTEM", - "TABLE", - "TABLES", - "TEMPORARY", - "TEST", - "THAN", - "THEN", - "TIES", - "TIMESTAMP", - "TIMESTAMP_ADD", - "TIMESTAMPADD", - "TIMESTAMP_DIFF", - "TIMESTAMPDIFF", - "TIMESTAMP_SUB", - "TIMESTAMPSUB", - "TO", - "TODATE", - "TODATETIME", - "TOP", - "TOTALS", - "TRACKING", - "TRAILING", - "TRANSACTION", - "TREE", - "TRIGGER", - "TRIM", - "TRIMBOTH", - "TRIMLEFT", - "TRIMRIGHT", - "TRUE", - "TRUNCATE", - "TTL", - "TUPLE", - "TYPE", - "UNBOUNDED", - "UNFREEZE", - "UNION", - "UNIQUE", - "UNSIGNED", - "UNTUPLE", - "UPDATE", - "URL", - "USE", - "USER", - "USING", - "UUID", - "VALUES", - "VARYING", - "VIEW", - "VIEWIFPERMITTED", - "VISIBLE", - "VOLUME", - "WATCH", - "WATERMARK", - "WEEK", - "WHEN", - "WHERE", - "WINDOW", - "WITH", - "WK", - "WRITABLE", - "YEAR", - "YYYY", - "ZKPATH" + static std::unordered_set instance; + + auto initialize = [&]() mutable + { + instance = { + "!=", + "", + "%", + "*", + "+", + "-", + "->", + ".", + "/", + ":", + "::", + "<", + "<=", + "<>", + "=", + "==", + "<=>", + ">", + ">=", + "?", + "[", + "]+", + "]+|[", + "^[", + "||", + "]+$" + }; + + auto & global_keywords = getAllKeyWords(); + std::copy(global_keywords.begin(), global_keywords.end(), std::inserter(instance, instance.begin())); + return true; + }; + + static bool initialized = initialize(); + (void) initialized; + + return instance; }; /// We want to keep some words inside quotes. For example we want to keep HOUR inside: @@ -1312,7 +952,7 @@ void obfuscateQueries( std::string whole_token_uppercase(whole_token); Poco::toUpperInPlace(whole_token_uppercase); - if (keywords.contains(whole_token_uppercase) + if (getObfuscateKeywords().contains(whole_token_uppercase) || known_identifier_func(whole_token)) { /// Keep keywords as is. diff --git a/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.reference b/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.reference index cf3e942adfe..78f8967263d 100644 --- a/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.reference +++ b/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.reference @@ -9,6 +9,7 @@ test_shard_localhost: OK default_path_test: OK default: OK uniqCombined64ForEach: OK +CHANGEABLE_IN_READONLY: OK system: OK aggregate_function_combinators: OK primary_key_bytes_in_memory_allocated: OK diff --git a/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.sh b/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.sh index db62dedb5b4..f757e32c56c 100755 --- a/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.sh +++ b/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.sh @@ -99,6 +99,8 @@ client_compwords_positive=( default # system.aggregate_function_combinators uniqCombined64ForEach + # system.keywords + CHANGEABLE_IN_READONLY # FIXME: one may add separate case for suggestion_limit # system.databases From 500379801bb8c27c1822fd1540dbcf3c1e16a8aa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=9A=D0=B8=D1=80=D0=B8=D0=BB=D0=BB=20=D0=93=D0=B0=D1=80?= =?UTF-8?q?=D0=B1=D0=B0=D1=80?= Date: Wed, 27 Mar 2024 23:09:14 +0300 Subject: [PATCH 0083/1018] Check query_settings nullptr --- src/Databases/DatabaseOrdinary.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 65d5f21bbea..f4191727b7c 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -31,7 +31,6 @@ #include #include #include -#include #include @@ -128,10 +127,10 @@ void DatabaseOrdinary::convertMergeTreeToReplicatedIfNeeded(ASTPtr ast, const Qu /// Get table's storage policy MergeTreeSettings default_settings = getContext()->getMergeTreeSettings(); - auto * query_settings = create_query->storage->settings->as(); auto policy = getContext()->getStoragePolicy(default_settings.storage_policy); - if (Field * policy_setting = query_settings->changes.tryGet("storage_policy")) - policy = getContext()->getStoragePolicy(policy_setting->get()); + if (auto * query_settings = create_query->storage->settings->as()) + if (Field * policy_setting = query_settings->changes.tryGet("storage_policy")) + policy = getContext()->getStoragePolicy(policy_setting->safeGet()); auto convert_to_replicated_flag_path = getConvertToReplicatedFlagPath(qualified_name.table, policy, false); From 9e719868a0b4141d4ee918f4116719f8227d4757 Mon Sep 17 00:00:00 2001 From: Maxim Alexeev Date: Wed, 27 Mar 2024 23:28:13 +0300 Subject: [PATCH 0084/1018] changed parameters --- src/Core/Settings.h | 3 ++- src/Core/SettingsChangesHistory.h | 2 ++ src/Interpreters/HashJoin.cpp | 4 ++-- src/Interpreters/TableJoin.cpp | 3 ++- src/Interpreters/TableJoin.h | 7 +++++-- 5 files changed, 13 insertions(+), 6 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b9ada75d581..a9f143124b6 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -452,7 +452,8 @@ class IColumn; M(OverflowMode, join_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ M(Bool, join_any_take_last_row, false, "When disabled (default) ANY JOIN will take the first found row for a key. When enabled, it will take the last row seen if there are multiple rows for the same key.", IMPORTANT) \ M(JoinAlgorithm, join_algorithm, JoinAlgorithm::DEFAULT, "Specify join algorithm.", 0) \ - M(Bool, cross_join_compress_blocks, false, "When enabled will compress blocks in CROSS JOIN", 0) \ + M(UInt64, cross_join_min_rows_to_compress, 1000, "Minimal count of rows to compress block in CROSS JOIN", 0) \ + M(UInt64, cross_join_min_bytes_to_compress, 10000, "Minimal size of block to compress in CROSS JOIN", 0) \ M(UInt64, default_max_bytes_in_join, 1000000000, "Maximum size of right-side table if limit is required but max_bytes_in_join is not set.", 0) \ M(UInt64, partial_merge_join_left_table_buffer_bytes, 0, "If not 0 group left table blocks in bigger ones for left-side table in partial merge join. It uses up to 2x of specified memory per joining thread.", 0) \ M(UInt64, partial_merge_join_rows_in_right_blocks, 65536, "Split right-hand joining data in blocks of specified size. It's a portion of data indexed by min-max values and possibly unloaded on disk.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index c107d953d52..d3addf8f40a 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -106,6 +106,8 @@ static std::map sett {"keeper_retry_max_backoff_ms", 5000, 5000, "Max backoff timeout for general keeper operations"}, {"s3queue_allow_experimental_sharded_mode", false, false, "Enable experimental sharded mode of S3Queue table engine. It is experimental because it will be rewritten"}, {"cross_join_compress_blocks", false, false, "A new setting."}, + {"cross_join_min_rows_to_compress", 1000, 1000, "A new setting."}, + {"cross_join_min_bytes_to_compress", 10000, 10000, "A new setting."}, {"merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability", 0.0, 0.0, "For testing of `PartsSplitter` - split read ranges into intersecting and non intersecting every time you read from MergeTree with the specified probability."}, {"allow_get_client_http_header", false, false, "Introduced a new function."}, {"output_format_pretty_row_numbers", false, true, "It is better for usability."}, diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 807ef556bb0..335d73248c3 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -826,9 +826,9 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) assertBlocksHaveEqualStructure(data->sample_block, block_to_save, "joined block"); - if (kind == JoinKind::Cross && table_join->crossJoinCompressBlocks()) + if (kind == JoinKind::Cross && block_to_save.allocatedBytes() >= table_join->crossJoinMinBytesToCompress() + && block_to_save.rows() >= table_join->crossJoinMinRowsToCompress()) block_to_save = block_to_save.compress(); - data->blocks_allocated_size += block_to_save.allocatedBytes(); data->blocks.emplace_back(std::move(block_to_save)); Block * stored_block = &data->blocks.back(); diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 8b2078c2cbe..48d59dd3b24 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -107,7 +107,8 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_) : size_limits(SizeLimits{settings.max_rows_in_join, settings.max_bytes_in_join, settings.join_overflow_mode}) , default_max_bytes(settings.default_max_bytes_in_join) , join_use_nulls(settings.join_use_nulls) - , cross_join_compress_blocks(settings.cross_join_compress_blocks) + , cross_join_min_rows_to_compress(settings.cross_join_min_rows_to_compress) + , cross_join_min_bytes_to_compress(settings.cross_join_min_bytes_to_compress) , max_joined_block_rows(settings.max_joined_block_size_rows) , join_algorithm(settings.join_algorithm) , partial_merge_join_rows_in_right_blocks(settings.partial_merge_join_rows_in_right_blocks) diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 1acba19f4f0..9038d2f5322 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -139,7 +139,8 @@ private: SizeLimits size_limits; const size_t default_max_bytes = 0; const bool join_use_nulls = false; - const bool cross_join_compress_blocks = false; + const UInt64 cross_join_min_rows_to_compress = 1000; + const UInt64 cross_join_min_bytes_to_compress = 10000; const size_t max_joined_block_rows = 0; std::vector join_algorithm; const size_t partial_merge_join_rows_in_right_blocks = 0; @@ -270,7 +271,9 @@ public: bool joinUseNulls() const { return join_use_nulls; } - bool crossJoinCompressBlocks() const { return cross_join_compress_blocks; } + UInt64 crossJoinMinRowsToCompress() const { return cross_join_min_rows_to_compress; } + + UInt64 crossJoinMinBytesToCompress() const { return cross_join_min_bytes_to_compress; } bool forceNullableRight() const { From 64c3c0bffc826a03c93198489382923ea580db7d Mon Sep 17 00:00:00 2001 From: Maxim Alexeev Date: Thu, 28 Mar 2024 01:09:02 +0300 Subject: [PATCH 0085/1018] Added test --- src/Interpreters/HashJoin.cpp | 1 + .../00181_cross_join_compression.reference | 1000 +++++++++++++++++ .../00181_cross_join_compression.sql | 3 + 3 files changed, 1004 insertions(+) create mode 100644 tests/queries/1_stateful/00181_cross_join_compression.reference create mode 100644 tests/queries/1_stateful/00181_cross_join_compression.sql diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 335d73248c3..a390a43c8d9 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -829,6 +829,7 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) if (kind == JoinKind::Cross && block_to_save.allocatedBytes() >= table_join->crossJoinMinBytesToCompress() && block_to_save.rows() >= table_join->crossJoinMinRowsToCompress()) block_to_save = block_to_save.compress(); + data->blocks_allocated_size += block_to_save.allocatedBytes(); data->blocks.emplace_back(std::move(block_to_save)); Block * stored_block = &data->blocks.back(); diff --git a/tests/queries/1_stateful/00181_cross_join_compression.reference b/tests/queries/1_stateful/00181_cross_join_compression.reference new file mode 100644 index 00000000000..da9f74397bd --- /dev/null +++ b/tests/queries/1_stateful/00181_cross_join_compression.reference @@ -0,0 +1,1000 @@ +5996666368919599438 57 https:\\%2Fvideo/search?text=фильма переводчик +5996666368919599438 57 https:\\%2Fvideo/search?text=коля- смотреть&clid=173 +5996666368919599438 57 https:\\%2Fvideo/search?text=коля- смотреть&clid=173 +5996666368919599438 57 https:\\%2Fvideo/07/subs=28626910825/ +5996666368919599438 57 https:\\%2Fv608531&search?cl4url=http://top/in/813631&lr=39 +5996666368919599438 57 https:\\%2Fv608531&search?cl4url=http://top/in/813631&lr=39 +5996666368919599438 57 https:\\%2Ferrovogo-ne-zapustini.ru/cgi/online.org.ru/neo2/#inbox/?language=view/2749144.html#44 +5996666368919599438 57 https:\\%2Ferrovogo-ne-zapustini.ru/cgi/online +5996666368919599438 57 https:\\%2Fekona.ru%2Fwmzonal +5996666368919599438 57 https:\\%2Feconomii/donbassamediaget%3D52%26prichen-pornoservice/7353135127410972&win=55&text=апплика с днепрография +5996666368919599438 57 https:\\%2Fe_847ad283c2950/?clid=195&lr=57&text +5996666368919599438 57 https:\\%2Fe%2F30859922.7848108_contraktoberton.com.tr +5996666368919599438 57 https:\\%2Fcs4039124670947&secret=506d9e3dfbd268e6b6630e58&domain.am +5996666368919599438 57 https:\\%2F\\%2Fv_krysh.com/ekonomerika.com.tr&user +5996666368919599438 57 https:\\%2F\\%2Fv_krymu_vidbuvalda-skonchili.html +5996666368919599438 57 https:\\%2F\\%2Fv_krymu_vidbuvalda-skonchili.html +5996666368919599438 57 https:\\%2F\\%2Fv_krymu_mne_powerTo=&page=html&lang +5996666368919599438 57 https:\\%2F\\%2Fv_krymu_mne_powerTo=&page=html&lang +5996666368919599438 57 https:\\%2F\\%2Fv_krymu_i_small.sport.tv/newsfrommail +5996666368919599438 57 https:\\%2F\\%2Fv_krymu_i_small.sport.tv/newsfrommail +5996666368919599438 57 https:\\%2F\\%2Fv_krymu13942993&group=2&page=0&parent +5996666368919599438 57 https:\\%2F\\%2Fv_krymu13942993&group=2&page=0&parent +5996666368919599438 57 https:\\%2F\\%2Fv_krymu/ +5996666368919599438 57 https:\\%2F\\%2Fv_krymu/ +5996666368919599438 57 https:\\%2F\\%2Fv_krymu/ +5996666368919599438 57 https:\\%2F\\%2Fv_krymu/ +5996666368919599438 57 https:\\%2F\\%2Fv_krymu/ +5996666368919599438 57 https:\\%2F\\%2Fv_krymu/ +5996666368919599438 57 https:\\%2F\\%2Fv_krymu/ +5996666368919599438 57 https:\\%2F\\%2Fv_krymu.html?account=4100139-yats +5996666368919599438 57 https:\\%2F\\%2Fv_krymu.html/cs/launchere=all&films +5996666368919599438 57 https:\\%2F\\%2Fv_krymu-zakryvaet-ranorama/13080.ru +5996666368919599438 57 https:\\%2F\\%2Fv_krymu-zakryvaet-ranorama/13080.ru +5996666368919599438 57 https:\\%2F\\%2Fv_krymu-sony/stratinations/1319230178522874 +5996666368919599438 57 https:\\%2F\\%2Fv_krymu-soczialnaja-shizirovka-tehniki +5996666368919599438 57 https:\\%2F\\%2Fv_krymu-soczialnaja-shizirovka-tehniki +5996666368919599438 57 https:\\%2F\\%2Fv_krymu-snials/es_ES/hotelstvo-goryachkoy +5996666368919599438 57 https:\\%2F\\%2Fv_krymu-snials/es_ES/hotelstvo-goryachkoy +5996666368919599438 57 https:\\%2F\\%2Fv_krymu-snials/es_ES/hotelstvo-goryachkoy +5996666368919599438 57 https:\\%2F\\%2Fv_krymu-podarks&counter.ru/matematika +5996666368919599438 57 https:\\%2F\\%2Fv_krymu-podarks&counter.ru/matematika +5996666368919599438 57 https:\\%2F\\%2Fv_krymu-planetcalculatorii/cher.exe +5996666368919599438 57 https:\\%2F\\%2Fv_krymu-planetcalculatorii/cher.exe +5996666368919599438 57 https:\\%2F\\%2Fv_krymu-planetcalculatorii/cher.exe +5996666368919599438 57 https:\\%2F\\%2Fv_krymu-planetcalculatorii/cher.exe +5996666368919599438 57 https:\\%2F\\%2Fv_krymu-oshini-yazyk.html&gm=1&lang +5996666368919599438 57 https:\\%2F\\%2Fv_krymu-oshini-yazyk.html&gm=1&lang +5996666368919599438 57 https:\\%2F\\%2Fv_krymu-odinotv/browser=ZmFTTVRNNU5U +5996666368919599438 57 https:\\%2F\\%2Fv_krymu-nasilovaja_tema-hamletradeafnet +5996666368919599438 57 https:\\%2F\\%2Fv_krymu-muzhchinu_polagalnyy-s-rochnikakh +5996666368919599438 57 https:\\%2F\\%2Fv_krymu-200976.140651.html&landinam +5996666368919599438 57 https:\\%2F\\%2Fv_krymu&lr=65&oprnd=105&redircnt=1395593 +5996666368919599438 57 https:\\%2F\\%2Fv_krymu&lr=65&oprnd=105&redircnt=1395593 +5996666368919599438 57 https:\\%2F\\%2Fv_krymskiy-remont-kresh.ru/24219420 +5996666368919599438 57 https:\\%2F\\%2Fv_krymskiy-remont-kresh.ru/24219420 +5996666368919599438 57 https:\\%2F\\%2Fv_krymskih_start.tv/video.mb.advweb +5996666368919599438 57 https:\\%2F\\%2Fv_krymskih_start.tv/video.mb.advweb +5996666368919599438 57 https:\\%2F\\%2Fv_krymskih-sostava.deti.html/ru/forum +5996666368919599438 57 https:\\%2F\\%2Fv_krymskih-sostava.deti.html/ru/forum +5996666368919599438 57 https:\\%2F\\%2Fv_krymskie/kozhare.com/iframe/frm +5996666368919599438 57 https:\\%2F\\%2Fv_krymskie/kozhare.com/iframe/frm +5996666368919599438 57 https:\\%2F\\%2Fv_krymskie/kozhare.com/iframe/frm +5996666368919599438 57 https:\\%2F\\%2Fv_krymskie/kozhare.com/iframe/frm +5996666368919599438 57 https:\\%2F\\%2Fv_krymski-200629.1395383820624343e +5996666368919599438 57 https:\\%2F\\%2Fv_krymski-200629.1395383820624343e +5996666368919599438 57 https:\\%2F\\%2Fv_krymskaya-cikti.avtory/igry-filenom +5996666368919599438 57 https:\\%2F\\%2Fv_krymskaya-cikti.avtory/igry-filenom +5996666368919599438 57 https:\\%2F\\%2Fv_krymskaya-cikti.avtory/igry-filenom +5996666368919599438 57 https:\\%2F\\%2Fv_krymskaya-cikti.avtory/igry-filenom +5996666368919599438 57 https:\\%2F\\%2Fv_krymom-efire.html/en/ari-erke.html +5996666368919599438 57 https:\\%2F\\%2Fv_krymolodeem/politolar/kvadrazhka +5996666368919599438 57 https:\\%2F\\%2Fv_krymolodeem/politolar/kvadrazhka +5996666368919599438 57 https:\\%2F\\%2Fv_krymolodeem/politolar/kvadrazhka +5996666368919599438 57 https:\\%2F\\%2Fv_krymolodeem/politolar/kvadrazhka +5996666368919599438 57 https:\\%2F\\%2Fv_kryme-ru42.html&lang=ru&lr=240320 +5996666368919599438 57 https:\\%2F\\%2Fv_krymchangi-chto-delaet-vosmertile +5996666368919599438 57 https:\\%2F\\%2Fv_krymchangi-chto-delaet-vosmertile +5996666368919599438 57 https:\\%2F\\%2Fv_krymanii-500-ictv.ua/action/sade +5996666368919599438 57 https:\\%2F\\%2Fv_krymanii-500-ictv.ua/action/sade +5996666368919599438 57 https:\\%2F\\%2Fv_krymanii-500-ictv.ua/action/sade +5996666368919599438 57 https:\\%2F\\%2Fv_kryman.org/galeries/football.tv +5996666368919599438 57 https:\\%2F\\%2Fv_kryman.com/iframe/igrushki/mult +5996666368919599438 57 https:\\%2F\\%2Fv_kryman.com/iframe/igrushki/mult +5996666368919599438 57 https:\\%2F\\%2Fv_kryman.com/iframe/igrushki/mult +5996666368919599438 57 https:\\%2F\\%2Fv_kryman-gecer-acoustom%3D200%26width +5996666368919599438 57 https:\\%2F\\%2Fv_kryman-gecer-acoustom%3D200%26width +5996666368919599438 57 https:\\%2F\\%2Fv_kryma_zarazzet.ru/public/?hash= +5996666368919599438 57 https:\\%2F\\%2Fv_kryma_place_id=40318/1518/82 +5996666368919599438 57 https:\\%2F\\%2Fv_kryma_place_id=40318/1518/82 +5996666368919599438 57 https:\\%2F\\%2Fv_kryma_i_kubka-dina.html?918/productDetails +5996666368919599438 57 https:\\%2F\\%2Fv_kryma.html/details/2plus.ru/novostiki +5996666368919599438 57 https:\\%2F\\%2Fv_kryma.html/details/2plus.ru/novostiki +5996666368919599438 57 https:\\%2F\\%2Fv_kryma-v-voprov_natsii-i-poezd-2014 +5996666368919599438 57 https:\\%2F\\%2Fv_kryma-v-voprov_natsii-i-poezd-2014 +5996666368919599438 57 https:\\%2F\\%2Fv_kryma-v-otvetskie.ru/eurosmichester +5996666368919599438 57 https:\\%2F\\%2Fv_kryma-v-otvetskie.ru/eurosmichester +5996666368919599438 57 https:\\%2F\\%2Fv_kryma-programmy.com/arm/proek.ru +5996666368919599438 57 https:\\%2F\\%2Fv_kryma-popular/default6BEQ8HTJXUVBBJRVBNwoIFjI2HA8E +5996666368919599438 57 https:\\%2F\\%2Fv_kryma-popular/default6BEQ8HTJXUVBBJRVBNwoIFjI2HA8E +5996666368919599438 57 https:\\%2F\\%2Fv_kryma-popular/default6BEQ8HTJXUVBBJRVBNwoIFjI2HA8E +5996666368919599438 57 https:\\%2F\\%2Fv_kryma-popular/default6BEQ8HTJXUVBBJRVBNwoIFjI2HA8E +5996666368919599438 57 https:\\%2F\\%2Fv_kryma-palanshowTranslate.xml?bannel +5996666368919599438 57 https:\\%2F\\%2Fv_kryma-palanshowTranslate.xml?bannel +5996666368919599438 57 https:\\%2F\\%2Fv_kryma-palanshowTranslate.xml?bannel +5996666368919599438 57 https:\\%2F\\%2Fv_kryma-nikov_i_stoilo-svitach%2Fcs31378 +5996666368919599438 57 https:\\%2F\\%2Fv_kryma-nikov_i_stoilo-svitach%2Fcs31378 +5996666368919599438 57 https:\\%2F\\%2Fv_kryma-nikov_i_stoilo-svitach%2Fcs31378 +5996666368919599438 57 https:\\%2F\\%2Fv_kryma-nikov_i_stoilo-svitach%2Fcs31378 +5996666368919599438 57 https:\\%2F\\%2Fv_kryma-nikov_i_stoilo-svitach%2Fcs31378 +5996666368919599438 57 https:\\%2F\\%2Fv_kryma-nikov_i_stoilo-svitach%2Fcs31378 +5996666368919599438 57 https:\\%2F\\%2Fv_kryma-ne-v-kriminus-kurinstrukciya +5996666368919599438 57 https:\\%2F\\%2Fv_kryma-ne-v-kriminus-kurinstrukciya +5996666368919599438 57 https:\\%2F\\%2Fv_kryma-kogo-tea.ru/Dianasayfa%2F19 +5996666368919599438 57 https:\\%2F\\%2Fv_kryma-k-spasseparama/society/30058fffa4baeva +5996666368919599438 57 https:\\%2F\\%2Fv_kryma-izobrazilii_ta_ninjali_podzhi +5996666368919599438 57 https:\\%2F\\%2Fv_kryma-izobrazilii_ta_ninjali_podzhi +5996666368919599438 57 https:\\%2F\\%2Fv_kryma-gemoryachenie-olu-ka-y-vyilezik +5996666368919599438 57 https:\\%2F\\%2Fv_kryma-gemoryachenie-olu-ka-y-vyilezik +5996666368919599438 57 https:\\%2F\\%2Fv_kryma&lang=ru&clid=46143&nohead +5996666368919599438 57 https:\\%2F\\%2Fv_kryma&lang=ru&clid=46143&nohead +5996666368919599438 57 https:\\%2F\\%2Fv_krym/bir-na-avtoront/indows.net +5996666368919599438 57 https:\\%2F\\%2Fv_krym/bir-na-avtoront/indows.net +5996666368919599438 57 https:\\%2F\\%2Fv_krym/bir-na-avtoront/indows.net +5996666368919599438 57 https:\\%2F\\%2Fv_krym/4391-mamy.html&scd=yes&rid +5996666368919599438 57 https:\\%2F\\%2Fv_krym/4391-mamy.html&scd=yes&rid +5996666368919599438 57 https:\\%2F\\%2Fv_krym/4391-mamy.html&scd=yes&rid +5996666368919599438 57 https:\\%2F\\%2Fv_krym/4391-mamy.html&scd=yes&rid +5996666368919599438 57 https:\\%2F\\%2Fv_krym/4391-mamy.html&scd=yes&rid +5996666368919599438 57 https:\\%2F\\%2Fv_krym/4391-mamy.html&scd=yes&rid +5996666368919599438 57 https:\\%2F\\%2Fv_krym/4391-mamy.html&scd=yes&rid +5996666368919599438 57 https:\\%2F\\%2Fv_krym/4391-mamy.html&scd=yes&rid +5996666368919599438 57 https:\\%2F\\%2Fv_krym/4368/view=http://image&lr=50 +5996666368919599438 57 https:\\%2F\\%2Fv_krym/4368/view=http://image&lr=50 +5996666368919599438 57 https:\\%2F\\%2Fv_krym/4368/view=http://image&lr=50 +5996666368919599438 57 https:\\%2F\\%2Fv_krym/4368/view=http://image&lr=50 +5996666368919599438 57 https:\\%2F\\%2Fv_krym/434356/#yator_3_1_1258_0.html +5996666368919599438 57 https:\\%2F\\%2Fv_krym/434356/#yator_3_1_1258_0.html +5996666368919599438 57 https:\\%2F\\%2Fv_krym/4321862038904&lr=68&noreask +5996666368919599438 57 https:\\%2F\\%2Fv_krym/4321862038904&lr=68&noreask +5996666368919599438 57 https:\\%2F\\%2Fv_krym/427546.html#i206949049&extras +5996666368919599438 57 https:\\%2F\\%2Fv_krym/427546.html#i206949049&extras +5996666368919599438 57 https:\\%2F\\%2Fv_krym.html/ru/land/doneshow.com +5996666368919599438 57 https:\\%2F\\%2Fv_krym.html/ru/land/doneshow.com +5996666368919599438 57 https:\\%2F\\%2Fv_krym-zamania.com/cat/sms/view%26rnd +5996666368919599438 57 https:\\%2F\\%2Fv_krym-v-mire-soygun.com/r/Thirdegil +5996666368919599438 57 https:\\%2F\\%2Fv_krym-v-mire-soygun.com/r/Thirdegil +5996666368919599438 57 https:\\%2F\\%2Fv_krym-ugrozitiv.tv/ar/search?clid +5996666368919599438 57 https:\\%2F\\%2Fv_krym-south/&referen-ekzamenon.ru +5996666368919599438 57 https:\\%2F\\%2Fv_krym-south/&referen-ekzamenon.ru +5996666368919599438 57 https:\\%2F\\%2Fv_krym-south/&referen-ekzamenon.ru +5996666368919599438 57 https:\\%2F\\%2Fv_krym-prayspondent.qtrax.com/sorular +5996666368919599438 57 https:\\%2F\\%2Fv_krym-polyatsent-type=2&sid=207177 +5996666368919599438 57 https:\\%2F\\%2Fv_krym-podsvetok-vdohnozoneid%3D300 +5996666368919599438 57 https:\\%2F\\%2Fv_krym-po-russkuyu-rebevka/divery +5996666368919599438 57 https:\\%2F\\%2Fv_krym-po-russkuyu-rebevka/divery +5996666368919599438 57 https:\\%2F\\%2Fv_krym-po-russkuyu-rebevka/divery +5996666368919599438 57 https:\\%2F\\%2Fv_krym-po-russkuyu-rebevka/divery +5996666368919599438 57 https:\\%2F\\%2Fv_krym-po-russkuyu-rebevka/divery +5996666368919599438 57 https:\\%2F\\%2Fv_krym-po-russkuyu-rebevka/divery +5996666368919599438 57 https:\\%2F\\%2Fv_krym-po-russkuyu-rebevka/divery +5996666368919599438 57 https:\\%2F\\%2Fv_krym-nizhniy_probegom/hu/private +5996666368919599438 57 https:\\%2F\\%2Fv_krym-nizhniy_probegom/hu/private +5996666368919599438 57 https:\\%2F\\%2Fv_krym-kabel=sireables-tv.ru/kak-zhenskogo +5996666368919599438 57 https:\\%2F\\%2Fv_krym-kabel=sireables-tv.ru/kak-zhenskogo +5996666368919599438 57 https:\\%2F\\%2Fv_krym-gosportal.net/news.yandex.ru +5996666368919599438 57 https:\\%2F\\%2Fv_krym-gosportal.net/news.yandex.ru +5996666368919599438 57 https:\\%2F\\%2Fv_krym-gosportal.net/news.yandex.ru +5996666368919599438 57 https:\\%2F\\%2Fv_krym-gosportal.net/news.yandex.ru +5996666368919599438 57 https:\\%2F\\%2Fv_krym-etotipad-1776798%26sid%3Dad +5996666368919599438 57 https:\\%2F\\%2Fv_krym-etotipad-1776798%26sid%3Dad +5996666368919599438 57 https:\\%2F\\%2Fv_krym-etotipad-1776798%26sid%3Dad +5996666368919599438 57 https:\\%2F\\%2Fv_krym-etotipad-1776798%26sid%3Dad +5996666368919599438 57 https:\\%2F\\%2Fv_krym-etotipad-1776798%26sid%3Dad +5996666368919599438 57 https:\\%2F\\%2Fv_krym-etotipad-1776798%26sid%3Dad +5996666368919599438 57 https:\\%2F\\%2Fv_krym-etotipad-1776798%26sid%3Dad +5996666368919599438 57 https:\\%2F\\%2Fv_krym-etotipad-1776798%26sid%3Dad +5996666368919599438 57 https:\\%2F\\%2Fv_krym-etotipad-1776798%26sid%3Dad +5996666368919599438 57 https:\\%2F\\%2Fv_krym-etotipad-1776798%26sid%3Dad +5996666368919599438 57 https:\\%2F\\%2Fv_krym-etotipad-1776798%26sid%3Dad +5996666368919599438 57 https:\\%2F\\%2Fv_krym-etotipad-1776798%26sid%3Dad +5996666368919599438 57 https:\\%2F\\%2Fv_krym-etotipad-1776798%26sid%3Dad +5996666368919599438 57 https:\\%2F\\%2Fv_krym-etotipad-1776798%26sid%3Dad +5996666368919599438 57 https:\\%2F\\%2Fv_krym-3655668&web=1&search?lr=112 +5996666368919599438 57 https:\\%2F\\%2Fv_krym-3655668&web=1&search?lr=112 +5996666368919599438 57 https:\\%2F\\%2Fv_krym-3652241440&is_app_user_id=1975 +5996666368919599438 57 https:\\%2F\\%2Fv_krym-10336c1d754408/?auto/2280&text +5996666368919599438 57 https:\\%2F\\%2Fv_krym-10336c1d754408/?auto/2280&text +5996666368919599438 57 https:\\%2F\\%2Fv_krym-10336c1d754408/?auto/2280&text +5996666368919599438 57 https:\\%2F\\%2Fv_kryitoe-views/363546998892732-avica +5996666368919599438 57 https:\\%2F\\%2Fv_kryitoe-views/363546998892732-avica +5996666368919599438 57 https:\\%2F\\%2Fv_kryitoe-views/363546998892732-avica +5996666368919599438 57 https:\\%2F\\%2Fv_kryitoe-views/363546998892732-avica +5996666368919599438 57 https:\\%2F\\%2Fv_kryima-chetverglashbox.ua/ru/Виза +5996666368919599438 57 https:\\%2F\\%2Fv_kryima-chetverglashbox.ua/ru/Виза +5996666368919599438 57 https:\\%2F\\%2Fv_kryima-chetverglashbox.ua/ru/Виза +5996666368919599438 57 https:\\%2F\\%2Fv_kryima-27-bolnie-ulas-skider.od +5996666368919599438 57 https:\\%2F\\%2Fv_kryima-27-bolnie-ulas-skider.od +5996666368919599438 57 https:\\%2F\\%2Fv_krivoronej.html?id=3463-sezon.ru +5996666368919599438 57 https:\\%2F\\%2Fv_krivor-kinogo_telejnye/43816840 +5996666368919599438 57 https:\\%2F\\%2Fv_krivor-kinogo_telejnye/43816840 +5996666368919599438 57 https:\\%2F\\%2Fv_krivor-kinogo_telejnye/43816840 +5996666368919599438 57 https:\\%2F\\%2Fv_krimulatnogo.net/maker.com.tr/category +5996666368919599438 57 https:\\%2F\\%2Fv_krimulatnogo.net/maker.com.tr/category +5996666368919599438 57 https:\\%2F\\%2Fv_krimskogo_znat_poka-murinstat/mail +5996666368919599438 57 https:\\%2F\\%2Fv_krimsk.ru/serialu.net/testvennie +5996666368919599438 57 https:\\%2F\\%2Fv_krimsk.ru/serialu.net/testvennie +5996666368919599438 57 https:\\%2F\\%2Fv_krimizi-ve-sluchsearch?cl4url=http +5996666368919599438 57 https:\\%2F\\%2Fv_krimi-rukamosti.ru/offiliatichestory +5996666368919599438 57 https:\\%2F\\%2Fv_krimi-rukamosti.ru/offiliatichestory +5996666368919599438 57 https:\\%2F\\%2Fv_krimi-rukamosti.ru/offiliatichestory +5996666368919599438 57 https:\\%2F\\%2Fv_krimi-rukamosti.ru/offiliatichestory +5996666368919599438 57 https:\\%2F\\%2Fv_krimenetimestion/1128042441ee036f80 +5996666368919599438 57 https:\\%2F\\%2Fv_krimenetimestion/1128042441ee036f80 +5996666368919599438 57 https:\\%2F\\%2Fv_krimenetimestion/1128042441ee036f80 +5996666368919599438 57 https:\\%2F\\%2Fv_krimenetimestion/1128042441ee036f80 +5996666368919599438 57 https:\\%2F\\%2Fv_krimea.com/en/stattovaya.net/onliner +5996666368919599438 57 https:\\%2F\\%2Fv_krim-simplektroapparams=rhost%3D2007 +5996666368919599438 57 https:\\%2F\\%2Fv_krim-simplektroapparams=rhost%3D2007 +5996666368919599438 57 https:\\%2F\\%2Fv_krim-simplektroapparams=rhost%3D2007 +5996666368919599438 57 https:\\%2F\\%2Fv_krim-simplektroapparams=rhost%3D2007 +5996666368919599438 57 https:\\%2F\\%2Fv_krim-is-ilanlar-110-var-vadiseaserver +5996666368919599438 57 https:\\%2F\\%2Fv_krayin-handlerManageral/1121945580 +5996666368919599438 57 https:\\%2F\\%2Fv_krayin-handlerManageral/1121945580 +5996666368919599438 57 https:\\%2F\\%2Fv_kray.org/inter=8147&redircnt=1395120 +5996666368919599438 57 https:\\%2F\\%2Fv_krasnoyarsk.24auto.yandsearch?cl4url +5996666368919599438 57 https:\\%2F\\%2Fv_krasnoyarsk.24auto.yandsearch?cl4url +5996666368919599438 57 https:\\%2F\\%2Fv_krasnodarom_el_6829-fw-102428074G +5996666368919599438 57 https:\\%2F\\%2Fv_krasnodarom_el_6829-fw-102428074G +5996666368919599438 57 https:\\%2F\\%2Fv_kraskrasnopeppandsearch?lr=206408 +5996666368919599438 57 https:\\%2F\\%2Fv_kraskrasnopeppandsearch?lr=206408 +5996666368919599438 57 https:\\%2F\\%2Fv_kraskrasnopeppandsearch?lr=206408 +5996666368919599438 57 https:\\%2F\\%2Fv_kraskrasnopeppandsearch?lr=206408 +5996666368919599438 57 https:\\%2F\\%2Fv_kraskrasnopeppandsearch?lr=206408 +5996666368919599438 57 https:\\%2F\\%2Fv_kraskrasnopeppandsearch?lr=206408 +5996666368919599438 57 https:\\%2F\\%2Fv_krashivai.ru/news%2FCSAFront.ru +5996666368919599438 57 https:\\%2F\\%2Fv_krashivai.ru/news%2FCSAFront.ru +5996666368919599438 57 https:\\%2F\\%2Fv_krashivai.ru/news%2FCSAFront.ru +5996666368919599438 57 https:\\%2F\\%2Fv_krainy-torrenchane-budet-na-krepic +5996666368919599438 57 https:\\%2F\\%2Fv60690544.html&lang=ru_RU +5996666368919599438 57 https:\\%2F\\%2Fv60690544.html&lang=ru_RU +5996666368919599438 57 https:\\%2F\\%2Fv32031_2008355&msid=208369546562 +5996666368919599438 57 https:\\%2F\\%2Fv3112150-wh-676-fw-783-chto_no +5996666368919599438 57 https:\\%2F\\%2Fimage&lr=1110.html +5996666368919599438 57 https:\\%2F\\%2Fcs61657064&web=0&language=http://forum +5996666368919599438 57 https:\\%2F\\%2Fcs61657064&web=0&language=http://forum +5996666368919599438 57 https:\\%2F\\%2Fcs608125&hashkan.net/news.ru/messa/1177 +5996666368919599438 57 https:\\%2F\\%2Fcs608125&hashkan.net/all/project_type=2&oprnd=6638e +5996666368919599438 57 https:\\%2F\\%2Fcs56700758779362/49651&dayfun.ru/launcherafication +5996666368919599438 57 https:\\%2F\\%2Fcs4091844038-fh-448-pd-1-wp-2/en/search +5996666368919599438 57 https:\\%2F\\%2Fcs40252497603-smi-rukoly-shkolazhiganiki +5996666368919599438 57 https:\\%2F\\%2Fcs40252497603-smi-rukoly-shkolazhiganiki +5996666368919599438 57 https:\\%2F\\%2Fcs40252497603-smi-rukoly-shkolazhiganiki +5996666368919599438 57 https:\\%2F\\%2Fcs3168421/comprosy.ru/boevik/getingi.ru +5996666368919599438 57 https:\\%2F\\%2Fcs307300-mlrd_evropeyskie-smut-sborno-onlajn +5996666368919599438 57 https:\\%2F\\%2Fcs307300-mlrd_evropeyskie-smut-sborno-onlajn +5996666368919599438 57 https:\\%2F\\%2Fcommunita-195.html&lang=ru&lr=10223480639.html/en +5996666368919599438 57 https:\\%2F\\%2Fclck%2F17%2F2014 +5996666368919599438 57 https:\\%2F\\%2Fcdn.anyoptik.ua +5996666368919599438 57 https:\\%2F\\%2Fcatalog_ar/?cid=1989613&lr=2&text=windom +5996666368919599438 57 https:\\%2F\\%2Fcatalog_ar/?cid=1989613&lr=2&text=windom +5996666368919599438 57 https:\\%2F\\%2Fcatalog/aavtomobile=0&sessiz-harch?text +5996666368919599438 57 https:\\%2F\\%2Fcatal-hacker=1724&is_mobi/wwwww2.webalta +5996666368919599438 57 https:\\%2F\\%2Fcatal-hacker=1724&is_mobi/wwwww2.webalta +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F9999&lang=ru&lr=143 +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F9999&lang=ru&lr=143 +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F243499000a5f0aa764cd8c862ccdaily +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F243499000a5f0aa764cd8c862ccdaily +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F243499000a5f0aa764cd8c862ccdaily +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F243499000a5f0aa764cd8c862ccdaily +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F243499000a5f0aa764cd8c862ccdaily +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F237053.html?id=1896842ceaf +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F237053.html?id=1896842ceaf +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F237053.html?id=1896842ceaf +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F21%2Fnazval-trana.ru +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F21%2Fnazval-trana.ru +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F21%2F19%2F6%2F10002 +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F21%2F19%2F6%2F10002 +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F21%2F19%2F6%2F10002 +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F21%2F19%2F6%2F10002 +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F21%2F19%2F6%2F10002 +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F21%2F19%2F6%2F10002 +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F21%2F19%2F6%2F10002 +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F21%2F19%2F6%2F10002 +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F21%2F19%2F6%2F10002 +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F21%2F19%2F6%2F10002 +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F21%2F19%2F6%2F10002 +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F21%2F19%2F6%2F10002 +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F21%2F19%2F6%2F10002 +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F21%2F19%2F6%2F10002 +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F21%2F&languagellenie +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F21%2F&languagellenie +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F2014/03/201334587-v +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F2014-03-06-20|curpass.php?id=41005 +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F2014-03-06-20|curpass.php?id=41005 +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F2014-03-06-20|curpass.php?id=41005 +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F2014-03-06-20|curpass.php?id=41005 +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F2014-03-06-20|curpass.php?id=41005 +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F2014-03-06-20|curpass.php?id=41005 +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F20%2F7671,55.30674 +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F20%26rnd%3D0%26height%3D451784 +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F19%2Fpost/1592921/2528 +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F19%2Fpost/1592921/2528 +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F19%2Fpost/1592921/2528 +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F17%2F9ac2\\%2F\\%2FOrl +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F17%2F9ac2\\%2F\\%2FOrl +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F1110km.ru/yandex.ru/ulya-k-ustanavtovzglyan +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F1110km.ru/yandex.ru/ulya-k-ustanavtovzglyan +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F1110km.ru/yandex.ru/ulya-k-ustanavtovzglyan +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F10007-1.html&landiysk +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F10007-1.html&landiysk +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F1.2.1395297.html&lr +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F1.2&name":"Рузаливны контакте +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F1.2&name":"Рузаливны контакте +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F03%2F19%2Freg-sostatus +5996666368919599438 57 https:\\%2F\\%2Fae%2F03%2F03%2F19%2Freg-sostatus +5996666368919599438 57 https:\\%2F\\%2Fae%2F01%2Fnews%2Fanasayfamiliyu-tebastaya.promo +5996666368919599438 57 https:\\%2F\\%2F\\%2Fvk.com/index.ru/download curre=all-nudistanks.ru +5996666368919599438 57 https:\\%2F\\%2F\\%2Fvk.com/index.ru/download curre=all-nudistanks.ru +5996666368919599438 57 https:\\%2F\\%2F\\%2Fvitie-rist.ru/hirdPart=false&needenii/mail=1&bt +5996666368919599438 57 https:\\%2F\\%2F\\%2Fv4163850&group +5996666368919599438 57 https:\\%2F\\%2F\\%2Fv4163850&group +5996666368919599438 57 https:\\%2F\\%2F\\%2Fv413856&text=черномочь в бездо кровь +5996666368919599438 57 https:\\%2F\\%2F\\%2Fv413856&text=черномочь в бездо кровь +5996666368919599438 57 https:\\%2F\\%2F\\%2Fv322938&text=авится +5996666368919599438 57 https:\\%2F\\%2F\\%2Fv%2Fpodrobnosti_i_dogovoryazanye-platform.kz/viders +5996666368919599438 57 https:\\%2F\\%2F\\%2Fv%2Fpodrobnosti_i_dogovoryazanye-platform.kz/viders +5996666368919599438 57 https:\\%2F\\%2F\\%2Fconsultati-sam.com/dukanal +5996666368919599438 57 https:\\%2F\\%2F\\%2F\\%2FWaZUOA4TFgaUSC1KX3GRqYm3qgeXlnsBMnRvQZnJFs2Di2Gia +5996666368919599438 57 https:\\%2F\\%2F\\%2F\\%2FWaZUOA4TFgaUSC1KX3GRqYm3qgeXlnsBMnRvQZnJFs2Di2Gia +5996666368919599438 57 https:\\%2F\\%2F\\%2FWh0LeEBISyd-NgpqegMPbkZaM18GVGl8CAZdVV14NM2JzDXF9dVMiM1ZpTB0LDHUpJSk0T3MINV8ycmpeBl0LNWpVAAByLAMTN3oxJT5 +5996666368919599438 57 https:\\%2F\\%2F\\%2FObnaruto-detochka.rv.ua/virtualnews/2014/dunya +5996666368919599438 57 https:\\%2F\\%2F\\%2FObnaruto-detochka.rv.ua/virtualnews/2014/dunya +5996666368919599438 57 https:\\%2F\\%2F\\%2F82610&text=стрование +5996666368919599438 57 https:\\%2F\\%2F\\%2F82610&text=стрование +5996666368919599438 57 https:\\%2F\\%2F\\%2F2014/0314%2F03%2F20%2F4369.shtml%3Fid%3D45178 +5996666368919599438 57 https:\\%2F\\%2F\\%2F2014/03/18/1525413550%26ref%3Dhttp://asbooks +5996666368919599438 57 https:\\%2F\\%2F\\%2F167948618211.aspx&ref=ser&from=0&parentertainme +5996666368919599438 57 https:\\%2F\\%2F\\%2F167948618211.aspx&ref=ser&from=0&parentertainme +5996666368919599438 57 https:\\%2F\\%2F705d5a18fc4b0a3af5900-ww-1349-wh-660-fw-782-kak-obuchen +5996666368919599438 57 https:\\%2F\\%2F705d5a18fc4b0a3af5900-ww-1349-wh-660-fw-782-kak-obuchen +5996666368919599438 57 https:\\%2F\\%2F479aae08bd385a4c1552246841136&secret=506d9e3dfbd268e6b6630e57 +5996666368919599438 57 https:\\%2F\\%2F19727.html_paramsResult={"respondent/kak_ot +5996666368919599438 57 https:\\%2F\\%2F19727.html_paramsResult={"respondent/kak_ot +5996666368919599438 57 https:\\%2F\\%2F19727.html?photostrategory/teamonkeys/dom +5996666368919599438 57 https:\\%2F\\%2F19727.html?photostrategory/teamonkeys/dom +5996666368919599438 57 https:\\%2F\\%2F19727.html?photostrategory/teamonkeys/dom +5996666368919599438 57 https:\\%2F\\%2F19727.html?photostrategory/teamonkeys/dom +5996666368919599438 57 https:\\%2F\\%2F19727.html?photostrategory/teamonkeys/dom +5996666368919599438 57 https:\\%2F\\%2F19727.html?photostrategory/teamonkeys/dom +5996666368919599438 57 https:\\%2F\\%2F19727.html?photostrategory/teamonkeys/dom +5996666368919599438 57 https:\\%2F\\%2F19727.html?photostrategory/teamonkeys/dom +5996666368919599438 57 https:\\%2F\\%2F19727.html?period]=index.ru/review/4000%26ar +5996666368919599438 57 https:\\%2F\\%2F19727.html?period]=index.ru/review/4000%26ar +5996666368919599438 57 https:\\%2F\\%2F19727.html?id=638461&apiconney-na-lication +5996666368919599438 57 https:\\%2F\\%2F19727.html?id=1108-vslednie_baslanding=windom +5996666368919599438 57 https:\\%2F\\%2F19727.html?id=1108-vslednie_baslanding=windom +5996666368919599438 57 https:\\%2F\\%2F19727.html?ext_messages.yandex.ru/Video/86 +5996666368919599438 57 https:\\%2F\\%2F19727.html?ext_messages.yandex.ru/Video/86 +5996666368919599438 57 https:\\%2F\\%2F19727.html/russiani-izle.org/?cuid=0&is_valysis +5996666368919599438 57 https:\\%2F\\%2F19727.html/russiani-izle.org/?cuid=0&is_valysis +5996666368919599438 57 https:\\%2F\\%2F19727.html/de/launcher/391414/?from=0m7g +5996666368919599438 57 https:\\%2F\\%2F19727.html/de/launcher/391414/?from=0m7g +5996666368919599438 57 https:\\%2FZa-volgaregistration/23033&lr=213&text=салоны.рф/?id=2288497.html +5996666368919599438 57 https:\\%2FZa-volgaregions/19003353836&yaso-dlya-patriplay/en/peppexperimet +5996666368919599438 57 https:\\%2FZa-volgaregion=succer.ru/api.php?model-urunk/?api_id=1763%26bid +5996666368919599438 57 https:\\%2FZa-volgaregion=page-dGiNdCwsTATUVKBZefTcAHxQSDFMwGw0OGXMADFJPMAYUMyAEDXIUZz8qWC4iIHJIKX59d1tXWUZKcQszX0Z5fAZR +5996666368919599438 57 https:\\%2FZa-volgaregion=DE,02,0.001259924_866_1000090&s=06d9e3dfbd268e6b6630e57&secret=0&clid=16974 +5996666368919599438 57 https:\\%2FZa-volgaregion%3D1196897.4049197709743.14/1934/page-2-onlineginal.com.tr%2Fanasayfa%2F12719&text +5996666368919599438 57 https:\\%2FZa-volgaregion%3D1196897.4049197709743.14/1934/page-2-onlineginal.com.tr%2Fanasayfa%2F12719&text +5996666368919599438 57 https:\\%2FZa-volgaregion%26carfinType%3Dad.adriver-85756897853437551e474af062af3ce6bf +5996666368919599438 57 https:\\%2FToWords.auto.ru/newsoskop +5996666368919599438 57 https:\\%2FToWORDERS&model=1569516450/?fromServiceBlock +5996666368919599438 57 https:\\%2FToWORDERS&mode=sl +5996666368919599438 57 https:\\%2F +5996666368919599438 57 https:\\%2F +5996666368919599438 57 https://zyali_dota2.ru/moskva_naznache.ru/guests/?tid +5996666368919599438 57 https://zp.promo=11070144db6bd +5996666368919599438 57 https://zoom/14-nova-orgazeta.ru/cgi-bin/msglist +5996666368919599438 57 https://zoom/14-nova-orgazeta.ru/cgi-bin/msglist +5996666368919599438 57 https://zoom/14-nova-orgazeta.ru/cgi-bin/msglist +5996666368919599438 57 https://zoom/11/07/sberbankalar.org +5996666368919599438 57 https://zoom/11/07/sberbankalar.org +5996666368919599438 57 https://zoom/11/07/sberbank.ru/yandsearch +5996666368919599438 57 https://zonwars.ru/messages/interer.ru/video.html#comment/2-1-0 +5996666368919599438 57 https://zonwars.ru/messages/interer.ru/video.html#comment/2-1-0 +5996666368919599438 57 https://zonwars.ru/messages/inter.ru/?links.eu/en/startv.com +5996666368919599438 57 https://zonwars.ru/messages/inter.ru/?links.eu/en/startv.com +5996666368919599438 57 https://zonwars.ru/messages/inbox/?back +5996666368919599438 57 https://zonwars.ru/messages/inbox/?back +5996666368919599438 57 https://zonwars.ru/messages.yandsearch?text=YouHTC Don\'t +5996666368919599438 57 https://zonwars.ru/messages.yandsearch?text=YouHTC Don\'t +5996666368919599438 57 https://zonwars.autok/1038-fh +5996666368919599438 57 https://zonwars.autok/1038-fh +5996666368919599438 57 https://zonwars.autok/1038-fh +5996666368919599438 57 https://zonwars.autok/1038-fh +5996666368919599438 57 https://zonwars.autok/1038-fh +5996666368919599438 57 https://znanij/128225094419/Komediction/lp3_1/?activestimeout=1440 +5996666368919599438 57 https://znakomstva-v-start=0&clid +5996666368919599438 57 https://znakomstva-v-start=0&clid +5996666368919599438 57 https://znakomstva-v-start=0&clid +5996666368919599438 57 https://znakomstva-v-start=0&clid +5996666368919599438 57 https://znakomstva-v-start=0&clid +5996666368919599438 57 https://znakomstva-v-start=0&clid +5996666368919599438 57 https://zik.ua/user_id=4100164364865&api_id=39094&text=мама тоснение называемые +5996666368919599438 57 https://zik.ua/user_id=41001044/184/putina-ogrency=RUB&litb_from=news.nnn.com/gameid=bc&o=likes +5996666368919599438 57 https://zik.ua/user_id=180893/vsenal.ru/yandsearch +5996666368919599438 57 https://zerogressages/inbox/?back_lishhe-ravni +5996666368919599438 57 https://zerogressages.yandsearch/?sid=207197114&clid=17700945780_510620953697.html&language=10 +5996666368919599438 57 https://zerogressage/24951.139559437/regions/news.qip.ru/yandex.ru/view_interesursant.ru/?clid=1959 +5996666368919599438 57 https://zerogressage/242204065.1395229&text=а стой компьютерея нату&fp=0&parent/ru/prog.php?id_n=38549 +5996666368919599438 57 https://zerogressage/2305140..913800200030515 +5996666368919599438 57 https://zelenta-android/igri_meinfo/news/?auth_key=c4bf18d78ffacce +5996666368919599438 57 https://zelenta-and-rosnezhednego.ru/file/video/search +5996666368919599438 57 https://zelennoven.html/ru/lite/index.ru +5996666368919599438 57 https://zelenkoj-obl.ru/imns/5446-1402280345&ref=advert.ru/stom_key=52416f439176b15d3e0c3131 +5996666368919599438 57 https://zelenkoj-obl.ru/imns/54413349524261.1354599&lang=ru&lang=ru&lr=10741 +5996666368919599438 57 https://zeleniya-ekateriya/obshcheski/psixologda-vyrashivai.ru/news.com/base.garantiseries/music-online-s-i-dlya-v-tepki_v +5996666368919599438 57 https://zelenieter.org/price_serial.ru/daily +5996666368919599438 57 https://zelenieter.org/price_serial.ru/daily +5996666368919599438 57 https://zelenie_djavolozov04_1395238057.html?photo_mans-love.mail=1&search&text=Познер: мнение +5996666368919599438 57 https://zelenie.forecommunity +5996666368919599438 57 https://zelenie.forecommunity +5996666368919599438 57 https://zelenie.forecommunity +5996666368919599438 57 https://zelenie.forecommunity +5996666368919599438 57 https://zelenie.forecommunity +5996666368919599438 57 https://zelendor=7391-magazin/films.ru/cms/boeviki_i_ustanova_Gavrilove.mail.rambler.ru/yandex.ru/novosibirsk +5996666368919599438 57 https://zelenda.eu/playcity/popular/1/#!ts +5996666368919599438 57 https://zelena.com.tr/2014/39549.html&lang=ru +5996666368919599438 57 https://zelena-perego-refererina=1#utm_source=wizard&filmseyret +5996666368919599438 57 https://zdrav.ru/?nocoonozhki +5996666368919599438 57 https://zarubezhnye_channo +5996666368919599438 57 https://zakrytie_svaro/foto/83503&text=ром безопасные +5996666368919599438 57 https://zagovorony-domomics/42-12-margaming.com.ua/zhilo_2013.html?id=52209&subishi-mitinsk.mihael +5996666368919599438 57 https://zagoves-russic.od.ua/putnik-rasslovary_frommail.yandex +5996666368919599438 57 https://zaderga-kollection=1.37871151008371&cm +5996666368919599438 57 https://zaderga-int/2313497,55.755768 +5996666368919599438 57 https://zaboleemaxx.com/iframe/default?abid=3591/?flt +5996666368919599438 57 https://zaboleemaxx.com/bina.html?modelidealnoe +5996666368919599438 57 https://zaboleemaxx.com/bina.html/ru/show_auth_sig=1 +5996666368919599438 57 https://za-magazin/clientLog=0&sessizle.html?pid +5996666368919599438 57 https://za-magazin/clientLog=0&sessizle.html?pid +5996666368919599438 57 https://yobt.com/online/news.yandsearch&text=кем рождения народные блок смотреть +5996666368919599438 57 https://yobt.com/online/news.yandsearch&text=кем рождения народные блок смотреть +5996666368919599438 57 https://yobt.com/13-08-169502817&z=10&targumen/usta.su/user=1&source=wiz&fp=0&userse_of_disco_rozhdeshebnik +5996666368919599438 57 https://yobt.com/13-08-169502817&z=10&targumen/usta.su/user=1&source=wiz&fp=0&userse_of_disco_rozhdeshebnik +5996666368919599438 57 https://yobt.com/13-08-10674931063&win=1012_142089&text=странический +5996666368919599438 57 https://yobt.com/13-08-10674931063&win=1012_142089&text=странический +5996666368919599438 57 https://yobt.com/13-08-1067493-pd-1.20061190752&text=что это&lr=1029998055-wh-629-milli_s_probegator-kokot.org/fanator +5996666368919599438 57 https://yobt.com/13-08-1067493-pd-1.20061190752&text=что это&lr=1029998055-wh-629-milli_s_probegator-kokot.org/fanator +5996666368919599438 57 https://yobt.com/13-08-1067493-pd-1.13951480629&lr=213&text=детский трудническая Народный +5996666368919599438 57 https://yobt.com/13-08-1067493-pd-1.13951480629&lr=213&text=детский трудническая Народный +5996666368919599438 57 https://yandsearch?text=крокомплектро +5996666368919599438 57 https://yandsearch?text=вк&lr=1440&is_secure=0&ad_info=ElsdCQBUSFRhDANSFQrYBs +5996666368919599438 57 https://yandsearch?text=zone.com +5996666368919599438 57 https://yandsearch?clid=1989604&s=1782904.139512684.139555851 +5996666368919599438 57 https://yandsearch?clid=1955484637.html/ru/list/avtomobile +5996666368919599438 57 https://yandsearch?clid=1955484637.html/ru/list/avtomobile +5996666368919599438 57 https://yandsearch?clid=13950406107002-28-29-08-1060-komnatnyiy +5996666368919599438 57 https://yandsearch?clid=13950406107002-28-29-08-1060-komnatnyiy +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch.php +5996666368919599438 57 https://yandsearch +5996666368919599438 57 https://yandex.ru/yandsearch?text=гвоздушный юмор&lr=10328-SHerlock=28673553 +5996666368919599438 57 https://yandex.ru/yandex +5996666368919599438 57 https://yandex.ru/video/merce/viewtopic/98553364232413140 +5996666368919599438 57 https://yandex.ru/video/merce/viewtopic/98553364232413140 +5996666368919599438 57 https://yandex.ru/video/merce/view_post/ru/trahovki-nabluke-sfotostatre +5996666368919599438 57 https://yandex.ru/proizvodstvo_domain/2014.xls&click.cgi?req=19 +5996666368919599438 57 https://yandex.ru/proizvodstvo_domain/2014.xls&click.cgi?req=19 +5996666368919599438 57 https://yandex.ru/prognoz-2014-03-27-bolshe-nuzhnoje/bz-ubore/1145370 +5996666368919599438 57 https://yandex.ru/prognoz-2014-03-27-bolshe-nuzhnoje/bz-ubore/1145370 +5996666368919599438 57 https://yandex.ru/prognoz-2014-03-27-bolshe-nuzhnoje/bz-ubore/1145370 +5996666368919599438 57 https://yandex.ru/prognoz-2014-03-27-bolshe-nuzhnoje/bz-ubore/1145370 +5996666368919599438 57 https://yandex.ru/prognoz-2014-03-27-bolshe-nuzhnoje/bz-ubore/1145370 +5996666368919599438 57 https://yandex.ru/prognoz-2014-03-27-bolshe-nuzhnoje/bz-ubore/1145370 +5996666368919599438 57 https://yandex.ru/prognoz-2014-03-27-bolshe-nuzhnoje/bz-ubore/1145370 +5996666368919599438 57 https://yandex.ru/prognoz-2014-03-27-bolshe-nuzhnoje/bz-ubore/1145370 +5996666368919599438 57 https://yandex.ru/prognoz-2014-03-27-bolshe-nuzhnoje/bz-ubore/1145370 +5996666368919599438 57 https://yandex.ru/prognoz-2014-03-27-bolshe-nuzhnoje/bz-ubore/1145370 +5996666368919599438 57 https://yandex.ru/prognoz-2014-03-27-bolshe-nuzhnoje/bz-ubore/1145370 +5996666368919599438 57 https://yandex.ru/prognoz-2014-03-27-bolshe-nuzhnoje/bz-ubore/1145370 +5996666368919599438 57 https://yandex.ru/prognoz-2014-03-27-bolshe-nuzhnoje/bz-ubore/1145370 +5996666368919599438 57 https://yandex.ru/prognoz-2014-03-27-bolshe-nuzhnoje/bz-ubore/1145370 +5996666368919599438 57 https://yandex.ru/produktory&clid=2014 +5996666368919599438 57 https://yandex.ru/produktory&clid=2014 +5996666368919599438 57 https://yandex.ru/prodazha/domkadryrova-nail%2Fnews +5996666368919599438 57 https://yandex.ru/prodazha/domkadryrova-nail%2Fnews +5996666368919599438 57 https://yandex.ru/prodazha/domkadryrova-nail%2Fnews +5996666368919599438 57 https://yandex.ru/prodazha/domkadryrova-nail%2Fnews +5996666368919599438 57 https://yandex.ru/prodazha/domkadryrova-nail%2Fnews +5996666368919599438 57 https://yandex.ru/prodazha/domkadryrova-nail%2Fnews +5996666368919599438 57 https://yandex.ru/prodau_kvarticles/61263-wh-13004681 +5996666368919599438 57 https://yandex.ru/prodau_kvarticles/61263-wh-13004681 +5996666368919599438 57 https://yandex.ru/prodau_kvarticles/61263-wh-13004681 +5996666368919599438 57 https://yandex.ru/prodau_kvarticles/61263-wh-13004681 +5996666368919599438 57 https://yandex.ru/prodau_kvarticles/61263-wh-13004681 +5996666368919599438 57 https://yandex.ru/prodau_kvarticles/61263-wh-13004681 +5996666368919599438 57 https://yandex.ru/prodau_kvarticles/61263-wh-13004681 +5996666368919599438 57 https://yandex.ru/prodau_kvarticles/61263-wh-13004681 +5996666368919599438 57 https://yandex.ru/prodau_kvarticles/61263-wh-13004681 +5996666368919599438 57 https://yandex.ru/prodau_kvarticles/61263-wh-13004681 +5996666368919599438 57 https://yandex.ru/prodagi.html/ru/cgi +5996666368919599438 57 https://yandex.ru/prodagi.html/ru/cgi +5996666368919599438 57 https://yandex.ru/proda/Motorozhzhukova-preobraslednik.ru/bloxing +5996666368919599438 57 https://yandex.ru/proda/Motorozhzhukova-preobraslednik.ru/bloxing +5996666368919599438 57 https://yandex.ru/proda/Motorozhzhukova-preobraslednik.ru/bloxing +5996666368919599438 57 https://yandex.ru/proda/Motorozhzhukova-preobraslednik.ru/bloxing +5996666368919599438 57 https://yandex.ru/pozner-galaxy +5996666368919599438 57 https://yandex.ru/pole-udarstva/2014-03-2014/91131809-Obraslet +5996666368919599438 57 https://yandex.ru/pole-udarstva/2014 года-дней +5996666368919599438 57 https://yandex.ru/neo2/#inbox +5996666368919599438 57 https://yandex.ru/neo2/#inbox +5996666368919599438 57 https://yandex.ru/neo2/#inbox +5996666368919599438 57 https://yandex.ru/neo2/#inbox +5996666368919599438 57 https://yandex.ru/neo2/#inbox +5996666368919599438 57 https://yandex.ru/neo2/#inbox +5996666368919599438 57 https://yandex.ru/neo2/#inbox +5996666368919599438 57 https://yandex.ru/neo2/#inbox +5996666368919599438 57 https://yandex.ru/neo2/#inbox +5996666368919599438 57 https://yandex.ru/neo2/#inbox +5996666368919599438 57 https://yandex.ru/namba.ru/yandex +5996666368919599438 57 https://yandex.ru/msearch?cl4url=http://pogoda.yandex.ru/neudachivai +5996666368919599438 57 https://yandex.ru/messages/interfax.com/li/rap +5996666368919599438 57 https://yandex.ru/messages/intent=roznik-prisoedinenow.ru/indeslek +5996666368919599438 57 https://yandex.ru/messages/index.ru/neo2/#inbox/?lang=uk&text=former_id=22902.173/?i=20919/ds.info/index +5996666368919599438 57 https://yandex.ru/messages/index.ru/neo2/#inbox/?lang=uk&text=former_id=22902.173/?i=20919/ds.info/index +5996666368919599438 57 https://yandex.ru/messages/inbox;?from=0m7g +5996666368919599438 57 https://yandex.ru/messages/inbox/metricolo1.adriver.ru/Pryg +5996666368919599438 57 https://yandex.ru/messages/inbox/metricolo1.adriver.ru/Pryg +5996666368919599438 57 https://yandex.ru/messages/inbox/messages.yandsearch?text +5996666368919599438 57 https://yandex.ru/messages/inbox/?backer=13&l=map,trfe&trfm=cursoriend_superlic-shop/graycell-banner_id=22876.html/ru/load/igrozhzhi +5996666368919599438 57 https://yandex.ru/messages/inbox/?backer=13&l=map,trfe&trfm=cursoriend_superlic-shop/graycell-banner_id=22876.html/ru/load/igrozhzhi +5996666368919599438 57 https://yandex.ru/messages/inbox/?backer.taka.com/survivorogies-games.ru/uzlovskiy-tsity=19&msid=2755.html +5996666368919599438 57 https://yandex.ru/messages/inbox/?backer.taka.com/survivorogies-games.ru/uzlovskiy-tsity=19&msid=2755.html +5996666368919599438 57 https://yandex.ru/messages/inbox/?backer.ru/video/embed/105755/?from +5996666368919599438 57 https://yandex.ru/messages/inbox/?backer.ru/video/embed/105755/?from +5996666368919599438 57 https://yandex.ru/messages/inbox/?back_url=www.intervye +5996666368919599438 57 https://yandex.ru/messages/inbox/?back_url=www.google +5996666368919599438 57 https://yandex.ru/messages/inbox/?back_url=www.google +5996666368919599438 57 https://yandex.ru/messages/inbox/?back_url=http://pogoda.yandsearch?lr=213&text=sale/506-smi +5996666368919599438 57 https://yandex.ru/messages/inbox/?back_url=http://pogoda.yandsearch?lr=213&text=sale/506-smi +5996666368919599438 57 https://yandex.ru/messages/inbox/?back_url=http +5996666368919599438 57 https://yandex.ru/messages/inbox/?back_url=http +5996666368919599438 57 https://yandex.ru/messages/inbox/?back_url +5996666368919599438 57 https://yandex.ru/messages/inbox/?back_list?folder=describe +5996666368919599438 57 https://yandex.ru/messages/inbox/?back_list?folder=describe +5996666368919599438 57 https://yandex.ru/messages/inbox/?back=1cf308ec3ee12d0c263bd778b92033-1-0-749 +5996666368919599438 57 https://yandex.ru/messages/inbox/?back=1cf308ec3ee12d0c263bd778b92033-1-0-749 +5996666368919599438 57 https://yandex.ru/messages/inbox/?back=1cf308e5d9321134426000 +5996666368919599438 57 https://yandex.ru/messages/inbox/?back=1cf308e5d9321134426000 +5996666368919599438 57 https://yandex.ru/messages/inbox/?back=102&clid=9582&lr=141&nomoovetskogo-rasproject.html?authorizantium.championat.com/direct.html +5996666368919599438 57 https://yandex.ru/messages/inbox/?back=102&clid=9582&lr=141&nomoovetskogo-rasproject.html?authorizantium.championat.com/direct.html +5996666368919599438 57 https://yandex.ru/messages/inbox/?back +5996666368919599438 57 https://yandex.ru/messages/inbox/?back +5996666368919599438 57 https://yandex.ru/messages/inbox/?_1ld=1_867_68787.html +5996666368919599438 57 https://yandex.ru/messages/inbox/?_1ld=1_867_68787.html +5996666368919599438 57 https://yandex.ru/inworldoftankin-kimlarings +5996666368919599438 57 https://yandex.ru/forum44/toly +5996666368919599438 57 https://yandex.ru/epz/map.php?SECTION_ID=2&age[year[1 +5996666368919599438 57 https://yandex.ru/epz/map.php?SECTION_ID=2&age[year[1 +5996666368919599438 57 https://yandex.ru/cgi-bilsya +5996666368919599438 57 https://yandex.ru/cars/user/64 +5996666368919599438 57 https://yandex.ru/add/gia_photo/filmId=Gbu1TE.jpg&pos +5996666368919599438 57 https://yandex.ru/add/gia_photo/filmId=Gbu1TE.jpg&pos +5996666368919599438 57 https://yandex.ru/?source=psearch +5996666368919599438 57 https://yandex.ru/2014%2F03%2F2014/03/2014 +5996666368919599438 57 https://yandex.ru&stype=2&star/politics/21/n_601871 +5996666368919599438 57 https://yandex.ru&stype=2&star/politics/21/n_601871 +5996666368919599438 57 https://yandex.php +5996666368919599438 57 https://yandex.html?wmid=6920%2F&tld +5996666368919599438 57 https://yandex.html&lr=963&spn=0.54236/0002-v-kolesa.kz/post +5996666368919599438 57 https://yandex.html&lr=213&text=дом2 +5996666368919599438 57 https://yandex.html&lr=194&msid=28:2014/03/19/seclub.com.ua/news.rambloger.net +5996666368919599438 57 https://yandex.html&lr=103695&lr=76&rpt=simage&lr=10443097817268.html?html?place_620/?from-prezident +5996666368919599438 57 https://yandex.html&lang=ru&text=сбербанк&lr +5996666368919599438 57 https://yandex.html&lang=ru&lr=172&text=сервис +5996666368919599438 57 https://yandex.html#/battle-ru28 +5996666368919599438 57 https://xxxtimeout=144039/965-fh-598-pd-1-wp-1/enter +5996666368919599438 57 https://xxxtime.net.ru/yandex.ru/yandex.ru/ +5996666368919599438 57 https://xxx-fans.mynet.ua/society/397155&lr=38&text=граница +5996666368919599438 57 https://xxx-fans.mynet.ua/society/397155&lr=38&text=граница +5996666368919599438 57 https://xcadr/42811&h=26e4d00e +5996666368919599438 57 https://wwwxnx-stoyanikyurnyh-karevody/?promoto/15255545447000081302&text=инстагранами-полимпиады +5996666368919599438 57 https://wwwxnx-stoyanikyurnyh-karevody/?promoto/15255545447000081302&text=инстагранами-полимпиады +5996666368919599438 57 https://wwwxnx-stoyanikyurnyh-karevody/?promoto/15254150-quarianty.ru/moskva/odessages +5996666368919599438 57 https://wwwxnx-stoyanikyurnitures.ru/blog_parazzivaet-skrytii-oboi.ru/4227535/?frommail.ru/works/champions +5996666368919599438 57 https://wwwxnx-stoyanikyurnbezsmertvecheniya-novosti.net/viewer_id=30 +5996666368919599438 57 https://wwwxnx-stoyanikyurnal/1121198-flag=10737000072585254563&logged_user/sole/57604 +5996666368919599438 57 https://wwwxnx-raste/default?query=media&markett.com/www.vedojki.ru/cat/tech.mail.ru/moi_rodova +5996666368919599438 57 https://wwwxnx-form_ok=1 +5996666368919599438 57 https://wwwxnx-club.osinka.com.tr/yandex?app=my&clid=19061466&text=лечень фотосессиках +5996666368919599438 57 https://wwwwwww.mn.ru/rus/news.yandex.com/indeks.html?complectblogs +5996666368919599438 57 https://wwwwwww.mn.ru/rus/news.me/blogs.yandex.ua/Zapornozaltin-derey-parata-dlya-mystitut.by/lite-anons.html_params=bid%3D0%26rleurl%3D//ad.evastopolna.tv/teley.co/u/wildberryingFrom/64230635&text=регистраница&clid=497999654742515450 +5996666368919599438 57 https://wwwwwww.mn.ru/rus/2014/911821636 +5996666368919599438 57 https://wwwwwww.hurriyet.com/webhp?espv=1&ie=UTF-8&l=vi&p=AhY_cQZUQQZNA1cHZlcJG1F4XldSeWtqV0BjeH8xIAEsQVgWCVtXd0B +5996666368919599438 57 https://wwwwwww.hurriyet.com/webhp?espv=1&ie=UTF-8&l=vi&p=AhY_cQZUQQZNA1cHZlcJG1F4XldSeWtqV0BjeH8xIAEsQVgWCVtXd0B +5996666368919599438 57 https://wwwwwww.hurriyet.com/webhp?espv=1&ie=UTF-8&l=vi&p=AhY_cQZUQQZNA1cHZlcJG1F4XldSeWtqV0BjeH8xIAEsQVgWCVtXd0B +5996666368919599438 57 https://wwwwwww.hurriyet.com/webhp?espv=1&ie=UTF-8&l=vi&p=AhY_cQZUQQZNA1cHZlcJG1F4XldSeWtqV0BjeH8xIAEsQVgWCVtXd0B +5996666368919599438 57 https://wwwwwww.hurriyet.com/webhp?espv=1&ie=UTF-8&l=vi&p=AhY_cQZUQQZNA1cHZlcJG1F4XldSeWtqV0BjeH8xIAEsQVgWCVtXd0B +5996666368919599438 57 https://wwwwwww.hurriyet.com/webhp?espv=1&ie=UTF-8&l=vi&p=AhY_cQZUQQZNA1cHZlcJG1F4XldSeWtqV0BjeH8xIAEsQVgWCVtXd0B +5996666368919599438 57 https://wwwwwww.hurriyet.com/webhp?espv=1&ie=UTF-8&l=vi&p=AhY_cQZUQQZNA1cHZlcJG1F4XldSeWtqV0BjeH8xIAEsQVgWCVtXd0B +5996666368919599438 57 https://wwwwwww.hurriyet.com/webhp?espv=1&ie=UTF-8&l=vi&p=AhY_cQZUQQZNA1cHZlcJG1F4XldSeWtqV0BjeH8xIAEsQVgWCVtXd0B +5996666368919599438 57 https://wwwwwww.hurriyet.com/webhp?espv=1&ie=UTF-8&l=vi&p=AhY_cQZUQQZNA1cHZlcJG1F4XldSeWtqV0BjeH8xIAEsQVgWCVtXd0B +5996666368919599438 57 https://wwwwwww.hurriyet.com/webhp?espv=1&ie=UTF-8 +5996666368919599438 57 https://wwwwwww.hurriyet.com/webhp?espv=1&ie=UTF-8 +5996666368919599438 57 https://wwwwww.novapost.ru/news/games +5996666368919599438 57 https://wwwwww.faylove555fa70d99bb1b5f0cce3dfbd268e6b6630e57&sec=1742459336 +5996666368919599438 57 https://wwwwww-1073143&text=туальный +5996666368919599438 57 https://wwwwww-1073143&text=торре все серии&lr=56&text=сколио упражнение отдых.рф +5996666368919599438 57 https://wwwwwRedircnt=139561&text=сочинокая +5996666368919599438 57 https://wwwwwRedircnt=1395132592/?flt=17565&aip=Y97&ap=0&come/?lang=ru_RU&back=1cf308ec3ee1aa41ba5a7e440f5e5b6 +5996666368919599438 57 https://wwwwwRedircnt=139503657.1394986/?promo=Keywords[0][refreak.tv/serialu.net/ru/1725119140457.html?4xFwkFaHlGTzhwVVF5qc5RDKCorTFqYY-FS0burEv5c1lnmMKw&text=одноклассності&clid=9582&text=высота_вакансии +5996666368919599438 57 https://wwwww2.sportbox.ru/travmatov/centry=750072468&lr=197&m=12074-smotrebnaya-ot-onlajn/2012 +5996666368919599438 57 https://wwwww2.sportbox.ru/travmatov/centry=750072468&lr=197&m=12074-smotrebnaya-ot-onlajn/2012 +5996666368919599438 57 https://wwwww2.sport.ru/launcher.tv/ar/search?lr=206591&text=олимеры +5996666368919599438 57 https://wwwww.yandex.ua/descriptiz-oralnye-golosyi +5996666368919599438 57 https://wwwww.yandex.ru/yandex240.html&lang=en&p=AhY_cQVSQQ9OAFsAYlcJG1F4XldReWVoX0die38xIAEsQVgWCVtXd0B_d2ACemhbU0dIJRUtcmRyXVdfR1RxCDBIFic4aAMMVQ1AA1MuTEhEGHVfV1dnY3wPFWVyGmMFVRgHC0JdUyd4NWJzDAdydVNXNi1pTW8OCwBdJF9GT3V +5996666368919599438 57 https://wwwww.virtualidad.ru/?url=L2FjY29yZGlsYV9qdircnt=1&md=17903&lr=45&noreask=1&sessune-serial_authorist +5996666368919599438 57 https://wwwww.virtualidad.rt.com/kupazh-onlajn/umnye-vernatures/znanie_kletki-prezident.aspx?q=loginza?back +5996666368919599438 57 https://wwwww.rg.ru/news%2Fapi.php/ust-medlentalk/forum.zr.ru/cgi-bin/msglist?lang=ru&lr=4&text +5996666368919599438 57 https://wwwww.rg.ru/messages/inbox/?_1ld=1_10009118726 +5996666368919599438 57 https://wwwww.radizma-iz-zapchasinochag-konchaya-prodam/1-k_kvartinki +5996666368919599438 57 https://wwwww.radiopoiskphotos.ru/games/5819104312412150228689-svetlyj +5996666368919599438 57 https://wwwww.radiopoiskphotos%2F2014/9126530317/99906591&lr=2&text=vancest-drakon.ru/sdoc.html?page=http://spec-pod-v +5996666368919599438 57 https://wwwww.hurriza/ +5996666368919599438 57 https://wwwww.hurriza/ +5996666368919599438 57 https://wwwww.hurriyetemlin.ru/yandex.ru/start/memuruz.net/details/?id=1959248000208&rpt=stor.html?fid=193&v=5000&src=895&secret=7709%2F124723008354/n4p/2119797464a9437.html/id/1597&lb_id=-2906.1395167769&buttoStartv.com.ua/News/2014%2F03%2F11002&yearfromWizard&film/5854731/substepway/1351-wh-1082&city +5996666368919599438 57 https://wwwww.hurriyetemlin.ru/yandex.ru/start/memuruz.net/details/?id=1959248000208&rpt=stor.html?fid=193&v=5000&src=895&secret=7709%2F124723008354/n4p/2119797464a9437.html/id/1597&lb_id=-2906.1395167769&buttoStartv.com.ua/News/2014%2F03%2F11002&yearfromWizard&film/5854731/substepway/1351-wh-1082&city +5996666368919599438 57 https://wwwww.hurriyetails/142/?toke +5996666368919599438 57 https://wwwww.hurriyetails/142/?toke +5996666368919599438 57 https://wwwww.hurriyetails&id=22894.2363&win=1139663/?PID=2189.php?id=2086311015302-kreditor-salon--170320/100000000 +5996666368919599438 57 https://wwwww.hurriyet.com/webmoney.yandex.ru/vykroika-s-kards +5996666368919599438 57 https://wwwww.hurriyet.com/usluginplanetiphone5,2&title=Мастений +5996666368919599438 57 https://wwwww.hurriyet.com/usluginplanetiphone5,2&title=Мастений +5996666368919599438 57 https://wwwww.hurriyet.com/usluginplanetiphone5,2&title=Мастений +5996666368919599438 57 https://wwwww.hurriyet.com/usluginplanetiphone5,2&title=Мастений +5996666368919599438 57 https://wwwww.hurriyet.com/usluginplanetiphone5,2&title=Мастений +5996666368919599438 57 https://wwwww.hurriyet.com/usluginplanetiphone5,2&title=Мастений +5996666368919599438 57 https://wwwww.hurriyet.com/user=164&lr=213&text=репель аморал +5996666368919599438 57 https://wwwww.hurriyet.com/ua/p2424435.html/pl/cat/1555768&spn=0.001.94140313/99924457 +5996666368919599438 57 https://wwwww.hurriyet.com/tr/public/?hash=lhZT8HbPFxN2FzqC8W+2gSKhWFFoKF2jJ1lO0qggo04 +5996666368919599438 57 https://wwwww.hurriyet.com/tr/public/?hash=lhZT8HbPFxN2FzqC8W+2gSKhWFFoKF2jJ1lO0qggo04 +5996666368919599438 57 https://wwwww.hurriyet.com/tr/public/?hash=lhZT8HbPFxN2FzqC8W+2gSKhWFFoKF2jJ1lO0qggo04 +5996666368919599438 57 https://wwwww.hurriyet.com/tr/public/?hash=lhZT8HbPFxN2FzqC8W+2gSKhWFFoKF2jJ1lO0qggo04 +5996666368919599438 57 https://wwwww.hurriyet.com/tr/public/?hash=lhZT8HbPFxN2FzqC8W+2gSKhWFFoKF2jJ1lO0qggo04 +5996666368919599438 57 https://wwwww.hurriyet.com/tr/public/?hash=lhZT8HbPFxN2FzqC8W+2gSKhWFFoKF2jJ1lO0qggo04 +5996666368919599438 57 https://wwwww.hurriyet.com/tr/public/?hash=lhZT8HbPFxN2FzqC8W+2gSKhWFFoKF2jJ1lO0qggo04 +5996666368919599438 57 https://wwwww.hurriyet.com/tr/public/?hash=lhZT8HbPFxN2FzqC8W+2gSKhWFFoKF2jJ1lO0qggo04 +5996666368919599438 57 https://wwwww.hurriyet.com/tr/public/?hash=lhZT8HbPFxN2FzqC8W+2gSKhWFFoKF2jJ1lO0qggo04 +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcherny-185-fw-930138002003556222-besplazmerovoe-tehnike South/aHR0cDovL3J1LmdvbG9zLnVhL3BvbGVfcjJfferendcounternoe-kinopoisk.com/api.vk.me%2F30883m.67819629&lr=11119&z=16&sort=mackdownload/mail.ru/?clid=91&aid=39097687938\\%2Fviperatsii_2013/1/?q=RUR&page=http://liver.ru/video/search?text=brandartid=666434/?from=email.ru/yandex.php?t=155164/şəxsiyyətin-serialy-domain=hurriyetter,mail.ru/CSAFront/index?appkey=8af5-8e7f-fee39b9f5c968177026623 +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/tr/launcher/frame/frm_index.ua/society +5996666368919599438 57 https://wwwww.hurriyet.com/th/special_2013_goda.yandex.ru +5996666368919599438 57 https://wwwww.hurriyet.com/th/special_2013_goda.yandex.ru +5996666368919599438 57 https://wwwww.hurriyet.com/th/special_2013_goda.yandex.ru +5996666368919599438 57 https://wwwww.hurriyet.com/th/special_2013_goda.yandex.ru +5996666368919599438 57 https://wwwww.hurriyet.com/th/special_2013_goda.yandex.ru +5996666368919599438 57 https://wwwww.hurriyet.com/th/special_2013_goda.yandex.ru +5996666368919599438 57 https://wwwww.hurriyet.com/th/special_2013_goda.yandex.ru +5996666368919599438 57 https://wwwww.hurriyet.com/th/special_2013_goda.yandex.ru +5996666368919599438 57 https://wwwww.hurriyet.com/surgut/kids/menu.php?f=19067250477.html +5996666368919599438 57 https://wwwww.hurriyet.com/s/shop.xml?cvred.net/flemontent +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Достоит все плавающие +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Достоит все плаваша и по фото&img_url=www.kommenta/index.ru/yandex.ru/blog/referer=http +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Достоит все плавать разблокированного&clid=228452 +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Достоит все плавать разблокированного&clid=228452 +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Достоит все плаванга&default.aspx&refplace=user +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Достоит все плава с отзывы&lr=28585.shtml?stat.ru +5996666368919599438 57 https://wwwww.hurriyet.com/ru/peterskii-irina%2F100002-intin-solitikal.com/uploads +5996666368919599438 57 https://wwwww.hurriyet.com/ru/petersk/kvarticle/83828&conds.html?modeli-gayduk_zakhvaty-esco_rooferti-mini +5996666368919599438 57 https://wwwww.hurriyet.com/ru/petersburg.mnogo.net/Game%3D0%26ad%3D%26xpid%3D3239658 +5996666368919599438 57 https://wwwww.hurriyet.com/ru/peters/xerox-cams.ru/213/balta.ru/articles +5996666368919599438 57 https://wwwww.hurriyet.com/ru/peterburgedu.by/news/n1705/sizzle/?mlocalenko-trax.com/beshchenie/subjects/movie/#page=http://kurity +5996666368919599438 57 https://wwwww.hurriyet.com/ru/peterburg/viewer_type=event/diskillforter97 +5996666368919599438 57 https://wwwww.hurriyet.com/ru/peterburg/viewer_type=event/diskillforter97 +5996666368919599438 57 https://wwwww.hurriyet.com/ru/peterburg/ru/news_block_new.aspx&refererdotov-i-smotret_org_id=48418&lr=1144 +5996666368919599438 57 https://wwwww.hurriyet.com/ru/peterburg/ru/launcher.html/al51.html&lang=ru&lr=143 +5996666368919599438 57 https://wwwww.hurriyet.com/ru/peterburg/ru/launcher.html/al51.html&lang=ru&lr=143 +5996666368919599438 57 https://wwwww.hurriyet.com/ru/peterburg/ru/launcher.html/al51.html&lang=ru&lr=143 +5996666368919599438 57 https://wwwww.hurriyet.com/ru/peterburg/ru/launcher.html/al51.html&lang=ru&lr=143 +5996666368919599438 57 https://wwwww.hurriyet.com/ru/peterburg.su/news.yandex-teatrad_polzovala-rossiya/avtovoj.html?id=203951571891519&lr=213&text +5996666368919599438 57 https://wwwww.hurriyet.com/ru/peterbanks.ru/economissi-bin/bankionline/1793.html +5996666368919599438 57 https://wwwww.hurriyet.com/ru/peterbanks.ru/economissi-bin/bankionline/1793.html +5996666368919599438 57 https://wwwww.hurriyet.com/ru/peterbanki-vannova-windows 7 SP1 1.htmlquot.zhtml?id=1359/commu-sveklops +5996666368919599438 57 https://wwwww.hurriyet.com/ru/message/2335.11555&aip=10735603405-rosti/862864 +5996666368919599438 57 https://wwwww.hurriyet.com/ru/message/2335.11555&aip=10735603405-rosti/862864 +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Pryg-i-Skok?ref=11&search?clid=21050&parent_lang=ru&lr=54 +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss +5996666368919599438 57 https://wwwww.hurriyet.com/ru/Planyrss diff --git a/tests/queries/1_stateful/00181_cross_join_compression.sql b/tests/queries/1_stateful/00181_cross_join_compression.sql new file mode 100644 index 00000000000..7457f545c5d --- /dev/null +++ b/tests/queries/1_stateful/00181_cross_join_compression.sql @@ -0,0 +1,3 @@ +CREATE VIEW trunc_hits AS (SELECT * FROM test.hits LIMIT 1); + +SELECT WatchID, CounterID, StartURL FROM trunc_hits, test.visits ORDER BY (WatchID, CounterID, StartURL) DESC LIMIT 1000; \ No newline at end of file From 4bb827e987781c9a754b1c7c5eb1f46acd28a90a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=B1=AA=E8=82=A5=E8=82=A5?= Date: Thu, 28 Mar 2024 14:28:42 +0800 Subject: [PATCH 0086/1018] fix npy big endianness --- src/Processors/Formats/Impl/NpyRowInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/NpyRowInputFormat.cpp b/src/Processors/Formats/Impl/NpyRowInputFormat.cpp index 795ad12ac98..65e0f9dd192 100644 --- a/src/Processors/Formats/Impl/NpyRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/NpyRowInputFormat.cpp @@ -131,7 +131,7 @@ std::shared_ptr parseType(String type) NumpyDataType::Endianness endianness; if (type[0] == '<') endianness = NumpyDataType::Endianness::LITTLE; - else if (type[1] == '>') + else if (type[0] == '>') endianness = NumpyDataType::Endianness::BIG; else if (type[0] == '|') endianness = NumpyDataType::Endianness::NONE; From 21d4cb775a6e9ab5840c3644ae0de33502760ed7 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 28 Mar 2024 09:12:43 -0300 Subject: [PATCH 0087/1018] rename db to see if this helps --- ...attach_table_if_active_replica_already_exists.sql | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sql b/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sql index 0129e7392f0..3706b3d1751 100644 --- a/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sql +++ b/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sql @@ -1,12 +1,12 @@ -drop database if exists at; -create database at engine = Atomic; +drop database if exists atomic_db; +create database atomic_db engine = Atomic; -drop table if exists at.t1 sync; -drop table if exists at.t2 sync; +drop table if exists atomic_db.t1 sync; +drop table if exists atomic_db.t2 sync; -create table at.t1 (a Int) +create table atomic_db.t1 (a Int) engine=ReplicatedMergeTree('/clickhouse/tables/{database}/test', 'r1') order by tuple() SETTINGS index_granularity = 8192; -attach table at.t2 UUID '6c32d92e-bebf-4730-ae73-c43e5748f829' +attach table atomic_db.t2 UUID '6c32d92e-bebf-4730-ae73-c43e5748f829' (a Int) engine=ReplicatedMergeTree('/clickhouse/tables/{database}/test', 'r1') order by tuple() SETTINGS index_granularity = 8192; -- { serverError REPLICA_ALREADY_EXISTS }; From 61b7c942ac85fc2a1be6b6564e8fe50a0da9fd99 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 28 Mar 2024 14:27:30 +0100 Subject: [PATCH 0088/1018] Fix clickhouse-test in case of missing .reference file Before it throws internal python error: $ ../tests/clickhouse-test 03033 Using queries from '/src/ch/clickhouse/tests/queries' directory Connecting to ClickHouse server... OK Connected to server 24.3.1.1 @ 3fa6d23730d8e7a7c72b194ee4e04972b9941d68 master Running 1 stateless tests (MainProcess). 03033_dist: [ UNKNOWN ] - Test internal error: TypeError expected str, bytes or os.PathLike object, not NoneType File "/src/ch/clickhouse/.cmake/../tests/clickhouse-test", line 1644, in run if not is_valid_utf_8(self.case_file) or not is_valid_utf_8( ^^^^^^^^^^^^^^^ File "/src/ch/clickhouse/.cmake/../tests/clickhouse-test", line 237, in is_valid_utf_8 with open(fname, "rb") as f: ^^^^^^^^^^^^^^^^^ 0 tests passed. 0 tests skipped. 0.01 s elapsed (MainProcess). Won't run stateful tests because test data wasn't loaded. All tests have finished. Now: $ ../tests/clickhouse-test 03033 Using queries from '/src/ch/clickhouse/tests/queries' directory Connecting to ClickHouse server... OK Connected to server 24.3.1.1 @ 3fa6d23730d8e7a7c72b194ee4e04972b9941d68 master Running 1 stateless tests (MainProcess). 03033_dist: [ UNKNOWN ] - no reference file 0 tests passed. 0 tests skipped. 0.11 s elapsed (MainProcess). Won't run stateful tests because test data wasn't loaded. All tests have finished. Signed-off-by: Azat Khuzhin --- tests/clickhouse-test | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 10851d23481..51498b43419 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1641,8 +1641,8 @@ class TestCase: client_options = self.add_random_settings(client_options) - if not is_valid_utf_8(self.case_file) or not is_valid_utf_8( - self.reference_file + if not is_valid_utf_8(self.case_file) or ( + self.reference_file and not is_valid_utf_8(self.reference_file) ): proc, stdout, stderr, debug_log, total_time = self.run_single_test( server_logs_level, client_options From 8436bb8d7275cee2297788d057ba54a941aebf58 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 28 Mar 2024 11:15:31 -0300 Subject: [PATCH 0089/1018] use random db name --- ...if_active_replica_already_exists.reference | 1 + ..._table_if_active_replica_already_exists.sh | 22 +++++++++++++++++++ ...table_if_active_replica_already_exists.sql | 12 ---------- 3 files changed, 23 insertions(+), 12 deletions(-) create mode 100755 tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sh delete mode 100644 tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sql diff --git a/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.reference b/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.reference index e69de29bb2d..7261e15108c 100644 --- a/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.reference +++ b/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.reference @@ -0,0 +1 @@ +REPLICA_ALREADY_EXISTS diff --git a/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sh b/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sh new file mode 100755 index 00000000000..e351375598e --- /dev/null +++ b/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sh @@ -0,0 +1,22 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +db_name=$(tr -dc A-Za-z &1 | grep -o 'REPLICA_ALREADY_EXISTS' diff --git a/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sql b/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sql deleted file mode 100644 index 3706b3d1751..00000000000 --- a/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sql +++ /dev/null @@ -1,12 +0,0 @@ -drop database if exists atomic_db; -create database atomic_db engine = Atomic; - -drop table if exists atomic_db.t1 sync; -drop table if exists atomic_db.t2 sync; - -create table atomic_db.t1 (a Int) - engine=ReplicatedMergeTree('/clickhouse/tables/{database}/test', 'r1') - order by tuple() SETTINGS index_granularity = 8192; -attach table atomic_db.t2 UUID '6c32d92e-bebf-4730-ae73-c43e5748f829' - (a Int) engine=ReplicatedMergeTree('/clickhouse/tables/{database}/test', 'r1') - order by tuple() SETTINGS index_granularity = 8192; -- { serverError REPLICA_ALREADY_EXISTS }; From 7e5b5420cb4270c424344ef4b33b40ed08370294 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 28 Mar 2024 14:44:28 +0000 Subject: [PATCH 0090/1018] impl --- src/Storages/StorageReplicatedMergeTree.cpp | 23 ++++++++++++++++----- 1 file changed, 18 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c41403e312b..e0d041dc96c 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5795,11 +5795,24 @@ bool StorageReplicatedMergeTree::optimize( if (select_decision != SelectPartsDecision::SELECTED) { - constexpr const char * message_fmt = "Cannot select parts for optimization: {}"; - assert(disable_reason != unknown_disable_reason); - if (!partition_id.empty()) - disable_reason += fmt::format(" (in partition {})", partition_id); - return handle_noop(message_fmt, disable_reason); + if (try_no + 1 < max_retries) + { + /// Here we trying to have a similar behaviour to ordinary MergeTree: if some merges are already in progress - let's wait for them to finish. + /// This way `optimize final` won't just silently be a noop (if also `optimize_throw_if_noop=false`), but will wait for the active merges and repeat an attempt to schedule final merge. + /// This guarantees are enough for tests, because there we have full control over insertions. + const auto wait_timeout = query_context->getSettingsRef().receive_timeout.totalMilliseconds() / max_retries; + /// DEFAULT (and not LIGHTWEIGHT) because merges are not condidered lightweight; empty `source_replicas` means "all replicas" + waitForProcessingQueue(wait_timeout, SyncReplicaMode::DEFAULT, {}); + continue; + } + else + { + constexpr const char * message_fmt = "Cannot select parts for optimization: {}"; + assert(disable_reason != unknown_disable_reason); + if (!partition_id.empty()) + disable_reason += fmt::format(" (in partition {})", partition_id); + return handle_noop(message_fmt, disable_reason); + } } ReplicatedMergeTreeLogEntryData merge_entry; From fd925770b2de0ea067541a156c796552a94f3d11 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 28 Mar 2024 17:00:54 +0100 Subject: [PATCH 0091/1018] Fixes after merge --- src/Interpreters/Cache/EvictionCandidates.h | 6 +- src/Interpreters/Cache/FileCache.cpp | 5 +- .../Cache/LRUFileCachePriority.cpp | 72 +++++++++---------- .../Cache/SLRUFileCachePriority.cpp | 2 +- 4 files changed, 43 insertions(+), 42 deletions(-) diff --git a/src/Interpreters/Cache/EvictionCandidates.h b/src/Interpreters/Cache/EvictionCandidates.h index 7468c159dae..b2bb6ae9e9f 100644 --- a/src/Interpreters/Cache/EvictionCandidates.h +++ b/src/Interpreters/Cache/EvictionCandidates.h @@ -7,15 +7,17 @@ namespace DB class EvictionCandidates { public: + using FinalizeEvictionFunc = std::function; + EvictionCandidates() = default; EvictionCandidates(EvictionCandidates && other) noexcept { candidates = std::move(other.candidates); candidates_size = std::move(other.candidates_size); + on_finalize = std::move(other.on_finalize); queue_entries_to_invalidate = std::move(other.queue_entries_to_invalidate); - finalize_eviction_func = std::move(other.finalize_eviction_func); + hold_space = std::move(other.hold_space); } - using FinalizeEvictionFunc = std::function; ~EvictionCandidates(); diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 8dfad6b8edf..455d11a3c1b 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -997,7 +997,7 @@ void FileCache::freeSpaceRatioKeepingThreadFunc() main_priority->getSize(lock), size_limit, main_priority->getElementsCount(lock), elements_limit, desired_size, desired_elements_num, - eviction_candidates.size(), stat.stat.non_releasable_count); + eviction_candidates.size(), stat.total_stat.non_releasable_count); lock.unlock(); eviction_candidates.evict(); @@ -1345,7 +1345,8 @@ void FileCache::deactivateBackgroundOperations() { shutdown.store(true); metadata.shutdown(); - keep_up_free_space_ratio_task->deactivate(); + if (keep_up_free_space_ratio_task) + keep_up_free_space_ratio_task->deactivate(); } std::vector FileCache::getFileSegmentInfos(const UserID & user_id) diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index 4e375813727..3780cd1750f 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -275,10 +275,43 @@ bool LRUFileCachePriority::collectCandidatesForEviction( auto can_fit = [&] { - return canFit(size, 1, stat.stat.releasable_size, stat.stat.releasable_count, nullptr, nullptr, lock); + return canFit(size, elements, stat.total_stat.releasable_size, stat.total_stat.releasable_count, lock); }; + iterateForEviction(res, stat, can_fit, lock); - return can_fit(); + + if (can_fit()) + { + /// As eviction is done without a cache priority lock, + /// then if some space was partially available and some needed + /// to be freed via eviction, we need to make sure that this + /// partially available space is still available + /// after we finish with eviction for non-available space. + /// So we create a space holder for the currently available part + /// of the required space for the duration of eviction of the other + /// currently non-available part of the space. + + const size_t hold_size = size > stat.total_stat.releasable_size + ? size - stat.total_stat.releasable_size + : 0; + + const size_t hold_elements = elements > stat.total_stat.releasable_count + ? elements - stat.total_stat.releasable_count + : 0; + + if (hold_size || hold_elements) + res.setSpaceHolder(hold_size, hold_elements, *this, lock); + + // LOG_TEST(log, "Collected {} candidates for eviction (total size: {}). " + // "Took hold of size {} and elements {}", + // res.size(), stat.total_stat.releasable_size, hold_size, hold_elements); + + return true; + } + else + { + return false; + } } EvictionCandidates LRUFileCachePriority::collectCandidatesForEviction( @@ -334,41 +367,6 @@ void LRUFileCachePriority::iterateForEviction( }, lock); } - - if (can_fit()) - { - /// As eviction is done without a cache priority lock, - /// then if some space was partially available and some needed - /// to be freed via eviction, we need to make sure that this - /// partially available space is still available - /// after we finish with eviction for non-available space. - /// So we create a space holder for the currently available part - /// of the required space for the duration of eviction of the other - /// currently non-available part of the space. - - const size_t hold_size = size > stat.total_stat.releasable_size - ? size - stat.total_stat.releasable_size - : 0; - - const size_t hold_elements = elements > stat.total_stat.releasable_count - ? elements - stat.total_stat.releasable_count - : 0; - - if (hold_size || hold_elements) - res.setSpaceHolder(hold_size, hold_elements, *this, lock); - - // LOG_TEST(log, "Collected {} candidates for eviction (total size: {}). " - // "Took hold of size {} and elements {}", - // res.size(), stat.total_stat.releasable_size, hold_size, hold_elements); - - return true; - } - else - { - return false; - } -} - LRUFileCachePriority::LRUIterator LRUFileCachePriority::move( LRUIterator & it, LRUFileCachePriority & other, diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.cpp b/src/Interpreters/Cache/SLRUFileCachePriority.cpp index 1503c23337c..96fd4185fbb 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/SLRUFileCachePriority.cpp @@ -266,7 +266,7 @@ EvictionCandidates SLRUFileCachePriority::collectCandidatesForEviction( desired_probationary_size, desired_probationary_elements_num, max_candidates_to_evict, stat, lock); chassert(res.size() <= max_candidates_to_evict); - chassert(res.size() == stat.stat.releasable_count); + chassert(res.size() == stat.total_stat.releasable_count); if (res.size() == max_candidates_to_evict) return res; From becedbdf4d79d3788f98e6eeb9faf3380c2b3d47 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 28 Mar 2024 18:22:58 +0000 Subject: [PATCH 0092/1018] add test --- .../03015_optimize_final_rmt.reference | 1 + .../0_stateless/03015_optimize_final_rmt.sh | 25 +++++++++++++++++++ 2 files changed, 26 insertions(+) create mode 100644 tests/queries/0_stateless/03015_optimize_final_rmt.reference create mode 100755 tests/queries/0_stateless/03015_optimize_final_rmt.sh diff --git a/tests/queries/0_stateless/03015_optimize_final_rmt.reference b/tests/queries/0_stateless/03015_optimize_final_rmt.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/03015_optimize_final_rmt.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/03015_optimize_final_rmt.sh b/tests/queries/0_stateless/03015_optimize_final_rmt.sh new file mode 100755 index 00000000000..96cb2ea22d1 --- /dev/null +++ b/tests/queries/0_stateless/03015_optimize_final_rmt.sh @@ -0,0 +1,25 @@ +#!/usr/bin/env bash +# Tags: long + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} -q "CREATE TABLE 03015_optimize_final_rmt(a UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/03015_optimize_final_rmt', 'r1') ORDER BY a SETTINGS min_age_to_force_merge_seconds=1, merge_selecting_sleep_ms=100" + +for _ in {0..10}; do + ${CLICKHOUSE_CLIENT} --insert_deduplicate 0 -q "INSERT INTO 03015_optimize_final_rmt select * from numbers_mt(1e6)" +done + +# trigger a merge if it is not already running +${CLICKHOUSE_CLIENT} -q "OPTIMIZE TABLE 03015_optimize_final_rmt FINAL" & + +# this query should wait for the running merges, not just return immediately +${CLICKHOUSE_CLIENT} -q "OPTIMIZE TABLE 03015_optimize_final_rmt FINAL" + +# then at this point we should have a single part +${CLICKHOUSE_CLIENT} -q "SELECT COUNT() FROM system.parts WHERE database = currentDatabase() AND table = '03015_optimize_final_rmt' AND active" + +wait + +${CLICKHOUSE_CLIENT} --query "DROP TABLE 03015_optimize_final_rmt" From b9727387541c3ac1a08691e6ea8abd272dfe9ccd Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 28 Mar 2024 15:55:33 -0300 Subject: [PATCH 0093/1018] :D --- ..._attach_table_if_active_replica_already_exists.sh | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sh b/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sh index e351375598e..85fb900bf83 100755 --- a/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sh +++ b/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sh @@ -6,17 +6,17 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) db_name=$(tr -dc A-Za-z &1 | grep -o 'REPLICA_ALREADY_EXISTS' + order by tuple() SETTINGS index_granularity = 8192" 2>&1 | grep -m 1 -o -F 'REPLICA_ALREADY_EXISTS' + +$CLICKHOUSE_CLIENT --query "drop database ${db_name} sync" From acd8a1f32e6640db767fb44e675c24cce4ea564c Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 29 Mar 2024 09:15:45 +0000 Subject: [PATCH 0094/1018] Enable custom parquet encoder by default --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 84e709294aa..f4e34aeea3c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1111,7 +1111,7 @@ class IColumn; M(ParquetVersion, output_format_parquet_version, "2.latest", "Parquet format version for output format. Supported versions: 1.0, 2.4, 2.6 and 2.latest (default)", 0) \ M(ParquetCompression, output_format_parquet_compression_method, "zstd", "Compression method for Parquet output format. Supported codecs: snappy, lz4, brotli, zstd, gzip, none (uncompressed)", 0) \ M(Bool, output_format_parquet_compliant_nested_types, true, "In parquet file schema, use name 'element' instead of 'item' for list elements. This is a historical artifact of Arrow library implementation. Generally increases compatibility, except perhaps with some old versions of Arrow.", 0) \ - M(Bool, output_format_parquet_use_custom_encoder, false, "Use a faster Parquet encoder implementation.", 0) \ + M(Bool, output_format_parquet_use_custom_encoder, true, "Use a faster Parquet encoder implementation.", 0) \ M(Bool, output_format_parquet_parallel_encoding, true, "Do Parquet encoding in multiple threads. Requires output_format_parquet_use_custom_encoder.", 0) \ M(UInt64, output_format_parquet_data_page_size, 1024 * 1024, "Target page size in bytes, before compression.", 0) \ M(UInt64, output_format_parquet_batch_size, 1024, "Check page size every this many rows. Consider decreasing if you have columns with average values size above a few KBs.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 170836cb980..7ea721ec955 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -125,6 +125,7 @@ static std::map sett {"azure_max_upload_part_size", 5ull*1024*1024*1024, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to Azure blob storage."}, {"azure_upload_part_size_multiply_factor", 2, 2, "Multiply azure_min_upload_part_size by this factor each time azure_multiply_parts_count_threshold parts were uploaded from a single write to Azure blob storage."}, {"azure_upload_part_size_multiply_parts_count_threshold", 500, 500, "Each time this number of parts was uploaded to Azure blob storage, azure_min_upload_part_size is multiplied by azure_upload_part_size_multiply_factor."}, + {"output_format_parquet_use_custom_encoder", false, true, "Enable custom Parquet encoder."}, }}, {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, From 3d7c1db763a6b25f4f2ec954851080f0c3df066f Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Sun, 31 Mar 2024 12:56:44 +0200 Subject: [PATCH 0095/1018] Copy comment when using CREATE AS statement --- src/Backups/DDLAdjustingForBackupVisitor.cpp | 2 +- src/Databases/DatabaseFilesystem.cpp | 2 +- src/Databases/DatabaseMemory.cpp | 2 +- src/Databases/DatabaseOnDisk.cpp | 5 ++--- src/Databases/DatabasesCommon.cpp | 4 ++-- src/Interpreters/InterpreterCreateQuery.cpp | 3 +++ src/Interpreters/SystemLog.cpp | 2 +- src/Parsers/ASTCreateQuery.cpp | 2 +- src/Parsers/ASTCreateQuery.h | 2 +- src/Parsers/ParserCreateQuery.cpp | 12 ++++++------ .../03033_create_as_copies_comment.reference | 3 +++ .../0_stateless/03033_create_as_copies_comment.sql | 11 +++++++++++ 12 files changed, 33 insertions(+), 17 deletions(-) create mode 100644 tests/queries/0_stateless/03033_create_as_copies_comment.reference create mode 100644 tests/queries/0_stateless/03033_create_as_copies_comment.sql diff --git a/src/Backups/DDLAdjustingForBackupVisitor.cpp b/src/Backups/DDLAdjustingForBackupVisitor.cpp index 5ea91094b75..89c24ac17b7 100644 --- a/src/Backups/DDLAdjustingForBackupVisitor.cpp +++ b/src/Backups/DDLAdjustingForBackupVisitor.cpp @@ -20,7 +20,7 @@ namespace /// If this is a definition of a system table we'll remove columns and comment because they're redundant for backups. auto & create = data.create_query->as(); create.reset(create.columns_list); - create.reset(create.comment); + create.comment.reset(); } void visitStorageReplicatedTableEngine(ASTStorage & storage, const DDLAdjustingForBackupVisitor::Data & data) diff --git a/src/Databases/DatabaseFilesystem.cpp b/src/Databases/DatabaseFilesystem.cpp index 05af0acf978..5d12c442700 100644 --- a/src/Databases/DatabaseFilesystem.cpp +++ b/src/Databases/DatabaseFilesystem.cpp @@ -192,7 +192,7 @@ ASTPtr DatabaseFilesystem::getCreateDatabaseQuery() const if (const auto database_comment = getDatabaseComment(); !database_comment.empty()) { auto & ast_create_query = ast->as(); - ast_create_query.set(ast_create_query.comment, std::make_shared(database_comment)); + ast_create_query.comment = std::make_shared(database_comment); } return ast; diff --git a/src/Databases/DatabaseMemory.cpp b/src/Databases/DatabaseMemory.cpp index 4ff7b3c7f2b..5b4e0fc3daf 100644 --- a/src/Databases/DatabaseMemory.cpp +++ b/src/Databases/DatabaseMemory.cpp @@ -107,7 +107,7 @@ ASTPtr DatabaseMemory::getCreateDatabaseQuery() const create_query->storage->set(create_query->storage->engine, engine); if (const auto comment_value = getDatabaseComment(); !comment_value.empty()) - create_query->set(create_query->comment, std::make_shared(comment_value)); + create_query->comment = std::make_shared(comment_value); return create_query; } diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 550f1a756cb..953143f6c1c 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -535,7 +535,7 @@ ASTPtr DatabaseOnDisk::getCreateDatabaseQuery() const if (const auto database_comment = getDatabaseComment(); !database_comment.empty()) { auto & ast_create_query = ast->as(); - ast_create_query.set(ast_create_query.comment, std::make_shared(database_comment)); + ast_create_query.comment = std::make_shared(database_comment); } return ast; @@ -784,8 +784,7 @@ ASTPtr DatabaseOnDisk::getCreateQueryFromStorage(const String & table_name, cons static_cast(settings.max_parser_backtracks), throw_on_error); - create_table_query->set(create_table_query->as()->comment, - std::make_shared("SYSTEM TABLE is built on the fly.")); + create_table_query->as()->comment = std::make_shared("SYSTEM TABLE is built on the fly."); return create_table_query; } diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index 4dffb16e486..924b785d8b8 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -114,9 +114,9 @@ void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemo } if (metadata.comment.empty()) - ast_create_query.reset(ast_create_query.comment); + ast_create_query.comment.reset(); else - ast_create_query.set(ast_create_query.comment, std::make_shared(metadata.comment)); + ast_create_query.comment = std::make_shared(metadata.comment); } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 7c3bed7388c..75208a501ca 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -785,6 +785,9 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti auto as_storage_metadata = as_storage->getInMemoryMetadataPtr(); properties.columns = as_storage_metadata->getColumns(); + if (!create.comment) + create.comment = std::make_shared(Field(as_storage_metadata->comment)); + /// Secondary indices and projections make sense only for MergeTree family of storage engines. /// We should not copy them for other storages. if (create.storage && endsWith(create.storage->engine->name, "MergeTree")) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index db73fe038c0..b0b18924010 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -656,7 +656,7 @@ ASTPtr SystemLog::getCreateTableQuery() StorageWithComment & storage_with_comment = storage_with_comment_ast->as(); create->set(create->storage, storage_with_comment.storage); - create->set(create->comment, storage_with_comment.comment); + create->comment = storage_with_comment.comment; /// Write additional (default) settings for MergeTree engine to make it make it possible to compare ASTs /// and recreate tables on settings changes. diff --git a/src/Parsers/ASTCreateQuery.cpp b/src/Parsers/ASTCreateQuery.cpp index 0403dc33164..b977c441de0 100644 --- a/src/Parsers/ASTCreateQuery.cpp +++ b/src/Parsers/ASTCreateQuery.cpp @@ -255,7 +255,7 @@ ASTPtr ASTCreateQuery::clone() const if (as_table_function) res->set(res->as_table_function, as_table_function->clone()); if (comment) - res->set(res->comment, comment->clone()); + res->comment = comment->clone(); cloneOutputOptions(*res); cloneTableOptions(*res); diff --git a/src/Parsers/ASTCreateQuery.h b/src/Parsers/ASTCreateQuery.h index 64e6bc8ce48..40e5fda300d 100644 --- a/src/Parsers/ASTCreateQuery.h +++ b/src/Parsers/ASTCreateQuery.h @@ -112,7 +112,7 @@ public: String as_table; IAST * as_table_function = nullptr; ASTSelectWithUnionQuery * select = nullptr; - IAST * comment = nullptr; + ASTPtr comment = nullptr; ASTPtr sql_security = nullptr; ASTTableOverrideList * table_overrides = nullptr; /// For CREATE DATABASE with engines that automatically create tables diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index f2c09e9b050..ab8bfcfbff9 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -861,7 +861,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe query->set(query->as_table_function, as_table_function); if (comment) - query->set(query->comment, comment); + query->comment = comment; if (query->columns_list && query->columns_list->primary_key) { @@ -1012,8 +1012,7 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e query->set(query->select, select); if (comment) - query->set(query->comment, comment); - + query->comment = comment; if (sql_security) query->sql_security = typeid_cast>(sql_security); @@ -1414,7 +1413,8 @@ bool ParserCreateDatabaseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e query->set(query->storage, storage); if (comment) - query->set(query->comment, comment); + query->comment = comment; + if (table_overrides && !table_overrides->children.empty()) query->set(query->table_overrides, table_overrides); @@ -1617,7 +1617,7 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (refresh_strategy) query->set(query->refresh_strategy, refresh_strategy); if (comment) - query->set(query->comment, comment); + query->comment = comment; if (sql_security) query->sql_security = typeid_cast>(sql_security); @@ -1794,7 +1794,7 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E query->cluster = cluster_str; if (comment) - query->set(query->comment, comment); + query->comment = comment; return true; } diff --git a/tests/queries/0_stateless/03033_create_as_copies_comment.reference b/tests/queries/0_stateless/03033_create_as_copies_comment.reference new file mode 100644 index 00000000000..98efe03b172 --- /dev/null +++ b/tests/queries/0_stateless/03033_create_as_copies_comment.reference @@ -0,0 +1,3 @@ +original comment +original comment +new comment diff --git a/tests/queries/0_stateless/03033_create_as_copies_comment.sql b/tests/queries/0_stateless/03033_create_as_copies_comment.sql new file mode 100644 index 00000000000..0583b6ed130 --- /dev/null +++ b/tests/queries/0_stateless/03033_create_as_copies_comment.sql @@ -0,0 +1,11 @@ +DROP TABLE IF EXISTS base; +DROP TABLE IF EXISTS copy_without_comment; +DROP TABLE IF EXISTS copy_with_comment; + +CREATE TABLE base (a Int32) ENGINE = MergeTree ORDER BY a COMMENT 'original comment'; +CREATE TABLE copy_without_comment as base; +CREATE TABLE copy_with_comment as base COMMENT 'new comment'; + +SELECT comment FROM system.tables WHERE name = 'base'; +SELECT comment FROM system.tables WHERE name = 'copy_without_comment'; +SELECT comment FROM system.tables WHERE name = 'copy_with_comment'; \ No newline at end of file From 8b1fcf37f832ff7e1fce1dbe5ddfe197fd661938 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 2 Apr 2024 08:46:15 +0200 Subject: [PATCH 0096/1018] Use CLICKHOUSE_DATABASE in SQL test --- .../03033_create_as_copies_comment.sql | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/03033_create_as_copies_comment.sql b/tests/queries/0_stateless/03033_create_as_copies_comment.sql index 0583b6ed130..f9458abd271 100644 --- a/tests/queries/0_stateless/03033_create_as_copies_comment.sql +++ b/tests/queries/0_stateless/03033_create_as_copies_comment.sql @@ -1,11 +1,10 @@ -DROP TABLE IF EXISTS base; -DROP TABLE IF EXISTS copy_without_comment; -DROP TABLE IF EXISTS copy_with_comment; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; +CREATE DATABASE {CLICKHOUSE_DATABASE:Identifier}; -CREATE TABLE base (a Int32) ENGINE = MergeTree ORDER BY a COMMENT 'original comment'; -CREATE TABLE copy_without_comment as base; -CREATE TABLE copy_with_comment as base COMMENT 'new comment'; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.base (a Int32) ENGINE = TinyLog COMMENT 'original comment'; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.copy_without_comment AS base; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.copy_with_comment AS base COMMENT 'new comment'; -SELECT comment FROM system.tables WHERE name = 'base'; -SELECT comment FROM system.tables WHERE name = 'copy_without_comment'; -SELECT comment FROM system.tables WHERE name = 'copy_with_comment'; \ No newline at end of file +SELECT comment FROM system.tables WHERE database = {CLICKHOUSE_DATABASE:String} AND name = 'base'; +SELECT comment FROM system.tables WHERE database = {CLICKHOUSE_DATABASE:String} AND name = 'copy_without_comment'; +SELECT comment FROM system.tables WHERE database = {CLICKHOUSE_DATABASE:String} AND name = 'copy_with_comment'; \ No newline at end of file From 73e56206e2683e008b19456e1d8b7cb6dd6f07e8 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 2 Apr 2024 09:12:59 +0200 Subject: [PATCH 0097/1018] Relax check-style condition because in SQL-based tests there's no $ --- utils/check-style/check-style | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 23e8b6b2bc4..a9cc61e551e 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -168,7 +168,7 @@ tests_with_query_log=( $( ) ) for test_case in "${tests_with_query_log[@]}"; do grep -qE current_database.*currentDatabase "$test_case" || { - grep -qE 'current_database.*\$CLICKHOUSE_DATABASE' "$test_case" + grep -qE 'current_database.*CLICKHOUSE_DATABASE' "$test_case" } || echo "Queries to system.query_log/system.query_thread_log does not have current_database = currentDatabase() condition in $test_case" done @@ -206,12 +206,12 @@ tests_with_database_column=( $( ) ) for test_case in "${tests_with_database_column[@]}"; do grep -qE database.*currentDatabase "$test_case" || { - grep -qE 'database.*\$CLICKHOUSE_DATABASE' "$test_case" + grep -qE 'database.*CLICKHOUSE_DATABASE' "$test_case" } || { # explicit database grep -qE "database[ ]*=[ ]*'" "$test_case" } || { - echo "Queries to ${tables_with_database_column[*]} does not have database = currentDatabase()/\$CLICKHOUSE_DATABASE condition in $test_case" + echo "Queries to ${tables_with_database_column[*]} does not have database = currentDatabase()/CLICKHOUSE_DATABASE condition in $test_case" } done From 98c2048d30d0a6e324875c98a4f138e7ee92734a Mon Sep 17 00:00:00 2001 From: zvonand Date: Sun, 31 Mar 2024 22:12:03 +0200 Subject: [PATCH 0098/1018] try to improve Storage S3 selection glob performance --- src/Storages/StorageS3.cpp | 122 +++++++++++++++++++++++++++++-------- 1 file changed, 98 insertions(+), 24 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 2d3aef312bf..cee9f11af95 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -210,32 +210,36 @@ public: if (globbed_uri.bucket.find_first_of("*?{") != globbed_uri.bucket.npos) throw Exception(ErrorCodes::UNEXPECTED_EXPRESSION, "Expression can not have wildcards inside bucket name"); - const String key_prefix = globbed_uri.key.substr(0, globbed_uri.key.find_first_of("*?{")); - - /// We don't have to list bucket, because there is no asterisks. - if (key_prefix.size() == globbed_uri.key.size()) + for (const auto & key : expandSelectionGlob(globbed_uri.key)) { - buffer.emplace_back(std::make_shared(globbed_uri.key, std::nullopt)); - buffer_iter = buffer.begin(); - is_finished = true; - return; + const String key_prefix = key.substr(0, key.find_first_of("*?{")); + + /// We don't have to list bucket, because there is no asterisks. + if (key_prefix.size() == key.size()) + { + buffer.emplace_back(std::make_shared(key, std::nullopt)); + buffer_iter = buffer.begin(); + is_finished = true; + return; + } + + request.SetBucket(globbed_uri.bucket); + request.SetPrefix(key_prefix); + request.SetMaxKeys(static_cast(request_settings.list_object_keys_size)); + + outcome_future = listObjectsAsync(); + + matcher = std::make_unique(makeRegexpPatternFromGlobs(key)); + if (!matcher->ok()) + throw Exception( + ErrorCodes::CANNOT_COMPILE_REGEXP, "Cannot compile regex from glob ({}): {}", key, matcher->error()); + + recursive = key == "/**"; + + filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); + updateInternalBufferAssumeLocked(); } - - request.SetBucket(globbed_uri.bucket); - request.SetPrefix(key_prefix); - request.SetMaxKeys(static_cast(request_settings.list_object_keys_size)); - - outcome_future = listObjectsAsync(); - - matcher = std::make_unique(makeRegexpPatternFromGlobs(globbed_uri.key)); - if (!matcher->ok()) - throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, - "Cannot compile regex from glob ({}): {}", globbed_uri.key, matcher->error()); - - recursive = globbed_uri.key == "/**" ? true : false; - - filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); - fillInternalBufferAssumeLocked(); + buffer_iter = buffer.begin(); } KeyWithInfoPtr next(size_t) @@ -301,6 +305,76 @@ private: } while (true); } + void updateInternalBufferAssumeLocked() + { + assert(outcome_future.valid()); + auto outcome = outcome_future.get(); + + if (!outcome.IsSuccess()) + { + throw S3Exception(outcome.GetError().GetErrorType(), "Could not list objects in bucket {} with prefix {}, S3 exception: {}, message: {}", + quoteString(request.GetBucket()), quoteString(request.GetPrefix()), + backQuote(outcome.GetError().GetExceptionName()), quoteString(outcome.GetError().GetMessage())); + } + + const auto & result_batch = outcome.GetResult().GetContents(); + + /// It returns false when all objects were returned + is_finished = !outcome.GetResult().GetIsTruncated(); + + if (!is_finished) + { + /// Even if task is finished the thread may be not freed in pool. + /// So wait until it will be freed before scheduling a new task. + list_objects_pool.wait(); + outcome_future = listObjectsAsync(); + } + + if (request_settings.throw_on_zero_files_match && result_batch.empty()) + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Can not match any files using prefix {}", request.GetPrefix()); + + KeysWithInfo temp_buffer; + temp_buffer.reserve(result_batch.size()); + + for (const auto & row : result_batch) + { + String key = row.GetKey(); + if (recursive || re2::RE2::FullMatch(key, *matcher)) + { + S3::ObjectInfo info = + { + .size = size_t(row.GetSize()), + .last_modification_time = row.GetLastModified().Millis() / 1000, + }; + temp_buffer.emplace_back(std::make_shared(std::move(key), std::move(info))); + } + } + + if (temp_buffer.empty()) + return; + + if (filter_dag) + { + std::vector paths; + paths.reserve(temp_buffer.size()); + for (const auto & key_with_info : temp_buffer) + paths.push_back(fs::path(globbed_uri.bucket) / key_with_info->key); + + VirtualColumnUtils::filterByPathOrFile(temp_buffer, paths, filter_dag, virtual_columns, getContext()); + } + + buffer.insert(buffer.end(), temp_buffer.begin(), temp_buffer.end()); + + if (read_keys) + read_keys->insert(read_keys->end(), temp_buffer.begin(), temp_buffer.end()); + + if (file_progress_callback) + { + for (const auto & key_with_info : buffer) + file_progress_callback(FileProgress(0, key_with_info->info->size)); + } + } + void fillInternalBufferAssumeLocked() { buffer.clear(); From 73b9ef99f4315e89d6f184d836e01da4345151ba Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 1 Apr 2024 17:40:40 +0200 Subject: [PATCH 0099/1018] Revert "try to improve Storage S3 selection glob performance" This reverts commit 9c9421b6897bf4a95346cef52171839ef67bd522. --- src/Storages/StorageS3.cpp | 122 ++++++++----------------------------- 1 file changed, 24 insertions(+), 98 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index cee9f11af95..2d3aef312bf 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -210,36 +210,32 @@ public: if (globbed_uri.bucket.find_first_of("*?{") != globbed_uri.bucket.npos) throw Exception(ErrorCodes::UNEXPECTED_EXPRESSION, "Expression can not have wildcards inside bucket name"); - for (const auto & key : expandSelectionGlob(globbed_uri.key)) + const String key_prefix = globbed_uri.key.substr(0, globbed_uri.key.find_first_of("*?{")); + + /// We don't have to list bucket, because there is no asterisks. + if (key_prefix.size() == globbed_uri.key.size()) { - const String key_prefix = key.substr(0, key.find_first_of("*?{")); - - /// We don't have to list bucket, because there is no asterisks. - if (key_prefix.size() == key.size()) - { - buffer.emplace_back(std::make_shared(key, std::nullopt)); - buffer_iter = buffer.begin(); - is_finished = true; - return; - } - - request.SetBucket(globbed_uri.bucket); - request.SetPrefix(key_prefix); - request.SetMaxKeys(static_cast(request_settings.list_object_keys_size)); - - outcome_future = listObjectsAsync(); - - matcher = std::make_unique(makeRegexpPatternFromGlobs(key)); - if (!matcher->ok()) - throw Exception( - ErrorCodes::CANNOT_COMPILE_REGEXP, "Cannot compile regex from glob ({}): {}", key, matcher->error()); - - recursive = key == "/**"; - - filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); - updateInternalBufferAssumeLocked(); + buffer.emplace_back(std::make_shared(globbed_uri.key, std::nullopt)); + buffer_iter = buffer.begin(); + is_finished = true; + return; } - buffer_iter = buffer.begin(); + + request.SetBucket(globbed_uri.bucket); + request.SetPrefix(key_prefix); + request.SetMaxKeys(static_cast(request_settings.list_object_keys_size)); + + outcome_future = listObjectsAsync(); + + matcher = std::make_unique(makeRegexpPatternFromGlobs(globbed_uri.key)); + if (!matcher->ok()) + throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, + "Cannot compile regex from glob ({}): {}", globbed_uri.key, matcher->error()); + + recursive = globbed_uri.key == "/**" ? true : false; + + filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); + fillInternalBufferAssumeLocked(); } KeyWithInfoPtr next(size_t) @@ -305,76 +301,6 @@ private: } while (true); } - void updateInternalBufferAssumeLocked() - { - assert(outcome_future.valid()); - auto outcome = outcome_future.get(); - - if (!outcome.IsSuccess()) - { - throw S3Exception(outcome.GetError().GetErrorType(), "Could not list objects in bucket {} with prefix {}, S3 exception: {}, message: {}", - quoteString(request.GetBucket()), quoteString(request.GetPrefix()), - backQuote(outcome.GetError().GetExceptionName()), quoteString(outcome.GetError().GetMessage())); - } - - const auto & result_batch = outcome.GetResult().GetContents(); - - /// It returns false when all objects were returned - is_finished = !outcome.GetResult().GetIsTruncated(); - - if (!is_finished) - { - /// Even if task is finished the thread may be not freed in pool. - /// So wait until it will be freed before scheduling a new task. - list_objects_pool.wait(); - outcome_future = listObjectsAsync(); - } - - if (request_settings.throw_on_zero_files_match && result_batch.empty()) - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Can not match any files using prefix {}", request.GetPrefix()); - - KeysWithInfo temp_buffer; - temp_buffer.reserve(result_batch.size()); - - for (const auto & row : result_batch) - { - String key = row.GetKey(); - if (recursive || re2::RE2::FullMatch(key, *matcher)) - { - S3::ObjectInfo info = - { - .size = size_t(row.GetSize()), - .last_modification_time = row.GetLastModified().Millis() / 1000, - }; - temp_buffer.emplace_back(std::make_shared(std::move(key), std::move(info))); - } - } - - if (temp_buffer.empty()) - return; - - if (filter_dag) - { - std::vector paths; - paths.reserve(temp_buffer.size()); - for (const auto & key_with_info : temp_buffer) - paths.push_back(fs::path(globbed_uri.bucket) / key_with_info->key); - - VirtualColumnUtils::filterByPathOrFile(temp_buffer, paths, filter_dag, virtual_columns, getContext()); - } - - buffer.insert(buffer.end(), temp_buffer.begin(), temp_buffer.end()); - - if (read_keys) - read_keys->insert(read_keys->end(), temp_buffer.begin(), temp_buffer.end()); - - if (file_progress_callback) - { - for (const auto & key_with_info : buffer) - file_progress_callback(FileProgress(0, key_with_info->info->size)); - } - } - void fillInternalBufferAssumeLocked() { buffer.clear(); From 70da13b9b01d0e9a86b313bdcf165b6c54a4b985 Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 1 Apr 2024 22:34:54 +0200 Subject: [PATCH 0100/1018] simpler way --- src/Storages/StorageS3.cpp | 76 +++++++++++++++++++++++++------------- 1 file changed, 51 insertions(+), 25 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 2d3aef312bf..844f5362ec2 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -191,7 +191,7 @@ public: Impl( const S3::Client & client_, const S3::URI & globbed_uri_, - const ActionsDAG::Node * predicate, + const ActionsDAG::Node * predicate_, const NamesAndTypesList & virtual_columns_, ContextPtr context_, KeysWithInfo * read_keys_, @@ -200,6 +200,7 @@ public: : WithContext(context_) , client(client_.clone()) , globbed_uri(globbed_uri_) + , predicate(predicate_) , virtual_columns(virtual_columns_) , read_keys(read_keys_) , request_settings(request_settings_) @@ -210,32 +211,13 @@ public: if (globbed_uri.bucket.find_first_of("*?{") != globbed_uri.bucket.npos) throw Exception(ErrorCodes::UNEXPECTED_EXPRESSION, "Expression can not have wildcards inside bucket name"); - const String key_prefix = globbed_uri.key.substr(0, globbed_uri.key.find_first_of("*?{")); + expanded_keys = expandSelectionGlob(globbed_uri.key); + expanded_keys_iter = expanded_keys.begin(); - /// We don't have to list bucket, because there is no asterisks. - if (key_prefix.size() == globbed_uri.key.size()) - { - buffer.emplace_back(std::make_shared(globbed_uri.key, std::nullopt)); - buffer_iter = buffer.begin(); + bool no_globs_in_key = fillBufferForKey(*expanded_keys_iter); + expanded_keys_iter++; + if (expanded_keys_iter == expanded_keys.end() && no_globs_in_key) is_finished = true; - return; - } - - request.SetBucket(globbed_uri.bucket); - request.SetPrefix(key_prefix); - request.SetMaxKeys(static_cast(request_settings.list_object_keys_size)); - - outcome_future = listObjectsAsync(); - - matcher = std::make_unique(makeRegexpPatternFromGlobs(globbed_uri.key)); - if (!matcher->ok()) - throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, - "Cannot compile regex from glob ({}): {}", globbed_uri.key, matcher->error()); - - recursive = globbed_uri.key == "/**" ? true : false; - - filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); - fillInternalBufferAssumeLocked(); } KeyWithInfoPtr next(size_t) @@ -257,6 +239,37 @@ public: private: using ListObjectsOutcome = Aws::S3::Model::ListObjectsV2Outcome; + bool fillBufferForKey(const std::string & uri_key) + { + const String key_prefix = uri_key.substr(0, uri_key.find_first_of("*?{")); + + /// We don't have to list bucket, because there is no asterisks. + if (key_prefix.size() == uri_key.size()) + { + buffer.clear(); + buffer.emplace_back(std::make_shared(uri_key, std::nullopt)); + buffer_iter = buffer.begin(); + return true; + } + + request.SetBucket(globbed_uri.bucket); + request.SetPrefix(key_prefix); + request.SetMaxKeys(static_cast(request_settings.list_object_keys_size)); + + outcome_future = listObjectsAsync(); + + matcher = std::make_unique(makeRegexpPatternFromGlobs(uri_key)); + if (!matcher->ok()) + throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, + "Cannot compile regex from glob ({}): {}", uri_key, matcher->error()); + + recursive = globbed_uri.key == "/**"; + + filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); + fillInternalBufferAssumeLocked(); + return false; + } + KeyWithInfoPtr nextAssumeLocked() { do @@ -278,6 +291,15 @@ private: return answer; } + if (expanded_keys_iter != expanded_keys.end()) + { + bool no_globs_in_key = fillBufferForKey(*expanded_keys_iter); + expanded_keys_iter++; + if (expanded_keys_iter == expanded_keys.end() && no_globs_in_key) + is_finished = true; + continue; + } + if (is_finished) return {}; @@ -399,8 +421,12 @@ private: KeysWithInfo buffer; KeysWithInfo::iterator buffer_iter; + std::vector expanded_keys; + std::vector::iterator expanded_keys_iter; + std::unique_ptr client; S3::URI globbed_uri; + const ActionsDAG::Node * predicate; ASTPtr query; NamesAndTypesList virtual_columns; ActionsDAGPtr filter_dag; From a177fbfd8cb52c64f096797a6c65fdc4dfeb828e Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 2 Apr 2024 00:05:53 +0200 Subject: [PATCH 0101/1018] ignore error when one of selection options not exist --- src/Storages/StorageS3.cpp | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 844f5362ec2..09a5ffc86a5 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -283,7 +283,18 @@ private: /// So we get object info lazily here on 'next()' request. if (!answer->info) { - answer->info = S3::getObjectInfo(*client, globbed_uri.bucket, answer->key, globbed_uri.version_id, request_settings); + try + { + answer->info = S3::getObjectInfo(*client, globbed_uri.bucket, answer->key, globbed_uri.version_id, request_settings); + } + catch (...) + { + /// if no such file AND there was no `{}` glob -- this is an exception + /// otherwise ignore it, this is acceptable + if (expanded_keys.size() == 1) + throw; + continue; + } if (file_progress_callback) file_progress_callback(FileProgress(0, answer->info->size)); } From eeb03182b8b3bc5a09dba6ebc4c5d41fd678357f Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 2 Apr 2024 12:39:13 +0200 Subject: [PATCH 0102/1018] Add link to DIctionaries section --- docs/en/sql-reference/functions/ym-dict-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/ym-dict-functions.md b/docs/en/sql-reference/functions/ym-dict-functions.md index a516f09d709..a0e760107a6 100644 --- a/docs/en/sql-reference/functions/ym-dict-functions.md +++ b/docs/en/sql-reference/functions/ym-dict-functions.md @@ -8,7 +8,7 @@ sidebar_label: Embedded Dictionaries In order for the functions below to work, the server config must specify the paths and addresses for getting all the embedded dictionaries. The dictionaries are loaded at the first call of any of these functions. If the reference lists can’t be loaded, an exception is thrown. -For information about creating reference lists, see the section “Dictionaries”. +For information about creating reference lists, see the section [“Dictionaries”](../../sql-reference/dictionaries/index.md). ## Multiple Geobases From fa79395127ed2d988bf5219339a319078fb67ebd Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 2 Apr 2024 12:58:06 +0200 Subject: [PATCH 0103/1018] Place warning about embedded dictionaries in a 'note' section --- .../functions/ym-dict-functions.md | 34 ++++++++++++++++++- 1 file changed, 33 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/ym-dict-functions.md b/docs/en/sql-reference/functions/ym-dict-functions.md index a0e760107a6..d1b17d09a58 100644 --- a/docs/en/sql-reference/functions/ym-dict-functions.md +++ b/docs/en/sql-reference/functions/ym-dict-functions.md @@ -6,7 +6,9 @@ sidebar_label: Embedded Dictionaries # Functions for Working with Embedded Dictionaries +:::note In order for the functions below to work, the server config must specify the paths and addresses for getting all the embedded dictionaries. The dictionaries are loaded at the first call of any of these functions. If the reference lists can’t be loaded, an exception is thrown. +::: For information about creating reference lists, see the section [“Dictionaries”](../../sql-reference/dictionaries/index.md). @@ -32,10 +34,40 @@ regionToCountry(RegionID, '') – Uses the default dictionary: /opt/geo/regions_ regionToCountry(RegionID, 'ua') – Uses the dictionary for the 'ua' key: /opt/geo/regions_hierarchy_ua.txt ``` -### regionToCity(id\[, geobase\]) +### regionToCity Accepts a UInt32 number – the region ID from the geobase. If this region is a city or part of a city, it returns the region ID for the appropriate city. Otherwise, returns 0. +**Syntax** + +```sql +regionToCity(id [, geobase]) +``` + +**Parameters** + +- `id`: the region ID from the geobase. [UInt32](../data-types/int-uint.md/#uint8-uint16-uint32-uint64-uint128-uint256-int8-int16-int32-int64-int128-int256) +- `geobase` (optional): geobase (dictionary key). [Dictionary Key](../../sql-reference/dictionaries/index.md/#dictionary-key-and-fields-dictionary-key-and-fields) + +**Returned value** + +- region ID for the appropriate city, if it exists. +- 0, otherwise. + +**Example** + +Query: + +```sql + +``` + +Result: + +```response + +``` + ### regionToArea(id\[, geobase\]) Converts a region to an area (type 5 in the geobase). In every other way, this function is the same as ‘regionToCity’. From 7232bf45768f56c768ac03ed4b34c085bc6f060a Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 2 Apr 2024 16:12:11 +0200 Subject: [PATCH 0104/1018] no reuse request --- src/Storages/StorageS3.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 09a5ffc86a5..28bfa3c32a9 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -252,6 +252,7 @@ private: return true; } + request = S3::ListObjectsV2Request{}; request.SetBucket(globbed_uri.bucket); request.SetPrefix(key_prefix); request.SetMaxKeys(static_cast(request_settings.list_object_keys_size)); From 4a65a066007173475f759f38d6f3fcb284b9d165 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 2 Apr 2024 16:48:08 +0200 Subject: [PATCH 0105/1018] Revert "Relax check-style condition because in SQL-based tests there's no $" This reverts commit 73e56206e2683e008b19456e1d8b7cb6dd6f07e8. --- utils/check-style/check-style | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index a9cc61e551e..23e8b6b2bc4 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -168,7 +168,7 @@ tests_with_query_log=( $( ) ) for test_case in "${tests_with_query_log[@]}"; do grep -qE current_database.*currentDatabase "$test_case" || { - grep -qE 'current_database.*CLICKHOUSE_DATABASE' "$test_case" + grep -qE 'current_database.*\$CLICKHOUSE_DATABASE' "$test_case" } || echo "Queries to system.query_log/system.query_thread_log does not have current_database = currentDatabase() condition in $test_case" done @@ -206,12 +206,12 @@ tests_with_database_column=( $( ) ) for test_case in "${tests_with_database_column[@]}"; do grep -qE database.*currentDatabase "$test_case" || { - grep -qE 'database.*CLICKHOUSE_DATABASE' "$test_case" + grep -qE 'database.*\$CLICKHOUSE_DATABASE' "$test_case" } || { # explicit database grep -qE "database[ ]*=[ ]*'" "$test_case" } || { - echo "Queries to ${tables_with_database_column[*]} does not have database = currentDatabase()/CLICKHOUSE_DATABASE condition in $test_case" + echo "Queries to ${tables_with_database_column[*]} does not have database = currentDatabase()/\$CLICKHOUSE_DATABASE condition in $test_case" } done From 836169af18c371104274c8108820aa592968e0f2 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 2 Apr 2024 16:50:03 +0200 Subject: [PATCH 0106/1018] Use currentDatabase() instead of CLICKHOUSE_DATABASE --- .../03033_create_as_copies_comment.sql | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/03033_create_as_copies_comment.sql b/tests/queries/0_stateless/03033_create_as_copies_comment.sql index f9458abd271..0de42f6cc79 100644 --- a/tests/queries/0_stateless/03033_create_as_copies_comment.sql +++ b/tests/queries/0_stateless/03033_create_as_copies_comment.sql @@ -1,10 +1,11 @@ -DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; -CREATE DATABASE {CLICKHOUSE_DATABASE:Identifier}; +DROP TABLE IF EXISTS base; +DROP TABLE IF EXISTS copy_without_comment; +DROP TABLE IF EXISTS copy_with_comment; -CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.base (a Int32) ENGINE = TinyLog COMMENT 'original comment'; -CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.copy_without_comment AS base; -CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.copy_with_comment AS base COMMENT 'new comment'; +CREATE TABLE base (a Int32) ENGINE = TinyLog COMMENT 'original comment'; +CREATE TABLE copy_without_comment AS base; +CREATE TABLE copy_with_comment AS base COMMENT 'new comment'; -SELECT comment FROM system.tables WHERE database = {CLICKHOUSE_DATABASE:String} AND name = 'base'; -SELECT comment FROM system.tables WHERE database = {CLICKHOUSE_DATABASE:String} AND name = 'copy_without_comment'; -SELECT comment FROM system.tables WHERE database = {CLICKHOUSE_DATABASE:String} AND name = 'copy_with_comment'; \ No newline at end of file +SELECT comment FROM system.tables WHERE database = currentDatabase() AND name = 'base'; +SELECT comment FROM system.tables WHERE database = currentDatabase() AND name = 'copy_without_comment'; +SELECT comment FROM system.tables WHERE database = currentDatabase() AND name = 'copy_with_comment'; \ No newline at end of file From 6d431184a87514eac72b8c71199bc33cf5d85149 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 2 Apr 2024 17:25:42 +0200 Subject: [PATCH 0107/1018] Only copy comment if it's not empty --- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 75208a501ca..c27d610474c 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -785,7 +785,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti auto as_storage_metadata = as_storage->getInMemoryMetadataPtr(); properties.columns = as_storage_metadata->getColumns(); - if (!create.comment) + if (!create.comment && !as_storage_metadata->comment.empty()) create.comment = std::make_shared(Field(as_storage_metadata->comment)); /// Secondary indices and projections make sense only for MergeTree family of storage engines. From e1041da183953a43ba7443ae608862b75c7924fa Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 2 Apr 2024 18:02:58 +0200 Subject: [PATCH 0108/1018] Update reference for 02904_distributed_settings_background_insert_compatibility --- ...ngs_background_insert_compatibility.reference | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/02904_distributed_settings_background_insert_compatibility.reference b/tests/queries/0_stateless/02904_distributed_settings_background_insert_compatibility.reference index 0694f895e38..2b1caf0d961 100644 --- a/tests/queries/0_stateless/02904_distributed_settings_background_insert_compatibility.reference +++ b/tests/queries/0_stateless/02904_distributed_settings_background_insert_compatibility.reference @@ -1,8 +1,8 @@ -CREATE TABLE default.dist_monitor_batch_inserts\n(\n `dummy` UInt8\n)\nENGINE = Distributed(\'test_shard_localhost\', \'system\', \'one\')\nSETTINGS monitor_batch_inserts = 1 -CREATE TABLE default.dist_monitor_split_batch_on_failure\n(\n `dummy` UInt8\n)\nENGINE = Distributed(\'test_shard_localhost\', \'system\', \'one\')\nSETTINGS monitor_split_batch_on_failure = 1 -CREATE TABLE default.dist_monitor_sleep_time_ms\n(\n `dummy` UInt8\n)\nENGINE = Distributed(\'test_shard_localhost\', \'system\', \'one\')\nSETTINGS monitor_sleep_time_ms = 1 -CREATE TABLE default.dist_monitor_max_sleep_time_ms\n(\n `dummy` UInt8\n)\nENGINE = Distributed(\'test_shard_localhost\', \'system\', \'one\')\nSETTINGS monitor_max_sleep_time_ms = 1 -CREATE TABLE default.dist_background_insert_batch\n(\n `dummy` UInt8\n)\nENGINE = Distributed(\'test_shard_localhost\', \'system\', \'one\')\nSETTINGS background_insert_batch = 1 -CREATE TABLE default.dist_background_insert_split_batch_on_failure\n(\n `dummy` UInt8\n)\nENGINE = Distributed(\'test_shard_localhost\', \'system\', \'one\')\nSETTINGS background_insert_split_batch_on_failure = 1 -CREATE TABLE default.dist_background_insert_sleep_time_ms\n(\n `dummy` UInt8\n)\nENGINE = Distributed(\'test_shard_localhost\', \'system\', \'one\')\nSETTINGS background_insert_sleep_time_ms = 1 -CREATE TABLE default.dist_background_insert_max_sleep_time_ms\n(\n `dummy` UInt8\n)\nENGINE = Distributed(\'test_shard_localhost\', \'system\', \'one\')\nSETTINGS background_insert_max_sleep_time_ms = 1 +CREATE TABLE default.dist_monitor_batch_inserts\n(\n `dummy` UInt8\n)\nENGINE = Distributed(\'test_shard_localhost\', \'system\', \'one\')\nSETTINGS monitor_batch_inserts = 1\nCOMMENT \'This table contains a single row with a single dummy UInt8 column containing the value 0. Used when the table is not specified explicitly, for example in queries like `SELECT 1`.\' +CREATE TABLE default.dist_monitor_split_batch_on_failure\n(\n `dummy` UInt8\n)\nENGINE = Distributed(\'test_shard_localhost\', \'system\', \'one\')\nSETTINGS monitor_split_batch_on_failure = 1\nCOMMENT \'This table contains a single row with a single dummy UInt8 column containing the value 0. Used when the table is not specified explicitly, for example in queries like `SELECT 1`.\' +CREATE TABLE default.dist_monitor_sleep_time_ms\n(\n `dummy` UInt8\n)\nENGINE = Distributed(\'test_shard_localhost\', \'system\', \'one\')\nSETTINGS monitor_sleep_time_ms = 1\nCOMMENT \'This table contains a single row with a single dummy UInt8 column containing the value 0. Used when the table is not specified explicitly, for example in queries like `SELECT 1`.\' +CREATE TABLE default.dist_monitor_max_sleep_time_ms\n(\n `dummy` UInt8\n)\nENGINE = Distributed(\'test_shard_localhost\', \'system\', \'one\')\nSETTINGS monitor_max_sleep_time_ms = 1\nCOMMENT \'This table contains a single row with a single dummy UInt8 column containing the value 0. Used when the table is not specified explicitly, for example in queries like `SELECT 1`.\' +CREATE TABLE default.dist_background_insert_batch\n(\n `dummy` UInt8\n)\nENGINE = Distributed(\'test_shard_localhost\', \'system\', \'one\')\nSETTINGS background_insert_batch = 1\nCOMMENT \'This table contains a single row with a single dummy UInt8 column containing the value 0. Used when the table is not specified explicitly, for example in queries like `SELECT 1`.\' +CREATE TABLE default.dist_background_insert_split_batch_on_failure\n(\n `dummy` UInt8\n)\nENGINE = Distributed(\'test_shard_localhost\', \'system\', \'one\')\nSETTINGS background_insert_split_batch_on_failure = 1\nCOMMENT \'This table contains a single row with a single dummy UInt8 column containing the value 0. Used when the table is not specified explicitly, for example in queries like `SELECT 1`.\' +CREATE TABLE default.dist_background_insert_sleep_time_ms\n(\n `dummy` UInt8\n)\nENGINE = Distributed(\'test_shard_localhost\', \'system\', \'one\')\nSETTINGS background_insert_sleep_time_ms = 1\nCOMMENT \'This table contains a single row with a single dummy UInt8 column containing the value 0. Used when the table is not specified explicitly, for example in queries like `SELECT 1`.\' +CREATE TABLE default.dist_background_insert_max_sleep_time_ms\n(\n `dummy` UInt8\n)\nENGINE = Distributed(\'test_shard_localhost\', \'system\', \'one\')\nSETTINGS background_insert_max_sleep_time_ms = 1\nCOMMENT \'This table contains a single row with a single dummy UInt8 column containing the value 0. Used when the table is not specified explicitly, for example in queries like `SELECT 1`.\' \ No newline at end of file From 1910e92346835203b796fd60c9fca7c0e5086d3c Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 2 Apr 2024 18:31:21 +0200 Subject: [PATCH 0109/1018] Add missing newline at end of reference file --- ...stributed_settings_background_insert_compatibility.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02904_distributed_settings_background_insert_compatibility.reference b/tests/queries/0_stateless/02904_distributed_settings_background_insert_compatibility.reference index 2b1caf0d961..c29bc62f795 100644 --- a/tests/queries/0_stateless/02904_distributed_settings_background_insert_compatibility.reference +++ b/tests/queries/0_stateless/02904_distributed_settings_background_insert_compatibility.reference @@ -5,4 +5,4 @@ CREATE TABLE default.dist_monitor_max_sleep_time_ms\n(\n `dummy` UInt8\n)\nEN CREATE TABLE default.dist_background_insert_batch\n(\n `dummy` UInt8\n)\nENGINE = Distributed(\'test_shard_localhost\', \'system\', \'one\')\nSETTINGS background_insert_batch = 1\nCOMMENT \'This table contains a single row with a single dummy UInt8 column containing the value 0. Used when the table is not specified explicitly, for example in queries like `SELECT 1`.\' CREATE TABLE default.dist_background_insert_split_batch_on_failure\n(\n `dummy` UInt8\n)\nENGINE = Distributed(\'test_shard_localhost\', \'system\', \'one\')\nSETTINGS background_insert_split_batch_on_failure = 1\nCOMMENT \'This table contains a single row with a single dummy UInt8 column containing the value 0. Used when the table is not specified explicitly, for example in queries like `SELECT 1`.\' CREATE TABLE default.dist_background_insert_sleep_time_ms\n(\n `dummy` UInt8\n)\nENGINE = Distributed(\'test_shard_localhost\', \'system\', \'one\')\nSETTINGS background_insert_sleep_time_ms = 1\nCOMMENT \'This table contains a single row with a single dummy UInt8 column containing the value 0. Used when the table is not specified explicitly, for example in queries like `SELECT 1`.\' -CREATE TABLE default.dist_background_insert_max_sleep_time_ms\n(\n `dummy` UInt8\n)\nENGINE = Distributed(\'test_shard_localhost\', \'system\', \'one\')\nSETTINGS background_insert_max_sleep_time_ms = 1\nCOMMENT \'This table contains a single row with a single dummy UInt8 column containing the value 0. Used when the table is not specified explicitly, for example in queries like `SELECT 1`.\' \ No newline at end of file +CREATE TABLE default.dist_background_insert_max_sleep_time_ms\n(\n `dummy` UInt8\n)\nENGINE = Distributed(\'test_shard_localhost\', \'system\', \'one\')\nSETTINGS background_insert_max_sleep_time_ms = 1\nCOMMENT \'This table contains a single row with a single dummy UInt8 column containing the value 0. Used when the table is not specified explicitly, for example in queries like `SELECT 1`.\' From 74ce999645f5598b6d79c24510fb7b28fb2b5f17 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 2 Apr 2024 19:47:26 +0200 Subject: [PATCH 0110/1018] Try fxi --- src/Storages/RabbitMQ/RabbitMQSettings.h | 1 + src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 8 +++++++- src/Storages/RabbitMQ/StorageRabbitMQ.h | 1 + 3 files changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Storages/RabbitMQ/RabbitMQSettings.h b/src/Storages/RabbitMQ/RabbitMQSettings.h index 36e092925fd..f2c0dae21c6 100644 --- a/src/Storages/RabbitMQ/RabbitMQSettings.h +++ b/src/Storages/RabbitMQ/RabbitMQSettings.h @@ -32,6 +32,7 @@ namespace DB M(Bool, rabbitmq_queue_consume, false, "Use user-defined queues and do not make any RabbitMQ setup: declaring exchanges, queues, bindings", 0) \ M(String, rabbitmq_username, "", "RabbitMQ username", 0) \ M(String, rabbitmq_password, "", "RabbitMQ password", 0) \ + M(Bool, reject_unhandled_messages, false, "Allow messages to be rejected in case they cannot be processed. This also automatically implies if there is a x-deadletter-exchange queue setting added", 0) \ M(Bool, rabbitmq_commit_on_select, false, "Commit messages when select query is made", 0) \ M(UInt64, rabbitmq_max_rows_per_message, 1, "The maximum number of rows produced in one message for row-based formats.", 0) \ M(StreamingHandleErrorMode, rabbitmq_handle_error_mode, StreamingHandleErrorMode::DEFAULT, "How to handle errors for RabbitMQ engine. Possible values: default (throw an exception after rabbitmq_skip_broken_messages broken messages), stream (save broken messages and errors in virtual columns _raw_message, _error).", 0) \ diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index b882fd2728c..d3cbcc1c6c7 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -64,6 +64,7 @@ namespace ExchangeType static const String HEADERS = "headers"; } +static const auto deadletter_exchange_setting = "x-dead-letter-exchange"; StorageRabbitMQ::StorageRabbitMQ( const StorageID & table_id_, @@ -93,6 +94,11 @@ StorageRabbitMQ::StorageRabbitMQ( , queue_size(std::max(QUEUE_SIZE, static_cast(getMaxBlockSize()))) , milliseconds_to_wait(rabbitmq_settings->rabbitmq_empty_queue_backoff_start_ms) { + reject_unhandled_messages = rabbitmq_settings->reject_unhandled_messages + || queue_settings_list.end() != + std::find_if(queue_settings_list.begin(), queue_settings_list.end(), + [](const String & name) { return name.starts_with(deadletter_exchange_setting); }); + const auto & config = getContext()->getConfigRef(); std::pair parsed_address; @@ -1166,7 +1172,7 @@ bool StorageRabbitMQ::tryStreamToViews() * the same channel will also commit all previously not-committed messages. Anyway I do not think that for ack frame this * will ever happen. */ - if (write_failed ? source->sendNack() : source->sendAck()) + if (write_failed && reject_unhandled_messages ? source->sendNack() : source->sendAck()) { /// Iterate loop to activate error callbacks if they happened connection->getHandler().iterateLoop(); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index e14741d9636..afef2920b07 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -91,6 +91,7 @@ private: String queue_base; Names queue_settings_list; size_t max_rows_per_message; + bool reject_unhandled_messages = false; /// For insert query. Mark messages as durable. const bool persistent; From 2b34d5e4f70dbb1730d8b6a017746d4113624ee9 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 2 Apr 2024 21:00:11 +0200 Subject: [PATCH 0111/1018] Add doc --- docs/en/engines/table-engines/integrations/rabbitmq.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/engines/table-engines/integrations/rabbitmq.md b/docs/en/engines/table-engines/integrations/rabbitmq.md index a4d0cf78066..ec7f7a7cec8 100644 --- a/docs/en/engines/table-engines/integrations/rabbitmq.md +++ b/docs/en/engines/table-engines/integrations/rabbitmq.md @@ -73,6 +73,7 @@ Optional parameters: - `rabbitmq_queue_consume` - Use user-defined queues and do not make any RabbitMQ setup: declaring exchanges, queues, bindings. Default: `false`. - `rabbitmq_username` - RabbitMQ username. - `rabbitmq_password` - RabbitMQ password. +- `reject_unhandled_messages` - Reject messages (send RabbitMQ Nack) in case of errors. This setting is automatically enabled if there is a `x-dead-letter-exchange` defined in `rabbitmq_queue_settings_list`. - `rabbitmq_commit_on_select` - Commit messages when select query is made. Default: `false`. - `rabbitmq_max_rows_per_message` — The maximum number of rows written in one RabbitMQ message for row-based formats. Default : `1`. - `rabbitmq_empty_queue_backoff_start` — A start backoff point to reschedule read if the rabbitmq queue is empty. From f49c15129b3343ac334c2c033cc2d6bad5574a05 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 2 Apr 2024 21:28:38 +0200 Subject: [PATCH 0112/1018] Fix build for PostgreSQL, MySQL and SQLite I didn't have them enabled in my build ^_^ --- src/Databases/MySQL/DatabaseMySQL.cpp | 2 +- src/Databases/PostgreSQL/DatabasePostgreSQL.cpp | 2 +- src/Databases/SQLite/DatabaseSQLite.cpp | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index d9b0f7f9ac7..5986be34ee8 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -204,7 +204,7 @@ ASTPtr DatabaseMySQL::getCreateDatabaseQuery() const create_query->set(create_query->storage, database_engine_define); if (const auto comment_value = getDatabaseComment(); !comment_value.empty()) - create_query->set(create_query->comment, std::make_shared(comment_value)); + create_query->comment = std::make_shared(comment_value); return create_query; } diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index b07b203f786..ac10fe60dbb 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -391,7 +391,7 @@ ASTPtr DatabasePostgreSQL::getCreateDatabaseQuery() const create_query->set(create_query->storage, database_engine_define); if (const auto comment_value = getDatabaseComment(); !comment_value.empty()) - create_query->set(create_query->comment, std::make_shared(comment_value)); + create_query->comment = std::make_shared(comment_value); return create_query; } diff --git a/src/Databases/SQLite/DatabaseSQLite.cpp b/src/Databases/SQLite/DatabaseSQLite.cpp index b7a82fd9d0f..530344f3c98 100644 --- a/src/Databases/SQLite/DatabaseSQLite.cpp +++ b/src/Databases/SQLite/DatabaseSQLite.cpp @@ -166,7 +166,7 @@ ASTPtr DatabaseSQLite::getCreateDatabaseQuery() const create_query->set(create_query->storage, database_engine_define); if (const auto comment_value = getDatabaseComment(); !comment_value.empty()) - create_query->set(create_query->comment, std::make_shared(comment_value)); + create_query->comment = std::make_shared(comment_value); return create_query; } From b6c059c184bf663a6bec37f27e09ac3cb0e9133d Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 2 Apr 2024 22:26:49 +0200 Subject: [PATCH 0113/1018] Fix build for HDFS and S3 --- src/Databases/DatabaseHDFS.cpp | 2 +- src/Databases/DatabaseS3.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Databases/DatabaseHDFS.cpp b/src/Databases/DatabaseHDFS.cpp index 2688ff2443c..cfa05db6137 100644 --- a/src/Databases/DatabaseHDFS.cpp +++ b/src/Databases/DatabaseHDFS.cpp @@ -188,7 +188,7 @@ ASTPtr DatabaseHDFS::getCreateDatabaseQuery() const if (const auto database_comment = getDatabaseComment(); !database_comment.empty()) { auto & ast_create_query = ast->as(); - ast_create_query.set(ast_create_query.comment, std::make_shared(database_comment)); + ast_create_query.comment = std::make_shared(database_comment); } return ast; diff --git a/src/Databases/DatabaseS3.cpp b/src/Databases/DatabaseS3.cpp index 159a5242dbe..ec237b7fe82 100644 --- a/src/Databases/DatabaseS3.cpp +++ b/src/Databases/DatabaseS3.cpp @@ -196,7 +196,7 @@ ASTPtr DatabaseS3::getCreateDatabaseQuery() const if (const auto database_comment = getDatabaseComment(); !database_comment.empty()) { auto & ast_create_query = ast->as(); - ast_create_query.set(ast_create_query.comment, std::make_shared(database_comment)); + ast_create_query.comment = std::make_shared(database_comment); } return ast; From 4dd35b4d4f98899da8cdf64f2d9921b7e32eafc9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=9A=D0=B8=D1=80=D0=B8=D0=BB=D0=BB=20=D0=93=D0=B0=D1=80?= =?UTF-8?q?=D0=B1=D0=B0=D1=80?= Date: Tue, 2 Apr 2024 23:36:49 +0300 Subject: [PATCH 0114/1018] Remove useless cast --- src/Databases/DatabaseOrdinary.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index f4191727b7c..5f3d8983936 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -128,7 +128,7 @@ void DatabaseOrdinary::convertMergeTreeToReplicatedIfNeeded(ASTPtr ast, const Qu /// Get table's storage policy MergeTreeSettings default_settings = getContext()->getMergeTreeSettings(); auto policy = getContext()->getStoragePolicy(default_settings.storage_policy); - if (auto * query_settings = create_query->storage->settings->as()) + if (auto * query_settings = create_query->storage->settings) if (Field * policy_setting = query_settings->changes.tryGet("storage_policy")) policy = getContext()->getStoragePolicy(policy_setting->safeGet()); From 8bdcac4b213b76f1add4ebaa790c1f0ea8eeea51 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 2 Apr 2024 23:38:32 +0200 Subject: [PATCH 0115/1018] Force a new build Seems 00028_shard_big_agg_aj_distributed timed out --- src/Parsers/ASTCreateQuery.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ASTCreateQuery.h b/src/Parsers/ASTCreateQuery.h index 40e5fda300d..fe7477f0e5f 100644 --- a/src/Parsers/ASTCreateQuery.h +++ b/src/Parsers/ASTCreateQuery.h @@ -112,7 +112,7 @@ public: String as_table; IAST * as_table_function = nullptr; ASTSelectWithUnionQuery * select = nullptr; - ASTPtr comment = nullptr; + ASTPtr comment; ASTPtr sql_security = nullptr; ASTTableOverrideList * table_overrides = nullptr; /// For CREATE DATABASE with engines that automatically create tables From daa07fbe9f88a5ea463649fbd029723cb827addc Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 3 Apr 2024 11:07:55 +0200 Subject: [PATCH 0116/1018] Update function names and syntax --- .../functions/ym-dict-functions.md | 110 +++++++++++++++--- 1 file changed, 95 insertions(+), 15 deletions(-) diff --git a/docs/en/sql-reference/functions/ym-dict-functions.md b/docs/en/sql-reference/functions/ym-dict-functions.md index d1b17d09a58..cd5cd33a240 100644 --- a/docs/en/sql-reference/functions/ym-dict-functions.md +++ b/docs/en/sql-reference/functions/ym-dict-functions.md @@ -36,7 +36,7 @@ regionToCountry(RegionID, 'ua') – Uses the dictionary for the 'ua' key: /opt/g ### regionToCity -Accepts a UInt32 number – the region ID from the geobase. If this region is a city or part of a city, it returns the region ID for the appropriate city. Otherwise, returns 0. +Accepts a region ID from the geobase. If this region is a city or part of a city, it returns the region ID for the appropriate city. Otherwise, returns 0. **Syntax** @@ -51,7 +51,7 @@ regionToCity(id [, geobase]) **Returned value** -- region ID for the appropriate city, if it exists. +- Region ID for the appropriate city, if it exists. - 0, otherwise. **Example** @@ -59,18 +59,36 @@ regionToCity(id [, geobase]) Query: ```sql - +SELECT regionToCity(toUInt32(5)); ``` Result: ```response - + ┌─regionToCity(toUInt32(5))─┐ +1. │ 5 │ + └───────────────────────────┘ ``` -### regionToArea(id\[, geobase\]) +### regionToArea -Converts a region to an area (type 5 in the geobase). In every other way, this function is the same as ‘regionToCity’. +Converts a region to an area (type 5 in the geobase). In every other way, this function is the same as [‘regionToCity’](#regiontocity). + +**Syntax** + +```sql +regionToArea(id [, geobase]) +``` + +**Parameters** + +- `id`: the region ID from the geobase. [UInt32](../data-types/int-uint.md/#uint8-uint16-uint32-uint64-uint128-uint256-int8-int16-int32-int64-int128-int256) +- `geobase` (optional): geobase (dictionary key). [Dictionary Key](../../sql-reference/dictionaries/index.md/#dictionary-key-and-fields-dictionary-key-and-fields) + +**Returned value** + +- Area +- 0, otherwise. ``` sql SELECT DISTINCT regionToName(regionToArea(toUInt32(number), 'ua')) @@ -98,16 +116,38 @@ LIMIT 15 └──────────────────────────────────────────────────────┘ ``` -### regionToDistrict(id\[, geobase\]) +### regionToDistrict Converts a region to a federal district (type 4 in the geobase). In every other way, this function is the same as ‘regionToCity’. +**Syntax** + +```sql +regionToDistrict(id [, geobase]) +``` + +**Parameters** + +- `id`: the region ID from the geobase. [UInt32](../data-types/int-uint.md/#uint8-uint16-uint32-uint64-uint128-uint256-int8-int16-int32-int64-int128-int256) +- `geobase` (optional): geobase (dictionary key). [Dictionary Key](../../sql-reference/dictionaries/index.md/#dictionary-key-and-fields-dictionary-key-and-fields) + +**Returned value** + +- Region ID for the appropriate city, if it exists. +- 0, otherwise. + +**Example** + +Query: + ``` sql SELECT DISTINCT regionToName(regionToDistrict(toUInt32(number), 'ua')) FROM system.numbers LIMIT 15 ``` +Result: + ``` text ┌─regionToName(regionToDistrict(toUInt32(number), \'ua\'))─┐ │ │ @@ -128,27 +168,43 @@ LIMIT 15 └──────────────────────────────────────────────────────────┘ ``` -### regionToCountry(id\[, geobase\]) +### regionToCountry Converts a region to a country. In every other way, this function is the same as ‘regionToCity’. Example: `regionToCountry(toUInt32(213)) = 225` converts Moscow (213) to Russia (225). -### regionToContinent(id\[, geobase\]) +**Syntax** + +```sql +regionToCountry(id [, geobase]) +``` + +### regionToContinent Converts a region to a continent. In every other way, this function is the same as ‘regionToCity’. Example: `regionToContinent(toUInt32(213)) = 10001` converts Moscow (213) to Eurasia (10001). -### regionToTopContinent(id\[, geobase\]) +**Syntax** + +```sql +regionToContinent(id [, geobase]) +``` + +### regionToTopContinent Finds the highest continent in the hierarchy for the region. +```sql +regionToTopContinent(id [, geobase]) +``` + **Syntax** ``` sql regionToTopContinent(id[, geobase]) ``` -**Arguments** +**Parameters** - `id` — Region ID from the geobase. [UInt32](../../sql-reference/data-types/int-uint.md). - `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../../sql-reference/data-types/string.md). Optional. @@ -160,25 +216,49 @@ regionToTopContinent(id[, geobase]) Type: `UInt32`. -### regionToPopulation(id\[, geobase\]) +### regionToPopulation Gets the population for a region. The population can be recorded in files with the geobase. See the section “Dictionaries”. If the population is not recorded for the region, it returns 0. In the geobase, the population might be recorded for child regions, but not for parent regions. -### regionIn(lhs, rhs\[, geobase\]) +**Syntax** + +``` sql +regionToPopulation(id[, geobase]) +``` + +### regionIn Checks whether a ‘lhs’ region belongs to a ‘rhs’ region. Returns a UInt8 number equal to 1 if it belongs, or 0 if it does not belong. The relationship is reflexive – any region also belongs to itself. -### regionHierarchy(id\[, geobase\]) +**Syntax** + +``` sql +regionIn(lhs, rhs\[, geobase\]) +``` + +### regionHierarchy Accepts a UInt32 number – the region ID from the geobase. Returns an array of region IDs consisting of the passed region and all parents along the chain. Example: `regionHierarchy(toUInt32(213)) = [213,1,3,225,10001,10000]`. -### regionToName(id\[, lang\]) +**Syntax** + +``` sql +regionHierarchy(id\[, geobase\]) +``` + +### regionToName Accepts a UInt32 number – the region ID from the geobase. A string with the name of the language can be passed as a second argument. Supported languages are: ru, en, ua, uk, by, kz, tr. If the second argument is omitted, the language ‘ru’ is used. If the language is not supported, an exception is thrown. Returns a string – the name of the region in the corresponding language. If the region with the specified ID does not exist, an empty string is returned. `ua` and `uk` both mean Ukrainian. + +**Syntax** + +``` sql +regionToName(id\[, lang\]) +``` From 7fd8d84fb67b6587b8f3ef5d4194dca04df9f4cb Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 3 Apr 2024 12:25:29 +0200 Subject: [PATCH 0117/1018] Fix style check --- docs/en/engines/table-engines/integrations/rabbitmq.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/integrations/rabbitmq.md b/docs/en/engines/table-engines/integrations/rabbitmq.md index ec7f7a7cec8..a8315a5ad9e 100644 --- a/docs/en/engines/table-engines/integrations/rabbitmq.md +++ b/docs/en/engines/table-engines/integrations/rabbitmq.md @@ -73,7 +73,7 @@ Optional parameters: - `rabbitmq_queue_consume` - Use user-defined queues and do not make any RabbitMQ setup: declaring exchanges, queues, bindings. Default: `false`. - `rabbitmq_username` - RabbitMQ username. - `rabbitmq_password` - RabbitMQ password. -- `reject_unhandled_messages` - Reject messages (send RabbitMQ Nack) in case of errors. This setting is automatically enabled if there is a `x-dead-letter-exchange` defined in `rabbitmq_queue_settings_list`. +- `reject_unhandled_messages` - Reject messages (send RabbitMQ negative acknowledgement) in case of errors. This setting is automatically enabled if there is a `x-dead-letter-exchange` defined in `rabbitmq_queue_settings_list`. - `rabbitmq_commit_on_select` - Commit messages when select query is made. Default: `false`. - `rabbitmq_max_rows_per_message` — The maximum number of rows written in one RabbitMQ message for row-based formats. Default : `1`. - `rabbitmq_empty_queue_backoff_start` — A start backoff point to reschedule read if the rabbitmq queue is empty. From d2c54459d8500f2bddaccb632a44ed4db0f86401 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Wed, 3 Apr 2024 19:55:51 +0200 Subject: [PATCH 0118/1018] Remove "URL-encoded" Change "URL-encoded" to "URL encoded" --- docs/en/interfaces/formats.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 9c94907a9d8..e3f70f888e4 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -2850,7 +2850,7 @@ The Form format can be used to read or write a single record in the application/ Examples: -Given a file `data.tmp` placed in the `user_files` path with some URL-encoded data: +Given a file `data.tmp` placed in the `user_files` path with some URL encoded data: ```text t_page=116&c.e=ls7xfkpm&c.tti.m=raf&rt.start=navigation&rt.bmr=390%2C11%2C10 From e01f1880c43b2457ac266b28e30355a003dfcac4 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Wed, 3 Apr 2024 19:56:44 +0200 Subject: [PATCH 0119/1018] Update aspell-dict.txt Add "URL" to aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index f6fb46dbf3f..f897eee7b7e 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -971,8 +971,8 @@ UMTS UNDROP UPDATEs URIs +URL URL's -URL-encoded URLHash URLHierarchy URLPathHierarchy From 25cab6f0713221e32b2c2fef844e2c2fde77e985 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 3 Apr 2024 20:57:10 +0200 Subject: [PATCH 0120/1018] fix schema inference cache (1) --- src/Storages/StorageS3.cpp | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 28bfa3c32a9..b19e61762d1 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -217,7 +217,7 @@ public: bool no_globs_in_key = fillBufferForKey(*expanded_keys_iter); expanded_keys_iter++; if (expanded_keys_iter == expanded_keys.end() && no_globs_in_key) - is_finished = true; + is_finished_for_key = true; } KeyWithInfoPtr next(size_t) @@ -241,6 +241,7 @@ private: bool fillBufferForKey(const std::string & uri_key) { + is_finished_for_key = false; const String key_prefix = uri_key.substr(0, uri_key.find_first_of("*?{")); /// We don't have to list bucket, because there is no asterisks. @@ -249,10 +250,12 @@ private: buffer.clear(); buffer.emplace_back(std::make_shared(uri_key, std::nullopt)); buffer_iter = buffer.begin(); + if (read_keys) + read_keys->insert(read_keys->end(), buffer.begin(), buffer.end()); return true; } - request = S3::ListObjectsV2Request{}; + request = {}; request.SetBucket(globbed_uri.bucket); request.SetPrefix(key_prefix); request.SetMaxKeys(static_cast(request_settings.list_object_keys_size)); @@ -308,11 +311,11 @@ private: bool no_globs_in_key = fillBufferForKey(*expanded_keys_iter); expanded_keys_iter++; if (expanded_keys_iter == expanded_keys.end() && no_globs_in_key) - is_finished = true; + is_finished_for_key = true; continue; } - if (is_finished) + if (is_finished_for_key) return {}; try @@ -327,7 +330,7 @@ private: /// it may take some time for threads to stop processors and they /// may still use this iterator after exception is thrown. /// To avoid this UB, reset the buffer and return defaults for further calls. - is_finished = true; + is_finished_for_key = true; buffer.clear(); buffer_iter = buffer.begin(); throw; @@ -351,9 +354,9 @@ private: const auto & result_batch = outcome.GetResult().GetContents(); /// It returns false when all objects were returned - is_finished = !outcome.GetResult().GetIsTruncated(); + is_finished_for_key = !outcome.GetResult().GetIsTruncated(); - if (!is_finished) + if (!is_finished_for_key) { /// Even if task is finished the thread may be not freed in pool. /// So wait until it will be freed before scheduling a new task. @@ -444,7 +447,7 @@ private: ActionsDAGPtr filter_dag; std::unique_ptr matcher; bool recursive{false}; - bool is_finished{false}; + bool is_finished_for_key{false}; KeysWithInfo * read_keys; S3::ListObjectsV2Request request; From b493e0654bf1e0b8b95597cd898baeeed89d570b Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 4 Apr 2024 11:50:30 +0200 Subject: [PATCH 0121/1018] More logging --- src/Storages/RabbitMQ/RabbitMQSource.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQSource.cpp b/src/Storages/RabbitMQ/RabbitMQSource.cpp index fc7c570ae1b..c8b8d9f2323 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSource.cpp @@ -200,8 +200,12 @@ Chunk RabbitMQSource::generateImpl() const auto exchange_name = storage.getExchange(); const auto & message = consumer->currentMessage(); - LOG_TEST(log, "Pulled {} rows, message delivery tag: {} (previous delivery tag: {}, redelivered: {})", - new_rows, message.delivery_tag, commit_info.delivery_tag, message.redelivered); + LOG_TEST(log, "Pulled {} rows, message delivery tag: {} " + "(previous delivery tag: {}, redelivered: {}, failed delivery tags by this moment: {}). " + "Exception message: {}", + new_rows, message.delivery_tag, commit_info.delivery_tag, message.redelivered, + commit_info.failed_delivery_tags.size(), + exception_message.has_value() ? exception_message.value() : "None"); if (exception_message.has_value()) { From 46d4d459e6d6e205a146f4379c6febfa253386f4 Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 4 Apr 2024 12:09:31 +0200 Subject: [PATCH 0122/1018] Remove unused code --- .../Formats/Impl/FormInputFormat.cpp | 41 ------------------- src/Processors/Formats/Impl/FormInputFormat.h | 7 ---- .../aspell-ignore/en/aspell-dict.txt | 1 + 3 files changed, 1 insertion(+), 48 deletions(-) diff --git a/src/Processors/Formats/Impl/FormInputFormat.cpp b/src/Processors/Formats/Impl/FormInputFormat.cpp index 0a90bf1da1c..03bf7844103 100644 --- a/src/Processors/Formats/Impl/FormInputFormat.cpp +++ b/src/Processors/Formats/Impl/FormInputFormat.cpp @@ -29,33 +29,6 @@ const String & FormInputFormat::columnName(size_t i) const return getPort().getHeader().getByPosition(i).name; } -StringRef readName(ReadBuffer & buf, StringRef & ref, String & tmp) -{ - tmp.clear(); - - while (!buf.eof()) - { - const char * next_pos = find_first_symbols<'=','&'>(buf.position(), buf.buffer().end()); - - if (next_pos == buf.buffer().end()) - { - tmp.append(buf.position(), next_pos - buf.position()); - buf.position() = buf.buffer().end(); - buf.next(); - continue; - } - - if (*next_pos == '=') - { - ref = StringRef(buf.position(), next_pos - buf.position()); - buf.position() += next_pos + 1 - buf.position(); - } - - return ref; - } - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected end of stream while reading key name from Form format"); -} - void FormInputFormat::readField(size_t index, MutableColumns & columns) { if (seen_columns[index]) @@ -83,20 +56,6 @@ String readFieldName(ReadBuffer & buf) return field; } -void FormInputFormat::skipUnknownFormField(StringRef name_ref) -{ - if (!format_settings.skip_unknown_fields) - throw Exception(ErrorCodes::INCORRECT_DATA, "Unknown field found while parsing Form format: {}", name_ref.toString()); - - /// read name and value but do nothing with them - if (!in->eof()) - { - readFieldName(*in); - String value; - readStringUntilAmpersand(value,*in); - } -} - void FormInputFormat::readFormData(MutableColumns & columns) { size_t index = 0; diff --git a/src/Processors/Formats/Impl/FormInputFormat.h b/src/Processors/Formats/Impl/FormInputFormat.h index e6275545b5e..93edf49e258 100644 --- a/src/Processors/Formats/Impl/FormInputFormat.h +++ b/src/Processors/Formats/Impl/FormInputFormat.h @@ -21,16 +21,9 @@ private: void readPrefix() override; bool readRow(MutableColumns & columns, RowReadExtension & ext) override; void readFormData(MutableColumns & columns); - void readNestedFormData(const String & name, MutableColumns & columns); void readField(size_t index, MutableColumns & columns); - void skipUnknownFormField(StringRef name_ref); const String & columnName(size_t i) const; - String name_buf; - - /// holds common prefix of nested column names - String current_column_name; - /// Hash table matches field name to position in the block using NameMap = HashMap; NameMap name_map; diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index f897eee7b7e..9a19e37b3f4 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2680,6 +2680,7 @@ uptime uptrace uring url +urlencoded urlCluster urls usearch From 836110ead5942d8c7505bd4a7ff79ad58a90cec1 Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 4 Apr 2024 14:56:47 +0200 Subject: [PATCH 0123/1018] Remove unused error code CANNOT_READ_ALL_DATA --- src/Processors/Formats/Impl/FormInputFormat.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Processors/Formats/Impl/FormInputFormat.cpp b/src/Processors/Formats/Impl/FormInputFormat.cpp index 03bf7844103..771da34e6c3 100644 --- a/src/Processors/Formats/Impl/FormInputFormat.cpp +++ b/src/Processors/Formats/Impl/FormInputFormat.cpp @@ -7,7 +7,6 @@ namespace DB namespace ErrorCodes { - extern const int CANNOT_READ_ALL_DATA; extern const int INCORRECT_DATA; } From ce3969e25d7ef8bd661fa6047ac0882735fd567a Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Thu, 4 Apr 2024 19:47:34 +0000 Subject: [PATCH 0124/1018] adapt test to new behavior --- tests/integration/test_storage_s3/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 6d5b84a8143..9d275575f8a 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1768,13 +1768,13 @@ def test_schema_inference_cache(started_cluster): check_cache(instance, []) run_describe_query(instance, files, storage_name, started_cluster, bucket) - check_cache_misses(instance, files, storage_name, started_cluster, bucket, 4) + check_cache_misses(instance, files, storage_name, started_cluster, bucket, 4 if storage_name == "url" else 1) instance.query("system drop schema cache") check_cache(instance, []) run_describe_query(instance, files, storage_name, started_cluster, bucket) - check_cache_misses(instance, files, storage_name, started_cluster, bucket, 4) + check_cache_misses(instance, files, storage_name, started_cluster, bucket, 4 if storage_name == "url" else 1) instance.query("system drop schema cache") From e3858107969d6f6363de343197608bf65693dd59 Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 4 Apr 2024 22:18:41 +0200 Subject: [PATCH 0125/1018] fix black --- tests/integration/test_storage_s3/test.py | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 9d275575f8a..a4ed94c815b 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1768,13 +1768,27 @@ def test_schema_inference_cache(started_cluster): check_cache(instance, []) run_describe_query(instance, files, storage_name, started_cluster, bucket) - check_cache_misses(instance, files, storage_name, started_cluster, bucket, 4 if storage_name == "url" else 1) + check_cache_misses( + instance, + files, + storage_name, + started_cluster, + bucket, + 4 if storage_name == "url" else 1, + ) instance.query("system drop schema cache") check_cache(instance, []) run_describe_query(instance, files, storage_name, started_cluster, bucket) - check_cache_misses(instance, files, storage_name, started_cluster, bucket, 4 if storage_name == "url" else 1) + check_cache_misses( + instance, + files, + storage_name, + started_cluster, + bucket, + 4 if storage_name == "url" else 1, + ) instance.query("system drop schema cache") From 1cd73b907c1493729699578231fa223f57de2f35 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 5 Apr 2024 12:59:57 +0200 Subject: [PATCH 0126/1018] Better --- src/Interpreters/Cache/EvictionCandidates.h | 11 -- src/Interpreters/Cache/FileCache.cpp | 123 ++++++++++++------ src/Interpreters/Cache/IFileCachePriority.h | 3 +- .../Cache/LRUFileCachePriority.cpp | 6 +- src/Interpreters/Cache/LRUFileCachePriority.h | 3 +- .../Cache/SLRUFileCachePriority.cpp | 33 +++-- .../Cache/SLRUFileCachePriority.h | 3 +- tests/config/config.d/storage_conf.xml | 6 +- 8 files changed, 116 insertions(+), 72 deletions(-) diff --git a/src/Interpreters/Cache/EvictionCandidates.h b/src/Interpreters/Cache/EvictionCandidates.h index b2bb6ae9e9f..db062561ad7 100644 --- a/src/Interpreters/Cache/EvictionCandidates.h +++ b/src/Interpreters/Cache/EvictionCandidates.h @@ -10,15 +10,6 @@ public: using FinalizeEvictionFunc = std::function; EvictionCandidates() = default; - EvictionCandidates(EvictionCandidates && other) noexcept - { - candidates = std::move(other.candidates); - candidates_size = std::move(other.candidates_size); - on_finalize = std::move(other.on_finalize); - queue_entries_to_invalidate = std::move(other.queue_entries_to_invalidate); - hold_space = std::move(other.hold_space); - } - ~EvictionCandidates(); void add( @@ -26,8 +17,6 @@ public: LockedKey & locked_key, const CachePriorityGuard::Lock &); - void add(const EvictionCandidates & other, const CachePriorityGuard::Lock &) { candidates.insert(other.candidates.begin(), other.candidates.end()); } - void evict(); void onFinalize(FinalizeEvictionFunc && func) { on_finalize.emplace_back(std::move(func)); } diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 3f16a8dde4e..61401c8ea0a 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -953,49 +953,71 @@ void FileCache::freeSpaceRatioKeepingThreadFunc() static constexpr auto space_ratio_satisfied_reschedule_ms = 5000; static constexpr auto general_reschedule_ms = 5000; - while (true) + if (shutdown) + return; + + Stopwatch watch; + + auto lock = tryLockCache(); + + /// To avoid deteriorating contention on cache, + /// proceed only if cache is not heavily used. + if (!lock) { + keep_up_free_space_ratio_task->scheduleAfter(lock_failed_reschedule_ms); + return; + } + + const size_t size_limit = main_priority->getSizeLimit(lock); + const size_t elements_limit = main_priority->getElementsLimit(lock); + + const size_t desired_size = std::lround(keep_current_size_to_max_ratio * size_limit); + const size_t desired_elements_num = std::lround(keep_current_elements_to_max_ratio * elements_limit); + + if ((size_limit == 0 || main_priority->getSize(lock) <= desired_size) + && (elements_limit == 0 || main_priority->getElementsCount(lock) <= desired_elements_num)) + { + /// Nothing to free - all limits are satisfied. + keep_up_free_space_ratio_task->scheduleAfter(space_ratio_satisfied_reschedule_ms); + return; + } + + FileCacheReserveStat stat; + EvictionCandidates eviction_candidates; + + try + { + /// Collect at most `keep_up_free_space_remove_batch` elements to evict, + /// (we use batches to make sure we do not block cache for too long, + /// by default the batch size is quite small). + const bool limits_satisfied = main_priority->collectCandidatesForEviction( + desired_size, desired_elements_num, keep_up_free_space_remove_batch, stat, eviction_candidates, lock); + +#ifdef ABORT_ON_LOGICAL_ERROR + /// Let's make sure that we correctly processed the limits. + if (limits_satisfied && eviction_candidates.size() < keep_up_free_space_remove_batch) + { + const auto current_size = main_priority->getSize(lock); + chassert(current_size >= stat.total_stat.releasable_size); + chassert(!size_limit + || current_size <= desired_size + || current_size - stat.total_stat.releasable_size <= desired_size); + + const auto current_elements_count = main_priority->getElementsCount(lock); + chassert(current_elements_count >= stat.total_stat.releasable_count); + chassert(!elements_limit + || current_elements_count <= desired_elements_num + || current_elements_count - stat.total_stat.releasable_count <= desired_elements_num); + } +#else + UNUSED(limits_satisfied); +#endif + if (shutdown) return; - auto lock = tryLockCache(); - if (!lock) + if (eviction_candidates.size() > 0) { - keep_up_free_space_ratio_task->scheduleAfter(lock_failed_reschedule_ms); - return; - } - - const size_t size_limit = main_priority->getSizeLimit(lock); - const size_t elements_limit = main_priority->getElementsLimit(lock); - - const size_t desired_size = std::lround(keep_current_size_to_max_ratio * size_limit); - const size_t desired_elements_num = std::lround(keep_current_elements_to_max_ratio * elements_limit); - - if ((size_limit == 0 || main_priority->getSize(lock) <= desired_size) - && (elements_limit == 0 || main_priority->getElementsCount(lock) <= desired_elements_num)) - { - /// Nothing to free - all limits are satisfied. - keep_up_free_space_ratio_task->scheduleAfter(space_ratio_satisfied_reschedule_ms); - return; - } - - try - { - FileCacheReserveStat stat; - auto eviction_candidates = main_priority->collectCandidatesForEviction( - desired_size, desired_elements_num, keep_up_free_space_remove_batch, stat, lock); - - if (shutdown) - return; - - if (eviction_candidates.size() == 0) - { - /// This case is impossible in realistic cache setup, - /// e.g. we should always be able to evict something. - keep_up_free_space_ratio_task->scheduleAfter(general_reschedule_ms); - return; - } - LOG_TRACE(log, "Current usage {}/{} in size, {}/{} in elements count " "(trying to keep size ration at {} and elements ratio at {}). " "Collected {} eviction candidates, " @@ -1006,20 +1028,35 @@ void FileCache::freeSpaceRatioKeepingThreadFunc() eviction_candidates.size(), stat.total_stat.non_releasable_count); lock.unlock(); + + /// Remove files from filesystem. eviction_candidates.evict(); + /// Take lock again to finalize eviction, + /// e.g. to update the in-memory state. lock.lock(); eviction_candidates.finalize(nullptr, lock); } - catch (...) + else { - tryLogCurrentException(__PRETTY_FUNCTION__); keep_up_free_space_ratio_task->scheduleAfter(general_reschedule_ms); - - /// Let's catch such cases in ci, in general there should not be exceptions. - chassert(false); } } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + + if (eviction_candidates.size() > 0) + eviction_candidates.finalize(nullptr, lockCache()); + + keep_up_free_space_ratio_task->scheduleAfter(general_reschedule_ms); + + /// Let's catch such cases in ci, + /// in general there should not be exceptions. + chassert(false); + } + + LOG_TRACE(log, "Free space ratio keeping thread finished in {} ms", watch.elapsedMilliseconds()); } void FileCache::iterate(IterateFunc && func, const UserID & user_id) diff --git a/src/Interpreters/Cache/IFileCachePriority.h b/src/Interpreters/Cache/IFileCachePriority.h index dba925ba37b..e1ac6c6661d 100644 --- a/src/Interpreters/Cache/IFileCachePriority.h +++ b/src/Interpreters/Cache/IFileCachePriority.h @@ -148,11 +148,12 @@ public: const CachePriorityGuard::Lock &) = 0; /// Collect eviction `candidates_num` candidates for eviction. - virtual EvictionCandidates collectCandidatesForEviction( + virtual bool collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, FileCacheReserveStat & stat, + EvictionCandidates & candidates, const CachePriorityGuard::Lock &) = 0; virtual bool modifySizeLimits(size_t max_size_, size_t max_elements_, double size_ratio_, const CachePriorityGuard::Lock &) = 0; diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index 203fc680b47..8ac60cd9e6e 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -322,24 +322,24 @@ bool LRUFileCachePriority::collectCandidatesForEviction( } } -EvictionCandidates LRUFileCachePriority::collectCandidatesForEviction( +bool LRUFileCachePriority::collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, FileCacheReserveStat & stat, + EvictionCandidates & res, const CachePriorityGuard::Lock & lock) { if (!max_candidates_to_evict) return {}; - EvictionCandidates res; auto stop_condition = [&, this]() { return (getSize(lock) <= desired_size && getElementsCount(lock) <= desired_elements_count) || res.size() >= max_candidates_to_evict; }; iterateForEviction(res, stat, stop_condition, lock); - return res; + return stop_condition(); } void LRUFileCachePriority::iterateForEviction( diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index d05c67fe5d0..4cc80f07664 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -62,11 +62,12 @@ public: const UserID & user_id, const CachePriorityGuard::Lock &) override; - EvictionCandidates collectCandidatesForEviction( + bool collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, FileCacheReserveStat & stat, + EvictionCandidates & res, const CachePriorityGuard::Lock &) override; void shuffle(const CachePriorityGuard::Lock &) override; diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.cpp b/src/Interpreters/Cache/SLRUFileCachePriority.cpp index a2730d88abb..4b6221d972a 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/SLRUFileCachePriority.cpp @@ -251,11 +251,12 @@ bool SLRUFileCachePriority::collectCandidatesForEvictionInProtected( return true; } -EvictionCandidates SLRUFileCachePriority::collectCandidatesForEviction( +bool SLRUFileCachePriority::collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, FileCacheReserveStat & stat, + EvictionCandidates & res, const CachePriorityGuard::Lock & lock) { if (!max_candidates_to_evict) @@ -264,22 +265,36 @@ EvictionCandidates SLRUFileCachePriority::collectCandidatesForEviction( const auto desired_probationary_size = getRatio(desired_size, 1 - size_ratio); const auto desired_probationary_elements_num = getRatio(desired_elements_count, 1 - size_ratio); - auto res = probationary_queue.collectCandidatesForEviction( - desired_probationary_size, desired_probationary_elements_num, max_candidates_to_evict, stat, lock); + FileCacheReserveStat probationary_stat; + const bool probationary_limit_satisfied = probationary_queue.collectCandidatesForEviction( + desired_probationary_size, desired_probationary_elements_num, + max_candidates_to_evict, probationary_stat, res, lock); + + stat += probationary_stat; + + LOG_TEST(log, "Collected {} to evict from probationary queue. Total size: {}", + res.size(), probationary_stat.total_stat.releasable_size); chassert(res.size() <= max_candidates_to_evict); chassert(res.size() == stat.total_stat.releasable_count); - if (res.size() == max_candidates_to_evict) - return res; + if (res.size() >= max_candidates_to_evict) + return probationary_limit_satisfied; const auto desired_protected_size = getRatio(max_size, size_ratio); const auto desired_protected_elements_num = getRatio(max_elements, size_ratio); - auto res_add = protected_queue.collectCandidatesForEviction( - desired_protected_size, desired_protected_elements_num, max_candidates_to_evict - res.size(), stat, lock); - res.add(res_add, lock); - return res; + FileCacheReserveStat protected_stat; + const bool protected_limit_satisfied = protected_queue.collectCandidatesForEviction( + desired_protected_size, desired_protected_elements_num, + max_candidates_to_evict - res.size(), protected_stat, res, lock); + + stat += protected_stat; + + LOG_TEST(log, "Collected {} to evict from protected queue. Total size: {}", + res.size(), protected_stat.total_stat.releasable_size); + + return probationary_limit_satisfied && protected_limit_satisfied; } void SLRUFileCachePriority::downgrade(IteratorPtr iterator, const CachePriorityGuard::Lock & lock) diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.h b/src/Interpreters/Cache/SLRUFileCachePriority.h index e837b8541c2..ee3cafe322d 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.h +++ b/src/Interpreters/Cache/SLRUFileCachePriority.h @@ -58,11 +58,12 @@ public: const UserID & user_id, const CachePriorityGuard::Lock &) override; - EvictionCandidates collectCandidatesForEviction( + bool collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, FileCacheReserveStat & stat, + EvictionCandidates & res, const CachePriorityGuard::Lock &) override; void shuffle(const CachePriorityGuard::Lock &) override; diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index 24e85031a3f..9d1f7e6b474 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -19,13 +19,13 @@ cache s3_disk s3_cache/ - 64Mi + 100Mi 1 100 LRU 0.3 - 0.1 - 0.1 + 0.15 + 0.15 cache From 832c7087a7c0cf0692140d349cd52b44278454d3 Mon Sep 17 00:00:00 2001 From: zhongyuankai <872237106@qq.com> Date: Sat, 6 Apr 2024 10:31:45 +0800 Subject: [PATCH 0127/1018] optimize drop tables --- src/Interpreters/DDLTask.cpp | 9 --------- src/Interpreters/InterpreterDropQuery.cpp | 2 +- src/Parsers/ASTDropQuery.h | 3 ++- src/Parsers/ParserDropQuery.cpp | 11 +++++++++++ src/Parsers/tests/gtest_dictionary_parser.cpp | 12 ++++-------- .../0_stateless/02961_drop_tables.reference | 3 +++ tests/queries/0_stateless/02961_drop_tables.sql | 14 ++++++++++++++ 7 files changed, 35 insertions(+), 19 deletions(-) diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index 37954850851..a37b4db029a 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -16,7 +16,6 @@ #include #include #include -#include namespace DB @@ -201,14 +200,6 @@ void DDLTaskBase::parseQueryFromEntry(ContextPtr context) ParserQuery parser_query(end, settings.allow_settings_after_format_in_insert); String description; query = parseQuery(parser_query, begin, end, description, 0, settings.max_parser_depth, settings.max_parser_backtracks); - if (auto * query_drop = query->as()) - { - ASTs drops = query_drop->getRewrittenASTsOfSingleTable(); - if (drops.size() > 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Not supports drop multiple tables for ddl task."); - - query = drops[0]; - } } void DDLTaskBase::formatRewrittenQuery(ContextPtr context) diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index e29e59ee4c3..0e591a7782a 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -557,7 +557,7 @@ bool InterpreterDropQuery::supportsTransactions() const return drop.cluster.empty() && !drop.temporary && drop.kind == ASTDropQuery::Kind::Truncate - && drop.database_and_tables; + && drop.table; } void registerInterpreterDropQuery(InterpreterFactory & factory) diff --git a/src/Parsers/ASTDropQuery.h b/src/Parsers/ASTDropQuery.h index e0e908733e5..7e25e990bc8 100644 --- a/src/Parsers/ASTDropQuery.h +++ b/src/Parsers/ASTDropQuery.h @@ -40,7 +40,7 @@ public: // We detach the object permanently, so it will not be reattached back during server restart. bool permanently{false}; - /// Example: Drop TABLE t1, t2, t3... + /// Used to drop multiple tables only, example: Drop TABLE t1, t2, t3... ASTPtr database_and_tables; /** Get the text that identifies this element. */ @@ -52,6 +52,7 @@ public: return removeOnCluster(clone(), params.default_database); } + /** Convert an AST that deletes multiple tables into multiple ASTs that delete a single table. */ ASTs getRewrittenASTsOfSingleTable(); QueryKind getQueryKind() const override { return QueryKind::Drop; } diff --git a/src/Parsers/ParserDropQuery.cpp b/src/Parsers/ParserDropQuery.cpp index 09f15e9649f..9fe8306c0c2 100644 --- a/src/Parsers/ParserDropQuery.cpp +++ b/src/Parsers/ParserDropQuery.cpp @@ -7,6 +7,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int SYNTAX_ERROR; +} + namespace { @@ -84,6 +89,9 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, cons if (!tables_p.parse(pos, database_and_tables, expected)) return false; + + if (database_and_tables->size() > 1 && kind != ASTDropQuery::Kind::Drop) + throw Exception(ErrorCodes::SYNTAX_ERROR, "Only Support DROP multiple tables currently"); } /// common for tables / dictionaries / databases @@ -123,6 +131,9 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, cons query->cluster = cluster_str; + if (database_and_tables && database_and_tables->size() == 1) + node = query->getRewrittenASTsOfSingleTable()[0]; + return true; } diff --git a/src/Parsers/tests/gtest_dictionary_parser.cpp b/src/Parsers/tests/gtest_dictionary_parser.cpp index 44205975cdc..a1ba46125a7 100644 --- a/src/Parsers/tests/gtest_dictionary_parser.cpp +++ b/src/Parsers/tests/gtest_dictionary_parser.cpp @@ -301,10 +301,8 @@ TEST(ParserDictionaryDDL, ParseDropQuery) ASTDropQuery * drop1 = ast1->as(); EXPECT_TRUE(drop1->is_dictionary); - auto & database_and_tables1 = drop1->database_and_tables->as(); - auto identifier1 = dynamic_pointer_cast(database_and_tables1.children[0]); - EXPECT_EQ(identifier1->getDatabaseName(), "test"); - EXPECT_EQ(identifier1->shortName(), "dict1"); + EXPECT_EQ(drop1->getDatabase(), "test"); + EXPECT_EQ(drop1->getTable(), "dict1"); auto str1 = serializeAST(*drop1); EXPECT_EQ(input1, str1); @@ -314,10 +312,8 @@ TEST(ParserDictionaryDDL, ParseDropQuery) ASTDropQuery * drop2 = ast2->as(); EXPECT_TRUE(drop2->is_dictionary); - auto & database_and_tables2 = drop2->database_and_tables->as(); - auto identifier2 = dynamic_pointer_cast(database_and_tables2.children[0]); - EXPECT_EQ(identifier2->getDatabaseName(), ""); - EXPECT_EQ(identifier2->shortName(), "dict2"); + EXPECT_EQ(drop2->getDatabase(), ""); + EXPECT_EQ(drop2->getTable(), "dict2"); auto str2 = serializeAST(*drop2); EXPECT_EQ(input2, str2); } diff --git a/tests/queries/0_stateless/02961_drop_tables.reference b/tests/queries/0_stateless/02961_drop_tables.reference index c0465dc592a..8ccdec0a1b0 100644 --- a/tests/queries/0_stateless/02961_drop_tables.reference +++ b/tests/queries/0_stateless/02961_drop_tables.reference @@ -6,3 +6,6 @@ Test when deletion of existing table fails -- check which tables exist in 02961_db1 -- check which tables exist in 02961_db2 02961_tb5 +Test when deletion of not empty table fails +tab2 +tab3 diff --git a/tests/queries/0_stateless/02961_drop_tables.sql b/tests/queries/0_stateless/02961_drop_tables.sql index e91ac4bfe19..f84fffbef75 100644 --- a/tests/queries/0_stateless/02961_drop_tables.sql +++ b/tests/queries/0_stateless/02961_drop_tables.sql @@ -27,6 +27,20 @@ SHOW TABLES FROM 02961_db1; SELECT '-- check which tables exist in 02961_db2'; SHOW TABLES FROM 02961_db2; +DROP TABLE IF EXISTS tab1, tab2, tab3; +CREATE TABLE IF NOT EXISTS tab1 (id UInt32) Engine=Memory(); +CREATE TABLE IF NOT EXISTS tab2 (id UInt32) Engine=Memory(); +CREATE TABLE IF NOT EXISTS tab3 (id UInt32) Engine=Memory(); + +INSERT INTO tab2 SELECT number FROM system.numbers limit 10; + +DROP TABLE IF EMPTY tab1, tab2, tab3; -- { serverError TABLE_NOT_EMPTY } +SELECT 'Test when deletion of not empty table fails'; +SHOW TABLES; + +TRUNCATE TABLE tab2, tab3; -- { clientError SYNTAX_ERROR } + +DROP TABLE IF EXISTS tab1, tab2, tab3; DROP DATABASE IF EXISTS 02961_db1; DROP DATABASE IF EXISTS 02961_db2; From d5aa23659043ae17a267bef3353a383e5383a24e Mon Sep 17 00:00:00 2001 From: zhongyuankai <54787696+zhongyuankai@users.noreply.github.com> Date: Mon, 8 Apr 2024 19:40:27 +0800 Subject: [PATCH 0128/1018] fix test --- src/Parsers/ParserDropQuery.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Parsers/ParserDropQuery.cpp b/src/Parsers/ParserDropQuery.cpp index 9fe8306c0c2..6efafa112d3 100644 --- a/src/Parsers/ParserDropQuery.cpp +++ b/src/Parsers/ParserDropQuery.cpp @@ -90,7 +90,7 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, cons if (!tables_p.parse(pos, database_and_tables, expected)) return false; - if (database_and_tables->size() > 1 && kind != ASTDropQuery::Kind::Drop) + if (database_and_tables->as().children.size() > 1 && kind != ASTDropQuery::Kind::Drop) throw Exception(ErrorCodes::SYNTAX_ERROR, "Only Support DROP multiple tables currently"); } @@ -131,7 +131,7 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, cons query->cluster = cluster_str; - if (database_and_tables && database_and_tables->size() == 1) + if (database_and_tables && database_and_tables->as().children.size() == 1) node = query->getRewrittenASTsOfSingleTable()[0]; return true; From 80c494a900474be5208e7be82793de86171408b3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 8 Apr 2024 15:23:41 +0200 Subject: [PATCH 0129/1018] Fxi --- src/Interpreters/Cache/FileCache.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 61401c8ea0a..83e17ddb2bb 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -193,6 +193,8 @@ void FileCache::initialize() if (keep_current_size_to_max_ratio != 1 || keep_current_elements_to_max_ratio != 1) keep_up_free_space_ratio_task = Context::getGlobalContextInstance()->getSchedulePool().createTask(log->name(), [this] { freeSpaceRatioKeepingThreadFunc(); }); + keep_up_free_space_ratio_task->schedule(); + is_initialized = true; } From 7404e7d27bf6ecadba46f97395a60d48894c1374 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 8 Apr 2024 15:24:00 +0200 Subject: [PATCH 0130/1018] Add a test --- .../integration/test_filesystem_cache/test.py | 57 +++++++++++++++++++ 1 file changed, 57 insertions(+) diff --git a/tests/integration/test_filesystem_cache/test.py b/tests/integration/test_filesystem_cache/test.py index dfab462732a..17a8dd8b6e1 100644 --- a/tests/integration/test_filesystem_cache/test.py +++ b/tests/integration/test_filesystem_cache/test.py @@ -501,3 +501,60 @@ INSERT INTO test SELECT 1, 'test'; node.query("SELECT * FROM test FORMAT Null") assert key not in node.query("SYSTEM SYNC FILESYSTEM CACHE") + + +def test_keep_up_size_ratio(cluster): + node = cluster.instances["node"] + max_elements = 20 + elements_ratio = 0.5 + cache_name = "keep_up_size_ratio" + node.query( + f""" +DROP TABLE IF EXISTS test; + +CREATE TABLE test (a String) +ENGINE = MergeTree() ORDER BY tuple() +SETTINGS disk = disk(type = cache, + name = {cache_name}, + max_size = '100Ki', + max_elements = {max_elements}, + max_file_segment_size = 10, + boundary_alignment = 10, + path = "test_keep_up_size_ratio", + keep_free_space_size_ratio = 0.5, + keep_free_space_elements_ratio = {elements_ratio}, + disk = hdd_blob), + min_bytes_for_wide_part = 10485760; + +INSERT INTO test SELECT randomString(200); + """ + ) + + query_id = "test_keep_up_size_ratio_1" + node.query( + "SELECT * FROM test FORMAT Null SETTINGS enable_filesystem_cache_log = 1", + query_id=query_id, + ) + count = int( + node.query( + f""" + SYSTEM FLUSH LOGS; + SELECT uniqExact(concat(key, toString(offset))) + FROM system.filesystem_cache_log + WHERE read_type = 'READ_FROM_FS_AND_DOWNLOADED_TO_CACHE'; + """ + ) + ) + assert count > max_elements + + expected = 10 + for _ in range(100): + elements = int( + node.query( + f"SELECT count() FROM system.filesystem_cache WHERE cache_name = '{cache_name}'" + ) + ) + if elements <= expected: + break + time.sleep(1) + assert elements <= expected From ed6e4fbe162e9398a5f3b2ce24ed4afb31141f97 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 8 Apr 2024 13:56:15 +0000 Subject: [PATCH 0131/1018] Improve trivial insert select from files, add separate max_parsing_threads setting --- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.h | 1 + src/Formats/FormatFactory.cpp | 2 +- src/Storages/StorageFile.cpp | 2 +- src/Storages/StorageS3.cpp | 4 +-- src/Storages/StorageURL.cpp | 6 ++-- .../trivial_insert_select_from_files.xml | 30 +++++++++++++++++++ 7 files changed, 39 insertions(+), 7 deletions(-) create mode 100644 tests/performance/trivial_insert_select_from_files.xml diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 84e709294aa..67a12d1e6d5 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -50,6 +50,7 @@ class IColumn; M(MaxThreads, max_threads, 0, "The maximum number of threads to execute the request. By default, it is determined automatically.", 0) \ M(Bool, use_concurrency_control, true, "Respect the server's concurrency control (see the `concurrent_threads_soft_limit_num` and `concurrent_threads_soft_limit_ratio_to_cores` global server settings). If disabled, it allows using a larger number of threads even if the server is overloaded (not recommended for normal usage, and needed mostly for tests).", 0) \ M(MaxThreads, max_download_threads, 4, "The maximum number of threads to download data (e.g. for URL engine).", 0) \ + M(MaxThreads, max_parsing_threads, 0, "The maximum number of threads to parse data in input formats that support parallel parsing. By default, it is determined automatically", 0) \ M(UInt64, max_download_buffer_size, 10*1024*1024, "The maximal size of buffer for parallel downloading (e.g. for URL engine) per each thread.", 0) \ M(UInt64, max_read_buffer_size, DBMS_DEFAULT_BUFFER_SIZE, "The maximum size of the buffer to read from the filesystem.", 0) \ M(UInt64, max_read_buffer_size_local_fs, 128*1024, "The maximum size of the buffer to read from local filesystem. If set to 0 then max_read_buffer_size will be used.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 768b6aa6cbd..a26e215ce35 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -86,6 +86,7 @@ namespace SettingsChangesHistory static std::map settings_changes_history = { {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, + {"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"}, }}, {"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"}, {"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"}, diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 8cbb1b9e563..1cc3dabd954 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -303,7 +303,7 @@ InputFormatPtr FormatFactory::getInput( auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context); const Settings & settings = context->getSettingsRef(); - size_t max_parsing_threads = _max_parsing_threads.value_or(settings.max_threads); + size_t max_parsing_threads = _max_parsing_threads.value_or(settings.max_parsing_threads); size_t max_download_threads = _max_download_threads.value_or(settings.max_download_threads); RowInputFormatParams row_input_format_params; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 0d220f2fd5d..54ba567b838 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1392,7 +1392,7 @@ Chunk StorageFileSource::generate() chassert(file_num > 0); - const auto max_parsing_threads = std::max(settings.max_threads / file_num, 1UL); + const auto max_parsing_threads = std::max(settings.max_parsing_threads / file_num, 1UL); input_format = FormatFactory::instance().getInput( storage->format_name, *read_buf, block_for_format, getContext(), max_block_size, storage->format_settings, max_parsing_threads, std::nullopt, /*is_remote_fs*/ false, CompressionMethod::None, need_only_count); diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 2d3aef312bf..c7b80f4912a 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1206,8 +1206,8 @@ void ReadFromStorageS3Step::initializePipeline(QueryPipelineBuilder & pipeline, /// Disclosed glob iterator can underestimate the amount of keys in some cases. We will keep one stream for this particular case. num_streams = 1; - const size_t max_threads = context->getSettingsRef().max_threads; - const size_t max_parsing_threads = num_streams >= max_threads ? 1 : (max_threads / std::max(num_streams, 1ul)); + const auto & settings = context->getSettingsRef(); + const size_t max_parsing_threads = num_streams >= settings.max_parsing_threads ? 1 : (settings.max_parsing_threads / std::max(num_streams, 1ul)); LOG_DEBUG(getLogger("StorageS3"), "Reading in {} streams, {} threads per stream", num_streams, max_parsing_threads); Pipes pipes; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index cc46cc8f8dc..be23ff50647 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -1172,8 +1172,8 @@ void ReadFromURL::initializePipeline(QueryPipelineBuilder & pipeline, const Buil Pipes pipes; pipes.reserve(num_streams); - const size_t max_threads = context->getSettingsRef().max_threads; - const size_t max_parsing_threads = num_streams >= max_threads ? 1 : (max_threads / num_streams); + const auto & settings = context->getSettingsRef(); + const size_t max_parsing_threads = num_streams >= settings.max_parsing_threads ? 1 : (settings.max_parsing_threads / num_streams); for (size_t i = 0; i < num_streams; ++i) { @@ -1204,7 +1204,7 @@ void ReadFromURL::initializePipeline(QueryPipelineBuilder & pipeline, const Buil auto pipe = Pipe::unitePipes(std::move(pipes)); size_t output_ports = pipe.numOutputPorts(); - const bool parallelize_output = context->getSettingsRef().parallelize_output_from_storages; + const bool parallelize_output = settings.parallelize_output_from_storages; if (parallelize_output && storage->parallelizeOutputAfterReading(context) && output_ports > 0 && output_ports < max_num_streams) pipe.resize(max_num_streams); diff --git a/tests/performance/trivial_insert_select_from_files.xml b/tests/performance/trivial_insert_select_from_files.xml new file mode 100644 index 00000000000..f6ec6500c10 --- /dev/null +++ b/tests/performance/trivial_insert_select_from_files.xml @@ -0,0 +1,30 @@ + + + + + format + + TabSeparated + TabSeparatedWithNames + TabSeparatedWithNamesAndTypes + CSV + CSVWithNames + JSONEachRow + JSONCompactEachRow + JSONCompactEachRowWithNamesAndTypes + TSKV + + + + +CREATE TABLE IF NOT EXISTS table_src_{format} ENGINE = File({format}) AS test.hits +CREATE TABLE IF NOT EXISTS table_dst_{format} AS test.hits + +INSERT INTO table_src_{format} SELECT * FROM test.hits LIMIT 100000 + +INSERT INTO table_dst_{format} SELECT * FROM table_src_{format} + +DROP TABLE IF EXISTS table_src_{format} +DROP TABLE IF EXISTS table_dst_{format} + + From 0fa4dfb2825a0c13e5adc84614c5de5c7a83c365 Mon Sep 17 00:00:00 2001 From: Blargian Date: Mon, 8 Apr 2024 17:11:18 +0200 Subject: [PATCH 0132/1018] Update regionXYZ documentation --- .../functions/ym-dict-functions.md | 316 ++++++++++++++++-- 1 file changed, 291 insertions(+), 25 deletions(-) diff --git a/docs/en/sql-reference/functions/ym-dict-functions.md b/docs/en/sql-reference/functions/ym-dict-functions.md index cd5cd33a240..f9d53ef1cb5 100644 --- a/docs/en/sql-reference/functions/ym-dict-functions.md +++ b/docs/en/sql-reference/functions/ym-dict-functions.md @@ -8,6 +8,8 @@ sidebar_label: Embedded Dictionaries :::note In order for the functions below to work, the server config must specify the paths and addresses for getting all the embedded dictionaries. The dictionaries are loaded at the first call of any of these functions. If the reference lists can’t be loaded, an exception is thrown. + +As such, the examples shown in this section cannot be run in [ClickHouse Fiddle](https://fiddle.clickhouse.com/) ::: For information about creating reference lists, see the section [“Dictionaries”](../../sql-reference/dictionaries/index.md). @@ -23,9 +25,10 @@ For example, it will also find the file `/opt/geo/regions_hierarchy_ua.txt`, if `ua` is called the dictionary key. For a dictionary without a suffix, the key is an empty string. -All the dictionaries are re-loaded in runtime (once every certain number of seconds, as defined in the builtin_dictionaries_reload_interval config parameter, or once an hour by default). However, the list of available dictionaries is defined one time, when the server starts. +All the dictionaries are re-loaded in runtime (once every certain number of seconds, as defined in the [`builtin_dictionaries_reload_interval`](../../operations/server-configuration-parameters/settings#builtin-dictionaries-reload-interval) config parameter, or once an hour by default). However, the list of available dictionaries is defined one time, when the server starts. All functions for working with regions have an optional argument at the end – the dictionary key. It is referred to as the geobase. + Example: ``` sql @@ -46,28 +49,42 @@ regionToCity(id [, geobase]) **Parameters** -- `id`: the region ID from the geobase. [UInt32](../data-types/int-uint.md/#uint8-uint16-uint32-uint64-uint128-uint256-int8-int16-int32-int64-int128-int256) -- `geobase` (optional): geobase (dictionary key). [Dictionary Key](../../sql-reference/dictionaries/index.md/#dictionary-key-and-fields-dictionary-key-and-fields) +- `id` — Region ID from the geobase. [UInt32](../../sql-reference/data-types/int-uint.md). +- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../../sql-reference/data-types/string.md). Optional. **Returned value** - Region ID for the appropriate city, if it exists. -- 0, otherwise. +- 0, if there is none. + +Type: `UInt32`. **Example** Query: ```sql -SELECT regionToCity(toUInt32(5)); +SELECT regionToName(number::UInt32, 'en'), regionToCity(number::UInt32) AS id, regionToName(id, 'en') FROM numbers(13); ``` Result: ```response - ┌─regionToCity(toUInt32(5))─┐ -1. │ 5 │ - └───────────────────────────┘ +┌─regionToName(CAST(number, 'UInt32'), 'en')─┬─id─┬─regionToName(regionToCity(CAST(number, 'UInt32')), 'en')─┐ +│ │ 0 │ │ +│ World │ 0 │ │ +│ USA │ 0 │ │ +│ Colorado │ 0 │ │ +│ Boulder County │ 0 │ │ +│ Boulder │ 5 │ Boulder │ +│ China │ 0 │ │ +│ Sichuan │ 0 │ │ +│ Chengdu │ 8 │ Chengdu │ +│ America │ 0 │ │ +│ North America │ 0 │ │ +│ Eurasia │ 0 │ │ +│ Asia │ 0 │ │ +└────────────────────────────────────────────┴────┴──────────────────────────────────────────────────────────┘ ``` ### regionToArea @@ -82,13 +99,19 @@ regionToArea(id [, geobase]) **Parameters** -- `id`: the region ID from the geobase. [UInt32](../data-types/int-uint.md/#uint8-uint16-uint32-uint64-uint128-uint256-int8-int16-int32-int64-int128-int256) -- `geobase` (optional): geobase (dictionary key). [Dictionary Key](../../sql-reference/dictionaries/index.md/#dictionary-key-and-fields-dictionary-key-and-fields) +- `id` — Region ID from the geobase. [UInt32](../../sql-reference/data-types/int-uint.md). +- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../../sql-reference/data-types/string.md). Optional. **Returned value** - Area -- 0, otherwise. +- 0, if there is none. + +Type: `UInt32`. + +**Example** + +Query: ``` sql SELECT DISTINCT regionToName(regionToArea(toUInt32(number), 'ua')) @@ -96,6 +119,8 @@ FROM system.numbers LIMIT 15 ``` +Result: + ``` text ┌─regionToName(regionToArea(toUInt32(number), \'ua\'))─┐ │ │ @@ -134,7 +159,9 @@ regionToDistrict(id [, geobase]) **Returned value** - Region ID for the appropriate city, if it exists. -- 0, otherwise. +- 0, if there is none. + +Type: `UInt32`. **Example** @@ -179,10 +206,49 @@ Example: `regionToCountry(toUInt32(213)) = 225` converts Moscow (213) to Russia regionToCountry(id [, geobase]) ``` +**Parameters** + +- `id` — Region ID from the geobase. [UInt32](../../sql-reference/data-types/int-uint.md). +- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../../sql-reference/data-types/string.md). Optional. + +**Returned value** + +- Region ID for the appropriate country, if it exists. +- 0, if there is none. + +Type: `UInt32`. + +**Example** + +Query: + +``` sql +SELECT regionToName(number::UInt32, 'en'), regionToCountry(number::UInt32) AS id, regionToName(id, 'en') FROM numbers(13); +``` + +Result: + +``` text +┌─regionToName(CAST(number, 'UInt32'), 'en')─┬─id─┬─regionToName(regionToCountry(CAST(number, 'UInt32')), 'en')─┐ +│ │ 0 │ │ +│ World │ 0 │ │ +│ USA │ 2 │ USA │ +│ Colorado │ 2 │ USA │ +│ Boulder County │ 2 │ USA │ +│ Boulder │ 2 │ USA │ +│ China │ 6 │ China │ +│ Sichuan │ 6 │ China │ +│ Chengdu │ 6 │ China │ +│ America │ 0 │ │ +│ North America │ 0 │ │ +│ Eurasia │ 0 │ │ +│ Asia │ 0 │ │ +└────────────────────────────────────────────┴────┴─────────────────────────────────────────────────────────────┘ +``` + ### regionToContinent Converts a region to a continent. In every other way, this function is the same as ‘regionToCity’. -Example: `regionToContinent(toUInt32(213)) = 10001` converts Moscow (213) to Eurasia (10001). **Syntax** @@ -190,14 +256,50 @@ Example: `regionToContinent(toUInt32(213)) = 10001` converts Moscow (213) to Eur regionToContinent(id [, geobase]) ``` +**Parameters** + +- `id` — Region ID from the geobase. [UInt32](../../sql-reference/data-types/int-uint.md). +- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../../sql-reference/data-types/string.md). Optional. + +**Returned value** + +- Region ID for the appropriate continent, if it exists. +- 0, if there is none. + +Type: `UInt32`. + +**Example** + +Query: + +``` sql +SELECT regionToName(number::UInt32, 'en'), regionToContinent(number::UInt32) AS id, regionToName(id, 'en') FROM numbers(13); +``` + +Result: + +``` text +┌─regionToName(CAST(number, 'UInt32'), 'en')─┬─id─┬─regionToName(regionToContinent(CAST(number, 'UInt32')), 'en')─┐ +│ │ 0 │ │ +│ World │ 0 │ │ +│ USA │ 10 │ North America │ +│ Colorado │ 10 │ North America │ +│ Boulder County │ 10 │ North America │ +│ Boulder │ 10 │ North America │ +│ China │ 12 │ Asia │ +│ Sichuan │ 12 │ Asia │ +│ Chengdu │ 12 │ Asia │ +│ America │ 9 │ America │ +│ North America │ 10 │ North America │ +│ Eurasia │ 11 │ Eurasia │ +│ Asia │ 12 │ Asia │ +└────────────────────────────────────────────┴────┴───────────────────────────────────────────────────────────────┘ +``` + ### regionToTopContinent Finds the highest continent in the hierarchy for the region. -```sql -regionToTopContinent(id [, geobase]) -``` - **Syntax** ``` sql @@ -216,12 +318,37 @@ regionToTopContinent(id[, geobase]) Type: `UInt32`. +**Example** + +Query: + +``` sql +SELECT regionToName(number::UInt32, 'en'), regionToTopContinent(number::UInt32) AS id, regionToName(id, 'en') FROM numbers(13); +``` + +Result: + +``` text +┌─regionToName(CAST(number, 'UInt32'), 'en')─┬─id─┬─regionToName(regionToTopContinent(CAST(number, 'UInt32')), 'en')─┐ +│ │ 0 │ │ +│ World │ 0 │ │ +│ USA │ 9 │ America │ +│ Colorado │ 9 │ America │ +│ Boulder County │ 9 │ America │ +│ Boulder │ 9 │ America │ +│ China │ 11 │ Eurasia │ +│ Sichuan │ 11 │ Eurasia │ +│ Chengdu │ 11 │ Eurasia │ +│ America │ 9 │ America │ +│ North America │ 9 │ America │ +│ Eurasia │ 11 │ Eurasia │ +│ Asia │ 11 │ Eurasia │ +└────────────────────────────────────────────┴────┴──────────────────────────────────────────────────────────────────┘ +``` + ### regionToPopulation -Gets the population for a region. -The population can be recorded in files with the geobase. See the section “Dictionaries”. -If the population is not recorded for the region, it returns 0. -In the geobase, the population might be recorded for child regions, but not for parent regions. +Gets the population for a region. The population can be recorded in files with the geobase. See the section ["Dictionaries"](../../sql-reference/dictionaries/index.md). If the population is not recorded for the region, it returns 0. In the geobase, the population might be recorded for child regions, but not for parent regions. **Syntax** @@ -229,10 +356,49 @@ In the geobase, the population might be recorded for child regions, but not for regionToPopulation(id[, geobase]) ``` +**Parameters** + +- `id` — Region ID from the geobase. [UInt32](../../sql-reference/data-types/int-uint.md). +- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../../sql-reference/data-types/string.md). Optional. + +**Returned value** + +- Population for the region. +- 0, if there is none. + +Type: `UInt32`. + +**Example** + +Query: + +``` sql +SELECT regionToName(number::UInt32, 'en'), regionToPopulation(number::UInt32) AS id, regionToName(id, 'en') FROM numbers(13); +``` + +Result: + +``` text +┌─regionToName(CAST(number, 'UInt32'), 'en')─┬─population─┐ +│ │ 0 │ +│ World │ 4294967295 │ +│ USA │ 330000000 │ +│ Colorado │ 5700000 │ +│ Boulder County │ 330000 │ +│ Boulder │ 100000 │ +│ China │ 1500000000 │ +│ Sichuan │ 83000000 │ +│ Chengdu │ 20000000 │ +│ America │ 1000000000 │ +│ North America │ 600000000 │ +│ Eurasia │ 4294967295 │ +│ Asia │ 4294967295 │ +└────────────────────────────────────────────┴────────────┘ +``` + ### regionIn Checks whether a ‘lhs’ region belongs to a ‘rhs’ region. Returns a UInt8 number equal to 1 if it belongs, or 0 if it does not belong. -The relationship is reflexive – any region also belongs to itself. **Syntax** @@ -240,6 +406,46 @@ The relationship is reflexive – any region also belongs to itself. regionIn(lhs, rhs\[, geobase\]) ``` +**Parameters** + +- `lhs` — Lhs region ID from the geobase. [UInt32](../../sql-reference/data-types/int-uint.md). +- `rhs` — Rhs region ID from the geobase. [UInt32](../../sql-reference/data-types/int-uint.md). +- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../../sql-reference/data-types/string.md). Optional. + +**Returned value** + +- 1, if it belongs. +- 0, if it doesn't belong. + +Type: `UInt8`. + +**Implementation details** + +The relationship is reflexive – any region also belongs to itself. + +**Example** + +Query: + +``` sql +SELECT regionToName(n1.number::UInt32, 'en') || (regionIn(n1.number::UInt32, n2.number::UInt32) ? ' is in ' : ' is not in ') || regionToName(n2.number::UInt32, 'en') FROM numbers(1,2) AS n1 CROSS JOIN numbers(1,5) AS n2; +``` + +Result: + +``` text +World is in World +World is not in USA +World is not in Colorado +World is not in Boulder County +World is not in Boulder +USA is in World +USA is in USA +USA is not in Colorado +USA is not in Boulder County +USA is not in Boulder +``` + ### regionHierarchy Accepts a UInt32 number – the region ID from the geobase. Returns an array of region IDs consisting of the passed region and all parents along the chain. @@ -251,14 +457,74 @@ Example: `regionHierarchy(toUInt32(213)) = [213,1,3,225,10001,10000]`. regionHierarchy(id\[, geobase\]) ``` +**Parameters** + +- `id` — Region ID from the geobase. [UInt32](../../sql-reference/data-types/int-uint.md). +- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../../sql-reference/data-types/string.md). Optional. + +**Returned value** + +- Array of region IDs consisting of the passed region and all parents along the chain. + +Type: `Array(UInt32)` + + +**Example** + +Query: + +``` sql +SELECT regionHierarchy(number::UInt32) AS arr, arrayMap(id -> regionToName(id, 'en'), arr) FROM numbers(5); +``` + +Result: + +``` text +┌─arr────────────┬─arrayMap(lambda(tuple(id), regionToName(id, 'en')), regionHierarchy(CAST(number, 'UInt32')))─┐ +│ [] │ [] │ +│ [1] │ ['World'] │ +│ [2,10,9,1] │ ['USA','North America','America','World'] │ +│ [3,2,10,9,1] │ ['Colorado','USA','North America','America','World'] │ +│ [4,3,2,10,9,1] │ ['Boulder County','Colorado','USA','North America','America','World'] │ +└────────────────┴──────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + ### regionToName -Accepts a UInt32 number – the region ID from the geobase. A string with the name of the language can be passed as a second argument. Supported languages are: ru, en, ua, uk, by, kz, tr. If the second argument is omitted, the language ‘ru’ is used. If the language is not supported, an exception is thrown. Returns a string – the name of the region in the corresponding language. If the region with the specified ID does not exist, an empty string is returned. - -`ua` and `uk` both mean Ukrainian. +Accepts a region ID and geobase and returns a string of the name of the region in the corresponding language. If the region with the specified ID does not exist, an empty string is returned. **Syntax** ``` sql regionToName(id\[, lang\]) ``` +**Parameters** + +- `id` — Region ID from the geobase. [UInt32](../../sql-reference/data-types/int-uint.md). +- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../../sql-reference/data-types/string.md). Optional. + +**Returned value** + +- Name of the region in the corresponding language specified by `geobase`. + +Type: String. + +**Example** + +Query: + +``` sql +SELECT regionToName(number::UInt32,'en') FROM numbers(0,5); +``` + +Result: + +``` text +┌─regionToName(CAST(number, 'UInt32'), 'en')─┐ +│ │ +│ World │ +│ USA │ +│ Colorado │ +│ Boulder County │ +└────────────────────────────────────────────┘ +``` \ No newline at end of file From 38ef6ac291f1efb01ee14997e7da9fc2dee6b8b0 Mon Sep 17 00:00:00 2001 From: Blargian Date: Mon, 8 Apr 2024 17:12:13 +0200 Subject: [PATCH 0133/1018] Add missing colon --- docs/en/sql-reference/functions/ym-dict-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/ym-dict-functions.md b/docs/en/sql-reference/functions/ym-dict-functions.md index f9d53ef1cb5..8722c534d77 100644 --- a/docs/en/sql-reference/functions/ym-dict-functions.md +++ b/docs/en/sql-reference/functions/ym-dict-functions.md @@ -9,7 +9,7 @@ sidebar_label: Embedded Dictionaries :::note In order for the functions below to work, the server config must specify the paths and addresses for getting all the embedded dictionaries. The dictionaries are loaded at the first call of any of these functions. If the reference lists can’t be loaded, an exception is thrown. -As such, the examples shown in this section cannot be run in [ClickHouse Fiddle](https://fiddle.clickhouse.com/) +As such, the examples shown in this section cannot be run in [ClickHouse Fiddle](https://fiddle.clickhouse.com/). ::: For information about creating reference lists, see the section [“Dictionaries”](../../sql-reference/dictionaries/index.md). From c895fd59e6dccc64d89b1059e0c6fe458f0b68d5 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 8 Apr 2024 18:53:06 +0200 Subject: [PATCH 0134/1018] Update src/Parsers/obfuscateQueries.cpp Co-authored-by: Nikita Taranov --- src/Parsers/obfuscateQueries.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/obfuscateQueries.cpp b/src/Parsers/obfuscateQueries.cpp index 49af925212d..4ba18f6265c 100644 --- a/src/Parsers/obfuscateQueries.cpp +++ b/src/Parsers/obfuscateQueries.cpp @@ -1,4 +1,4 @@ -#include "Parsers/CommonParsers.h" +#include #include #include From 93408fda6bfa99faeef705dd09db6db0b32fb5f5 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 8 Apr 2024 16:59:54 +0000 Subject: [PATCH 0135/1018] Comments --- src/Parsers/obfuscateQueries.cpp | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/src/Parsers/obfuscateQueries.cpp b/src/Parsers/obfuscateQueries.cpp index 4ba18f6265c..fc1dd7289ff 100644 --- a/src/Parsers/obfuscateQueries.cpp +++ b/src/Parsers/obfuscateQueries.cpp @@ -29,11 +29,9 @@ namespace const std::unordered_set & getObfuscateKeywords() { - static std::unordered_set instance; - - auto initialize = [&]() mutable + auto initialize = []() { - instance = { + std::unordered_set instance = { "!=", "", "%", @@ -64,12 +62,10 @@ const std::unordered_set & getObfuscateKeywords() auto & global_keywords = getAllKeyWords(); std::copy(global_keywords.begin(), global_keywords.end(), std::inserter(instance, instance.begin())); - return true; + return instance; }; - static bool initialized = initialize(); - (void) initialized; - + static std::unordered_set instance = initialize(); return instance; }; From f1b565048ee5056dfb3211323bed0fff2bfcf559 Mon Sep 17 00:00:00 2001 From: Blargian Date: Mon, 8 Apr 2024 19:09:26 +0200 Subject: [PATCH 0136/1018] Formatting fixes --- .../functions/ym-dict-functions.md | 176 ++++++++---------- 1 file changed, 77 insertions(+), 99 deletions(-) diff --git a/docs/en/sql-reference/functions/ym-dict-functions.md b/docs/en/sql-reference/functions/ym-dict-functions.md index 8722c534d77..dba6c7f6460 100644 --- a/docs/en/sql-reference/functions/ym-dict-functions.md +++ b/docs/en/sql-reference/functions/ym-dict-functions.md @@ -9,23 +9,23 @@ sidebar_label: Embedded Dictionaries :::note In order for the functions below to work, the server config must specify the paths and addresses for getting all the embedded dictionaries. The dictionaries are loaded at the first call of any of these functions. If the reference lists can’t be loaded, an exception is thrown. -As such, the examples shown in this section cannot be run in [ClickHouse Fiddle](https://fiddle.clickhouse.com/). +As such, the examples shown in this section will throw an exception in [ClickHouse Fiddle](https://fiddle.clickhouse.com/) and in quick release and production deployments by default, unless first configured. ::: -For information about creating reference lists, see the section [“Dictionaries”](../../sql-reference/dictionaries/index.md). +For information about creating reference lists, see the section [“Dictionaries”](../../sql-reference/dictionaries/index.md#embedded-dictionaries). ## Multiple Geobases ClickHouse supports working with multiple alternative geobases (regional hierarchies) simultaneously, in order to support various perspectives on which countries certain regions belong to. -The ‘clickhouse-server’ config specifies the file with the regional hierarchy::`/opt/geo/regions_hierarchy.txt` +The ‘clickhouse-server’ config specifies the file with the regional hierarchy: -Besides this file, it also searches for files nearby that have the _ symbol and any suffix appended to the name (before the file extension). -For example, it will also find the file `/opt/geo/regions_hierarchy_ua.txt`, if present. +```/opt/geo/regions_hierarchy.txt``` -`ua` is called the dictionary key. For a dictionary without a suffix, the key is an empty string. +Besides this file, it also searches for files nearby that have the `_` symbol and any suffix appended to the name (before the file extension). +For example, it will also find the file `/opt/geo/regions_hierarchy_ua.txt`, if present. Here `ua` is called the dictionary key. For a dictionary without a suffix, the key is an empty string. -All the dictionaries are re-loaded in runtime (once every certain number of seconds, as defined in the [`builtin_dictionaries_reload_interval`](../../operations/server-configuration-parameters/settings#builtin-dictionaries-reload-interval) config parameter, or once an hour by default). However, the list of available dictionaries is defined one time, when the server starts. +All the dictionaries are re-loaded during runtime (once every certain number of seconds, as defined in the [`builtin_dictionaries_reload_interval`](../../operations/server-configuration-parameters/settings#builtin-dictionaries-reload-interval) config parameter, or once an hour by default). However, the list of available dictionaries is defined once, when the server starts. All functions for working with regions have an optional argument at the end – the dictionary key. It is referred to as the geobase. @@ -37,6 +37,45 @@ regionToCountry(RegionID, '') – Uses the default dictionary: /opt/geo/regions_ regionToCountry(RegionID, 'ua') – Uses the dictionary for the 'ua' key: /opt/geo/regions_hierarchy_ua.txt ``` +### regionToName + +Accepts a region ID and geobase and returns a string of the name of the region in the corresponding language. If the region with the specified ID does not exist, an empty string is returned. + +**Syntax** + +``` sql +regionToName(id\[, lang\]) +``` +**Parameters** + +- `id` — Region ID from the geobase. [UInt32](../data-types/int-uint). +- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../data-types/string). Optional. + +**Returned value** + +- Name of the region in the corresponding language specified by `geobase`. [String](../data-types/string). +- Otherwise, an empty string. + +**Example** + +Query: + +``` sql +SELECT regionToName(number::UInt32,'en') FROM numbers(0,5); +``` + +Result: + +``` text +┌─regionToName(CAST(number, 'UInt32'), 'en')─┐ +│ │ +│ World │ +│ USA │ +│ Colorado │ +│ Boulder County │ +└────────────────────────────────────────────┘ +``` + ### regionToCity Accepts a region ID from the geobase. If this region is a city or part of a city, it returns the region ID for the appropriate city. Otherwise, returns 0. @@ -49,16 +88,14 @@ regionToCity(id [, geobase]) **Parameters** -- `id` — Region ID from the geobase. [UInt32](../../sql-reference/data-types/int-uint.md). -- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../../sql-reference/data-types/string.md). Optional. +- `id` — Region ID from the geobase. [UInt32](../data-types/int-uint). +- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../data-types/string). Optional. **Returned value** -- Region ID for the appropriate city, if it exists. +- Region ID for the appropriate city, if it exists. [UInt32](../data-types/int-uint). - 0, if there is none. -Type: `UInt32`. - **Example** Query: @@ -99,16 +136,14 @@ regionToArea(id [, geobase]) **Parameters** -- `id` — Region ID from the geobase. [UInt32](../../sql-reference/data-types/int-uint.md). -- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../../sql-reference/data-types/string.md). Optional. +- `id` — Region ID from the geobase. [UInt32](../data-types/int-uint). +- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../data-types/string). Optional. **Returned value** -- Area +- Region ID for the appropriate area, if it exists. [UInt32](../data-types/int-uint). - 0, if there is none. -Type: `UInt32`. - **Example** Query: @@ -153,16 +188,14 @@ regionToDistrict(id [, geobase]) **Parameters** -- `id`: the region ID from the geobase. [UInt32](../data-types/int-uint.md/#uint8-uint16-uint32-uint64-uint128-uint256-int8-int16-int32-int64-int128-int256) -- `geobase` (optional): geobase (dictionary key). [Dictionary Key](../../sql-reference/dictionaries/index.md/#dictionary-key-and-fields-dictionary-key-and-fields) +- `id` — Region ID from the geobase. [UInt32](../data-types/int-uint). +- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../data-types/string). Optional. **Returned value** -- Region ID for the appropriate city, if it exists. +- Region ID for the appropriate city, if it exists. [UInt32](../data-types/int-uint). - 0, if there is none. -Type: `UInt32`. - **Example** Query: @@ -197,8 +230,7 @@ Result: ### regionToCountry -Converts a region to a country. In every other way, this function is the same as ‘regionToCity’. -Example: `regionToCountry(toUInt32(213)) = 225` converts Moscow (213) to Russia (225). +Converts a region to a country (type 3 in the geobase). In every other way, this function is the same as ‘regionToCity’. **Syntax** @@ -208,16 +240,14 @@ regionToCountry(id [, geobase]) **Parameters** -- `id` — Region ID from the geobase. [UInt32](../../sql-reference/data-types/int-uint.md). -- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../../sql-reference/data-types/string.md). Optional. +- `id` — Region ID from the geobase. [UInt32](../data-types/int-uint). +- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../data-types/string). Optional. **Returned value** -- Region ID for the appropriate country, if it exists. +- Region ID for the appropriate country, if it exists. [UInt32](../data-types/int-uint). - 0, if there is none. -Type: `UInt32`. - **Example** Query: @@ -248,7 +278,7 @@ Result: ### regionToContinent -Converts a region to a continent. In every other way, this function is the same as ‘regionToCity’. +Converts a region to a continent (type 1 in the geobase). In every other way, this function is the same as ‘regionToCity’. **Syntax** @@ -258,16 +288,14 @@ regionToContinent(id [, geobase]) **Parameters** -- `id` — Region ID from the geobase. [UInt32](../../sql-reference/data-types/int-uint.md). -- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../../sql-reference/data-types/string.md). Optional. +- `id` — Region ID from the geobase. [UInt32](../data-types/int-uint). +- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../data-types/string). Optional. **Returned value** -- Region ID for the appropriate continent, if it exists. +- Region ID for the appropriate continent, if it exists. [UInt32](../data-types/int-uint). - 0, if there is none. -Type: `UInt32`. - **Example** Query: @@ -308,16 +336,14 @@ regionToTopContinent(id[, geobase]) **Parameters** -- `id` — Region ID from the geobase. [UInt32](../../sql-reference/data-types/int-uint.md). -- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../../sql-reference/data-types/string.md). Optional. +- `id` — Region ID from the geobase. [UInt32](../data-types/int-uint). +- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../data-types/string). Optional. **Returned value** -- Identifier of the top level continent (the latter when you climb the hierarchy of regions). +- Identifier of the top level continent (the latter when you climb the hierarchy of regions).[UInt32](../data-types/int-uint). - 0, if there is none. -Type: `UInt32`. - **Example** Query: @@ -358,16 +384,14 @@ regionToPopulation(id[, geobase]) **Parameters** -- `id` — Region ID from the geobase. [UInt32](../../sql-reference/data-types/int-uint.md). -- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../../sql-reference/data-types/string.md). Optional. +- `id` — Region ID from the geobase. [UInt32](../data-types/int-uint). +- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../data-types/string). Optional. **Returned value** -- Population for the region. +- Population for the region. [UInt32](../data-types/int-uint). - 0, if there is none. -Type: `UInt32`. - **Example** Query: @@ -398,7 +422,7 @@ Result: ### regionIn -Checks whether a ‘lhs’ region belongs to a ‘rhs’ region. Returns a UInt8 number equal to 1 if it belongs, or 0 if it does not belong. +Checks whether a `lhs` region belongs to a `rhs` region. Returns a UInt8 number equal to 1 if it belongs, or 0 if it does not belong. **Syntax** @@ -408,17 +432,15 @@ regionIn(lhs, rhs\[, geobase\]) **Parameters** -- `lhs` — Lhs region ID from the geobase. [UInt32](../../sql-reference/data-types/int-uint.md). -- `rhs` — Rhs region ID from the geobase. [UInt32](../../sql-reference/data-types/int-uint.md). -- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../../sql-reference/data-types/string.md). Optional. +- `lhs` — Lhs region ID from the geobase. [UInt32](../../sql-reference/data-types/int-uint). +- `rhs` — Rhs region ID from the geobase. [UInt32](../../sql-reference/data-types/int-uint). +- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../data-types/string). Optional. **Returned value** -- 1, if it belongs. +- 1, if it belongs. [UInt8](../../sql-reference/data-types/int-uint). - 0, if it doesn't belong. -Type: `UInt8`. - **Implementation details** The relationship is reflexive – any region also belongs to itself. @@ -449,7 +471,6 @@ USA is not in Boulder ### regionHierarchy Accepts a UInt32 number – the region ID from the geobase. Returns an array of region IDs consisting of the passed region and all parents along the chain. -Example: `regionHierarchy(toUInt32(213)) = [213,1,3,225,10001,10000]`. **Syntax** @@ -459,15 +480,12 @@ regionHierarchy(id\[, geobase\]) **Parameters** -- `id` — Region ID from the geobase. [UInt32](../../sql-reference/data-types/int-uint.md). -- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../../sql-reference/data-types/string.md). Optional. +- `id` — Region ID from the geobase. [UInt32](../data-types/int-uint). +- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../data-types/string). Optional. **Returned value** -- Array of region IDs consisting of the passed region and all parents along the chain. - -Type: `Array(UInt32)` - +- Array of region IDs consisting of the passed region and all parents along the chain. [Array](../data-types/array)([UInt32](../data-types/int-uint.md)). **Example** @@ -487,44 +505,4 @@ Result: │ [3,2,10,9,1] │ ['Colorado','USA','North America','America','World'] │ │ [4,3,2,10,9,1] │ ['Boulder County','Colorado','USA','North America','America','World'] │ └────────────────┴──────────────────────────────────────────────────────────────────────────────────────────────┘ -``` - -### regionToName - -Accepts a region ID and geobase and returns a string of the name of the region in the corresponding language. If the region with the specified ID does not exist, an empty string is returned. - -**Syntax** - -``` sql -regionToName(id\[, lang\]) -``` -**Parameters** - -- `id` — Region ID from the geobase. [UInt32](../../sql-reference/data-types/int-uint.md). -- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../../sql-reference/data-types/string.md). Optional. - -**Returned value** - -- Name of the region in the corresponding language specified by `geobase`. - -Type: String. - -**Example** - -Query: - -``` sql -SELECT regionToName(number::UInt32,'en') FROM numbers(0,5); -``` - -Result: - -``` text -┌─regionToName(CAST(number, 'UInt32'), 'en')─┐ -│ │ -│ World │ -│ USA │ -│ Colorado │ -│ Boulder County │ -└────────────────────────────────────────────┘ ``` \ No newline at end of file From 8960a7f08dd5f068d9de7a5c526780234fbce180 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 8 Apr 2024 18:05:05 +0000 Subject: [PATCH 0137/1018] Add back-pressure for `MOVE PARTITION TO TABLE` as for `INSERT`s --- src/Storages/StorageMergeTree.cpp | 3 +++ src/Storages/StorageReplicatedMergeTree.cpp | 3 +++ 2 files changed, 6 insertions(+) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index c9f451b6bb1..ce5db1963fd 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2178,6 +2178,9 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const this->getStoragePolicy()->getName(), dest_table_storage->getStorageID().getNameForLogs(), dest_table_storage->getStoragePolicy()->getName()); + // Use the same back-pressure (delay/throw) logic as for INSERTs to be consistent and avoid possibility of exceeding part limits using MOVE PARTITION queries + dest_table_storage->delayInsertOrThrowIfNeeded(nullptr, query_context, true); + auto dest_metadata_snapshot = dest_table->getInMemoryMetadataPtr(); auto metadata_snapshot = getInMemoryMetadataPtr(); Stopwatch watch; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ffc7ade5253..21fbef395fc 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8224,6 +8224,9 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta this->getStoragePolicy()->getName(), getStorageID().getNameForLogs(), dest_table_storage->getStoragePolicy()->getName()); + // Use the same back-pressure (delay/throw) logic as for INSERTs to be consistent and avoid possibility of exceeding part limits using MOVE PARTITION queries + dest_table_storage->delayInsertOrThrowIfNeeded(nullptr, query_context, true); + auto dest_metadata_snapshot = dest_table->getInMemoryMetadataPtr(); auto metadata_snapshot = getInMemoryMetadataPtr(); From ba172e7693bb02bc9f7a6fabcd8e681bbf84551a Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Mon, 8 Apr 2024 20:16:09 +0200 Subject: [PATCH 0138/1018] Fix broken links --- docs/en/sql-reference/functions/ym-dict-functions.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/ym-dict-functions.md b/docs/en/sql-reference/functions/ym-dict-functions.md index dba6c7f6460..d69f476f2e3 100644 --- a/docs/en/sql-reference/functions/ym-dict-functions.md +++ b/docs/en/sql-reference/functions/ym-dict-functions.md @@ -12,7 +12,7 @@ In order for the functions below to work, the server config must specify the pat As such, the examples shown in this section will throw an exception in [ClickHouse Fiddle](https://fiddle.clickhouse.com/) and in quick release and production deployments by default, unless first configured. ::: -For information about creating reference lists, see the section [“Dictionaries”](../../sql-reference/dictionaries/index.md#embedded-dictionaries). +For information about creating reference lists, see the section [“Dictionaries”](../../sql-reference/dictionaries/index#embedded-dictionaries). ## Multiple Geobases @@ -374,7 +374,7 @@ Result: ### regionToPopulation -Gets the population for a region. The population can be recorded in files with the geobase. See the section ["Dictionaries"](../../sql-reference/dictionaries/index.md). If the population is not recorded for the region, it returns 0. In the geobase, the population might be recorded for child regions, but not for parent regions. +Gets the population for a region. The population can be recorded in files with the geobase. See the section ["Dictionaries"](../../sql-reference/dictionaries/index#embedded-dictionaries). If the population is not recorded for the region, it returns 0. In the geobase, the population might be recorded for child regions, but not for parent regions. **Syntax** @@ -485,7 +485,7 @@ regionHierarchy(id\[, geobase\]) **Returned value** -- Array of region IDs consisting of the passed region and all parents along the chain. [Array](../data-types/array)([UInt32](../data-types/int-uint.md)). +- Array of region IDs consisting of the passed region and all parents along the chain. [Array](../data-types/array)([UInt32](../data-types/int-uint)). **Example** @@ -505,4 +505,4 @@ Result: │ [3,2,10,9,1] │ ['Colorado','USA','North America','America','World'] │ │ [4,3,2,10,9,1] │ ['Boulder County','Colorado','USA','North America','America','World'] │ └────────────────┴──────────────────────────────────────────────────────────────────────────────────────────────┘ -``` \ No newline at end of file +``` From 88548eb19eb07ea29c8d603d77a43039844a351c Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 8 Apr 2024 18:59:04 +0000 Subject: [PATCH 0139/1018] Fix exception message during writing to partitioned s3 path woth globs --- src/Storages/StorageS3.cpp | 22 +++++++++++++------ src/Storages/StorageS3.h | 6 +++-- ...rite_to_globbed_partitioned_path.reference | 0 ...7_s3_write_to_globbed_partitioned_path.sql | 2 ++ 4 files changed, 21 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/03037_s3_write_to_globbed_partitioned_path.reference create mode 100644 tests/queries/0_stateless/03037_s3_write_to_globbed_partitioned_path.sql diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 6cda0fca60b..c0802d02120 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -207,7 +207,7 @@ public: , list_objects_scheduler(threadPoolCallbackRunner(list_objects_pool, "ListObjects")) , file_progress_callback(file_progress_callback_) { - if (globbed_uri.bucket.find_first_of("*?{") != std::string::npos) + if (globbed_uri.bucket.find_first_of("*?{") != globbed_uri.bucket.npos) throw Exception(ErrorCodes::UNEXPECTED_EXPRESSION, "Expression can not have wildcards inside bucket name"); const String key_prefix = globbed_uri.key.substr(0, globbed_uri.key.find_first_of("*?{")); @@ -1194,7 +1194,7 @@ void ReadFromStorageS3Step::createIterator(const ActionsDAG::Node * predicate) void ReadFromStorageS3Step::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - if (storage.partition_by && query_configuration.withWildcard()) + if (storage.partition_by && query_configuration.withPartitionWildcard()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Reading from a partitioned S3 storage is not implemented yet"); createIterator(nullptr); @@ -1249,12 +1249,16 @@ SinkToStoragePtr StorageS3::write(const ASTPtr & query, const StorageMetadataPtr { auto query_configuration = updateConfigurationAndGetCopy(local_context); + if (query_configuration.withGlobsIgnorePartitionWildcard()) + throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, + "S3 key '{}' contains globs, so the table is in readonly mode", query_configuration.url.key); + auto sample_block = metadata_snapshot->getSampleBlock(); auto chosen_compression_method = chooseCompressionMethod(query_configuration.keys.back(), query_configuration.compression_method); auto insert_query = std::dynamic_pointer_cast(query); auto partition_by_ast = insert_query ? (insert_query->partition_by ? insert_query->partition_by : partition_by) : nullptr; - bool is_partitioned_implementation = partition_by_ast && query_configuration.withWildcard(); + bool is_partitioned_implementation = partition_by_ast && query_configuration.withPartitionWildcard(); if (is_partitioned_implementation) { @@ -1271,10 +1275,6 @@ SinkToStoragePtr StorageS3::write(const ASTPtr & query, const StorageMetadataPtr } else { - if (query_configuration.withGlobs()) - throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, - "S3 key '{}' contains globs, so the table is in readonly mode", query_configuration.url.key); - bool truncate_in_insert = local_context->getSettingsRef().s3_truncate_on_insert; if (!truncate_in_insert && S3::objectExists(*query_configuration.client, query_configuration.url.bucket, query_configuration.keys.back(), query_configuration.url.version_id, query_configuration.request_settings)) @@ -1460,6 +1460,14 @@ void StorageS3::Configuration::connect(const ContextPtr & context) credentials.GetSessionToken()); } +bool StorageS3::Configuration::withGlobsIgnorePartitionWildcard() const +{ + if (!withPartitionWildcard()) + return withGlobs(); + + return PartitionedSink::replaceWildcards(getPath(), "").find_first_of("*?{") != std::string::npos; +} + void StorageS3::processNamedCollectionResult(StorageS3::Configuration & configuration, const NamedCollection & collection) { validateNamedCollection(collection, required_configuration_keys, optional_configuration_keys); diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 19cbfaa6f08..e657db7bd35 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -274,7 +274,7 @@ public: { Configuration() = default; - String getPath() const { return url.key; } + const String & getPath() const { return url.key; } bool update(const ContextPtr & context); @@ -282,13 +282,15 @@ public: bool withGlobs() const { return url.key.find_first_of("*?{") != std::string::npos; } - bool withWildcard() const + bool withPartitionWildcard() const { static const String PARTITION_ID_WILDCARD = "{_partition_id}"; return url.bucket.find(PARTITION_ID_WILDCARD) != String::npos || keys.back().find(PARTITION_ID_WILDCARD) != String::npos; } + bool withGlobsIgnorePartitionWildcard() const; + S3::URI url; S3::AuthSettings auth_settings; S3Settings::RequestSettings request_settings; diff --git a/tests/queries/0_stateless/03037_s3_write_to_globbed_partitioned_path.reference b/tests/queries/0_stateless/03037_s3_write_to_globbed_partitioned_path.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03037_s3_write_to_globbed_partitioned_path.sql b/tests/queries/0_stateless/03037_s3_write_to_globbed_partitioned_path.sql new file mode 100644 index 00000000000..400bc790f0b --- /dev/null +++ b/tests/queries/0_stateless/03037_s3_write_to_globbed_partitioned_path.sql @@ -0,0 +1,2 @@ +insert into function s3('http://localhost:11111/test/data_*_{_partition_id}.csv') partition by number % 3 select * from numbers(10); -- {serverError DATABASE_ACCESS_DENIED} + From cc38765cda0e149d0ba978a76b1f85a44d24795c Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 8 Apr 2024 18:59:19 +0000 Subject: [PATCH 0140/1018] fix build --- 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 ce5db1963fd..1398fb2c375 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2179,7 +2179,7 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const dest_table_storage->getStoragePolicy()->getName()); // Use the same back-pressure (delay/throw) logic as for INSERTs to be consistent and avoid possibility of exceeding part limits using MOVE PARTITION queries - dest_table_storage->delayInsertOrThrowIfNeeded(nullptr, query_context, true); + dest_table_storage->delayInsertOrThrowIfNeeded(nullptr, local_context, true); auto dest_metadata_snapshot = dest_table->getInMemoryMetadataPtr(); auto metadata_snapshot = getInMemoryMetadataPtr(); From 21ab0e4ea27b389ab697cb5f045a9600bd0fa7c2 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 8 Apr 2024 19:01:00 +0000 Subject: [PATCH 0141/1018] Remove bad change --- src/Storages/StorageS3.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index c0802d02120..8de139ce366 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -207,7 +207,7 @@ public: , list_objects_scheduler(threadPoolCallbackRunner(list_objects_pool, "ListObjects")) , file_progress_callback(file_progress_callback_) { - if (globbed_uri.bucket.find_first_of("*?{") != globbed_uri.bucket.npos) + if (globbed_uri.bucket.find_first_of("*?{") != std::string::npos) throw Exception(ErrorCodes::UNEXPECTED_EXPRESSION, "Expression can not have wildcards inside bucket name"); const String key_prefix = globbed_uri.key.substr(0, globbed_uri.key.find_first_of("*?{")); From d2c3a015c1cf81f2e5cf9184da1879c8a87d215d Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Mon, 8 Apr 2024 21:03:58 +0200 Subject: [PATCH 0142/1018] Update ym-dict-functions.md Fix broken links --- docs/en/sql-reference/functions/ym-dict-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/ym-dict-functions.md b/docs/en/sql-reference/functions/ym-dict-functions.md index d69f476f2e3..dfcabd82586 100644 --- a/docs/en/sql-reference/functions/ym-dict-functions.md +++ b/docs/en/sql-reference/functions/ym-dict-functions.md @@ -12,7 +12,7 @@ In order for the functions below to work, the server config must specify the pat As such, the examples shown in this section will throw an exception in [ClickHouse Fiddle](https://fiddle.clickhouse.com/) and in quick release and production deployments by default, unless first configured. ::: -For information about creating reference lists, see the section [“Dictionaries”](../../sql-reference/dictionaries/index#embedded-dictionaries). +For information about creating reference lists, see the section [“Dictionaries”](../dictionaries/index#embedded-dictionaries). ## Multiple Geobases @@ -374,7 +374,7 @@ Result: ### regionToPopulation -Gets the population for a region. The population can be recorded in files with the geobase. See the section ["Dictionaries"](../../sql-reference/dictionaries/index#embedded-dictionaries). If the population is not recorded for the region, it returns 0. In the geobase, the population might be recorded for child regions, but not for parent regions. +Gets the population for a region. The population can be recorded in files with the geobase. See the section ["Dictionaries"](../dictionaries/index#embedded-dictionaries). If the population is not recorded for the region, it returns 0. In the geobase, the population might be recorded for child regions, but not for parent regions. **Syntax** From 78dd23fd83e541c73b2fec0bc6bdf19d8d5b9297 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 8 Apr 2024 21:23:51 +0200 Subject: [PATCH 0143/1018] Don't run new test in fasttest --- .../0_stateless/03037_s3_write_to_globbed_partitioned_path.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03037_s3_write_to_globbed_partitioned_path.sql b/tests/queries/0_stateless/03037_s3_write_to_globbed_partitioned_path.sql index 400bc790f0b..1de89a593b0 100644 --- a/tests/queries/0_stateless/03037_s3_write_to_globbed_partitioned_path.sql +++ b/tests/queries/0_stateless/03037_s3_write_to_globbed_partitioned_path.sql @@ -1,2 +1,4 @@ +-- Tags: no-fasttest + insert into function s3('http://localhost:11111/test/data_*_{_partition_id}.csv') partition by number % 3 select * from numbers(10); -- {serverError DATABASE_ACCESS_DENIED} From 80eae8765e62e5443d23cddf1d0c267663007b64 Mon Sep 17 00:00:00 2001 From: Blargian Date: Mon, 8 Apr 2024 21:51:18 +0200 Subject: [PATCH 0144/1018] Actually fix broken links this time --- docs/en/sql-reference/functions/ym-dict-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/ym-dict-functions.md b/docs/en/sql-reference/functions/ym-dict-functions.md index dfcabd82586..043686889c4 100644 --- a/docs/en/sql-reference/functions/ym-dict-functions.md +++ b/docs/en/sql-reference/functions/ym-dict-functions.md @@ -12,7 +12,7 @@ In order for the functions below to work, the server config must specify the pat As such, the examples shown in this section will throw an exception in [ClickHouse Fiddle](https://fiddle.clickhouse.com/) and in quick release and production deployments by default, unless first configured. ::: -For information about creating reference lists, see the section [“Dictionaries”](../dictionaries/index#embedded-dictionaries). +For information about creating reference lists, see the section [“Dictionaries”](../dictionaries#embedded-dictionaries). ## Multiple Geobases @@ -374,7 +374,7 @@ Result: ### regionToPopulation -Gets the population for a region. The population can be recorded in files with the geobase. See the section ["Dictionaries"](../dictionaries/index#embedded-dictionaries). If the population is not recorded for the region, it returns 0. In the geobase, the population might be recorded for child regions, but not for parent regions. +Gets the population for a region. The population can be recorded in files with the geobase. See the section [“Dictionaries”](../dictionaries#embedded-dictionaries). If the population is not recorded for the region, it returns 0. In the geobase, the population might be recorded for child regions, but not for parent regions. **Syntax** From 8accf395fb9cb3f2b896151ab09a902c674b9809 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 8 Apr 2024 20:37:06 +0000 Subject: [PATCH 0145/1018] Fix for HDFS and Azure --- src/Storages/HDFS/StorageHDFS.cpp | 4 ++++ src/Storages/StorageAzureBlob.cpp | 19 +++++++++++++------ src/Storages/StorageAzureBlob.h | 4 +++- .../test_storage_azure_blob_storage/test.py | 14 ++++++++++++++ tests/integration/test_storage_hdfs/test.py | 10 ++++++++++ 5 files changed, 44 insertions(+), 7 deletions(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 93cf64f30a0..fe5d590770e 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -1065,6 +1065,10 @@ SinkToStoragePtr StorageHDFS::write(const ASTPtr & query, const StorageMetadataP if (is_partitioned_implementation) { + String path = current_uri.substr(current_uri.find('/', current_uri.find("//") + 2)); + if (PartitionedSink::replaceWildcards(path, "").find_first_of("*?{") != std::string::npos) + throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "URI '{}' contains globs, so the table is in readonly mode", uris.back()); + return std::make_shared( partition_by_ast, current_uri, diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 306a5eac8e5..1eb18021653 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -462,6 +462,13 @@ Poco::URI StorageAzureBlob::Configuration::getConnectionURL() const return Poco::URI(parsed_connection_string.BlobServiceUrl.GetAbsoluteUrl()); } +bool StorageAzureBlob::Configuration::withGlobsIgnorePartitionWildcard() const +{ + if (!withPartitionWildcard()) + return withGlobs(); + + return PartitionedSink::replaceWildcards(getPath(), "").find_first_of("*?{") != std::string::npos; +} StorageAzureBlob::StorageAzureBlob( const Configuration & configuration_, @@ -749,7 +756,7 @@ void StorageAzureBlob::read( size_t max_block_size, size_t num_streams) { - if (partition_by && configuration.withWildcard()) + if (partition_by && configuration.withPartitionWildcard()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Reading from a partitioned Azure storage is not implemented yet"); auto this_ptr = std::static_pointer_cast(shared_from_this()); @@ -836,12 +843,16 @@ void ReadFromAzureBlob::initializePipeline(QueryPipelineBuilder & pipeline, cons SinkToStoragePtr StorageAzureBlob::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { + if (configuration.withGlobsIgnorePartitionWildcard()) + throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, + "AzureBlobStorage key '{}' contains globs, so the table is in readonly mode", configuration.blob_path); + auto sample_block = metadata_snapshot->getSampleBlock(); auto chosen_compression_method = chooseCompressionMethod(configuration.blobs_paths.back(), configuration.compression_method); auto insert_query = std::dynamic_pointer_cast(query); auto partition_by_ast = insert_query ? (insert_query->partition_by ? insert_query->partition_by : partition_by) : nullptr; - bool is_partitioned_implementation = partition_by_ast && configuration.withWildcard(); + bool is_partitioned_implementation = partition_by_ast && configuration.withPartitionWildcard(); if (is_partitioned_implementation) { @@ -857,10 +868,6 @@ SinkToStoragePtr StorageAzureBlob::write(const ASTPtr & query, const StorageMeta } else { - if (configuration.withGlobs()) - throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, - "AzureBlobStorage key '{}' contains globs, so the table is in readonly mode", configuration.blob_path); - bool truncate_in_insert = local_context->getSettingsRef().azure_truncate_on_insert; if (!truncate_in_insert && object_storage->exists(StoredObject(configuration.blob_path))) diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h index 3f1ba33f636..928a8204bb2 100644 --- a/src/Storages/StorageAzureBlob.h +++ b/src/Storages/StorageAzureBlob.h @@ -37,12 +37,14 @@ public: bool withGlobs() const { return blob_path.find_first_of("*?{") != std::string::npos; } - bool withWildcard() const + bool withPartitionWildcard() const { static const String PARTITION_ID_WILDCARD = "{_partition_id}"; return blobs_paths.back().find(PARTITION_ID_WILDCARD) != String::npos; } + bool withGlobsIgnorePartitionWildcard() const; + Poco::URI getConnectionURL() const; std::string connection_url; diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 7d30265e4f8..1c87feee297 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -1323,6 +1323,20 @@ def test_format_detection(cluster): assert result == expected_result +def test_write_to_globbed_partitioned_path(cluster): + node = cluster.instances["node"] + storage_account_url = cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"] + account_name = "devstoreaccount1" + account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==" + error = azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_data_*_{{_partition_id}}', '{account_name}', '{account_key}', 'CSV', 'auto', 'x UInt64') partition by 42 select 42", + expect_error="true", + ) + + assert "DATABASE_ACCESS_DENIED" in error + + def test_parallel_read(cluster): node = cluster.instances["node"] connection_string = cluster.env_variables["AZURITE_CONNECTION_STRING"] diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 9dec1954406..5daf8618036 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -1116,6 +1116,16 @@ def test_format_detection(started_cluster): assert expected_result == result +def test_write_to_globbed_partitioned_path(started_cluster): + node = started_cluster.instances["node1"] + + error = node.query_and_get_error( + "insert into function hdfs('hdfs://hdfs1:9000/test_data_*_{_partition_id}.csv') partition by 42 select 42" + ) + + assert "DATABASE_ACCESS_DENIED" in error + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From 45b2619f7ad765bfd96eb96f11e9f632849b923f Mon Sep 17 00:00:00 2001 From: peter279k Date: Tue, 9 Apr 2024 13:29:34 +0800 Subject: [PATCH 0146/1018] Validating zip and fix query result in Recipes doc --- docs/en/getting-started/example-datasets/recipes.md | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/docs/en/getting-started/example-datasets/recipes.md b/docs/en/getting-started/example-datasets/recipes.md index e0a66022d37..a8808e376e0 100644 --- a/docs/en/getting-started/example-datasets/recipes.md +++ b/docs/en/getting-started/example-datasets/recipes.md @@ -10,7 +10,8 @@ The RecipeNLG dataset is available for download [here](https://recipenlg.cs.put. 1. Go to the download page [https://recipenlg.cs.put.poznan.pl/dataset](https://recipenlg.cs.put.poznan.pl/dataset). 1. Accept Terms and Conditions and download zip file. -1. Unpack the zip file with `unzip`. You will get the `full_dataset.csv` file. +1. Option: Using the `md5sum dataset.zip` to validate the zip file and it should be equal to `3a168dfd0912bb034225619b3586ce76`. +1. Unpack the zip file with `unzip dataset.zip`. You will get the `full_dataset.csv` file in the `dataset` directory. ## Create a Table @@ -72,7 +73,7 @@ Result: ``` text ┌─count()─┐ -│ 2231141 │ +│ 2231142 │ └─────────┘ ``` @@ -115,7 +116,7 @@ Result: │ egg │ 160507 │ │ baking powder │ 148277 │ │ lemon juice │ 146414 │ -│ Salt │ 122557 │ +│ Salt │ 122558 │ │ cinnamon │ 117927 │ │ sour cream │ 116682 │ │ cream cheese │ 114423 │ From c3a1ec30b1c2432700c72863103415a1b295b5fa Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 9 Apr 2024 11:02:10 +0200 Subject: [PATCH 0147/1018] Fix flakey tests failing --- src/Processors/Formats/Impl/FormInputFormat.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/FormInputFormat.cpp b/src/Processors/Formats/Impl/FormInputFormat.cpp index 771da34e6c3..b3b62e21a99 100644 --- a/src/Processors/Formats/Impl/FormInputFormat.cpp +++ b/src/Processors/Formats/Impl/FormInputFormat.cpp @@ -58,12 +58,14 @@ String readFieldName(ReadBuffer & buf) void FormInputFormat::readFormData(MutableColumns & columns) { size_t index = 0; + StringRef name_ref; while (true) { if (in->eof()) break; - StringRef name_ref = readFieldName(*in); + auto tmp = readFieldName(*in); + name_ref = StringRef(tmp); auto * it = name_map.find(name_ref); if (!it) From 19f00e7c42bd9ba0af63be4987a89aad3457b3c8 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 9 Apr 2024 12:33:18 +0200 Subject: [PATCH 0148/1018] Restart CI --- tests/performance/trivial_insert_select_from_files.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/performance/trivial_insert_select_from_files.xml b/tests/performance/trivial_insert_select_from_files.xml index f6ec6500c10..64bd3e54b95 100644 --- a/tests/performance/trivial_insert_select_from_files.xml +++ b/tests/performance/trivial_insert_select_from_files.xml @@ -19,7 +19,6 @@ CREATE TABLE IF NOT EXISTS table_src_{format} ENGINE = File({format}) AS test.hits CREATE TABLE IF NOT EXISTS table_dst_{format} AS test.hits - INSERT INTO table_src_{format} SELECT * FROM test.hits LIMIT 100000 INSERT INTO table_dst_{format} SELECT * FROM table_src_{format} From 1b20bb8e3472fb0d616f405c92199e3827ac6101 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 9 Apr 2024 14:05:09 +0000 Subject: [PATCH 0149/1018] do not wait under locks --- src/Storages/StorageMergeTree.cpp | 8 ++++---- src/Storages/StorageReplicatedMergeTree.cpp | 8 ++++---- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 1398fb2c375..3a6f9996bab 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2161,10 +2161,6 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, ContextPtr local_context) { - auto lock1 = lockForShare(local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout); - auto lock2 = dest_table->lockForShare(local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout); - auto merges_blocker = stopMergesAndWait(); - auto dest_table_storage = std::dynamic_pointer_cast(dest_table); if (!dest_table_storage) throw Exception(ErrorCodes::NOT_IMPLEMENTED, @@ -2181,6 +2177,10 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const // Use the same back-pressure (delay/throw) logic as for INSERTs to be consistent and avoid possibility of exceeding part limits using MOVE PARTITION queries dest_table_storage->delayInsertOrThrowIfNeeded(nullptr, local_context, true); + auto lock1 = lockForShare(local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout); + auto lock2 = dest_table->lockForShare(local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout); + auto merges_blocker = stopMergesAndWait(); + auto dest_metadata_snapshot = dest_table->getInMemoryMetadataPtr(); auto metadata_snapshot = getInMemoryMetadataPtr(); Stopwatch watch; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 21fbef395fc..fec667ac35c 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8207,10 +8207,6 @@ void StorageReplicatedMergeTree::replacePartitionFrom( void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, ContextPtr query_context) { - auto lock1 = lockForShare(query_context->getCurrentQueryId(), query_context->getSettingsRef().lock_acquire_timeout); - auto lock2 = dest_table->lockForShare(query_context->getCurrentQueryId(), query_context->getSettingsRef().lock_acquire_timeout); - auto storage_settings_ptr = getSettings(); - auto dest_table_storage = std::dynamic_pointer_cast(dest_table); if (!dest_table_storage) throw Exception(ErrorCodes::NOT_IMPLEMENTED, @@ -8227,6 +8223,10 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta // Use the same back-pressure (delay/throw) logic as for INSERTs to be consistent and avoid possibility of exceeding part limits using MOVE PARTITION queries dest_table_storage->delayInsertOrThrowIfNeeded(nullptr, query_context, true); + auto lock1 = lockForShare(query_context->getCurrentQueryId(), query_context->getSettingsRef().lock_acquire_timeout); + auto lock2 = dest_table->lockForShare(query_context->getCurrentQueryId(), query_context->getSettingsRef().lock_acquire_timeout); + auto storage_settings_ptr = getSettings(); + auto dest_metadata_snapshot = dest_table->getInMemoryMetadataPtr(); auto metadata_snapshot = getInMemoryMetadataPtr(); From 3c58e5873b24e0aec20a4c5b97c3ab6bb849c47e Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Tue, 9 Apr 2024 19:06:14 +0000 Subject: [PATCH 0150/1018] fix reading of {} with more than 1000 objects under each --- src/Storages/StorageS3.cpp | 25 ++++++++++++------------- 1 file changed, 12 insertions(+), 13 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 85d9b45291c..ffe3213a4bc 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -214,10 +214,8 @@ public: expanded_keys = expandSelectionGlob(globbed_uri.key); expanded_keys_iter = expanded_keys.begin(); - bool no_globs_in_key = fillBufferForKey(*expanded_keys_iter); + fillBufferForKey(*expanded_keys_iter); expanded_keys_iter++; - if (expanded_keys_iter == expanded_keys.end() && no_globs_in_key) - is_finished_for_key = true; } KeyWithInfoPtr next(size_t) @@ -252,6 +250,7 @@ private: buffer_iter = buffer.begin(); if (read_keys) read_keys->insert(read_keys->end(), buffer.begin(), buffer.end()); + is_finished_for_key = true; return true; } @@ -306,17 +305,17 @@ private: return answer; } - if (expanded_keys_iter != expanded_keys.end()) - { - bool no_globs_in_key = fillBufferForKey(*expanded_keys_iter); - expanded_keys_iter++; - if (expanded_keys_iter == expanded_keys.end() && no_globs_in_key) - is_finished_for_key = true; - continue; - } - if (is_finished_for_key) - return {}; + { + if (expanded_keys_iter != expanded_keys.end()) + { + fillBufferForKey(*expanded_keys_iter); + expanded_keys_iter++; + continue; + } + else + return {}; + } try { From 093b71b8585161e91f571d9991e2d351effd10fa Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Tue, 9 Apr 2024 21:01:01 +0000 Subject: [PATCH 0151/1018] added test for selection globs with many files under --- tests/integration/test_storage_s3/test.py | 38 +++++++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index a4ed94c815b..60b0e8792d7 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -678,6 +678,44 @@ def test_s3_glob_scheherazade(started_cluster): assert run_query(instance, query).splitlines() == ["1001\t1001\t1001\t1001"] +# a bit modified version of scheherazade test +# checks e.g. `prefix{1,2}/file*.csv`, where there are more than 1000 files under each of prefix1, prefix2. +def test_s3_glob_many_objects_under_selection(started_cluster): + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + values = "(1, 1, 1)" + jobs = [] + for file_num in range(1100): + + def create_files(file_num): + for folder_num in range(1, 3): + path = f"folder{folder_num}/file{file_num}.csv" + query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values {}".format( + started_cluster.minio_ip, + MINIO_INTERNAL_PORT, + bucket, + path, + table_format, + values, + ) + run_query(instance, query) + + jobs.append(threading.Thread(target=create_files, args=(file_num,))) + jobs[-1].start() + + for job in jobs: + job.join() + + query = "select count(), sum(column1), sum(column2), sum(column3) from s3('http://{}:{}/{}/folder{{1,2}}/file*.csv', 'CSV', '{}')".format( + started_cluster.minio_redirect_host, + started_cluster.minio_redirect_port, + bucket, + table_format, + ) + assert run_query(instance, query).splitlines() == ["2200\t2200\t2200\t2200"] + + def run_s3_mocks(started_cluster): script_dir = os.path.join(os.path.dirname(__file__), "s3_mocks") start_mock_servers( From 8418efe06daa5ea2fae57ae098b7ab109b312f60 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 10 Apr 2024 02:12:02 +0200 Subject: [PATCH 0152/1018] Support UBSan on Clang-19 (master) --- cmake/linux/default_libs.cmake | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/cmake/linux/default_libs.cmake b/cmake/linux/default_libs.cmake index 8552097fa57..61d40b4fd04 100644 --- a/cmake/linux/default_libs.cmake +++ b/cmake/linux/default_libs.cmake @@ -8,6 +8,10 @@ set (DEFAULT_LIBS "-nodefaultlibs") if (COMPILER_CLANG) execute_process (COMMAND ${CMAKE_CXX_COMPILER} --target=${CMAKE_CXX_COMPILER_TARGET} --print-libgcc-file-name --rtlib=compiler-rt OUTPUT_VARIABLE BUILTINS_LIBRARY OUTPUT_STRIP_TRAILING_WHITESPACE) + if (SANITIZE STREQUAL undefined) + string(REPLACE "builtins.a" "ubsan_standalone_cxx.a" EXTRA_BUILTINS_LIBRARY "${BUILTINS_LIBRARY}") + endif () + if (NOT EXISTS "${BUILTINS_LIBRARY}") set (BUILTINS_LIBRARY "-lgcc") endif () @@ -15,11 +19,11 @@ endif () if (OS_ANDROID) # pthread and rt are included in libc - set (DEFAULT_LIBS "${DEFAULT_LIBS} ${BUILTINS_LIBRARY} ${COVERAGE_OPTION} -lc -lm -ldl") + set (DEFAULT_LIBS "${DEFAULT_LIBS} ${BUILTINS_LIBRARY} ${EXTRA_BUILTINS_LIBRARY} ${COVERAGE_OPTION} -lc -lm -ldl") elseif (USE_MUSL) - set (DEFAULT_LIBS "${DEFAULT_LIBS} ${BUILTINS_LIBRARY} ${COVERAGE_OPTION} -static -lc") + set (DEFAULT_LIBS "${DEFAULT_LIBS} ${BUILTINS_LIBRARY} ${EXTRA_BUILTINS_LIBRARY} ${COVERAGE_OPTION} -static -lc") else () - set (DEFAULT_LIBS "${DEFAULT_LIBS} ${BUILTINS_LIBRARY} ${COVERAGE_OPTION} -lc -lm -lrt -lpthread -ldl") + set (DEFAULT_LIBS "${DEFAULT_LIBS} ${BUILTINS_LIBRARY} ${EXTRA_BUILTINS_LIBRARY} ${COVERAGE_OPTION} -lc -lm -lrt -lpthread -ldl") endif () message(STATUS "Default libraries: ${DEFAULT_LIBS}") From 41e0720c0e4e1f3dd1262ffc5932285e700c8dd4 Mon Sep 17 00:00:00 2001 From: zhongyuankai Date: Wed, 10 Apr 2024 09:50:47 +0800 Subject: [PATCH 0153/1018] cmment --- src/Parsers/ASTDropQuery.h | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Parsers/ASTDropQuery.h b/src/Parsers/ASTDropQuery.h index 7e25e990bc8..e18043b771b 100644 --- a/src/Parsers/ASTDropQuery.h +++ b/src/Parsers/ASTDropQuery.h @@ -37,13 +37,13 @@ public: bool sync{false}; - // We detach the object permanently, so it will not be reattached back during server restart. + /// We detach the object permanently, so it will not be reattached back during server restart. bool permanently{false}; - /// Used to drop multiple tables only, example: Drop TABLE t1, t2, t3... + /// Used to drop multiple tables only, example: DROP TABLE t1, t2, t3... ASTPtr database_and_tables; - /** Get the text that identifies this element. */ + /// Get the text that identifies this element. String getID(char) const override; ASTPtr clone() const override; @@ -52,7 +52,7 @@ public: return removeOnCluster(clone(), params.default_database); } - /** Convert an AST that deletes multiple tables into multiple ASTs that delete a single table. */ + /// Convert an AST that deletes multiple tables into multiple ASTs that delete a single table. ASTs getRewrittenASTsOfSingleTable(); QueryKind getQueryKind() const override { return QueryKind::Drop; } From b6b7c3f80f6461e8198e9d70f6b4742f62d6435d Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 10 Apr 2024 13:41:21 +0200 Subject: [PATCH 0154/1018] Fxi --- src/Interpreters/Cache/FileCache.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 564dd03dd3c..29836dc1d15 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -191,9 +191,10 @@ void FileCache::initialize() metadata.startup(); if (keep_current_size_to_max_ratio != 1 || keep_current_elements_to_max_ratio != 1) + { keep_up_free_space_ratio_task = Context::getGlobalContextInstance()->getSchedulePool().createTask(log->name(), [this] { freeSpaceRatioKeepingThreadFunc(); }); - - keep_up_free_space_ratio_task->schedule(); + keep_up_free_space_ratio_task->schedule(); + } is_initialized = true; } From c9b05eac022254c71323d2715a0dc1a32ae9c2f7 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Wed, 10 Apr 2024 12:02:01 +0000 Subject: [PATCH 0155/1018] fix test_s3_glob_many_objects_under_selection --- tests/integration/test_storage_s3/test.py | 26 ++++++++++++++++------- 1 file changed, 18 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 60b0e8792d7..28b70911b77 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -678,19 +678,19 @@ def test_s3_glob_scheherazade(started_cluster): assert run_query(instance, query).splitlines() == ["1001\t1001\t1001\t1001"] -# a bit modified version of scheherazade test -# checks e.g. `prefix{1,2}/file*.csv`, where there are more than 1000 files under each of prefix1, prefix2. +# a bit simplified version of scheherazade test +# checks e.g. `prefix{1,2}/file*.csv`, where there are more than 1000 files under prefix1. def test_s3_glob_many_objects_under_selection(started_cluster): bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" values = "(1, 1, 1)" jobs = [] - for file_num in range(1100): + for thread_num in range(16): - def create_files(file_num): - for folder_num in range(1, 3): - path = f"folder{folder_num}/file{file_num}.csv" + def create_files(thread_num): + for f_num in range(thread_num * 63, thread_num * 63 + 63): + path = f"folder1/file{f_num}.csv" query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values {}".format( started_cluster.minio_ip, MINIO_INTERNAL_PORT, @@ -701,9 +701,19 @@ def test_s3_glob_many_objects_under_selection(started_cluster): ) run_query(instance, query) - jobs.append(threading.Thread(target=create_files, args=(file_num,))) + jobs.append(threading.Thread(target=create_files, args=(thread_num,))) jobs[-1].start() + query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values {}".format( + started_cluster.minio_ip, + MINIO_INTERNAL_PORT, + bucket, + f"folder2/file0.csv", + table_format, + values, + ) + run_query(instance, query) + for job in jobs: job.join() @@ -713,7 +723,7 @@ def test_s3_glob_many_objects_under_selection(started_cluster): bucket, table_format, ) - assert run_query(instance, query).splitlines() == ["2200\t2200\t2200\t2200"] + assert run_query(instance, query).splitlines() == ["1009\t1009\t1009\t1009"] def run_s3_mocks(started_cluster): From dd45151106c4f9c4bb25320c5fd6e7e46c56d21a Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 10 Apr 2024 15:28:52 +0000 Subject: [PATCH 0156/1018] fix test --- tests/queries/0_stateless/03015_optimize_final_rmt.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03015_optimize_final_rmt.sh b/tests/queries/0_stateless/03015_optimize_final_rmt.sh index 96cb2ea22d1..f822a401c49 100755 --- a/tests/queries/0_stateless/03015_optimize_final_rmt.sh +++ b/tests/queries/0_stateless/03015_optimize_final_rmt.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long +# Tags: long, no-random-settings, no-random-merge-tree-settings CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From dc07496fd668f5db3e28a123fbfb9347b3ce7c63 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 10 Apr 2024 15:45:03 +0000 Subject: [PATCH 0157/1018] Fix validation of special MergeTree columns --- src/Storages/MergeTree/MergeTreeData.cpp | 90 ++++++++++++++++++- .../03093_special_column_errors.reference | 0 .../03093_special_column_errors.sql | 36 ++++++++ 3 files changed, 122 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/03093_special_column_errors.reference create mode 100644 tests/queries/0_stateless/03093_special_column_errors.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 3e4350c2fbb..7eb862c8c72 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -861,6 +861,42 @@ void MergeTreeData::checkTTLExpressions(const StorageInMemoryMetadata & new_meta } } +namespace +{ +template +void checkSpecialColumn(const std::string_view column_meta_name, const AlterCommand & command) +{ + if (command.type == AlterCommand::MODIFY_COLUMN) + { + if (!typeid_cast(command.data_type.get())) + { + throw Exception( + ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, + "Cannot alter {} column ({}) to type {}, because it must have type {}", + column_meta_name, + command.column_name, + command.data_type->getName(), + TypeName); + } + } + else if (command.type == AlterCommand::DROP_COLUMN) + { + throw Exception( + ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, + "Trying to ALTER DROP {} ({}) column", + column_meta_name, + backQuoteIfNeed(command.column_name)); + } + else if (command.type == AlterCommand::RENAME_COLUMN) + { + throw Exception( + ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, + "Trying to ALTER RENAME {} ({}) column", + column_meta_name, + backQuoteIfNeed(command.column_name)); + } +}; +} void MergeTreeData::checkStoragePolicy(const StoragePolicyPtr & new_storage_policy) const { @@ -993,10 +1029,20 @@ void MergeTreeData::MergingParams::check(const StorageInMemoryMetadata & metadat { return column_to_sum == Nested::extractTableName(name_and_type.name); }; - if (columns.end() == std::find_if(columns.begin(), columns.end(), check_column_to_sum_exists)) - throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, - "Column {} listed in columns to sum does not exist in table declaration.", - column_to_sum); + const auto column_it = std::find_if(columns.begin(), columns.end(), check_column_to_sum_exists); + + if (columns.end() == column_it) + { + throw Exception( + ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, + "Column {} listed in columns to sum does not exist in table declaration", + column_to_sum); + } + else if (!isNumber(column_it->type)) + { + throw Exception( + ErrorCodes::BAD_TYPE_OF_FIELD, "Column {} listed in columns to sum does not have number type", column_to_sum); + } } /// Check that summing columns are not in partition key. @@ -3114,6 +3160,42 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context } } + // Technically it is possible to specify the same column for `version` and `is_deleted`, thus let's be sure and don't use -if-else here + if (command.column_name == merging_params.is_deleted_column) + { + checkSpecialColumn("is_deleted", command); + } + else if (command.column_name == merging_params.sign_column) + { + checkSpecialColumn("sign", command); + } + else if (std::ranges::any_of( + merging_params.columns_to_sum, [&](const String & column_to_sum) { return command.column_name == column_to_sum; })) + { + if (command.type == AlterCommand::MODIFY_COLUMN && !isNumber(command.data_type)) + { + throw Exception( + ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, + "Cannot alter summarizing column ({}) to type {}, because it must have numeric type", + command.column_name, + command.data_type->getName()); + } + else if (command.type == AlterCommand::DROP_COLUMN) + { + throw Exception( + ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, + "Trying to ALTER DROP summarizing column ({})", + backQuoteIfNeed(command.column_name)); + } + else if (command.type == AlterCommand::RENAME_COLUMN) + { + throw Exception( + ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, + "Trying to ALTER RENAME summarizing column ({})", + backQuoteIfNeed(command.column_name)); + } + } + if (command.type == AlterCommand::MODIFY_QUERY) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "ALTER MODIFY QUERY is not supported by MergeTree engines family"); diff --git a/tests/queries/0_stateless/03093_special_column_errors.reference b/tests/queries/0_stateless/03093_special_column_errors.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03093_special_column_errors.sql b/tests/queries/0_stateless/03093_special_column_errors.sql new file mode 100644 index 00000000000..2bff551738e --- /dev/null +++ b/tests/queries/0_stateless/03093_special_column_errors.sql @@ -0,0 +1,36 @@ +CREATE TABLE replacing_wrong (key Int64, ver Int64, is_deleted UInt16) ENGINE = ReplacingMergeTree(ver, is_deleted) ORDER BY key; -- { serverError BAD_TYPE_OF_FIELD } +CREATE TABLE replacing_wrong (key Int64, ver String, is_deleted UInt8) ENGINE = ReplacingMergeTree(ver, is_deleted) ORDER BY key; -- { serverError BAD_TYPE_OF_FIELD } + +CREATE TABLE replacing (key Int64, ver Int64, is_deleted UInt8) ENGINE = ReplacingMergeTree(ver, is_deleted) ORDER BY key; +ALTER TABLE replacing MODIFY COLUMN ver String; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE replacing MODIFY COLUMN ver Int128; +ALTER TABLE replacing MODIFY COLUMN is_deleted String; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE replacing MODIFY COLUMN is_deleted UInt16; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE replacing MODIFY COLUMN is_deleted Int8; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE replacing DROP COLUMN ver; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE replacing DROP COLUMN is_deleted; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE replacing RENAME COLUMN ver TO ver2; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE replacing RENAME COLUMN is_deleted TO is_deleted2; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } + +CREATE TABLE summing_wrong (key Int64, sum1 Int64, sum2 String) ENGINE = SummingMergeTree((sum1, sum2)) ORDER BY key; -- { serverError BAD_TYPE_OF_FIELD } +CREATE TABLE summing_wrong (key Int64, sum1 String, sum2 Int64) ENGINE = SummingMergeTree((sum1, sum2)) ORDER BY key; -- { serverError BAD_TYPE_OF_FIELD } +CREATE TABLE summing_wrong (key Int64, sum1 String, sum2 Int64) ENGINE = SummingMergeTree(sum_doesnt_exists) ORDER BY key; -- { serverError NO_SUCH_COLUMN_IN_TABLE } + +CREATE TABLE summing (key Int64, sum1 Int64, sum2 UInt64, not_sum String) ENGINE = SummingMergeTree((sum1, sum2)) ORDER BY key; +ALTER TABLE summing MODIFY COLUMN sum1 Int32, MODIFY COLUMN sum2 IPv4; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE summing MODIFY COLUMN sum2 String; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE summing MODIFY COLUMN sum1 Int32, MODIFY COLUMN sum2 UInt256; +ALTER TABLE summing DROP COLUMN sum1; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE summing DROP COLUMN sum2; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE summing RENAME COLUMN sum1 TO sum3; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE summing RENAME COLUMN sum2 TO sum3; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE summing RENAME COLUMN not_sum TO still_not_sum; + +CREATE TABLE collapsing_wrong (key Int64, sign Int16) ENGINE = CollapsingMergeTree(sign) ORDER BY key; -- { serverError BAD_TYPE_OF_FIELD } +CREATE TABLE collapsing_wrong (key Int64, sign UInt8) ENGINE = CollapsingMergeTree(sign) ORDER BY key; -- { serverError BAD_TYPE_OF_FIELD } +CREATE TABLE collapsing_wrong (key Int64, sign UInt8) ENGINE = CollapsingMergeTree(not_existing) ORDER BY key; -- { serverError NO_SUCH_COLUMN_IN_TABLE } + +CREATE TABLE collapsing (key Int64, sign Int8) ENGINE = CollapsingMergeTree(sign) ORDER BY key; +ALTER TABLE collapsing MODIFY COLUMN sign String; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE collapsing DROP COLUMN sign; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE collapsing RENAME COLUMN sign TO sign2; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } From 8d2f8cccec336e592e52d4403ec7a14cca4465c0 Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 10 Apr 2024 19:12:04 +0200 Subject: [PATCH 0158/1018] Add flattenTuple to documentation --- .../functions/tuple-functions.md | 36 +++++++++++++++++++ 1 file changed, 36 insertions(+) diff --git a/docs/en/sql-reference/functions/tuple-functions.md b/docs/en/sql-reference/functions/tuple-functions.md index b3cec1206b8..0f3839ee617 100644 --- a/docs/en/sql-reference/functions/tuple-functions.md +++ b/docs/en/sql-reference/functions/tuple-functions.md @@ -856,6 +856,42 @@ Result: └─────────────────────────────────────┘ ``` +## flattenTuple + +Returns a flattened `output` Tuple from a nested named `input` Tuple. Elements of the `output` Tuple are the paths from the original `Tuple`. For instance: `Tuple(a Int, Tuple(b Int, c Int)) -> Tuple(a Int, b Int, c Int)`. `flattenTuple` can be used to select all paths from type `Object` as separate columns. + +**Syntax** + +```sql +flattenTuple(input) +``` + +**Parameters** + +- `input`: Nested named Tuple to flatten. [Tuple](../data-types/tuple). + +**Returned value** + +- `output` Tuple whose elements are paths from the original `input`. [Tuple](../data-types/tuple). + +**Examples** + +Query: + +``` sql +CREATE TABLE t_flatten_tuple(t Tuple(t1 Nested(a UInt32, s String), b UInt32, t2 Tuple(k String, v UInt32))) ENGINE = Memory; +INSERT INTO t_flatten_tuple VALUES (([(1, 'a'), (2, 'b')], 3, ('c', 4))); +SELECT flattenTuple(t) FROM t_flatten_tuple; +``` + +Result: + +``` text +┌─flattenTuple(t)───────────┐ +│ ([1,2],['a','b'],3,'c',4) │ +└───────────────────────────┘ +``` + ## Distance functions All supported functions are described in [distance functions documentation](../../sql-reference/functions/distance-functions.md). From 6df90c5c25f3e14916695420166414dfb8f3c942 Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 10 Apr 2024 19:14:57 +0200 Subject: [PATCH 0159/1018] Change examples -> example --- docs/en/sql-reference/functions/tuple-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/tuple-functions.md b/docs/en/sql-reference/functions/tuple-functions.md index 0f3839ee617..5775da34a1e 100644 --- a/docs/en/sql-reference/functions/tuple-functions.md +++ b/docs/en/sql-reference/functions/tuple-functions.md @@ -874,7 +874,7 @@ flattenTuple(input) - `output` Tuple whose elements are paths from the original `input`. [Tuple](../data-types/tuple). -**Examples** +**Example** Query: From 8344167d8bb038bce36da5bb473d6298757be09d Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 10 Apr 2024 19:17:21 +0200 Subject: [PATCH 0160/1018] Try fix tests --- src/Storages/RabbitMQ/RabbitMQSource.cpp | 6 +++++- src/Storages/RabbitMQ/RabbitMQSource.h | 9 ++++++--- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 4 ++-- 3 files changed, 13 insertions(+), 6 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQSource.cpp b/src/Storages/RabbitMQ/RabbitMQSource.cpp index c8b8d9f2323..0badb6346b3 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSource.cpp @@ -46,6 +46,7 @@ RabbitMQSource::RabbitMQSource( size_t max_block_size_, UInt64 max_execution_time_, StreamingHandleErrorMode handle_error_mode_, + bool nack_broken_messages_, bool ack_in_suffix_) : RabbitMQSource( storage_, @@ -56,6 +57,7 @@ RabbitMQSource::RabbitMQSource( max_block_size_, max_execution_time_, handle_error_mode_, + nack_broken_messages_, ack_in_suffix_) { } @@ -69,6 +71,7 @@ RabbitMQSource::RabbitMQSource( size_t max_block_size_, UInt64 max_execution_time_, StreamingHandleErrorMode handle_error_mode_, + bool nack_broken_messages_, bool ack_in_suffix_) : ISource(getSampleBlock(headers.first, headers.second)) , storage(storage_) @@ -78,6 +81,7 @@ RabbitMQSource::RabbitMQSource( , max_block_size(max_block_size_) , handle_error_mode(handle_error_mode_) , ack_in_suffix(ack_in_suffix_) + , nack_broken_messages(nack_broken_messages_) , non_virtual_header(std::move(headers.first)) , virtual_header(std::move(headers.second)) , log(getLogger("RabbitMQSource")) @@ -207,7 +211,7 @@ Chunk RabbitMQSource::generateImpl() commit_info.failed_delivery_tags.size(), exception_message.has_value() ? exception_message.value() : "None"); - if (exception_message.has_value()) + if (exception_message.has_value() && nack_broken_messages) { commit_info.failed_delivery_tags.push_back(message.delivery_tag); } diff --git a/src/Storages/RabbitMQ/RabbitMQSource.h b/src/Storages/RabbitMQ/RabbitMQSource.h index 0d6fad97054..8d939e5c57c 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.h +++ b/src/Storages/RabbitMQ/RabbitMQSource.h @@ -20,6 +20,7 @@ public: size_t max_block_size_, UInt64 max_execution_time_, StreamingHandleErrorMode handle_error_mode_, + bool nack_broken_messages_, bool ack_in_suffix = false); ~RabbitMQSource() override; @@ -39,10 +40,11 @@ private: StorageRabbitMQ & storage; StorageSnapshotPtr storage_snapshot; ContextPtr context; - Names column_names; + const Names column_names; const size_t max_block_size; - StreamingHandleErrorMode handle_error_mode; - bool ack_in_suffix; + const StreamingHandleErrorMode handle_error_mode; + const bool ack_in_suffix; + const bool nack_broken_messages; bool is_finished = false; const Block non_virtual_header; @@ -65,6 +67,7 @@ private: size_t max_block_size_, UInt64 max_execution_time_, StreamingHandleErrorMode handle_error_mode_, + bool nack_broken_messages_, bool ack_in_suffix); Chunk generateImpl(); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index d3cbcc1c6c7..15435ba2988 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -746,7 +746,7 @@ void StorageRabbitMQ::read( { auto rabbit_source = std::make_shared( *this, storage_snapshot, modified_context, column_names, 1, - max_execution_time_ms, rabbitmq_settings->rabbitmq_handle_error_mode, rabbitmq_settings->rabbitmq_commit_on_select); + max_execution_time_ms, rabbitmq_settings->rabbitmq_handle_error_mode, reject_unhandled_messages, rabbitmq_settings->rabbitmq_commit_on_select); auto converting_dag = ActionsDAG::makeConvertingActions( rabbit_source->getPort().getHeader().getColumnsWithTypeAndName(), @@ -1082,7 +1082,7 @@ bool StorageRabbitMQ::tryStreamToViews() { auto source = std::make_shared( *this, storage_snapshot, rabbitmq_context, Names{}, block_size, - max_execution_time_ms, rabbitmq_settings->rabbitmq_handle_error_mode); + max_execution_time_ms, rabbitmq_settings->rabbitmq_handle_error_mode, reject_unhandled_messages); sources.emplace_back(source); pipes.emplace_back(source); From 05f55504bb487148d5ced68feb2f9e1ad9ca2597 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 10 Apr 2024 19:32:11 +0200 Subject: [PATCH 0161/1018] Fix schedule --- src/Interpreters/Cache/FileCache.cpp | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 29836dc1d15..0a2f0da0305 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -988,12 +988,13 @@ void FileCache::freeSpaceRatioKeepingThreadFunc() FileCacheReserveStat stat; EvictionCandidates eviction_candidates; + bool limits_satisfied = true; try { /// Collect at most `keep_up_free_space_remove_batch` elements to evict, /// (we use batches to make sure we do not block cache for too long, /// by default the batch size is quite small). - const bool limits_satisfied = main_priority->collectCandidatesForEviction( + limits_satisfied = main_priority->collectCandidatesForEviction( desired_size, desired_elements_num, keep_up_free_space_remove_batch, stat, eviction_candidates, lock); #ifdef ABORT_ON_LOGICAL_ERROR @@ -1012,8 +1013,6 @@ void FileCache::freeSpaceRatioKeepingThreadFunc() || current_elements_count <= desired_elements_num || current_elements_count - stat.total_stat.releasable_count <= desired_elements_num); } -#else - UNUSED(limits_satisfied); #endif if (shutdown) @@ -1040,10 +1039,6 @@ void FileCache::freeSpaceRatioKeepingThreadFunc() lock.lock(); eviction_candidates.finalize(nullptr, lock); } - else - { - keep_up_free_space_ratio_task->scheduleAfter(general_reschedule_ms); - } } catch (...) { @@ -1052,14 +1047,17 @@ void FileCache::freeSpaceRatioKeepingThreadFunc() if (eviction_candidates.size() > 0) eviction_candidates.finalize(nullptr, lockCache()); - keep_up_free_space_ratio_task->scheduleAfter(general_reschedule_ms); - /// Let's catch such cases in ci, /// in general there should not be exceptions. chassert(false); } LOG_TRACE(log, "Free space ratio keeping thread finished in {} ms", watch.elapsedMilliseconds()); + + if (limits_satisfied) + keep_up_free_space_ratio_task->scheduleAfter(general_reschedule_ms); + else + keep_up_free_space_ratio_task->schedule(); } void FileCache::iterate(IterateFunc && func, const UserID & user_id) From 0669591e35e0b6f19c148ff941c2497a0e38435c Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Wed, 10 Apr 2024 17:33:48 +0000 Subject: [PATCH 0162/1018] small code cleanup --- src/Storages/StorageS3.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index ffe3213a4bc..acef213c1f4 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -237,7 +237,7 @@ public: private: using ListObjectsOutcome = Aws::S3::Model::ListObjectsV2Outcome; - bool fillBufferForKey(const std::string & uri_key) + void fillBufferForKey(const std::string & uri_key) { is_finished_for_key = false; const String key_prefix = uri_key.substr(0, uri_key.find_first_of("*?{")); @@ -251,7 +251,7 @@ private: if (read_keys) read_keys->insert(read_keys->end(), buffer.begin(), buffer.end()); is_finished_for_key = true; - return true; + return; } request = {}; @@ -270,7 +270,7 @@ private: filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); fillInternalBufferAssumeLocked(); - return false; + return; } KeyWithInfoPtr nextAssumeLocked() From e044b4bf0219f16563f75153d4da3d10806e1e51 Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 10 Apr 2024 20:31:45 +0200 Subject: [PATCH 0163/1018] add flattenTuple to aspell-dict and replace Tuple with tuple --- docs/en/sql-reference/functions/tuple-functions.md | 6 +++--- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-functions.md b/docs/en/sql-reference/functions/tuple-functions.md index 5775da34a1e..454d3cc1966 100644 --- a/docs/en/sql-reference/functions/tuple-functions.md +++ b/docs/en/sql-reference/functions/tuple-functions.md @@ -858,7 +858,7 @@ Result: ## flattenTuple -Returns a flattened `output` Tuple from a nested named `input` Tuple. Elements of the `output` Tuple are the paths from the original `Tuple`. For instance: `Tuple(a Int, Tuple(b Int, c Int)) -> Tuple(a Int, b Int, c Int)`. `flattenTuple` can be used to select all paths from type `Object` as separate columns. +Returns a flattened `output` tuple from a nested named `input` tuple. Elements of the `output` tuple are the paths from the original `input` tuple. For instance: `Tuple(a Int, Tuple(b Int, c Int)) -> Tuple(a Int, b Int, c Int)`. `flattenTuple` can be used to select all paths from type `Object` as separate columns. **Syntax** @@ -868,11 +868,11 @@ flattenTuple(input) **Parameters** -- `input`: Nested named Tuple to flatten. [Tuple](../data-types/tuple). +- `input`: Nested named tuple to flatten. [Tuple](../data-types/tuple). **Returned value** -- `output` Tuple whose elements are paths from the original `input`. [Tuple](../data-types/tuple). +- `output` tuple whose elements are paths from the original `input`. [Tuple](../data-types/tuple). **Example** diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 697f93f78c9..7eda00f6386 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1566,6 +1566,7 @@ firstSignificantSubdomainCustom fixedstring flamegraph flatbuffers +flattenTuple flink fluentd fmtlib From 619d1b42e14be65e77bd0f1980e2b16696f74070 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 10 Apr 2024 18:35:20 +0000 Subject: [PATCH 0164/1018] Increase timeout --- src/Client/Connection.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 057cceb68df..9708ab6d914 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -615,7 +615,7 @@ TablesStatusResponse Connection::getTablesStatus(const ConnectionTimeouts & time out->next(); fiu_do_on(FailPoints::receive_timeout_on_table_status_response, { - sleepForSeconds(5); + sleepForSeconds(10); throw NetException(ErrorCodes::SOCKET_TIMEOUT, "Injected timeout exceeded while reading from socket ({}:{})", host, port); }); From 263dcee5eab06151310132ac3ee12d6ee41627aa Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 10 Apr 2024 22:41:25 +0200 Subject: [PATCH 0165/1018] Update RabbitMQSource.cpp --- src/Storages/RabbitMQ/RabbitMQSource.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQSource.cpp b/src/Storages/RabbitMQ/RabbitMQSource.cpp index 0badb6346b3..b69a533fd92 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSource.cpp @@ -204,9 +204,8 @@ Chunk RabbitMQSource::generateImpl() const auto exchange_name = storage.getExchange(); const auto & message = consumer->currentMessage(); - LOG_TEST(log, "Pulled {} rows, message delivery tag: {} " - "(previous delivery tag: {}, redelivered: {}, failed delivery tags by this moment: {}). " - "Exception message: {}", + LOG_TEST(log, "Pulled {} rows, message delivery tag: {}, " + "previous delivery tag: {}, redelivered: {}, failed delivery tags by this moment: {}, exception message: {}", new_rows, message.delivery_tag, commit_info.delivery_tag, message.redelivered, commit_info.failed_delivery_tags.size(), exception_message.has_value() ? exception_message.value() : "None"); From b8c77a4453d3baaa21cc45e35eb3cb1c6b1aa357 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 11 Apr 2024 10:13:41 +0200 Subject: [PATCH 0166/1018] Support parallel write buffer for AzureBlobStorage --- src/Backups/BackupIO_AzureBlobStorage.cpp | 3 ++- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.h | 1 + src/Disks/IDisk.cpp | 1 + .../ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp | 7 ++++++- src/IO/WriteSettings.h | 1 + src/Interpreters/Context.cpp | 1 + 7 files changed, 13 insertions(+), 2 deletions(-) diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index b9b208e321c..5b8aa687b1c 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -278,7 +278,8 @@ std::unique_ptr BackupWriterAzureBlobStorage::writeFile(const Strin key, DBMS_DEFAULT_BUFFER_SIZE, write_settings, - settings); + settings, + threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupWRAzure")); } void BackupWriterAzureBlobStorage::removeFile(const String & file_name) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6eb088edc9a..2356a63f24c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -112,6 +112,7 @@ class IColumn; M(Bool, azure_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in azure engine tables", 0) \ M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \ M(Bool, s3_allow_parallel_part_upload, true, "Use multiple threads for s3 multipart upload. It may lead to slightly higher memory usage", 0) \ + M(Bool, azure_allow_parallel_part_upload, true, "Use multiple threads for azure multipart upload.", 0) \ M(Bool, s3_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ M(Bool, s3_disable_checksum, false, "Do not calculate a checksum when sending a file to S3. This speeds up writes by avoiding excessive processing passes on a file. It is mostly safe as the data of MergeTree tables is checksummed by ClickHouse anyway, and when S3 is accessed with HTTPS, the TLS layer already provides integrity while transferring through the network. While additional checksums on S3 give defense in depth.", 0) \ M(UInt64, s3_retry_attempts, 100, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index eb4ab2d4438..d6482eb293f 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -89,6 +89,7 @@ static std::map sett {"ignore_drop_queries_probability", 0, 0, "Allow to ignore drop queries in server with specified probability for testing purposes"}, {"lightweight_deletes_sync", 2, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes"}, {"query_cache_system_table_handling", "save", "throw", "The query cache no longer caches results of queries against system tables"}, + {"azure_allow_parallel_part_upload", "true", "true", "Use multiple threads for azure multipart upload."}, }}, {"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"}, {"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"}, diff --git a/src/Disks/IDisk.cpp b/src/Disks/IDisk.cpp index 066acc250a2..14d5f94ef46 100644 --- a/src/Disks/IDisk.cpp +++ b/src/Disks/IDisk.cpp @@ -154,6 +154,7 @@ void IDisk::copyThroughBuffers( /// Disable parallel write. We already copy in parallel. /// Avoid high memory usage. See test_s3_zero_copy_ttl/test.py::test_move_and_s3_memory_usage write_settings.s3_allow_parallel_part_upload = false; + write_settings.azure_allow_parallel_part_upload = false; asyncCopy(*this, from_path, *to_disk, to_path, copying_thread_pool, results, copy_root_dir, read_settings, write_settings, cancellation_hook); diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index e0614613c3f..f6b510bf973 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -262,12 +262,17 @@ std::unique_ptr AzureObjectStorage::writeObject( /// NO LOG_TEST(log, "Writing file: {}", object.remote_path); + ThreadPoolCallbackRunner scheduler; + if (write_settings.azure_allow_parallel_part_upload) + scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "VFSWrite"); + return std::make_unique( client.get(), object.remote_path, buf_size, patchSettings(write_settings), - settings.get()); + settings.get(), + std::move(scheduler)); } /// Remove file. Throws exception if file doesn't exists or it's a directory. diff --git a/src/IO/WriteSettings.h b/src/IO/WriteSettings.h index 7d36677b468..4da3806e51d 100644 --- a/src/IO/WriteSettings.h +++ b/src/IO/WriteSettings.h @@ -23,6 +23,7 @@ struct WriteSettings size_t filesystem_cache_reserve_space_wait_lock_timeout_milliseconds = 1000; bool s3_allow_parallel_part_upload = true; + bool azure_allow_parallel_part_upload = true; /// Monitoring bool for_object_storage = false; // to choose which profile events should be incremented diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 04b761e8b2b..0b63904c3e5 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -5256,6 +5256,7 @@ WriteSettings Context::getWriteSettings() const res.filesystem_cache_reserve_space_wait_lock_timeout_milliseconds = settings.filesystem_cache_reserve_space_wait_lock_timeout_milliseconds; res.s3_allow_parallel_part_upload = settings.s3_allow_parallel_part_upload; + res.azure_allow_parallel_part_upload = settings.azure_allow_parallel_part_upload; res.remote_throttler = getRemoteWriteThrottler(); res.local_throttler = getLocalWriteThrottler(); From ce457e76256e397bea451db5c1012b8c49e8d70f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Thu, 11 Apr 2024 09:01:14 +0000 Subject: [PATCH 0167/1018] Add checks to prevent using the same column for multiple special columns --- src/Storages/MergeTree/MergeTreeData.cpp | 15 ++++++++++++--- .../0_stateless/03093_special_column_errors.sql | 6 ++++++ 2 files changed, 18 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 7eb862c8c72..8f97ec9cbc2 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1022,6 +1022,11 @@ void MergeTreeData::MergingParams::check(const StorageInMemoryMetadata & metadat if (mode == MergingParams::Summing) { + auto columns_to_sum_copy = columns_to_sum; + std::sort(columns_to_sum_copy.begin(), columns_to_sum_copy.end()); + if (const auto it = std::adjacent_find(columns_to_sum_copy.begin(), columns_to_sum_copy.end()); it != columns_to_sum_copy.end()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Column {} is listed multiple times in the list of columns to sum", *it); + /// If columns_to_sum are set, then check that such columns exist. for (const auto & column_to_sum : columns_to_sum) { @@ -1063,12 +1068,18 @@ void MergeTreeData::MergingParams::check(const StorageInMemoryMetadata & metadat if (mode == MergingParams::Replacing) { + if (version_column == is_deleted_column) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The version and is_deleted column cannot be the same column ({})", version_column); + check_is_deleted_column(true, "ReplacingMergeTree"); check_version_column(true, "ReplacingMergeTree"); } if (mode == MergingParams::VersionedCollapsing) { + if (version_column == sign_column) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The version and sign column cannot be the same column ({})", version_column); + check_sign_column(false, "VersionedCollapsingMergeTree"); check_version_column(false, "VersionedCollapsingMergeTree"); } @@ -3159,9 +3170,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context "Trying to ALTER RENAME version {} column", backQuoteIfNeed(command.column_name)); } } - - // Technically it is possible to specify the same column for `version` and `is_deleted`, thus let's be sure and don't use -if-else here - if (command.column_name == merging_params.is_deleted_column) + else if (command.column_name == merging_params.is_deleted_column) { checkSpecialColumn("is_deleted", command); } diff --git a/tests/queries/0_stateless/03093_special_column_errors.sql b/tests/queries/0_stateless/03093_special_column_errors.sql index 2bff551738e..bbdf1bb7183 100644 --- a/tests/queries/0_stateless/03093_special_column_errors.sql +++ b/tests/queries/0_stateless/03093_special_column_errors.sql @@ -1,5 +1,6 @@ CREATE TABLE replacing_wrong (key Int64, ver Int64, is_deleted UInt16) ENGINE = ReplacingMergeTree(ver, is_deleted) ORDER BY key; -- { serverError BAD_TYPE_OF_FIELD } CREATE TABLE replacing_wrong (key Int64, ver String, is_deleted UInt8) ENGINE = ReplacingMergeTree(ver, is_deleted) ORDER BY key; -- { serverError BAD_TYPE_OF_FIELD } +CREATE TABLE replacing_wrong (key Int64, ver Int64, is_deleted UInt8) ENGINE = ReplacingMergeTree(is_deleted, is_deleted) ORDER BY key; -- { serverError BAD_ARGUMENTS } CREATE TABLE replacing (key Int64, ver Int64, is_deleted UInt8) ENGINE = ReplacingMergeTree(ver, is_deleted) ORDER BY key; ALTER TABLE replacing MODIFY COLUMN ver String; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } @@ -15,6 +16,7 @@ ALTER TABLE replacing RENAME COLUMN is_deleted TO is_deleted2; -- { serverError CREATE TABLE summing_wrong (key Int64, sum1 Int64, sum2 String) ENGINE = SummingMergeTree((sum1, sum2)) ORDER BY key; -- { serverError BAD_TYPE_OF_FIELD } CREATE TABLE summing_wrong (key Int64, sum1 String, sum2 Int64) ENGINE = SummingMergeTree((sum1, sum2)) ORDER BY key; -- { serverError BAD_TYPE_OF_FIELD } CREATE TABLE summing_wrong (key Int64, sum1 String, sum2 Int64) ENGINE = SummingMergeTree(sum_doesnt_exists) ORDER BY key; -- { serverError NO_SUCH_COLUMN_IN_TABLE } +CREATE TABLE summing_wrong (key Int64, sum1 String, sum2 Int64) ENGINE = SummingMergeTree((sum1, sum2, sum1)) ORDER BY key; -- { serverError BAD_ARGUMENTS } CREATE TABLE summing (key Int64, sum1 Int64, sum2 UInt64, not_sum String) ENGINE = SummingMergeTree((sum1, sum2)) ORDER BY key; ALTER TABLE summing MODIFY COLUMN sum1 Int32, MODIFY COLUMN sum2 IPv4; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } @@ -34,3 +36,7 @@ CREATE TABLE collapsing (key Int64, sign Int8) ENGINE = CollapsingMergeTree(sign ALTER TABLE collapsing MODIFY COLUMN sign String; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } ALTER TABLE collapsing DROP COLUMN sign; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } ALTER TABLE collapsing RENAME COLUMN sign TO sign2; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } + +CREATE TABLE versioned_collapsing_wrong (key Int64, version UInt8, sign Int8) ENGINE = VersionedCollapsingMergeTree(sign, sign) ORDER BY key; -- { serverError BAD_ARGUMENTS } + +CREATE TABLE versioned_collapsing (key Int64, version UInt8, sign Int8) ENGINE = VersionedCollapsingMergeTree(sign, version) ORDER BY key; From 583bdff2bc51c3dbff275d76fbdbd7e9e209915c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Thu, 11 Apr 2024 10:57:33 +0000 Subject: [PATCH 0168/1018] Abandon validating the special columns of SummingMergeTree, as they are way more complex --- src/Storages/MergeTree/MergeTreeData.cpp | 39 +------------------ .../03093_special_column_errors.sql | 15 ------- 2 files changed, 2 insertions(+), 52 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 8f97ec9cbc2..d09bb23de90 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1034,20 +1034,11 @@ void MergeTreeData::MergingParams::check(const StorageInMemoryMetadata & metadat { return column_to_sum == Nested::extractTableName(name_and_type.name); }; - const auto column_it = std::find_if(columns.begin(), columns.end(), check_column_to_sum_exists); - - if (columns.end() == column_it) - { + if (columns.end() == std::find_if(columns.begin(), columns.end(), check_column_to_sum_exists)) throw Exception( ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, - "Column {} listed in columns to sum does not exist in table declaration", + "Column {} listed in columns to sum does not exist in table declaration.", column_to_sum); - } - else if (!isNumber(column_it->type)) - { - throw Exception( - ErrorCodes::BAD_TYPE_OF_FIELD, "Column {} listed in columns to sum does not have number type", column_to_sum); - } } /// Check that summing columns are not in partition key. @@ -3178,32 +3169,6 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context { checkSpecialColumn("sign", command); } - else if (std::ranges::any_of( - merging_params.columns_to_sum, [&](const String & column_to_sum) { return command.column_name == column_to_sum; })) - { - if (command.type == AlterCommand::MODIFY_COLUMN && !isNumber(command.data_type)) - { - throw Exception( - ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, - "Cannot alter summarizing column ({}) to type {}, because it must have numeric type", - command.column_name, - command.data_type->getName()); - } - else if (command.type == AlterCommand::DROP_COLUMN) - { - throw Exception( - ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, - "Trying to ALTER DROP summarizing column ({})", - backQuoteIfNeed(command.column_name)); - } - else if (command.type == AlterCommand::RENAME_COLUMN) - { - throw Exception( - ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, - "Trying to ALTER RENAME summarizing column ({})", - backQuoteIfNeed(command.column_name)); - } - } if (command.type == AlterCommand::MODIFY_QUERY) throw Exception(ErrorCodes::NOT_IMPLEMENTED, diff --git a/tests/queries/0_stateless/03093_special_column_errors.sql b/tests/queries/0_stateless/03093_special_column_errors.sql index bbdf1bb7183..1464927db7e 100644 --- a/tests/queries/0_stateless/03093_special_column_errors.sql +++ b/tests/queries/0_stateless/03093_special_column_errors.sql @@ -13,21 +13,6 @@ ALTER TABLE replacing DROP COLUMN is_deleted; -- { serverError ALTER_OF_COLUMN_I ALTER TABLE replacing RENAME COLUMN ver TO ver2; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } ALTER TABLE replacing RENAME COLUMN is_deleted TO is_deleted2; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } -CREATE TABLE summing_wrong (key Int64, sum1 Int64, sum2 String) ENGINE = SummingMergeTree((sum1, sum2)) ORDER BY key; -- { serverError BAD_TYPE_OF_FIELD } -CREATE TABLE summing_wrong (key Int64, sum1 String, sum2 Int64) ENGINE = SummingMergeTree((sum1, sum2)) ORDER BY key; -- { serverError BAD_TYPE_OF_FIELD } -CREATE TABLE summing_wrong (key Int64, sum1 String, sum2 Int64) ENGINE = SummingMergeTree(sum_doesnt_exists) ORDER BY key; -- { serverError NO_SUCH_COLUMN_IN_TABLE } -CREATE TABLE summing_wrong (key Int64, sum1 String, sum2 Int64) ENGINE = SummingMergeTree((sum1, sum2, sum1)) ORDER BY key; -- { serverError BAD_ARGUMENTS } - -CREATE TABLE summing (key Int64, sum1 Int64, sum2 UInt64, not_sum String) ENGINE = SummingMergeTree((sum1, sum2)) ORDER BY key; -ALTER TABLE summing MODIFY COLUMN sum1 Int32, MODIFY COLUMN sum2 IPv4; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } -ALTER TABLE summing MODIFY COLUMN sum2 String; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } -ALTER TABLE summing MODIFY COLUMN sum1 Int32, MODIFY COLUMN sum2 UInt256; -ALTER TABLE summing DROP COLUMN sum1; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } -ALTER TABLE summing DROP COLUMN sum2; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } -ALTER TABLE summing RENAME COLUMN sum1 TO sum3; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } -ALTER TABLE summing RENAME COLUMN sum2 TO sum3; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } -ALTER TABLE summing RENAME COLUMN not_sum TO still_not_sum; - CREATE TABLE collapsing_wrong (key Int64, sign Int16) ENGINE = CollapsingMergeTree(sign) ORDER BY key; -- { serverError BAD_TYPE_OF_FIELD } CREATE TABLE collapsing_wrong (key Int64, sign UInt8) ENGINE = CollapsingMergeTree(sign) ORDER BY key; -- { serverError BAD_TYPE_OF_FIELD } CREATE TABLE collapsing_wrong (key Int64, sign UInt8) ENGINE = CollapsingMergeTree(not_existing) ORDER BY key; -- { serverError NO_SUCH_COLUMN_IN_TABLE } From 78c34916f93c11a913bc16149e028c9b688c902d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Thu, 11 Apr 2024 13:44:23 +0000 Subject: [PATCH 0169/1018] Fix incorrect error when no special columns are specified --- src/Storages/MergeTree/MergeTreeData.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d09bb23de90..7a138331207 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1059,7 +1059,7 @@ void MergeTreeData::MergingParams::check(const StorageInMemoryMetadata & metadat if (mode == MergingParams::Replacing) { - if (version_column == is_deleted_column) + if (!version_column.empty() && version_column == is_deleted_column) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The version and is_deleted column cannot be the same column ({})", version_column); check_is_deleted_column(true, "ReplacingMergeTree"); @@ -1068,7 +1068,7 @@ void MergeTreeData::MergingParams::check(const StorageInMemoryMetadata & metadat if (mode == MergingParams::VersionedCollapsing) { - if (version_column == sign_column) + if (!version_column.empty() && version_column == sign_column) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The version and sign column cannot be the same column ({})", version_column); check_sign_column(false, "VersionedCollapsingMergeTree"); From 45258bc565bb6440cfd4716f6eab6e76472508ab Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Thu, 11 Apr 2024 14:50:18 +0000 Subject: [PATCH 0170/1018] CI: Add tests with Azure storage --- .github/PULL_REQUEST_TEMPLATE.md | 5 ++--- tests/ci/ci_config.py | 9 +++++++++ tests/ci/functional_test_check.py | 3 +++ tests/ci/stress_check.py | 6 ++++-- .../config/config.d/azure_storage_policy_by_default.xml | 5 +++++ tests/config/install.sh | 4 +++- 6 files changed, 26 insertions(+), 6 deletions(-) create mode 100644 tests/config/config.d/azure_storage_policy_by_default.xml diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index 85b1d460833..7f5942d3f74 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -56,6 +56,7 @@ At a minimum, the following information should be added (but add more as needed) - [ ] All with ASAN - [ ] All with TSAN - [ ] All with Analyzer +- [ ] All with Azure storage - [ ] Add your option here #### Exclude tests: @@ -70,7 +71,7 @@ At a minimum, the following information should be added (but add more as needed) - [ ] All with UBSAN - [ ] All with Coverage - [ ] All with Aarch64 -- [ ] Add your option here +- [ ] All with Azure storage #### Extra options: - [ ] do not test (only style check) @@ -80,5 +81,3 @@ At a minimum, the following information should be added (but add more as needed) #### Only specified batches in multi-batch jobs: - [ ] 1 - [ ] 2 -- [ ] 3 -- [ ] 4 diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 8d1dcefcf1b..b208fc9d092 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -107,6 +107,7 @@ class JobNames(metaclass=WithIter): # STATELESS_TEST_S3_RELEASE = "Stateless tests (release, s3 storage)" STATELESS_TEST_S3_DEBUG = "Stateless tests (debug, s3 storage)" STATELESS_TEST_S3_TSAN = "Stateless tests (tsan, s3 storage)" + STATELESS_TEST_AZURE_ASAN = "Stateless tests (azure, asan)" STATELESS_TEST_FLAKY_ASAN = "Stateless tests flaky check (asan)" STATEFUL_TEST_DEBUG = "Stateful tests (debug)" @@ -129,6 +130,7 @@ class JobNames(metaclass=WithIter): STRESS_TEST_UBSAN = "Stress test (ubsan)" STRESS_TEST_MSAN = "Stress test (msan)" STRESS_TEST_DEBUG = "Stress test (debug)" + STRESS_TEST_AZURE_TSAN = "Stress test (azure, tsan)" INTEGRATION_TEST = "Integration tests (release)" INTEGRATION_TEST_ASAN = "Integration tests (asan)" @@ -1193,6 +1195,10 @@ CI_CONFIG = CIConfig( Build.PACKAGE_DEBUG, job_config=JobConfig(num_batches=6, **statless_test_common_params), # type: ignore ), + JobNames.STATELESS_TEST_AZURE_ASAN: TestConfig( + Build.PACKAGE_ASAN, + job_config=JobConfig(num_batches=4, **statless_test_common_params), # type: ignore + ), JobNames.STATELESS_TEST_S3_TSAN: TestConfig( Build.PACKAGE_TSAN, job_config=JobConfig(num_batches=5, **statless_test_common_params), # type: ignore @@ -1215,6 +1221,9 @@ CI_CONFIG = CIConfig( JobNames.UPGRADE_TEST_ASAN: TestConfig( Build.PACKAGE_ASAN, job_config=JobConfig(pr_only=True, random_bucket="upgrade_with_sanitizer", **upgrade_test_common_params) # type: ignore ), + JobNames.STRESS_TEST_TSAN: TestConfig( + Build.PACKAGE_TSAN, job_config=JobConfig(**stress_test_common_params) # type: ignore + ), JobNames.UPGRADE_TEST_TSAN: TestConfig( Build.PACKAGE_TSAN, job_config=JobConfig(pr_only=True, random_bucket="upgrade_with_sanitizer", **upgrade_test_common_params) # type: ignore ), diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index a0c977ea25f..4c7bb52d2a7 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -40,6 +40,9 @@ def get_additional_envs( result.append("RANDOMIZE_OBJECT_KEY_TYPE=1") if "analyzer" in check_name: result.append("USE_OLD_ANALYZER=1") + if "azure" in check_name: + assert "USE_S3_STORAGE_FOR_MERGE_TREE=1" not in result + result.append("USE_AZURE_STORAGE_FOR_MERGE_TREE=1") if run_by_hash_total != 0: result.append(f"RUN_BY_HASH_NUM={run_by_hash_num}") diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index 0f2c0cdd222..c26be7a9d4b 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -18,12 +18,14 @@ from stopwatch import Stopwatch from tee_popen import TeePopen -def get_additional_envs() -> List[str]: +def get_additional_envs(check_name) -> List[str]: result = [] # some cloud-specific features require feature flags enabled # so we need this ENV to be able to disable the randomization # of feature flags result.append("RANDOMIZE_KEEPER_FEATURE_FLAGS=1") + if "azure" in check_name: + result.append("USE_AZURE_STORAGE_FOR_MERGE_TREE=1") return result @@ -143,7 +145,7 @@ def run_stress_test(docker_image_name: str) -> None: pr_info, stopwatch.start_time_str, check_name ) - additional_envs = get_additional_envs() + additional_envs = get_additional_envs(check_name) run_command = get_run_command( packages_path, diff --git a/tests/config/config.d/azure_storage_policy_by_default.xml b/tests/config/config.d/azure_storage_policy_by_default.xml new file mode 100644 index 00000000000..cab8a106f1b --- /dev/null +++ b/tests/config/config.d/azure_storage_policy_by_default.xml @@ -0,0 +1,5 @@ + + + azure_cache + + diff --git a/tests/config/install.sh b/tests/config/install.sh index 06f2f5fe902..ee8b077e1d3 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -150,7 +150,7 @@ if [[ -n "$USE_DATABASE_ORDINARY" ]] && [[ "$USE_DATABASE_ORDINARY" -eq 1 ]]; th ln -sf $SRC_PATH/users.d/database_ordinary.xml $DEST_SERVER_PATH/users.d/ fi -if [[ -n "$USE_S3_STORAGE_FOR_MERGE_TREE" ]] && [[ "$USE_S3_STORAGE_FOR_MERGE_TREE" -eq 1 ]]; then +if [[ "$USE_S3_STORAGE_FOR_MERGE_TREE" == "1" ]]; then object_key_types_options=("generate-suffix" "generate-full-key" "generate-template-key") object_key_type="${object_key_types_options[0]}" @@ -171,6 +171,8 @@ if [[ -n "$USE_S3_STORAGE_FOR_MERGE_TREE" ]] && [[ "$USE_S3_STORAGE_FOR_MERGE_TR ln -sf $SRC_PATH/config.d/s3_storage_policy_by_default.xml $DEST_SERVER_PATH/config.d/ ;; esac +elif [[ "$USE_AZURE_STORAGE_FOR_MERGE_TREE" == "1" ]]; then + ln -sf $SRC_PATH/config.d/azure_storage_policy_by_default.xml $DEST_SERVER_PATH/config.d/ fi ARM="aarch64" From 252d2a05a6f8a6d00e88dfdda2a1bc944b78d9fb Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Thu, 11 Apr 2024 15:15:47 +0000 Subject: [PATCH 0171/1018] fix unittest --- tests/ci/ci_config.py | 2 +- tests/ci/stress_check.py | 2 +- tests/ci/test_ci_options.py | 87 ++++++++++++++++++++++++++++++++++--- 3 files changed, 83 insertions(+), 8 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index b208fc9d092..ef83346d6ba 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -1221,7 +1221,7 @@ CI_CONFIG = CIConfig( JobNames.UPGRADE_TEST_ASAN: TestConfig( Build.PACKAGE_ASAN, job_config=JobConfig(pr_only=True, random_bucket="upgrade_with_sanitizer", **upgrade_test_common_params) # type: ignore ), - JobNames.STRESS_TEST_TSAN: TestConfig( + JobNames.STRESS_TEST_AZURE_TSAN: TestConfig( Build.PACKAGE_TSAN, job_config=JobConfig(**stress_test_common_params) # type: ignore ), JobNames.UPGRADE_TEST_TSAN: TestConfig( diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index c26be7a9d4b..58db4b0db1b 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -18,7 +18,7 @@ from stopwatch import Stopwatch from tee_popen import TeePopen -def get_additional_envs(check_name) -> List[str]: +def get_additional_envs(check_name: str) -> List[str]: result = [] # some cloud-specific features require feature flags enabled # so we need this ENV to be able to disable the randomization diff --git a/tests/ci/test_ci_options.py b/tests/ci/test_ci_options.py index 7c87144a84d..f336b917644 100644 --- a/tests/ci/test_ci_options.py +++ b/tests/ci/test_ci_options.py @@ -4,7 +4,6 @@ import unittest from ci import CiOptions -from ci_config import JobNames _TEST_BODY_1 = """ #### Run only: @@ -44,6 +43,85 @@ _TEST_BODY_3 = """ - [x] Must include all tests for analyzer """ +_TEST_JOB_LIST = [ + "Style check", + "Fast test", + "package_release", + "package_asan", + "Docker server image", + "Docker keeper image", + "Install packages (amd64)", + "Install packages (arm64)", + "Stateless tests (debug)", + "Stateless tests (release)", + "Stateless tests (coverage)", + "Stateless tests (aarch64)", + "Stateless tests (asan)", + "Stateless tests (tsan)", + "Stateless tests (msan)", + "Stateless tests (ubsan)", + "Stateless tests (release, old analyzer, s3, DatabaseReplicated)", + "Stateless tests (debug, s3 storage)", + "Stateless tests (tsan, s3 storage)", + "Stateless tests flaky check (asan)", + "Stateful tests (debug)", + "Stateful tests (release)", + "Stateful tests (coverage)", + "Stateful tests (aarch64)", + "Stateful tests (asan)", + "Stateful tests (tsan)", + "Stateful tests (msan)", + "Stateful tests (ubsan)", + "Stateful tests (release, ParallelReplicas)", + "Stateful tests (debug, ParallelReplicas)", + "Stateful tests (asan, ParallelReplicas)", + "Stateful tests (msan, ParallelReplicas)", + "Stateful tests (ubsan, ParallelReplicas)", + "Stateful tests (tsan, ParallelReplicas)", + "Stress test (asan)", + "Stress test (tsan)", + "Stress test (ubsan)", + "Stress test (msan)", + "Stress test (debug)", + "Integration tests (release)", + "Integration tests (asan)", + "Integration tests (asan, old analyzer)", + "Integration tests (tsan)", + "Integration tests (aarch64)", + "Integration tests flaky check (asan)", + "Upgrade check (debug)", + "Upgrade check (asan)", + "Upgrade check (tsan)", + "Upgrade check (msan)", + "Unit tests (release)", + "Unit tests (asan)", + "Unit tests (msan)", + "Unit tests (tsan)", + "Unit tests (ubsan)", + "AST fuzzer (debug)", + "AST fuzzer (asan)", + "AST fuzzer (msan)", + "AST fuzzer (tsan)", + "AST fuzzer (ubsan)", + "ClickHouse Keeper Jepsen", + "ClickHouse Server Jepsen", + "Performance Comparison", + "Performance Comparison Aarch64", + "Sqllogic test (release)", + "SQLancer (release)", + "SQLancer (debug)", + "SQLTest", + "Compatibility check (amd64)", + "Compatibility check (aarch64)", + "ClickBench (amd64)", + "ClickBench (aarch64)", + "libFuzzer tests", + "ClickHouse build check", + "ClickHouse special build check", + "Docs check", + "Bugfix validation", +] + class TestCIOptions(unittest.TestCase): def test_pr_body_parsing(self): @@ -69,7 +147,7 @@ class TestCIOptions(unittest.TestCase): ci_options.exclude_keywords, ["tsan", "aarch64", "analyzer", "s3_storage", "coverage"], ) - jobs_to_do = list(JobNames) + jobs_to_do = list(_TEST_JOB_LIST) jobs_to_skip = [] job_params = {} jobs_to_do, jobs_to_skip, job_params = ci_options.apply( @@ -81,9 +159,6 @@ class TestCIOptions(unittest.TestCase): "Style check", "package_release", "package_asan", - "package_ubsan", - "package_debug", - "package_msan", "Stateless tests (asan)", "Stateless tests flaky check (asan)", "Stateless tests (msan)", @@ -103,7 +178,7 @@ class TestCIOptions(unittest.TestCase): ) self.assertCountEqual(ci_options.include_keywords, ["analyzer"]) self.assertIsNone(ci_options.exclude_keywords) - jobs_to_do = list(JobNames) + jobs_to_do = list(_TEST_JOB_LIST) jobs_to_skip = [] job_params = {} jobs_to_do, jobs_to_skip, job_params = ci_options.apply( From 053c3cc0c1de1dbb3555dec40258eb5367afbbfc Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Fri, 12 Apr 2024 10:22:03 +0000 Subject: [PATCH 0172/1018] redirect azure logs --- docker/test/stateful/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index c2e9fdfe41d..dabd4d09228 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -19,7 +19,7 @@ ln -s /usr/share/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test # install test configs /usr/share/clickhouse-test/config/install.sh -azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --debug /azurite_log & +azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --debug /azurite_log >> /test_output/garbage.log 2>&1 & ./setup_minio.sh stateful config_logs_export_cluster /etc/clickhouse-server/config.d/system_logs_export.yaml From e2acd0e93f2a5819bdb22a28b87640935bcee308 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Fri, 12 Apr 2024 10:18:43 +0000 Subject: [PATCH 0173/1018] CI: add FT to MQ remove Style from master --- .github/workflows/master.yml | 21 ++++++++-------- .github/workflows/pull_request.yml | 4 +-- tests/ci/ci.py | 39 ++++++++++++++---------------- tests/ci/sync_pr.py | 2 ++ 4 files changed, 33 insertions(+), 33 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 9a719a205d4..384141b78e7 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -59,16 +59,17 @@ jobs: uses: ./.github/workflows/reusable_docker.yml with: data: ${{ needs.RunConfig.outputs.data }} - StyleCheck: - needs: [RunConfig, BuildDockers] - if: ${{ !failure() && !cancelled() }} - uses: ./.github/workflows/reusable_test.yml - with: - test_name: Style check - runner_type: style-checker - data: ${{ needs.RunConfig.outputs.data }} - run_command: | - python3 style_check.py --no-push + # Tested in MQ + # StyleCheck: + # needs: [RunConfig, BuildDockers] + # if: ${{ !failure() && !cancelled() }} + # uses: ./.github/workflows/reusable_test.yml + # with: + # test_name: Style check + # runner_type: style-checker + # data: ${{ needs.RunConfig.outputs.data }} + # run_command: | + # python3 style_check.py --no-push CompatibilityCheckX86: needs: [RunConfig, BuilderDebRelease] if: ${{ !failure() && !cancelled() }} diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 0eac9a9a722..c2e76de5e14 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -83,7 +83,7 @@ jobs: ${{secrets.ROBOT_CLICKHOUSE_SSH_KEY}} RCSK FastTest: - needs: [RunConfig, StyleCheck] + needs: [RunConfig, BuildDockers] if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'Fast test') }} uses: ./.github/workflows/reusable_test.yml with: @@ -97,7 +97,7 @@ jobs: # for main CI chain # Builds_1: - needs: [RunConfig, FastTest] + needs: [RunConfig, StyleCheck, FastTest] if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).stages_data.stages_to_do, 'Builds_1') }} # using callable wf (reusable_stage.yml) allows to group all nested jobs under a tab uses: ./.github/workflows/reusable_build_stage.yml diff --git a/tests/ci/ci.py b/tests/ci/ci.py index f11d62e9136..bd6117dc858 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -1261,18 +1261,6 @@ def _print_results(result: Any, outfile: Optional[str], pretty: bool = False) -> raise AssertionError(f"Unexpected type for 'res': {type(result)}") -def _update_config_for_docs_only(jobs_data: dict) -> None: - DOCS_CHECK_JOBS = [JobNames.DOCS_CHECK, JobNames.STYLE_CHECK] - print(f"NOTE: Will keep only docs related jobs: [{DOCS_CHECK_JOBS}]") - jobs_to_do = jobs_data.get("jobs_to_do", []) - jobs_data["jobs_to_do"] = [job for job in jobs_to_do if job in DOCS_CHECK_JOBS] - jobs_data["jobs_to_wait"] = { - job: params - for job, params in jobs_data["jobs_to_wait"].items() - if job in DOCS_CHECK_JOBS - } - - def _configure_docker_jobs(docker_digest_or_latest: bool) -> Dict: print("::group::Docker images check") # generate docker jobs data @@ -1332,8 +1320,20 @@ def _configure_jobs( jobs_to_skip: List[str] = [] digests: Dict[str, str] = {} + # FIXME: find better place for these config variables + DOCS_CHECK_JOBS = [JobNames.DOCS_CHECK, JobNames.STYLE_CHECK] + MQ_JOBS = [JobNames.STYLE_CHECK, JobNames.FAST_TEST] + if pr_info.has_changes_in_documentation_only(): + print(f"WARNING: Only docs are changed - will run only [{DOCS_CHECK_JOBS}]") + if pr_info.is_merge_queue(): + print(f"WARNING: It's a MQ run - will run only [{MQ_JOBS}]") + print("::group::Job Digests") for job in CI_CONFIG.job_generator(pr_info.head_ref if CI else "dummy_branch_name"): + if pr_info.is_merge_queue() and job not in MQ_JOBS: + continue + if pr_info.has_changes_in_documentation_only() and job not in DOCS_CHECK_JOBS: + continue digest = job_digester.get_job_digest(CI_CONFIG.get_digest_config(job)) digests[job] = digest print(f" job [{job.rjust(50)}] has digest [{digest}]") @@ -1436,10 +1436,11 @@ def _configure_jobs( ) if pr_info.is_merge_queue(): - # FIXME: Quick support for MQ workflow which is only StyleCheck for now - jobs_to_do = [JobNames.STYLE_CHECK] - jobs_to_skip = [] - print(f"NOTE: This is Merge Queue CI: set jobs to do: [{jobs_to_do}]") + # no need to run pending job in MQ, since it's pending - it's not affected by current checnge + for job_to_wait in jobs_to_wait: + if job_to_wait in jobs_to_do: + print(f"Remove pending job [{job_to_wait}] from MQ workflow") + jobs_to_do.remove(job_to_wait) return { "digests": digests, @@ -1902,9 +1903,6 @@ def main() -> int: else {} ) - if not args.skip_jobs and pr_info.has_changes_in_documentation_only(): - _update_config_for_docs_only(jobs_data) - if not args.skip_jobs: ci_cache = CiCache(s3, jobs_data["digests"]) @@ -1928,8 +1926,7 @@ def main() -> int: jobs_to_skip.append(job) del jobs_params[job] - # set planned jobs as pending in the CI cache if on the master - if pr_info.is_master(): + # set planned jobs as pending in the CI cache if on the master for job in jobs_data["jobs_to_do"]: config = CI_CONFIG.get_job_config(job) if config.run_always or config.run_by_label: diff --git a/tests/ci/sync_pr.py b/tests/ci/sync_pr.py index acff7ba541b..639436ab3ac 100644 --- a/tests/ci/sync_pr.py +++ b/tests/ci/sync_pr.py @@ -54,10 +54,12 @@ def merge_sync_pr(gh, sync_pr): def set_sync_status(gh, pr_info, sync_pr): if not sync_pr or not sync_pr.mergeable: + print("Sync PR is not mergeable") post_commit_status( get_commit(gh, pr_info.sha), FAILURE, "", "Sync PR failure", "A Sync" ) else: + print("Sync PR is mergeable") post_commit_status(get_commit(gh, pr_info.sha), SUCCESS, "", "", "A Sync") From 830598c83e09f4eeb514e0c5300cb520f365b9e9 Mon Sep 17 00:00:00 2001 From: peter279k Date: Fri, 12 Apr 2024 19:25:24 +0800 Subject: [PATCH 0174/1018] Add functions tryBase58 and tryBase64Decode usage --- .../functions/string-functions.md | 52 +++++++++++++++++++ 1 file changed, 52 insertions(+) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index f4d66730011..87950d65cd5 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -867,6 +867,34 @@ Result: Like `base58Decode` but returns an empty string in case of error. +**Syntax** + +```sql +tryBase58Decode(encoded) +``` + +**Parameters** + +- `encoded`: [String](../../sql-reference/data-types/string.md) column or constant. If the string is not a valid Base58-encoded value, returns an empty string in case of error. + +**Returned value** + +- A string containing the decoded value of the argument. + +**Examples** + +Query: + +```sql +SELECT tryBase58Decode('3dc8KtHrwM') as res; +``` + +```response +┌─res─────┐ +│ Encoded │ +└─────────┘ +``` + ## base64Encode Encodes a String or FixedString as base64. @@ -883,6 +911,30 @@ Alias: `FROM_BASE64`. Like `base64Decode` but returns an empty string in case of error. +**Syntax** + +```sql +tryBase64Decode(encoded) +``` + +**Parameters** + +- `encoded`: [String](../../sql-reference/data-types/string.md) column or constant. If the string is not a valid Base58-encoded value, returns an empty string in case of error. + +**Examples** + +Query: + +```sql +SELECT tryBase64Decode('RW5jb2RlZA==') as res; +``` + +```response +┌─res─────┐ +│ Encoded │ +└─────────┘ +``` + ## endsWith {#endswith} Returns whether string `str` ends with `suffix`. From 669e3b44b0041db102c010bec2a69b8a1df269c3 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 12 Apr 2024 15:59:19 +0000 Subject: [PATCH 0175/1018] Fix: msan in UUIDStringToNum --- src/Functions/FunctionsCodingUUID.cpp | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/src/Functions/FunctionsCodingUUID.cpp b/src/Functions/FunctionsCodingUUID.cpp index dd9170e44ad..f88ec255787 100644 --- a/src/Functions/FunctionsCodingUUID.cpp +++ b/src/Functions/FunctionsCodingUUID.cpp @@ -128,10 +128,14 @@ UUIDSerializer::Variant parseVariant(const DB::ColumnsWithTypeAndName & argument if (arguments.size() < 2) return UUIDSerializer::Variant::Default; - const auto representation = static_cast>(arguments[1].column->getInt(0)); - const auto as_enum = magic_enum::enum_cast(representation); + Int64 value = static_cast(UUIDSerializer::Variant::Default); + if (!arguments[1].column->empty()) + value = arguments[1].column->getInt(0); + + const auto as_enum + = magic_enum::enum_cast(static_cast(value)); if (!as_enum) - throw DB::Exception(DB::ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Expected UUID variant, got {}", representation); + throw DB::Exception(DB::ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Expected UUID variant, got {}", value); return *as_enum; } From 1f1e1b4bc91c0c7c4f1f3e0d73530592b4d1e0bd Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 12 Apr 2024 16:20:57 +0000 Subject: [PATCH 0176/1018] Test --- .../queries/0_stateless/03095_msan_uuid_string_to_num.reference | 0 tests/queries/0_stateless/03095_msan_uuid_string_to_num.sql | 1 + 2 files changed, 1 insertion(+) create mode 100644 tests/queries/0_stateless/03095_msan_uuid_string_to_num.reference create mode 100644 tests/queries/0_stateless/03095_msan_uuid_string_to_num.sql diff --git a/tests/queries/0_stateless/03095_msan_uuid_string_to_num.reference b/tests/queries/0_stateless/03095_msan_uuid_string_to_num.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03095_msan_uuid_string_to_num.sql b/tests/queries/0_stateless/03095_msan_uuid_string_to_num.sql new file mode 100644 index 00000000000..73ba5589681 --- /dev/null +++ b/tests/queries/0_stateless/03095_msan_uuid_string_to_num.sql @@ -0,0 +1 @@ +SELECT UUIDStringToNum('00112233-4455-6677-8899-aabbccddeeff', materialize(2)) -- { serverError ILLEGAL_COLUMN } From c8f014591c71b1b3f48c37d98f6ef76edfc06c4a Mon Sep 17 00:00:00 2001 From: Blargian Date: Sat, 13 Apr 2024 11:05:43 +0200 Subject: [PATCH 0177/1018] Update countMatchesCaseInsensitive function --- .../functions/string-search-functions.md | 35 ++++++++++++++++++- 1 file changed, 34 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index df266b224fb..2d47c217fab 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -1421,7 +1421,40 @@ Result: ## countMatchesCaseInsensitive -Like `countMatches(haystack, pattern)` but matching ignores the case. +Like [`countMatches`](#countmatches) but matching ignores the case. + +**Syntax** + +``` sql +countMatchesCaseInsensitive(haystack, pattern) +``` + +**Arguments** + +- `haystack` — The string to search in. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `pattern` — The regular expression with [re2 syntax](https://github.com/google/re2/wiki/Syntax). [String](../../sql-reference/data-types/string.md). + +**Returned value** + +- The number of matches. + +Type: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Examples** + +Query: + +``` sql +SELECT countMatchesCaseInsensitive('AAAA', 'aa'); +``` + +Result: + +``` text +┌─countMatchesCaseInsensitive('AAAA', 'aa')────┐ +│ 2 │ +└──────────────────────────────────────────────┘ +``` ## regexpExtract From f18cdaa004559512284e15b8b646b4d7bd7e7320 Mon Sep 17 00:00:00 2001 From: Blargian Date: Sat, 13 Apr 2024 11:25:08 +0200 Subject: [PATCH 0178/1018] Add variants --- .../functions/string-search-functions.md | 103 +++++++++++++++++- 1 file changed, 101 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index 2d47c217fab..14e484bfc81 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -1322,9 +1322,9 @@ Result: ## countSubstrings -Returns how often substring `needle` occurs in string `haystack`. +Returns how often a substring `needle` occurs in string `haystack`. -Functions `countSubstringsCaseInsensitive` and `countSubstringsCaseInsensitiveUTF8` provide a case-insensitive and case-insensitive + UTF-8 variants of this function. +Functions [`countSubstringsCaseInsensitive`](#countsubstringscaseinsensitive) and [`countSubstringsCaseInsensitiveUTF8`](#countsubstringscaseinsensitiveutf8) provide case-insensitive and case-insensitive + UTF-8 variants of this function respectively. **Syntax** @@ -1371,6 +1371,105 @@ Result: │ 1 │ └────────────────────────────────────────┘ ``` +## countSubstringsCaseInsensitive + +Returns how often a substring `needle` occurs in string `haystack`. Ignores case. + +**Syntax** + +``` sql +countSubstringsCaseInsensitive(haystack, needle[, start_pos]) +``` + +**Arguments** + +- `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `needle` — Substring to be searched. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `start_pos` – Position (1-based) in `haystack` at which the search starts. [UInt](../../sql-reference/data-types/int-uint.md). Optional. + +**Returned values** + +- The number of occurrences. + +Type: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Examples** + +``` sql +SELECT countSubstringsCaseInsensitive('AAAA', 'aa'); +``` + +Result: + +``` text +┌─countSubstringsCaseInsensitive('AAAA', 'aa')─┐ +│ 2 │ +└──────────────────────────────────────────────┘ +``` + +Example with `start_pos` argument: + +```sql +SELECT countSubstringsCaseInsensitive('abc___ABC___abc', 'abc', 4); +``` + +Result: + +``` text +┌─countSubstringsCaseInsensitive('abc___ABC___abc', 'abc', 4)─┐ +│ 2 │ +└─────────────────────────────────────────────────────────────┘ +``` + +## countSubstringsCaseInsensitiveUTF8 + +Returns how often a substring `needle` occurs in string `haystack`. Ignores case and assumes that `haystack` is a UTF8 string. + +**Syntax** + +``` sql +countSubstringsCaseInsensitiveUTF8(haystack, needle[, start_pos]) +``` + +**Arguments** + +- `haystack` — UTF-8 string in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `needle` — Substring to be searched. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `start_pos` – Position (1-based) in `haystack` at which the search starts. [UInt](../../sql-reference/data-types/int-uint.md). Optional. + +**Returned values** + +- The number of occurrences. + +Type: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Examples** + +``` sql +SELECT countSubstringsCaseInsensitiveUTF8('ложка, кошка, картошка', 'КА'); +``` + +Result: + +``` text +┌─countSubstringsCaseInsensitiveUTF8('ложка, кошка, картошка', 'КА')─┐ +│ 4 │ +└────────────────────────────────────────────────────────────────────┘ +``` + +Example with `start_pos` argument: + +```sql +SELECT countSubstringsCaseInsensitiveUTF8('ложка, кошка, картошка', 'КА', 13); +``` + +Result: + +``` text +┌─countSubstringsCaseInsensitiveUTF8('ложка, кошка, картошка', 'КА', 13)─┐ +│ 2 │ +└────────────────────────────────────────────────────────────────────────┘ +``` ## countMatches From 13f84a59574c5d3aa1fa044dc29681cb3fd0820e Mon Sep 17 00:00:00 2001 From: Blargian Date: Sat, 13 Apr 2024 11:32:40 +0200 Subject: [PATCH 0179/1018] minor fixes --- .../functions/string-search-functions.md | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index 14e484bfc81..9720bb49ec0 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -1322,7 +1322,7 @@ Result: ## countSubstrings -Returns how often a substring `needle` occurs in string `haystack`. +Returns how often a substring `needle` occurs in a string `haystack`. Functions [`countSubstringsCaseInsensitive`](#countsubstringscaseinsensitive) and [`countSubstringsCaseInsensitiveUTF8`](#countsubstringscaseinsensitiveutf8) provide case-insensitive and case-insensitive + UTF-8 variants of this function respectively. @@ -1373,7 +1373,7 @@ Result: ``` ## countSubstringsCaseInsensitive -Returns how often a substring `needle` occurs in string `haystack`. Ignores case. +Returns how often a substring `needle` occurs in a string `haystack`. Ignores case. **Syntax** @@ -1395,6 +1395,8 @@ Type: [UInt64](../../sql-reference/data-types/int-uint.md). **Examples** +Query: + ``` sql SELECT countSubstringsCaseInsensitive('AAAA', 'aa'); ``` @@ -1409,6 +1411,8 @@ Result: Example with `start_pos` argument: +Query: + ```sql SELECT countSubstringsCaseInsensitive('abc___ABC___abc', 'abc', 4); ``` @@ -1423,7 +1427,7 @@ Result: ## countSubstringsCaseInsensitiveUTF8 -Returns how often a substring `needle` occurs in string `haystack`. Ignores case and assumes that `haystack` is a UTF8 string. +Returns how often a substring `needle` occurs in a string `haystack`. Ignores case and assumes that `haystack` is a UTF8 string. **Syntax** @@ -1445,6 +1449,8 @@ Type: [UInt64](../../sql-reference/data-types/int-uint.md). **Examples** +Query: + ``` sql SELECT countSubstringsCaseInsensitiveUTF8('ложка, кошка, картошка', 'КА'); ``` @@ -1459,6 +1465,8 @@ Result: Example with `start_pos` argument: +Query: + ```sql SELECT countSubstringsCaseInsensitiveUTF8('ложка, кошка, картошка', 'КА', 13); ``` From 3ed8463d08ad35841f47edf5b3efe59f5feba0fd Mon Sep 17 00:00:00 2001 From: Blargian Date: Sat, 13 Apr 2024 11:34:57 +0200 Subject: [PATCH 0180/1018] Give explicit description to countMatchesCaseInsensitive in addition to a link to its base function --- docs/en/sql-reference/functions/string-search-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index 9720bb49ec0..9738c19bf3c 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -1528,7 +1528,7 @@ Result: ## countMatchesCaseInsensitive -Like [`countMatches`](#countmatches) but matching ignores the case. +Returns the number of regular expression matches for a pattern in a haystack like [`countMatches`](#countmatches) but matching ignores the case. **Syntax** From 12d4682a4777b790ecf3c521d03fd146e86cc697 Mon Sep 17 00:00:00 2001 From: Blargian Date: Sat, 13 Apr 2024 11:38:35 +0200 Subject: [PATCH 0181/1018] Add variants to aspell-ignore --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index c65a593c632..ca67cef4711 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1371,6 +1371,8 @@ countEqual countMatches countMatchesCaseInsensitive countSubstrings +countSubstringsCaseInsensitive +countSubstringsCaseInsensitiveUTF8 covarPop covarSamp covariates From 9d87cec8433442af9f560c3d1d72f0d3e01e0c4f Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Sun, 14 Apr 2024 17:38:01 +0200 Subject: [PATCH 0182/1018] Don't use ASTPtr for the comment following feedback in the PR --- src/Backups/DDLAdjustingForBackupVisitor.cpp | 2 +- src/Databases/DatabaseFilesystem.cpp | 2 +- src/Databases/DatabaseHDFS.cpp | 2 +- src/Databases/DatabaseMemory.cpp | 2 +- src/Databases/DatabaseOnDisk.cpp | 5 +++-- src/Databases/DatabaseS3.cpp | 2 +- src/Databases/DatabasesCommon.cpp | 4 ++-- src/Databases/MySQL/DatabaseMySQL.cpp | 2 +- src/Databases/PostgreSQL/DatabasePostgreSQL.cpp | 2 +- src/Databases/SQLite/DatabaseSQLite.cpp | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- src/Interpreters/SystemLog.cpp | 2 +- src/Parsers/ASTCreateQuery.cpp | 2 +- src/Parsers/ASTCreateQuery.h | 2 +- src/Parsers/ParserCreateQuery.cpp | 12 ++++++------ 15 files changed, 23 insertions(+), 22 deletions(-) diff --git a/src/Backups/DDLAdjustingForBackupVisitor.cpp b/src/Backups/DDLAdjustingForBackupVisitor.cpp index 89c24ac17b7..5ea91094b75 100644 --- a/src/Backups/DDLAdjustingForBackupVisitor.cpp +++ b/src/Backups/DDLAdjustingForBackupVisitor.cpp @@ -20,7 +20,7 @@ namespace /// If this is a definition of a system table we'll remove columns and comment because they're redundant for backups. auto & create = data.create_query->as(); create.reset(create.columns_list); - create.comment.reset(); + create.reset(create.comment); } void visitStorageReplicatedTableEngine(ASTStorage & storage, const DDLAdjustingForBackupVisitor::Data & data) diff --git a/src/Databases/DatabaseFilesystem.cpp b/src/Databases/DatabaseFilesystem.cpp index 61d19f34422..b27a816a60d 100644 --- a/src/Databases/DatabaseFilesystem.cpp +++ b/src/Databases/DatabaseFilesystem.cpp @@ -192,7 +192,7 @@ ASTPtr DatabaseFilesystem::getCreateDatabaseQuery() const if (const auto database_comment = getDatabaseComment(); !database_comment.empty()) { auto & ast_create_query = ast->as(); - ast_create_query.comment = std::make_shared(database_comment); + ast_create_query.set(ast_create_query.comment, std::make_shared(database_comment)); } return ast; diff --git a/src/Databases/DatabaseHDFS.cpp b/src/Databases/DatabaseHDFS.cpp index 8a7f0c7cb60..1de7f80f512 100644 --- a/src/Databases/DatabaseHDFS.cpp +++ b/src/Databases/DatabaseHDFS.cpp @@ -188,7 +188,7 @@ ASTPtr DatabaseHDFS::getCreateDatabaseQuery() const if (const auto database_comment = getDatabaseComment(); !database_comment.empty()) { auto & ast_create_query = ast->as(); - ast_create_query.comment = std::make_shared(database_comment); + ast_create_query.set(ast_create_query.comment, std::make_shared(database_comment)); } return ast; diff --git a/src/Databases/DatabaseMemory.cpp b/src/Databases/DatabaseMemory.cpp index 5b4e0fc3daf..4ff7b3c7f2b 100644 --- a/src/Databases/DatabaseMemory.cpp +++ b/src/Databases/DatabaseMemory.cpp @@ -107,7 +107,7 @@ ASTPtr DatabaseMemory::getCreateDatabaseQuery() const create_query->storage->set(create_query->storage->engine, engine); if (const auto comment_value = getDatabaseComment(); !comment_value.empty()) - create_query->comment = std::make_shared(comment_value); + create_query->set(create_query->comment, std::make_shared(comment_value)); return create_query; } diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 7836775a905..674e9afa8ac 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -535,7 +535,7 @@ ASTPtr DatabaseOnDisk::getCreateDatabaseQuery() const if (const auto database_comment = getDatabaseComment(); !database_comment.empty()) { auto & ast_create_query = ast->as(); - ast_create_query.comment = std::make_shared(database_comment); + ast_create_query.set(ast_create_query.comment, std::make_shared(database_comment)); } return ast; @@ -784,7 +784,8 @@ ASTPtr DatabaseOnDisk::getCreateQueryFromStorage(const String & table_name, cons static_cast(settings.max_parser_backtracks), throw_on_error); - create_table_query->as()->comment = std::make_shared("SYSTEM TABLE is built on the fly."); + create_table_query->set(create_table_query->as()->comment, + std::make_shared("SYSTEM TABLE is built on the fly.")); return create_table_query; } diff --git a/src/Databases/DatabaseS3.cpp b/src/Databases/DatabaseS3.cpp index dbadddb9aa6..1589cc1c75d 100644 --- a/src/Databases/DatabaseS3.cpp +++ b/src/Databases/DatabaseS3.cpp @@ -196,7 +196,7 @@ ASTPtr DatabaseS3::getCreateDatabaseQuery() const if (const auto database_comment = getDatabaseComment(); !database_comment.empty()) { auto & ast_create_query = ast->as(); - ast_create_query.comment = std::make_shared(database_comment); + ast_create_query.set(ast_create_query.comment, std::make_shared(database_comment)); } return ast; diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index 9fbe4de6656..fc75f8e44b9 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -114,9 +114,9 @@ void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemo } if (metadata.comment.empty()) - ast_create_query.comment.reset(); + ast_create_query.reset(ast_create_query.comment); else - ast_create_query.comment = std::make_shared(metadata.comment); + ast_create_query.set(ast_create_query.comment, std::make_shared(metadata.comment)); } diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index 9fa7abaf2ec..b2e199735db 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -204,7 +204,7 @@ ASTPtr DatabaseMySQL::getCreateDatabaseQuery() const create_query->set(create_query->storage, database_engine_define); if (const auto comment_value = getDatabaseComment(); !comment_value.empty()) - create_query->comment = std::make_shared(comment_value); + create_query->set(create_query->comment, std::make_shared(comment_value)); return create_query; } diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 727c37a92c4..3f62b9719d2 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -391,7 +391,7 @@ ASTPtr DatabasePostgreSQL::getCreateDatabaseQuery() const create_query->set(create_query->storage, database_engine_define); if (const auto comment_value = getDatabaseComment(); !comment_value.empty()) - create_query->comment = std::make_shared(comment_value); + create_query->set(create_query->comment, std::make_shared(comment_value)); return create_query; } diff --git a/src/Databases/SQLite/DatabaseSQLite.cpp b/src/Databases/SQLite/DatabaseSQLite.cpp index e2040364c8e..e758ea35de5 100644 --- a/src/Databases/SQLite/DatabaseSQLite.cpp +++ b/src/Databases/SQLite/DatabaseSQLite.cpp @@ -166,7 +166,7 @@ ASTPtr DatabaseSQLite::getCreateDatabaseQuery() const create_query->set(create_query->storage, database_engine_define); if (const auto comment_value = getDatabaseComment(); !comment_value.empty()) - create_query->comment = std::make_shared(comment_value); + create_query->set(create_query->comment, std::make_shared(comment_value)); return create_query; } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 6a39f7dfa75..1a376836ee5 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -786,7 +786,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti properties.columns = as_storage_metadata->getColumns(); if (!create.comment && !as_storage_metadata->comment.empty()) - create.comment = std::make_shared(Field(as_storage_metadata->comment)); + create.set(create.comment, std::make_shared(as_storage_metadata->comment)); /// Secondary indices and projections make sense only for MergeTree family of storage engines. /// We should not copy them for other storages. diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index c51956218ae..3af8761ff8e 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -656,7 +656,7 @@ ASTPtr SystemLog::getCreateTableQuery() StorageWithComment & storage_with_comment = storage_with_comment_ast->as(); create->set(create->storage, storage_with_comment.storage); - create->comment = storage_with_comment.comment; + create->set(create->comment, storage_with_comment.comment); /// Write additional (default) settings for MergeTree engine to make it make it possible to compare ASTs /// and recreate tables on settings changes. diff --git a/src/Parsers/ASTCreateQuery.cpp b/src/Parsers/ASTCreateQuery.cpp index b977c441de0..0403dc33164 100644 --- a/src/Parsers/ASTCreateQuery.cpp +++ b/src/Parsers/ASTCreateQuery.cpp @@ -255,7 +255,7 @@ ASTPtr ASTCreateQuery::clone() const if (as_table_function) res->set(res->as_table_function, as_table_function->clone()); if (comment) - res->comment = comment->clone(); + res->set(res->comment, comment->clone()); cloneOutputOptions(*res); cloneTableOptions(*res); diff --git a/src/Parsers/ASTCreateQuery.h b/src/Parsers/ASTCreateQuery.h index fe7477f0e5f..64e6bc8ce48 100644 --- a/src/Parsers/ASTCreateQuery.h +++ b/src/Parsers/ASTCreateQuery.h @@ -112,7 +112,7 @@ public: String as_table; IAST * as_table_function = nullptr; ASTSelectWithUnionQuery * select = nullptr; - ASTPtr comment; + IAST * comment = nullptr; ASTPtr sql_security = nullptr; ASTTableOverrideList * table_overrides = nullptr; /// For CREATE DATABASE with engines that automatically create tables diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 22480c717e2..ff88b58760b 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -859,7 +859,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe query->set(query->as_table_function, as_table_function); if (comment) - query->comment = comment; + query->set(query->comment, comment); if (query->columns_list && query->columns_list->primary_key) { @@ -1010,7 +1010,8 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e query->set(query->select, select); if (comment) - query->comment = comment; + query->set(query->comment, comment); + if (sql_security) query->sql_security = typeid_cast>(sql_security); @@ -1411,8 +1412,7 @@ bool ParserCreateDatabaseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e query->set(query->storage, storage); if (comment) - query->comment = comment; - + query->set(query->comment, comment); if (table_overrides && !table_overrides->children.empty()) query->set(query->table_overrides, table_overrides); @@ -1615,7 +1615,7 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (refresh_strategy) query->set(query->refresh_strategy, refresh_strategy); if (comment) - query->comment = comment; + query->set(query->comment, comment); if (sql_security) query->sql_security = typeid_cast>(sql_security); @@ -1792,7 +1792,7 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E query->cluster = cluster_str; if (comment) - query->comment = comment; + query->set(query->comment, comment); return true; } From 210efe162113d74f3894bacd8e6a4b4a5fce975c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Pawe=C5=82=20Kudzia?= <30438339+pakud@users.noreply.github.com> Date: Mon, 15 Apr 2024 08:50:01 +0200 Subject: [PATCH 0183/1018] documenting alternative syntax that's supported for DATE_ADD, DATE_SUB this is related to issue https://github.com/ClickHouse/ClickHouse/issues/25815 i'm adding examples for: date_add(date, INTERVAL value unit) and date_sub(date, INTERVAL value unit) both are covered with a test from https://github.com/ClickHouse/ClickHouse/blob/e3c09e97034b7d26b2280487fa8422a8037caa51/tests/queries/0_stateless/02160_special_functions.sql#L33 ; i think it's worth documenting them as the syntax is familiar to MySQL users [ https://dev.mysql.com/doc/refman/8.0/en/date-and-time-functions.html#function_date-add ] --- .../functions/date-time-functions.md | 40 +++++++++++++++++++ 1 file changed, 40 insertions(+) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 3bb9d4e7dbe..7c790e739e0 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1902,6 +1902,12 @@ If the addition results in a value outside the bounds of the data type, the resu date_add(unit, value, date) ``` +Alternative syntax: + +``` sql +date_add(date, INTERVAL value unit) +``` + Aliases: `dateAdd`, `DATE_ADD`. **Arguments** @@ -1941,6 +1947,20 @@ Result: └───────────────────────────────────────────────┘ ``` +```sql +SELECT date_add(toDate('2018-01-01'), INTERVAL 3 YEAR); +``` + +Result: + +```text +┌─plus(toDate('2018-01-01'), toIntervalYear(3))─┐ +│ 2021-01-01 │ +└───────────────────────────────────────────────┘ +``` + + + **See Also** - [addDate](#addDate) @@ -1957,6 +1977,13 @@ If the subtraction results in a value outside the bounds of the data type, the r date_sub(unit, value, date) ``` +Alternative syntax: + +``` sql +date_sub(date, INTERVAL value unit) +``` + + Aliases: `dateSub`, `DATE_SUB`. **Arguments** @@ -1997,6 +2024,19 @@ Result: └────────────────────────────────────────────────┘ ``` +``` sql +SELECT date_sub(toDate('2018-01-01'), INTERVAL 3 YEAR); +``` + +Result: + +``` text +┌─minus(toDate('2018-01-01'), toIntervalYear(3))─┐ +│ 2015-01-01 │ +└────────────────────────────────────────────────┘ +``` + + **See Also** - [subDate](#subDate) From 5f1d85a92e3627caedbaaeb8dd58715d802a807c Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 15 Apr 2024 10:11:39 +0100 Subject: [PATCH 0184/1018] impl --- tests/queries/0_stateless/02366_kql_summarize.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02366_kql_summarize.sql b/tests/queries/0_stateless/02366_kql_summarize.sql index 21a1b643d98..861811711f0 100644 --- a/tests/queries/0_stateless/02366_kql_summarize.sql +++ b/tests/queries/0_stateless/02366_kql_summarize.sql @@ -89,9 +89,9 @@ print '-- Summarize following sort --'; Customers | sort by FirstName | summarize count() by Occupation | sort by Occupation; print '-- summarize with bin --'; -EventLog | summarize count=count() by bin(Created, 1000); -EventLog | summarize count=count() by bin(unixtime_seconds_todatetime(Created/1000), 1s); -EventLog | summarize count=count() by time_label=bin(Created/1000, 1s); +EventLog | summarize count=count() by bin(Created, 1000) | sort by Created asc; +EventLog | summarize count=count() by bin(unixtime_seconds_todatetime(Created/1000), 1s) | sort by Columns1 asc; +EventLog | summarize count=count() by time_label=bin(Created/1000, 1s) | sort by time_label asc; Dates | project bin(datetime(EventTime), 1m); print '-- make_list_with_nulls --'; Customers | summarize t = make_list_with_nulls(FirstName); From 32ed192434f61ec6009d34852ed54113651bb121 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 28 Mar 2024 14:38:28 +0100 Subject: [PATCH 0185/1018] Fix incorrect sharding key types for optimize_skip_unused_shards_rewrite_in Before it always uses the first column from sample block, while it is not guaranteed. Signed-off-by: Azat Khuzhin --- src/Interpreters/ClusterProxy/executeQuery.cpp | 2 -- .../OptimizeShardingKeyRewriteInVisitor.cpp | 14 ++++++++------ .../OptimizeShardingKeyRewriteInVisitor.h | 2 -- 3 files changed, 8 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 07ef7aa6c96..16a873fe10e 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -249,7 +249,6 @@ void executeQuery( { OptimizeShardingKeyRewriteInVisitor::Data visitor_data{ sharding_key_expr, - sharding_key_expr->getSampleBlock().getByPosition(0).type, sharding_key_column_name, shard_info, not_optimized_cluster->getSlotToShard(), @@ -286,7 +285,6 @@ void executeQuery( { OptimizeShardingKeyRewriteInVisitor::Data visitor_data{ sharding_key_expr, - sharding_key_expr->getSampleBlock().getByPosition(0).type, sharding_key_column_name, shard_info, not_optimized_cluster->getSlotToShard(), diff --git a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp index 84279282d92..54515ea072a 100644 --- a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp +++ b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp @@ -38,25 +38,27 @@ Field executeFunctionOnField( return (*ret.column)[0]; } -/// @param sharding_column_value - one of values from IN +/// @param column_value - one of values from IN /// @param sharding_column_name - name of that column /// @return true if shard may contain such value (or it is unknown), otherwise false. bool shardContains( - Field sharding_column_value, + Field column_value, const std::string & sharding_column_name, const OptimizeShardingKeyRewriteInMatcher::Data & data) { + /// Type of column in storage (used for implicit conversion from i.e. String to Int) + const DataTypePtr & column_type = data.sharding_key_expr->getSampleBlock().getByName(sharding_column_name).type; /// Implicit conversion. - sharding_column_value = convertFieldToType(sharding_column_value, *data.sharding_key_type); + column_value = convertFieldToType(column_value, *column_type); /// NULL is not allowed in sharding key, /// so it should be safe to assume that shard cannot contain it. - if (sharding_column_value.isNull()) + if (column_value.isNull()) return false; Field sharding_value = executeFunctionOnField( - sharding_column_value, sharding_column_name, - data.sharding_key_expr, data.sharding_key_type, + column_value, sharding_column_name, + data.sharding_key_expr, column_type, data.sharding_key_column_name); /// The value from IN can be non-numeric, /// but in this case it should be convertible to numeric type, let's try. diff --git a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.h b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.h index 1b2d5a8fc15..78e4f42a912 100644 --- a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.h +++ b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.h @@ -28,8 +28,6 @@ struct OptimizeShardingKeyRewriteInMatcher { /// Expression of sharding_key for the Distributed() table const ExpressionActionsPtr & sharding_key_expr; - /// Type of sharding_key column. - const DataTypePtr & sharding_key_type; /// Name of the column for sharding_expr const std::string & sharding_key_column_name; /// Info for the current shard (to compare shard_num with calculated) From 8a1fe333046fba9c4750d153ddbb34d5165fb7e6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 28 Mar 2024 16:16:21 +0100 Subject: [PATCH 0186/1018] Disable optimize_skip_unused_shards_rewrite_in for composite keys It never worked, since shardContains() accept only one value, and it will throw "Not found column X in block" when sharding key consist from multiple columns. Signed-off-by: Azat Khuzhin --- src/Interpreters/ClusterProxy/executeQuery.cpp | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 16a873fe10e..f7727f70ff7 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -245,7 +245,12 @@ void executeQuery( const auto & shard_info = cluster->getShardsInfo()[i]; auto query_for_shard = query_info.query_tree->clone(); - if (sharding_key_expr && query_info.optimized_cluster && settings.optimize_skip_unused_shards_rewrite_in && shards > 1) + if (sharding_key_expr && + query_info.optimized_cluster && + settings.optimize_skip_unused_shards_rewrite_in && + shards > 1 && + /// TODO: support composite sharding key + sharding_key_expr->getRequiredColumns().size() == 1) { OptimizeShardingKeyRewriteInVisitor::Data visitor_data{ sharding_key_expr, @@ -281,7 +286,12 @@ void executeQuery( const auto & shard_info = cluster->getShardsInfo()[i]; ASTPtr query_ast_for_shard = query_info.query->clone(); - if (sharding_key_expr && query_info.optimized_cluster && settings.optimize_skip_unused_shards_rewrite_in && shards > 1) + if (sharding_key_expr && + query_info.optimized_cluster && + settings.optimize_skip_unused_shards_rewrite_in && + shards > 1 && + /// TODO: support composite sharding key + sharding_key_expr->getRequiredColumns().size() == 1) { OptimizeShardingKeyRewriteInVisitor::Data visitor_data{ sharding_key_expr, From 70eaf43deb03f12157c445574e370789af4d4967 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 28 Mar 2024 16:17:28 +0100 Subject: [PATCH 0187/1018] Add a test for settings.optimize_skip_unused_shards_rewrite_in with composite sharding key v2: use distributed_foreground_insert=1 to avoid flakiness Signed-off-by: Azat Khuzhin --- ...rds_rewrite_in_composite_sharding_key.reference | 10 ++++++++++ ...ed_shards_rewrite_in_composite_sharding_key.sql | 14 ++++++++++++++ 2 files changed, 24 insertions(+) create mode 100644 tests/queries/0_stateless/03033_dist_settings.optimize_skip_unused_shards_rewrite_in_composite_sharding_key.reference create mode 100644 tests/queries/0_stateless/03033_dist_settings.optimize_skip_unused_shards_rewrite_in_composite_sharding_key.sql diff --git a/tests/queries/0_stateless/03033_dist_settings.optimize_skip_unused_shards_rewrite_in_composite_sharding_key.reference b/tests/queries/0_stateless/03033_dist_settings.optimize_skip_unused_shards_rewrite_in_composite_sharding_key.reference new file mode 100644 index 00000000000..93f0f004541 --- /dev/null +++ b/tests/queries/0_stateless/03033_dist_settings.optimize_skip_unused_shards_rewrite_in_composite_sharding_key.reference @@ -0,0 +1,10 @@ +-- { echoOn } +SELECT shardNum(), count() FROM dt WHERE (tag_id, tag_name) IN ((1, 'foo1'), (1, 'foo2')) GROUP BY 1 ORDER BY 1; +1 2 +2 2 +SELECT shardNum(), count() FROM dt WHERE tag_id IN (1, 1) AND tag_name IN ('foo1', 'foo2') GROUP BY 1 ORDER BY 1; +1 2 +2 2 +SELECT shardNum(), count() FROM dt WHERE tag_id = 1 AND tag_name IN ('foo1', 'foo2') GROUP BY 1 ORDER BY 1; +1 2 +2 2 diff --git a/tests/queries/0_stateless/03033_dist_settings.optimize_skip_unused_shards_rewrite_in_composite_sharding_key.sql b/tests/queries/0_stateless/03033_dist_settings.optimize_skip_unused_shards_rewrite_in_composite_sharding_key.sql new file mode 100644 index 00000000000..b68fc27722e --- /dev/null +++ b/tests/queries/0_stateless/03033_dist_settings.optimize_skip_unused_shards_rewrite_in_composite_sharding_key.sql @@ -0,0 +1,14 @@ +DROP TABLE IF EXISTS t; +DROP TABLE IF EXISTS dt; + +CREATE TABLE t (tag_id UInt64, tag_name String) ENGINE = MergeTree ORDER BY tuple(); +CREATE TABLE dt AS t ENGINE = Distributed('test_cluster_two_shards_localhost', currentDatabase(), 't', cityHash64(concat(tag_id, tag_name))); + +INSERT INTO dt SETTINGS distributed_foreground_insert=1 VALUES (1, 'foo1'); -- shard0 +INSERT INTO dt SETTINGS distributed_foreground_insert=1 VALUES (1, 'foo2'); -- shard1 + +SET optimize_skip_unused_shards=1, optimize_skip_unused_shards_rewrite_in=1; +-- { echoOn } +SELECT shardNum(), count() FROM dt WHERE (tag_id, tag_name) IN ((1, 'foo1'), (1, 'foo2')) GROUP BY 1 ORDER BY 1; +SELECT shardNum(), count() FROM dt WHERE tag_id IN (1, 1) AND tag_name IN ('foo1', 'foo2') GROUP BY 1 ORDER BY 1; +SELECT shardNum(), count() FROM dt WHERE tag_id = 1 AND tag_name IN ('foo1', 'foo2') GROUP BY 1 ORDER BY 1; From daea269a28e5a43378f0a9b6bd4c451b005aab00 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 15 Apr 2024 14:42:20 +0200 Subject: [PATCH 0188/1018] Try without debug logs --- docker/test/stateless/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 271f30d187b..ed6eca96020 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -46,7 +46,7 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] elif [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then echo "Azure is disabled" else - azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --debug /azurite_log & + azurite-blob --blobHost 0.0.0.0 --blobPort 10000 & fi ./setup_minio.sh stateless From 0f2cc6e00a3c6c58f55b67a3bbfd0e210d1501aa Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 15 Apr 2024 15:17:36 +0200 Subject: [PATCH 0189/1018] Disable logs --- docker/test/stateless/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index ed6eca96020..60153d1e957 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -46,7 +46,7 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] elif [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then echo "Azure is disabled" else - azurite-blob --blobHost 0.0.0.0 --blobPort 10000 & + azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --silent & fi ./setup_minio.sh stateless From 949ec8d44c3babe95f0afc6a55c640c9455a297f Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 15 Apr 2024 15:33:25 +0200 Subject: [PATCH 0190/1018] Silence --- docker/test/stateful/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index dabd4d09228..3660865cb3e 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -19,7 +19,7 @@ ln -s /usr/share/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test # install test configs /usr/share/clickhouse-test/config/install.sh -azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --debug /azurite_log >> /test_output/garbage.log 2>&1 & +azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --silent & ./setup_minio.sh stateful config_logs_export_cluster /etc/clickhouse-server/config.d/system_logs_export.yaml From d56608067000000609400044675dfa7c1ef48fef Mon Sep 17 00:00:00 2001 From: Jus <40656180+jus1096@users.noreply.github.com> Date: Mon, 15 Apr 2024 17:38:20 +0400 Subject: [PATCH 0191/1018] Update argmax.md add description for argmax --- .../aggregate-functions/reference/argmax.md | 59 ++++++++++++++++++- 1 file changed, 58 insertions(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/argmax.md b/docs/ru/sql-reference/aggregate-functions/reference/argmax.md index 0489606b3c1..7277da91af5 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/argmax.md @@ -5,7 +5,7 @@ sidebar_position: 106 # argMax {#agg-function-argmax} -Вычисляет значение `arg` при максимальном значении `val`. +Вычисляет значение `arg` при максимальном значении `val`. Если несколько строк имеют одинаковое `val`, в которых равное значение является максимальным, то возвращаемое `arg` не является детерминированным. Обе части, arg и max, ведут себя как агрегатные функции, они обе пропускают Null во время обработки и возвращают не Null значения, если Null значения доступны. **Синтаксис** @@ -49,3 +49,60 @@ SELECT argMax(user, salary), argMax(tuple(user, salary), salary) FROM salary; │ director │ ('director',5000) │ └──────────────────────┴─────────────────────────────────────┘ ``` + +**Дополнительный пример** + +```sql +CREATE TABLE test +( + a Nullable(String), + b Nullable(Int64) +) +ENGINE = Memory AS +SELECT * +FROM VALUES(('a', 1), ('b', 2), ('c', 2), (NULL, 3), (NULL, NULL), ('d', NULL)); + +select * from test; +┌─a────┬────b─┐ +│ a │ 1 │ +│ b │ 2 │ +│ c │ 2 │ +│ ᴺᵁᴸᴸ │ 3 │ +│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ +│ d │ ᴺᵁᴸᴸ │ +└──────┴──────┘ + +SELECT argMax(a, b), max(b) FROM test; +┌─argMax(a, b)─┬─max(b)─┐ +│ b │ 3 │ -- argMax = 'b' потому что это первое not Null значение, max(b) из другой строки! +└──────────────┴────────┘ + +SELECT argMax(tuple(a), b) FROM test; +┌─argMax(tuple(a), b)─┐ +│ (NULL) │ -- Кортеж `Tuple`, который содержит только `NULL` значения является не `NULL` кортежем, поэтому агрегатыне функции не будут пропускать эту строку с `NULL` значениями. +└─────────────────────┘ + +SELECT (argMax((a, b), b) as t).1 argMaxA, t.2 argMaxB FROM test; +┌─argMaxA─┬─argMaxB─┐ +│ ᴺᵁᴸᴸ │ 3 │ -- Вы можете использовать кортеж Tuple и получить оба значения для соответсвующего max(b). +└─────────┴─────────┘ + +SELECT argMax(a, b), max(b) FROM test WHERE a IS NULL AND b IS NULL; +┌─argMax(a, b)─┬─max(b)─┐ +│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ -- Все агрегированные строки содержат хотя бы одно `NULL` значение, поэтому все строки пропускаются и результатом будет `NULL`. +└──────────────┴────────┘ + +SELECT argMax(a, (b,a)) FROM test; +┌─argMax(a, tuple(b, a))─┐ +│ c │ -- Есть две строки с b=2, кортеж `Tuple` в функции `Max` позволяет получить не первый `arg`. +└────────────────────────┘ + +SELECT argMax(a, tuple(b)) FROM test; +┌─argMax(a, tuple(b))─┐ +│ b │ -- Кортеж `Tuple` может использоваться в `Max`, чтобы не пропускать `NULL` значения в `Max`. +└─────────────────────┘ +``` + +**Смотрите также** + +- [Tuple](/docs/ru/sql-reference/data-types/tuple.md) From 45d49ca53d3e705073735526aa29a6766e344174 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 15 Apr 2024 15:46:41 +0200 Subject: [PATCH 0192/1018] I'll shut you down --- docker/test/stress/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 6c6caf872e9..9d334a77223 100644 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -52,7 +52,7 @@ export ZOOKEEPER_FAULT_INJECTION=1 # available for dump via clickhouse-local configure -azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --debug /azurite_log & +azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --silence & ./setup_minio.sh stateless # to have a proper environment config_logs_export_cluster /etc/clickhouse-server/config.d/system_logs_export.yaml From 03c0b20c2e2fbefd035fc92507a01ba5190f5313 Mon Sep 17 00:00:00 2001 From: Jus <40656180+jus1096@users.noreply.github.com> Date: Mon, 15 Apr 2024 17:53:36 +0400 Subject: [PATCH 0193/1018] Update argmax.md small fix --- docs/ru/sql-reference/aggregate-functions/reference/argmax.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/argmax.md b/docs/ru/sql-reference/aggregate-functions/reference/argmax.md index 7277da91af5..bbc8b4fcb07 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/argmax.md @@ -5,7 +5,7 @@ sidebar_position: 106 # argMax {#agg-function-argmax} -Вычисляет значение `arg` при максимальном значении `val`. Если несколько строк имеют одинаковое `val`, в которых равное значение является максимальным, то возвращаемое `arg` не является детерминированным. Обе части, arg и max, ведут себя как агрегатные функции, они обе пропускают Null во время обработки и возвращают не Null значения, если Null значения доступны. +Вычисляет значение `arg` при максимальном значении `val`. Если несколько строк имеют одинаковое `val`, в которых равное значение является максимальным, то возвращаемое `arg` не является детерминированным. Обе части, arg и max, ведут себя как агрегатные функции, они обе пропускают Null во время обработки и возвращают не Null значения, если не Null значения доступны. **Синтаксис** From 91d4284bdf37efca0c1a2250495e93fcbd8b0850 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 15 Apr 2024 16:21:09 +0200 Subject: [PATCH 0194/1018] don't allow relative paths when installing --- programs/install/Install.cpp | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index f2ef3857d63..6bed114238a 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -729,6 +729,15 @@ int mainEntryClickHouseInstall(int argc, char ** argv) } } + /// Don't allow relative paths because install script may cd to / when installing + /// And having path=./ may break the system + if (log_path.is_relative()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Log path is relative: {}", log_path.string()); + if (data_path.is_relative()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Data path is relative: {}", data_path.string()); + if (pid_path.is_relative()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Pid path is relative: {}", pid_path.string()); + /// Create directories for data and log. if (fs::exists(log_path)) From c0c8c3be990d4b4087c5cb632c18ce810fe3050d Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 15 Apr 2024 17:28:22 +0200 Subject: [PATCH 0195/1018] Fix --- docker/test/stress/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 9d334a77223..702d2db1db6 100644 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -52,7 +52,7 @@ export ZOOKEEPER_FAULT_INJECTION=1 # available for dump via clickhouse-local configure -azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --silence & +azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --silent & ./setup_minio.sh stateless # to have a proper environment config_logs_export_cluster /etc/clickhouse-server/config.d/system_logs_export.yaml From 90ef5cc516eea9b6de60fc283074858598f396c8 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 15 Apr 2024 18:52:17 +0200 Subject: [PATCH 0196/1018] better --- docker/test/stateless/run.sh | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 60153d1e957..11638ebac45 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -213,6 +213,14 @@ function run_tests() ADDITIONAL_OPTIONS+=('--s3-storage') fi + if [[ -n "$USE_AZURE_STORAGE_FOR_MERGE_TREE" ]] && [[ "$USE_AZURE_STORAGE_FOR_MERGE_TREE" -eq 1 ]]; then + # to disable the same tests + ADDITIONAL_OPTIONS+=('--s3-storage') + # azurite is slow, but with these two settings it can be super slow + ADDITIONAL_OPTIONS+=('--no-random-settings') + ADDITIONAL_OPTIONS+=('--no-random-merge-tree-settings') + fi + if [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then ADDITIONAL_OPTIONS+=('--shared-catalog') fi From 1e1f35fac040ab663289258e49c48b7a596e570a Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Tue, 16 Apr 2024 10:35:18 +0200 Subject: [PATCH 0197/1018] Try release job --- tests/ci/ci_config.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index ef83346d6ba..ab5282259fe 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -107,7 +107,7 @@ class JobNames(metaclass=WithIter): # STATELESS_TEST_S3_RELEASE = "Stateless tests (release, s3 storage)" STATELESS_TEST_S3_DEBUG = "Stateless tests (debug, s3 storage)" STATELESS_TEST_S3_TSAN = "Stateless tests (tsan, s3 storage)" - STATELESS_TEST_AZURE_ASAN = "Stateless tests (azure, asan)" + STATELESS_TEST_AZURE_RELEASE = "Stateless tests (azure, release)" STATELESS_TEST_FLAKY_ASAN = "Stateless tests flaky check (asan)" STATEFUL_TEST_DEBUG = "Stateful tests (debug)" @@ -1195,8 +1195,8 @@ CI_CONFIG = CIConfig( Build.PACKAGE_DEBUG, job_config=JobConfig(num_batches=6, **statless_test_common_params), # type: ignore ), - JobNames.STATELESS_TEST_AZURE_ASAN: TestConfig( - Build.PACKAGE_ASAN, + JobNames.STATELESS_TEST_AZURE_RELEASE: TestConfig( + Build.PACKAGE_RELEASE, job_config=JobConfig(num_batches=4, **statless_test_common_params), # type: ignore ), JobNames.STATELESS_TEST_S3_TSAN: TestConfig( From 5b11815bcc7336b7796c7a36a7b2ca16dbc3978a Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 16 Apr 2024 14:20:46 +0100 Subject: [PATCH 0198/1018] impl --- .../QueryPlan/ReadFromMergeTree.cpp | 5 +++++ ...ading_bug_with_parallel_replicas.reference | 0 ...093_reading_bug_with_parallel_replicas.sql | 19 +++++++++++++++++++ 3 files changed, 24 insertions(+) create mode 100644 tests/queries/0_stateless/03093_reading_bug_with_parallel_replicas.reference create mode 100644 tests/queries/0_stateless/03093_reading_bug_with_parallel_replicas.sql diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index bee42c3ddde..e15a421c092 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1791,6 +1791,11 @@ bool ReadFromMergeTree::requestOutputEachPartitionThroughSeparatePort() if (isQueryWithFinal()) return false; + /// With parallel replicas we have to have only a single instance of `MergeTreeReadPoolParallelReplicas` per replica. + /// With aggregation-by-partitions optimisation we might create a separate pool for each partition. + if (is_parallel_reading_from_replicas) + return false; + const auto & settings = context->getSettingsRef(); const auto partitions_cnt = countPartitions(prepared_parts); diff --git a/tests/queries/0_stateless/03093_reading_bug_with_parallel_replicas.reference b/tests/queries/0_stateless/03093_reading_bug_with_parallel_replicas.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03093_reading_bug_with_parallel_replicas.sql b/tests/queries/0_stateless/03093_reading_bug_with_parallel_replicas.sql new file mode 100644 index 00000000000..9592764baa6 --- /dev/null +++ b/tests/queries/0_stateless/03093_reading_bug_with_parallel_replicas.sql @@ -0,0 +1,19 @@ + +set max_threads = 16; +set use_hedged_requests = 0; +set max_parallel_replicas = 3; +set cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; +set allow_experimental_parallel_reading_from_replicas = 1; +set parallel_replicas_for_non_replicated_merge_tree = 1; +set allow_aggregate_partitions_independently = 1; + +drop table if exists t2; + +create table t2(a Int16) engine=MergeTree order by tuple() partition by a % 8 SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; + +system stop merges t2; + +insert into t2 select number from numbers_mt(1e6); +insert into t2 select number from numbers_mt(1e6); + +select a from t2 group by a format Null; From 7806f64226e7bdba5240088ee8fd8213d0a4586e Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 16 Apr 2024 14:31:54 +0100 Subject: [PATCH 0199/1018] Test --- src/Storages/RabbitMQ/RabbitMQConsumer.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQConsumer.cpp b/src/Storages/RabbitMQ/RabbitMQConsumer.cpp index ed16175b987..1d99581b4a5 100644 --- a/src/Storages/RabbitMQ/RabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/RabbitMQConsumer.cpp @@ -119,8 +119,9 @@ bool RabbitMQConsumer::ackMessages(const CommitInfo & commit_info) } /// Duplicate ack? - if (commit_info.delivery_tag > last_commited_delivery_tag - && consumer_channel->ack(commit_info.delivery_tag, AMQP::multiple)) + // if (commit_info.delivery_tag > last_commited_delivery_tag + // && consumer_channel->ack(commit_info.delivery_tag, AMQP::multiple)) + if (consumer_channel->ack(commit_info.delivery_tag, AMQP::multiple)) { last_commited_delivery_tag = commit_info.delivery_tag; From 9896c50c3f41cfc80017bc234ac7fdff912dbc2f Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Wed, 17 Apr 2024 14:53:45 +0200 Subject: [PATCH 0200/1018] azurite only in memory --- docker/test/stateful/run.sh | 2 +- docker/test/stateless/run.sh | 2 +- docker/test/stress/run.sh | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 3660865cb3e..1ae3c61fe87 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -19,7 +19,7 @@ ln -s /usr/share/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test # install test configs /usr/share/clickhouse-test/config/install.sh -azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --silent & +azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --silent --inMemoryPersistence & ./setup_minio.sh stateful config_logs_export_cluster /etc/clickhouse-server/config.d/system_logs_export.yaml diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index d97d5d57b41..1cc8e24d0a1 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -47,7 +47,7 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] elif [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then echo "Azure is disabled" else - azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --silent & + azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --silent --inMemoryPersistence & fi ./setup_minio.sh stateless diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index c9a16cbdc4a..9a60b4e78ec 100644 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -52,7 +52,7 @@ export ZOOKEEPER_FAULT_INJECTION=1 # available for dump via clickhouse-local configure -azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --silent & +azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --silent --inMemoryPersistence & ./setup_minio.sh stateless # to have a proper environment config_logs_export_cluster /etc/clickhouse-server/config.d/system_logs_export.yaml From 4a9f490159f6e9df6a4f4bc892845cc845c2a6b4 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 17 Apr 2024 14:22:41 +0100 Subject: [PATCH 0201/1018] Update StorageRabbitMQ.cpp --- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 15435ba2988..fc3dc1da8ad 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -1172,7 +1172,7 @@ bool StorageRabbitMQ::tryStreamToViews() * the same channel will also commit all previously not-committed messages. Anyway I do not think that for ack frame this * will ever happen. */ - if (write_failed && reject_unhandled_messages ? source->sendNack() : source->sendAck()) + if (write_failed ? source->sendNack() : source->sendAck()) { /// Iterate loop to activate error callbacks if they happened connection->getHandler().iterateLoop(); From 5cad63b9d50df2e88dd568aa842ec40d3ee4f25e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 10 Apr 2024 18:56:30 +0000 Subject: [PATCH 0202/1018] Add test for bug 33446 --- tests/performance/one_thousand_joins.xml | 6 ++++++ .../queries/0_stateless/03094_one_thousand_joins.reference | 1 + tests/queries/0_stateless/03094_one_thousand_joins.sql | 4 ++++ 3 files changed, 11 insertions(+) create mode 100644 tests/performance/one_thousand_joins.xml create mode 100644 tests/queries/0_stateless/03094_one_thousand_joins.reference create mode 100644 tests/queries/0_stateless/03094_one_thousand_joins.sql diff --git a/tests/performance/one_thousand_joins.xml b/tests/performance/one_thousand_joins.xml new file mode 100644 index 00000000000..d7e66bb1c67 --- /dev/null +++ b/tests/performance/one_thousand_joins.xml @@ -0,0 +1,6 @@ + + + +SELECT * FROM (SELECT 1 AS x) t1 JOIN (SELECT 1 AS x) t2 ON t1.x = t2.x JOIN (SELECT 1 AS x) t3 ON t1.x = t3.x JOIN (SELECT 1 AS x) t4 ON t1.x = t4.x JOIN (SELECT 1 AS x) t5 ON t1.x = t5.x JOIN (SELECT 1 AS x) t6 ON t1.x = t6.x JOIN (SELECT 1 AS x) t7 ON t1.x = t7.x JOIN (SELECT 1 AS x) t8 ON t1.x = t8.x JOIN (SELECT 1 AS x) t9 ON t1.x = t9.x JOIN (SELECT 1 AS x) t10 ON t1.x = t10.x JOIN (SELECT 1 AS x) t11 ON t1.x = t11.x JOIN (SELECT 1 AS x) t12 ON t1.x = t12.x JOIN (SELECT 1 AS x) t13 ON t1.x = t13.x JOIN (SELECT 1 AS x) t14 ON t1.x = t14.x JOIN (SELECT 1 AS x) t15 ON t1.x = t15.x JOIN (SELECT 1 AS x) t16 ON t1.x = t16.x JOIN (SELECT 1 AS x) t17 ON t1.x = t17.x JOIN (SELECT 1 AS x) t18 ON t1.x = t18.x JOIN (SELECT 1 AS x) t19 ON t1.x = t19.x JOIN (SELECT 1 AS x) t20 ON t1.x = t20.x JOIN (SELECT 1 AS x) t21 ON t1.x = t21.x JOIN (SELECT 1 AS x) t22 ON t1.x = t22.x JOIN (SELECT 1 AS x) t23 ON t1.x = t23.x JOIN (SELECT 1 AS x) t24 ON t1.x = t24.x JOIN (SELECT 1 AS x) t25 ON t1.x = t25.x JOIN (SELECT 1 AS x) t26 ON t1.x = t26.x JOIN (SELECT 1 AS x) t27 ON t1.x = t27.x JOIN (SELECT 1 AS x) t28 ON t1.x = t28.x JOIN (SELECT 1 AS x) t29 ON t1.x = t29.x JOIN (SELECT 1 AS x) t30 ON t1.x = t30.x JOIN (SELECT 1 AS x) t31 ON t1.x = t31.x JOIN (SELECT 1 AS x) t32 ON t1.x = t32.x JOIN (SELECT 1 AS x) t33 ON t1.x = t33.x JOIN (SELECT 1 AS x) t34 ON t1.x = t34.x JOIN (SELECT 1 AS x) t35 ON t1.x = t35.x JOIN (SELECT 1 AS x) t36 ON t1.x = t36.x JOIN (SELECT 1 AS x) t37 ON t1.x = t37.x JOIN (SELECT 1 AS x) t38 ON t1.x = t38.x JOIN (SELECT 1 AS x) t39 ON t1.x = t39.x JOIN (SELECT 1 AS x) t40 ON t1.x = t40.x JOIN (SELECT 1 AS x) t41 ON t1.x = t41.x JOIN (SELECT 1 AS x) t42 ON t1.x = t42.x JOIN (SELECT 1 AS x) t43 ON t1.x = t43.x JOIN (SELECT 1 AS x) t44 ON t1.x = t44.x JOIN (SELECT 1 AS x) t45 ON t1.x = t45.x JOIN (SELECT 1 AS x) t46 ON t1.x = t46.x JOIN (SELECT 1 AS x) t47 ON t1.x = t47.x JOIN (SELECT 1 AS x) t48 ON t1.x = t48.x JOIN (SELECT 1 AS x) t49 ON t1.x = t49.x JOIN (SELECT 1 AS x) t50 ON t1.x = t50.x JOIN (SELECT 1 AS x) t51 ON t1.x = t51.x JOIN (SELECT 1 AS x) t52 ON t1.x = t52.x JOIN (SELECT 1 AS x) t53 ON t1.x = t53.x JOIN (SELECT 1 AS x) t54 ON t1.x = t54.x JOIN (SELECT 1 AS x) t55 ON t1.x = t55.x JOIN (SELECT 1 AS x) t56 ON t1.x = t56.x JOIN (SELECT 1 AS x) t57 ON t1.x = t57.x JOIN (SELECT 1 AS x) t58 ON t1.x = t58.x JOIN (SELECT 1 AS x) t59 ON t1.x = t59.x JOIN (SELECT 1 AS x) t60 ON t1.x = t60.x JOIN (SELECT 1 AS x) t61 ON t1.x = t61.x JOIN (SELECT 1 AS x) t62 ON t1.x = t62.x JOIN (SELECT 1 AS x) t63 ON t1.x = t63.x JOIN (SELECT 1 AS x) t64 ON t1.x = t64.x JOIN (SELECT 1 AS x) t65 ON t1.x = t65.x JOIN (SELECT 1 AS x) t66 ON t1.x = t66.x JOIN (SELECT 1 AS x) t67 ON t1.x = t67.x JOIN (SELECT 1 AS x) t68 ON t1.x = t68.x JOIN (SELECT 1 AS x) t69 ON t1.x = t69.x JOIN (SELECT 1 AS x) t70 ON t1.x = t70.x JOIN (SELECT 1 AS x) t71 ON t1.x = t71.x JOIN (SELECT 1 AS x) t72 ON t1.x = t72.x JOIN (SELECT 1 AS x) t73 ON t1.x = t73.x JOIN (SELECT 1 AS x) t74 ON t1.x = t74.x JOIN (SELECT 1 AS x) t75 ON t1.x = t75.x JOIN (SELECT 1 AS x) t76 ON t1.x = t76.x JOIN (SELECT 1 AS x) t77 ON t1.x = t77.x JOIN (SELECT 1 AS x) t78 ON t1.x = t78.x JOIN (SELECT 1 AS x) t79 ON t1.x = t79.x JOIN (SELECT 1 AS x) t80 ON t1.x = t80.x JOIN (SELECT 1 AS x) t81 ON t1.x = t81.x JOIN (SELECT 1 AS x) t82 ON t1.x = t82.x JOIN (SELECT 1 AS x) t83 ON t1.x = t83.x JOIN (SELECT 1 AS x) t84 ON t1.x = t84.x JOIN (SELECT 1 AS x) t85 ON t1.x = t85.x JOIN (SELECT 1 AS x) t86 ON t1.x = t86.x JOIN (SELECT 1 AS x) t87 ON t1.x = t87.x JOIN (SELECT 1 AS x) t88 ON t1.x = t88.x JOIN (SELECT 1 AS x) t89 ON t1.x = t89.x JOIN (SELECT 1 AS x) t90 ON t1.x = t90.x JOIN (SELECT 1 AS x) t91 ON t1.x = t91.x JOIN (SELECT 1 AS x) t92 ON t1.x = t92.x JOIN (SELECT 1 AS x) t93 ON t1.x = t93.x JOIN (SELECT 1 AS x) t94 ON t1.x = t94.x JOIN (SELECT 1 AS x) t95 ON t1.x = t95.x JOIN (SELECT 1 AS x) t96 ON t1.x = t96.x JOIN (SELECT 1 AS x) t97 ON t1.x = t97.x JOIN (SELECT 1 AS x) t98 ON t1.x = t98.x JOIN (SELECT 1 AS x) t99 ON t1.x = t99.x JOIN (SELECT 1 AS x) t100 ON t1.x = t100.x JOIN (SELECT 1 AS x) t101 ON t1.x = t101.x JOIN (SELECT 1 AS x) t102 ON t1.x = t102.x JOIN (SELECT 1 AS x) t103 ON t1.x = t103.x JOIN (SELECT 1 AS x) t104 ON t1.x = t104.x JOIN (SELECT 1 AS x) t105 ON t1.x = t105.x JOIN (SELECT 1 AS x) t106 ON t1.x = t106.x JOIN (SELECT 1 AS x) t107 ON t1.x = t107.x JOIN (SELECT 1 AS x) t108 ON t1.x = t108.x JOIN (SELECT 1 AS x) t109 ON t1.x = t109.x JOIN (SELECT 1 AS x) t110 ON t1.x = t110.x JOIN (SELECT 1 AS x) t111 ON t1.x = t111.x JOIN (SELECT 1 AS x) t112 ON t1.x = t112.x JOIN (SELECT 1 AS x) t113 ON t1.x = t113.x JOIN (SELECT 1 AS x) t114 ON t1.x = t114.x JOIN (SELECT 1 AS x) t115 ON t1.x = t115.x JOIN (SELECT 1 AS x) t116 ON t1.x = t116.x JOIN (SELECT 1 AS x) t117 ON t1.x = t117.x JOIN (SELECT 1 AS x) t118 ON t1.x = t118.x JOIN (SELECT 1 AS x) t119 ON t1.x = t119.x JOIN (SELECT 1 AS x) t120 ON t1.x = t120.x JOIN (SELECT 1 AS x) t121 ON t1.x = t121.x JOIN (SELECT 1 AS x) t122 ON t1.x = t122.x JOIN (SELECT 1 AS x) t123 ON t1.x = t123.x JOIN (SELECT 1 AS x) t124 ON t1.x = t124.x JOIN (SELECT 1 AS x) t125 ON t1.x = t125.x JOIN (SELECT 1 AS x) t126 ON t1.x = t126.x JOIN (SELECT 1 AS x) t127 ON t1.x = t127.x JOIN (SELECT 1 AS x) t128 ON t1.x = t128.x JOIN (SELECT 1 AS x) t129 ON t1.x = t129.x JOIN (SELECT 1 AS x) t130 ON t1.x = t130.x JOIN (SELECT 1 AS x) t131 ON t1.x = t131.x JOIN (SELECT 1 AS x) t132 ON t1.x = t132.x JOIN (SELECT 1 AS x) t133 ON t1.x = t133.x JOIN (SELECT 1 AS x) t134 ON t1.x = t134.x JOIN (SELECT 1 AS x) t135 ON t1.x = t135.x JOIN (SELECT 1 AS x) t136 ON t1.x = t136.x JOIN (SELECT 1 AS x) t137 ON t1.x = t137.x JOIN (SELECT 1 AS x) t138 ON t1.x = t138.x JOIN (SELECT 1 AS x) t139 ON t1.x = t139.x JOIN (SELECT 1 AS x) t140 ON t1.x = t140.x JOIN (SELECT 1 AS x) t141 ON t1.x = t141.x JOIN (SELECT 1 AS x) t142 ON t1.x = t142.x JOIN (SELECT 1 AS x) t143 ON t1.x = t143.x JOIN (SELECT 1 AS x) t144 ON t1.x = t144.x JOIN (SELECT 1 AS x) t145 ON t1.x = t145.x JOIN (SELECT 1 AS x) t146 ON t1.x = t146.x JOIN (SELECT 1 AS x) t147 ON t1.x = t147.x JOIN (SELECT 1 AS x) t148 ON t1.x = t148.x JOIN (SELECT 1 AS x) t149 ON t1.x = t149.x JOIN (SELECT 1 AS x) t150 ON t1.x = t150.x JOIN (SELECT 1 AS x) t151 ON t1.x = t151.x JOIN (SELECT 1 AS x) t152 ON t1.x = t152.x JOIN (SELECT 1 AS x) t153 ON t1.x = t153.x JOIN (SELECT 1 AS x) t154 ON t1.x = t154.x JOIN (SELECT 1 AS x) t155 ON t1.x = t155.x JOIN (SELECT 1 AS x) t156 ON t1.x = t156.x JOIN (SELECT 1 AS x) t157 ON t1.x = t157.x JOIN (SELECT 1 AS x) t158 ON t1.x = t158.x JOIN (SELECT 1 AS x) t159 ON t1.x = t159.x JOIN (SELECT 1 AS x) t160 ON t1.x = t160.x JOIN (SELECT 1 AS x) t161 ON t1.x = t161.x JOIN (SELECT 1 AS x) t162 ON t1.x = t162.x JOIN (SELECT 1 AS x) t163 ON t1.x = t163.x JOIN (SELECT 1 AS x) t164 ON t1.x = t164.x JOIN (SELECT 1 AS x) t165 ON t1.x = t165.x JOIN (SELECT 1 AS x) t166 ON t1.x = t166.x JOIN (SELECT 1 AS x) t167 ON t1.x = t167.x JOIN (SELECT 1 AS x) t168 ON t1.x = t168.x JOIN (SELECT 1 AS x) t169 ON t1.x = t169.x JOIN (SELECT 1 AS x) t170 ON t1.x = t170.x JOIN (SELECT 1 AS x) t171 ON t1.x = t171.x JOIN (SELECT 1 AS x) t172 ON t1.x = t172.x JOIN (SELECT 1 AS x) t173 ON t1.x = t173.x JOIN (SELECT 1 AS x) t174 ON t1.x = t174.x JOIN (SELECT 1 AS x) t175 ON t1.x = t175.x JOIN (SELECT 1 AS x) t176 ON t1.x = t176.x JOIN (SELECT 1 AS x) t177 ON t1.x = t177.x JOIN (SELECT 1 AS x) t178 ON t1.x = t178.x JOIN (SELECT 1 AS x) t179 ON t1.x = t179.x JOIN (SELECT 1 AS x) t180 ON t1.x = t180.x JOIN (SELECT 1 AS x) t181 ON t1.x = t181.x JOIN (SELECT 1 AS x) t182 ON t1.x = t182.x JOIN (SELECT 1 AS x) t183 ON t1.x = t183.x JOIN (SELECT 1 AS x) t184 ON t1.x = t184.x JOIN (SELECT 1 AS x) t185 ON t1.x = t185.x JOIN (SELECT 1 AS x) t186 ON t1.x = t186.x JOIN (SELECT 1 AS x) t187 ON t1.x = t187.x JOIN (SELECT 1 AS x) t188 ON t1.x = t188.x JOIN (SELECT 1 AS x) t189 ON t1.x = t189.x JOIN (SELECT 1 AS x) t190 ON t1.x = t190.x JOIN (SELECT 1 AS x) t191 ON t1.x = t191.x JOIN (SELECT 1 AS x) t192 ON t1.x = t192.x JOIN (SELECT 1 AS x) t193 ON t1.x = t193.x JOIN (SELECT 1 AS x) t194 ON t1.x = t194.x JOIN (SELECT 1 AS x) t195 ON t1.x = t195.x JOIN (SELECT 1 AS x) t196 ON t1.x = t196.x JOIN (SELECT 1 AS x) t197 ON t1.x = t197.x JOIN (SELECT 1 AS x) t198 ON t1.x = t198.x JOIN (SELECT 1 AS x) t199 ON t1.x = t199.x JOIN (SELECT 1 AS x) t200 ON t1.x = t200.x JOIN (SELECT 1 AS x) t201 ON t1.x = t201.x JOIN (SELECT 1 AS x) t202 ON t1.x = t202.x JOIN (SELECT 1 AS x) t203 ON t1.x = t203.x JOIN (SELECT 1 AS x) t204 ON t1.x = t204.x JOIN (SELECT 1 AS x) t205 ON t1.x = t205.x JOIN (SELECT 1 AS x) t206 ON t1.x = t206.x JOIN (SELECT 1 AS x) t207 ON t1.x = t207.x JOIN (SELECT 1 AS x) t208 ON t1.x = t208.x JOIN (SELECT 1 AS x) t209 ON t1.x = t209.x JOIN (SELECT 1 AS x) t210 ON t1.x = t210.x JOIN (SELECT 1 AS x) t211 ON t1.x = t211.x JOIN (SELECT 1 AS x) t212 ON t1.x = t212.x JOIN (SELECT 1 AS x) t213 ON t1.x = t213.x JOIN (SELECT 1 AS x) t214 ON t1.x = t214.x JOIN (SELECT 1 AS x) t215 ON t1.x = t215.x JOIN (SELECT 1 AS x) t216 ON t1.x = t216.x JOIN (SELECT 1 AS x) t217 ON t1.x = t217.x JOIN (SELECT 1 AS x) t218 ON t1.x = t218.x JOIN (SELECT 1 AS x) t219 ON t1.x = t219.x JOIN (SELECT 1 AS x) t220 ON t1.x = t220.x JOIN (SELECT 1 AS x) t221 ON t1.x = t221.x JOIN (SELECT 1 AS x) t222 ON t1.x = t222.x JOIN (SELECT 1 AS x) t223 ON t1.x = t223.x JOIN (SELECT 1 AS x) t224 ON t1.x = t224.x JOIN (SELECT 1 AS x) t225 ON t1.x = t225.x JOIN (SELECT 1 AS x) t226 ON t1.x = t226.x JOIN (SELECT 1 AS x) t227 ON t1.x = t227.x JOIN (SELECT 1 AS x) t228 ON t1.x = t228.x JOIN (SELECT 1 AS x) t229 ON t1.x = t229.x JOIN (SELECT 1 AS x) t230 ON t1.x = t230.x JOIN (SELECT 1 AS x) t231 ON t1.x = t231.x JOIN (SELECT 1 AS x) t232 ON t1.x = t232.x JOIN (SELECT 1 AS x) t233 ON t1.x = t233.x JOIN (SELECT 1 AS x) t234 ON t1.x = t234.x JOIN (SELECT 1 AS x) t235 ON t1.x = t235.x JOIN (SELECT 1 AS x) t236 ON t1.x = t236.x JOIN (SELECT 1 AS x) t237 ON t1.x = t237.x JOIN (SELECT 1 AS x) t238 ON t1.x = t238.x JOIN (SELECT 1 AS x) t239 ON t1.x = t239.x JOIN (SELECT 1 AS x) t240 ON t1.x = t240.x JOIN (SELECT 1 AS x) t241 ON t1.x = t241.x JOIN (SELECT 1 AS x) t242 ON t1.x = t242.x JOIN (SELECT 1 AS x) t243 ON t1.x = t243.x JOIN (SELECT 1 AS x) t244 ON t1.x = t244.x JOIN (SELECT 1 AS x) t245 ON t1.x = t245.x JOIN (SELECT 1 AS x) t246 ON t1.x = t246.x JOIN (SELECT 1 AS x) t247 ON t1.x = t247.x JOIN (SELECT 1 AS x) t248 ON t1.x = t248.x JOIN (SELECT 1 AS x) t249 ON t1.x = t249.x JOIN (SELECT 1 AS x) t250 ON t1.x = t250.x JOIN (SELECT 1 AS x) t251 ON t1.x = t251.x JOIN (SELECT 1 AS x) t252 ON t1.x = t252.x JOIN (SELECT 1 AS x) t253 ON t1.x = t253.x JOIN (SELECT 1 AS x) t254 ON t1.x = t254.x JOIN (SELECT 1 AS x) t255 ON t1.x = t255.x JOIN (SELECT 1 AS x) t256 ON t1.x = t256.x JOIN (SELECT 1 AS x) t257 ON t1.x = t257.x JOIN (SELECT 1 AS x) t258 ON t1.x = t258.x JOIN (SELECT 1 AS x) t259 ON t1.x = t259.x JOIN (SELECT 1 AS x) t260 ON t1.x = t260.x JOIN (SELECT 1 AS x) t261 ON t1.x = t261.x JOIN (SELECT 1 AS x) t262 ON t1.x = t262.x JOIN (SELECT 1 AS x) t263 ON t1.x = t263.x JOIN (SELECT 1 AS x) t264 ON t1.x = t264.x JOIN (SELECT 1 AS x) t265 ON t1.x = t265.x JOIN (SELECT 1 AS x) t266 ON t1.x = t266.x JOIN (SELECT 1 AS x) t267 ON t1.x = t267.x JOIN (SELECT 1 AS x) t268 ON t1.x = t268.x JOIN (SELECT 1 AS x) t269 ON t1.x = t269.x JOIN (SELECT 1 AS x) t270 ON t1.x = t270.x JOIN (SELECT 1 AS x) t271 ON t1.x = t271.x JOIN (SELECT 1 AS x) t272 ON t1.x = t272.x JOIN (SELECT 1 AS x) t273 ON t1.x = t273.x JOIN (SELECT 1 AS x) t274 ON t1.x = t274.x JOIN (SELECT 1 AS x) t275 ON t1.x = t275.x JOIN (SELECT 1 AS x) t276 ON t1.x = t276.x JOIN (SELECT 1 AS x) t277 ON t1.x = t277.x JOIN (SELECT 1 AS x) t278 ON t1.x = t278.x JOIN (SELECT 1 AS x) t279 ON t1.x = t279.x JOIN (SELECT 1 AS x) t280 ON t1.x = t280.x JOIN (SELECT 1 AS x) t281 ON t1.x = t281.x JOIN (SELECT 1 AS x) t282 ON t1.x = t282.x JOIN (SELECT 1 AS x) t283 ON t1.x = t283.x JOIN (SELECT 1 AS x) t284 ON t1.x = t284.x JOIN (SELECT 1 AS x) t285 ON t1.x = t285.x JOIN (SELECT 1 AS x) t286 ON t1.x = t286.x JOIN (SELECT 1 AS x) t287 ON t1.x = t287.x JOIN (SELECT 1 AS x) t288 ON t1.x = t288.x JOIN (SELECT 1 AS x) t289 ON t1.x = t289.x JOIN (SELECT 1 AS x) t290 ON t1.x = t290.x JOIN (SELECT 1 AS x) t291 ON t1.x = t291.x JOIN (SELECT 1 AS x) t292 ON t1.x = t292.x JOIN (SELECT 1 AS x) t293 ON t1.x = t293.x JOIN (SELECT 1 AS x) t294 ON t1.x = t294.x JOIN (SELECT 1 AS x) t295 ON t1.x = t295.x JOIN (SELECT 1 AS x) t296 ON t1.x = t296.x JOIN (SELECT 1 AS x) t297 ON t1.x = t297.x JOIN (SELECT 1 AS x) t298 ON t1.x = t298.x JOIN (SELECT 1 AS x) t299 ON t1.x = t299.x JOIN (SELECT 1 AS x) t300 ON t1.x = t300.x JOIN (SELECT 1 AS x) t301 ON t1.x = t301.x JOIN (SELECT 1 AS x) t302 ON t1.x = t302.x JOIN (SELECT 1 AS x) t303 ON t1.x = t303.x JOIN (SELECT 1 AS x) t304 ON t1.x = t304.x JOIN (SELECT 1 AS x) t305 ON t1.x = t305.x JOIN (SELECT 1 AS x) t306 ON t1.x = t306.x JOIN (SELECT 1 AS x) t307 ON t1.x = t307.x JOIN (SELECT 1 AS x) t308 ON t1.x = t308.x JOIN (SELECT 1 AS x) t309 ON t1.x = t309.x JOIN (SELECT 1 AS x) t310 ON t1.x = t310.x JOIN (SELECT 1 AS x) t311 ON t1.x = t311.x JOIN (SELECT 1 AS x) t312 ON t1.x = t312.x JOIN (SELECT 1 AS x) t313 ON t1.x = t313.x JOIN (SELECT 1 AS x) t314 ON t1.x = t314.x JOIN (SELECT 1 AS x) t315 ON t1.x = t315.x JOIN (SELECT 1 AS x) t316 ON t1.x = t316.x JOIN (SELECT 1 AS x) t317 ON t1.x = t317.x JOIN (SELECT 1 AS x) t318 ON t1.x = t318.x JOIN (SELECT 1 AS x) t319 ON t1.x = t319.x JOIN (SELECT 1 AS x) t320 ON t1.x = t320.x JOIN (SELECT 1 AS x) t321 ON t1.x = t321.x JOIN (SELECT 1 AS x) t322 ON t1.x = t322.x JOIN (SELECT 1 AS x) t323 ON t1.x = t323.x JOIN (SELECT 1 AS x) t324 ON t1.x = t324.x JOIN (SELECT 1 AS x) t325 ON t1.x = t325.x JOIN (SELECT 1 AS x) t326 ON t1.x = t326.x JOIN (SELECT 1 AS x) t327 ON t1.x = t327.x JOIN (SELECT 1 AS x) t328 ON t1.x = t328.x JOIN (SELECT 1 AS x) t329 ON t1.x = t329.x JOIN (SELECT 1 AS x) t330 ON t1.x = t330.x JOIN (SELECT 1 AS x) t331 ON t1.x = t331.x JOIN (SELECT 1 AS x) t332 ON t1.x = t332.x JOIN (SELECT 1 AS x) t333 ON t1.x = t333.x JOIN (SELECT 1 AS x) t334 ON t1.x = t334.x JOIN (SELECT 1 AS x) t335 ON t1.x = t335.x JOIN (SELECT 1 AS x) t336 ON t1.x = t336.x JOIN (SELECT 1 AS x) t337 ON t1.x = t337.x JOIN (SELECT 1 AS x) t338 ON t1.x = t338.x JOIN (SELECT 1 AS x) t339 ON t1.x = t339.x JOIN (SELECT 1 AS x) t340 ON t1.x = t340.x JOIN (SELECT 1 AS x) t341 ON t1.x = t341.x JOIN (SELECT 1 AS x) t342 ON t1.x = t342.x JOIN (SELECT 1 AS x) t343 ON t1.x = t343.x JOIN (SELECT 1 AS x) t344 ON t1.x = t344.x JOIN (SELECT 1 AS x) t345 ON t1.x = t345.x JOIN (SELECT 1 AS x) t346 ON t1.x = t346.x JOIN (SELECT 1 AS x) t347 ON t1.x = t347.x JOIN (SELECT 1 AS x) t348 ON t1.x = t348.x JOIN (SELECT 1 AS x) t349 ON t1.x = t349.x JOIN (SELECT 1 AS x) t350 ON t1.x = t350.x JOIN (SELECT 1 AS x) t351 ON t1.x = t351.x JOIN (SELECT 1 AS x) t352 ON t1.x = t352.x JOIN (SELECT 1 AS x) t353 ON t1.x = t353.x JOIN (SELECT 1 AS x) t354 ON t1.x = t354.x JOIN (SELECT 1 AS x) t355 ON t1.x = t355.x JOIN (SELECT 1 AS x) t356 ON t1.x = t356.x JOIN (SELECT 1 AS x) t357 ON t1.x = t357.x JOIN (SELECT 1 AS x) t358 ON t1.x = t358.x JOIN (SELECT 1 AS x) t359 ON t1.x = t359.x JOIN (SELECT 1 AS x) t360 ON t1.x = t360.x JOIN (SELECT 1 AS x) t361 ON t1.x = t361.x JOIN (SELECT 1 AS x) t362 ON t1.x = t362.x JOIN (SELECT 1 AS x) t363 ON t1.x = t363.x JOIN (SELECT 1 AS x) t364 ON t1.x = t364.x JOIN (SELECT 1 AS x) t365 ON t1.x = t365.x JOIN (SELECT 1 AS x) t366 ON t1.x = t366.x JOIN (SELECT 1 AS x) t367 ON t1.x = t367.x JOIN (SELECT 1 AS x) t368 ON t1.x = t368.x JOIN (SELECT 1 AS x) t369 ON t1.x = t369.x JOIN (SELECT 1 AS x) t370 ON t1.x = t370.x JOIN (SELECT 1 AS x) t371 ON t1.x = t371.x JOIN (SELECT 1 AS x) t372 ON t1.x = t372.x JOIN (SELECT 1 AS x) t373 ON t1.x = t373.x JOIN (SELECT 1 AS x) t374 ON t1.x = t374.x JOIN (SELECT 1 AS x) t375 ON t1.x = t375.x JOIN (SELECT 1 AS x) t376 ON t1.x = t376.x JOIN (SELECT 1 AS x) t377 ON t1.x = t377.x JOIN (SELECT 1 AS x) t378 ON t1.x = t378.x JOIN (SELECT 1 AS x) t379 ON t1.x = t379.x JOIN (SELECT 1 AS x) t380 ON t1.x = t380.x JOIN (SELECT 1 AS x) t381 ON t1.x = t381.x JOIN (SELECT 1 AS x) t382 ON t1.x = t382.x JOIN (SELECT 1 AS x) t383 ON t1.x = t383.x JOIN (SELECT 1 AS x) t384 ON t1.x = t384.x JOIN (SELECT 1 AS x) t385 ON t1.x = t385.x JOIN (SELECT 1 AS x) t386 ON t1.x = t386.x JOIN (SELECT 1 AS x) t387 ON t1.x = t387.x JOIN (SELECT 1 AS x) t388 ON t1.x = t388.x JOIN (SELECT 1 AS x) t389 ON t1.x = t389.x JOIN (SELECT 1 AS x) t390 ON t1.x = t390.x JOIN (SELECT 1 AS x) t391 ON t1.x = t391.x JOIN (SELECT 1 AS x) t392 ON t1.x = t392.x JOIN (SELECT 1 AS x) t393 ON t1.x = t393.x JOIN (SELECT 1 AS x) t394 ON t1.x = t394.x JOIN (SELECT 1 AS x) t395 ON t1.x = t395.x JOIN (SELECT 1 AS x) t396 ON t1.x = t396.x JOIN (SELECT 1 AS x) t397 ON t1.x = t397.x JOIN (SELECT 1 AS x) t398 ON t1.x = t398.x JOIN (SELECT 1 AS x) t399 ON t1.x = t399.x JOIN (SELECT 1 AS x) t400 ON t1.x = t400.x JOIN (SELECT 1 AS x) t401 ON t1.x = t401.x JOIN (SELECT 1 AS x) t402 ON t1.x = t402.x JOIN (SELECT 1 AS x) t403 ON t1.x = t403.x JOIN (SELECT 1 AS x) t404 ON t1.x = t404.x JOIN (SELECT 1 AS x) t405 ON t1.x = t405.x JOIN (SELECT 1 AS x) t406 ON t1.x = t406.x JOIN (SELECT 1 AS x) t407 ON t1.x = t407.x JOIN (SELECT 1 AS x) t408 ON t1.x = t408.x JOIN (SELECT 1 AS x) t409 ON t1.x = t409.x JOIN (SELECT 1 AS x) t410 ON t1.x = t410.x JOIN (SELECT 1 AS x) t411 ON t1.x = t411.x JOIN (SELECT 1 AS x) t412 ON t1.x = t412.x JOIN (SELECT 1 AS x) t413 ON t1.x = t413.x JOIN (SELECT 1 AS x) t414 ON t1.x = t414.x JOIN (SELECT 1 AS x) t415 ON t1.x = t415.x JOIN (SELECT 1 AS x) t416 ON t1.x = t416.x JOIN (SELECT 1 AS x) t417 ON t1.x = t417.x JOIN (SELECT 1 AS x) t418 ON t1.x = t418.x JOIN (SELECT 1 AS x) t419 ON t1.x = t419.x JOIN (SELECT 1 AS x) t420 ON t1.x = t420.x JOIN (SELECT 1 AS x) t421 ON t1.x = t421.x JOIN (SELECT 1 AS x) t422 ON t1.x = t422.x JOIN (SELECT 1 AS x) t423 ON t1.x = t423.x JOIN (SELECT 1 AS x) t424 ON t1.x = t424.x JOIN (SELECT 1 AS x) t425 ON t1.x = t425.x JOIN (SELECT 1 AS x) t426 ON t1.x = t426.x JOIN (SELECT 1 AS x) t427 ON t1.x = t427.x JOIN (SELECT 1 AS x) t428 ON t1.x = t428.x JOIN (SELECT 1 AS x) t429 ON t1.x = t429.x JOIN (SELECT 1 AS x) t430 ON t1.x = t430.x JOIN (SELECT 1 AS x) t431 ON t1.x = t431.x JOIN (SELECT 1 AS x) t432 ON t1.x = t432.x JOIN (SELECT 1 AS x) t433 ON t1.x = t433.x JOIN (SELECT 1 AS x) t434 ON t1.x = t434.x JOIN (SELECT 1 AS x) t435 ON t1.x = t435.x JOIN (SELECT 1 AS x) t436 ON t1.x = t436.x JOIN (SELECT 1 AS x) t437 ON t1.x = t437.x JOIN (SELECT 1 AS x) t438 ON t1.x = t438.x JOIN (SELECT 1 AS x) t439 ON t1.x = t439.x JOIN (SELECT 1 AS x) t440 ON t1.x = t440.x JOIN (SELECT 1 AS x) t441 ON t1.x = t441.x JOIN (SELECT 1 AS x) t442 ON t1.x = t442.x JOIN (SELECT 1 AS x) t443 ON t1.x = t443.x JOIN (SELECT 1 AS x) t444 ON t1.x = t444.x JOIN (SELECT 1 AS x) t445 ON t1.x = t445.x JOIN (SELECT 1 AS x) t446 ON t1.x = t446.x JOIN (SELECT 1 AS x) t447 ON t1.x = t447.x JOIN (SELECT 1 AS x) t448 ON t1.x = t448.x JOIN (SELECT 1 AS x) t449 ON t1.x = t449.x JOIN (SELECT 1 AS x) t450 ON t1.x = t450.x JOIN (SELECT 1 AS x) t451 ON t1.x = t451.x JOIN (SELECT 1 AS x) t452 ON t1.x = t452.x JOIN (SELECT 1 AS x) t453 ON t1.x = t453.x JOIN (SELECT 1 AS x) t454 ON t1.x = t454.x JOIN (SELECT 1 AS x) t455 ON t1.x = t455.x JOIN (SELECT 1 AS x) t456 ON t1.x = t456.x JOIN (SELECT 1 AS x) t457 ON t1.x = t457.x JOIN (SELECT 1 AS x) t458 ON t1.x = t458.x JOIN (SELECT 1 AS x) t459 ON t1.x = t459.x JOIN (SELECT 1 AS x) t460 ON t1.x = t460.x JOIN (SELECT 1 AS x) t461 ON t1.x = t461.x JOIN (SELECT 1 AS x) t462 ON t1.x = t462.x JOIN (SELECT 1 AS x) t463 ON t1.x = t463.x JOIN (SELECT 1 AS x) t464 ON t1.x = t464.x JOIN (SELECT 1 AS x) t465 ON t1.x = t465.x JOIN (SELECT 1 AS x) t466 ON t1.x = t466.x JOIN (SELECT 1 AS x) t467 ON t1.x = t467.x JOIN (SELECT 1 AS x) t468 ON t1.x = t468.x JOIN (SELECT 1 AS x) t469 ON t1.x = t469.x JOIN (SELECT 1 AS x) t470 ON t1.x = t470.x JOIN (SELECT 1 AS x) t471 ON t1.x = t471.x JOIN (SELECT 1 AS x) t472 ON t1.x = t472.x JOIN (SELECT 1 AS x) t473 ON t1.x = t473.x JOIN (SELECT 1 AS x) t474 ON t1.x = t474.x JOIN (SELECT 1 AS x) t475 ON t1.x = t475.x JOIN (SELECT 1 AS x) t476 ON t1.x = t476.x JOIN (SELECT 1 AS x) t477 ON t1.x = t477.x JOIN (SELECT 1 AS x) t478 ON t1.x = t478.x JOIN (SELECT 1 AS x) t479 ON t1.x = t479.x JOIN (SELECT 1 AS x) t480 ON t1.x = t480.x JOIN (SELECT 1 AS x) t481 ON t1.x = t481.x JOIN (SELECT 1 AS x) t482 ON t1.x = t482.x JOIN (SELECT 1 AS x) t483 ON t1.x = t483.x JOIN (SELECT 1 AS x) t484 ON t1.x = t484.x JOIN (SELECT 1 AS x) t485 ON t1.x = t485.x JOIN (SELECT 1 AS x) t486 ON t1.x = t486.x JOIN (SELECT 1 AS x) t487 ON t1.x = t487.x JOIN (SELECT 1 AS x) t488 ON t1.x = t488.x JOIN (SELECT 1 AS x) t489 ON t1.x = t489.x JOIN (SELECT 1 AS x) t490 ON t1.x = t490.x JOIN (SELECT 1 AS x) t491 ON t1.x = t491.x JOIN (SELECT 1 AS x) t492 ON t1.x = t492.x JOIN (SELECT 1 AS x) t493 ON t1.x = t493.x JOIN (SELECT 1 AS x) t494 ON t1.x = t494.x JOIN (SELECT 1 AS x) t495 ON t1.x = t495.x JOIN (SELECT 1 AS x) t496 ON t1.x = t496.x JOIN (SELECT 1 AS x) t497 ON t1.x = t497.x JOIN (SELECT 1 AS x) t498 ON t1.x = t498.x JOIN (SELECT 1 AS x) t499 ON t1.x = t499.x JOIN (SELECT 1 AS x) t500 ON t1.x = t500.x JOIN (SELECT 1 AS x) t501 ON t1.x = t501.x JOIN (SELECT 1 AS x) t502 ON t1.x = t502.x JOIN (SELECT 1 AS x) t503 ON t1.x = t503.x JOIN (SELECT 1 AS x) t504 ON t1.x = t504.x JOIN (SELECT 1 AS x) t505 ON t1.x = t505.x JOIN (SELECT 1 AS x) t506 ON t1.x = t506.x JOIN (SELECT 1 AS x) t507 ON t1.x = t507.x JOIN (SELECT 1 AS x) t508 ON t1.x = t508.x JOIN (SELECT 1 AS x) t509 ON t1.x = t509.x JOIN (SELECT 1 AS x) t510 ON t1.x = t510.x JOIN (SELECT 1 AS x) t511 ON t1.x = t511.x JOIN (SELECT 1 AS x) t512 ON t1.x = t512.x JOIN (SELECT 1 AS x) t513 ON t1.x = t513.x JOIN (SELECT 1 AS x) t514 ON t1.x = t514.x JOIN (SELECT 1 AS x) t515 ON t1.x = t515.x JOIN (SELECT 1 AS x) t516 ON t1.x = t516.x JOIN (SELECT 1 AS x) t517 ON t1.x = t517.x JOIN (SELECT 1 AS x) t518 ON t1.x = t518.x JOIN (SELECT 1 AS x) t519 ON t1.x = t519.x JOIN (SELECT 1 AS x) t520 ON t1.x = t520.x JOIN (SELECT 1 AS x) t521 ON t1.x = t521.x JOIN (SELECT 1 AS x) t522 ON t1.x = t522.x JOIN (SELECT 1 AS x) t523 ON t1.x = t523.x JOIN (SELECT 1 AS x) t524 ON t1.x = t524.x JOIN (SELECT 1 AS x) t525 ON t1.x = t525.x JOIN (SELECT 1 AS x) t526 ON t1.x = t526.x JOIN (SELECT 1 AS x) t527 ON t1.x = t527.x JOIN (SELECT 1 AS x) t528 ON t1.x = t528.x JOIN (SELECT 1 AS x) t529 ON t1.x = t529.x JOIN (SELECT 1 AS x) t530 ON t1.x = t530.x JOIN (SELECT 1 AS x) t531 ON t1.x = t531.x JOIN (SELECT 1 AS x) t532 ON t1.x = t532.x JOIN (SELECT 1 AS x) t533 ON t1.x = t533.x JOIN (SELECT 1 AS x) t534 ON t1.x = t534.x JOIN (SELECT 1 AS x) t535 ON t1.x = t535.x JOIN (SELECT 1 AS x) t536 ON t1.x = t536.x JOIN (SELECT 1 AS x) t537 ON t1.x = t537.x JOIN (SELECT 1 AS x) t538 ON t1.x = t538.x JOIN (SELECT 1 AS x) t539 ON t1.x = t539.x JOIN (SELECT 1 AS x) t540 ON t1.x = t540.x JOIN (SELECT 1 AS x) t541 ON t1.x = t541.x JOIN (SELECT 1 AS x) t542 ON t1.x = t542.x JOIN (SELECT 1 AS x) t543 ON t1.x = t543.x JOIN (SELECT 1 AS x) t544 ON t1.x = t544.x JOIN (SELECT 1 AS x) t545 ON t1.x = t545.x JOIN (SELECT 1 AS x) t546 ON t1.x = t546.x JOIN (SELECT 1 AS x) t547 ON t1.x = t547.x JOIN (SELECT 1 AS x) t548 ON t1.x = t548.x JOIN (SELECT 1 AS x) t549 ON t1.x = t549.x JOIN (SELECT 1 AS x) t550 ON t1.x = t550.x JOIN (SELECT 1 AS x) t551 ON t1.x = t551.x JOIN (SELECT 1 AS x) t552 ON t1.x = t552.x JOIN (SELECT 1 AS x) t553 ON t1.x = t553.x JOIN (SELECT 1 AS x) t554 ON t1.x = t554.x JOIN (SELECT 1 AS x) t555 ON t1.x = t555.x JOIN (SELECT 1 AS x) t556 ON t1.x = t556.x JOIN (SELECT 1 AS x) t557 ON t1.x = t557.x JOIN (SELECT 1 AS x) t558 ON t1.x = t558.x JOIN (SELECT 1 AS x) t559 ON t1.x = t559.x JOIN (SELECT 1 AS x) t560 ON t1.x = t560.x JOIN (SELECT 1 AS x) t561 ON t1.x = t561.x JOIN (SELECT 1 AS x) t562 ON t1.x = t562.x JOIN (SELECT 1 AS x) t563 ON t1.x = t563.x JOIN (SELECT 1 AS x) t564 ON t1.x = t564.x JOIN (SELECT 1 AS x) t565 ON t1.x = t565.x JOIN (SELECT 1 AS x) t566 ON t1.x = t566.x JOIN (SELECT 1 AS x) t567 ON t1.x = t567.x JOIN (SELECT 1 AS x) t568 ON t1.x = t568.x JOIN (SELECT 1 AS x) t569 ON t1.x = t569.x JOIN (SELECT 1 AS x) t570 ON t1.x = t570.x JOIN (SELECT 1 AS x) t571 ON t1.x = t571.x JOIN (SELECT 1 AS x) t572 ON t1.x = t572.x JOIN (SELECT 1 AS x) t573 ON t1.x = t573.x JOIN (SELECT 1 AS x) t574 ON t1.x = t574.x JOIN (SELECT 1 AS x) t575 ON t1.x = t575.x JOIN (SELECT 1 AS x) t576 ON t1.x = t576.x JOIN (SELECT 1 AS x) t577 ON t1.x = t577.x JOIN (SELECT 1 AS x) t578 ON t1.x = t578.x JOIN (SELECT 1 AS x) t579 ON t1.x = t579.x JOIN (SELECT 1 AS x) t580 ON t1.x = t580.x JOIN (SELECT 1 AS x) t581 ON t1.x = t581.x JOIN (SELECT 1 AS x) t582 ON t1.x = t582.x JOIN (SELECT 1 AS x) t583 ON t1.x = t583.x JOIN (SELECT 1 AS x) t584 ON t1.x = t584.x JOIN (SELECT 1 AS x) t585 ON t1.x = t585.x JOIN (SELECT 1 AS x) t586 ON t1.x = t586.x JOIN (SELECT 1 AS x) t587 ON t1.x = t587.x JOIN (SELECT 1 AS x) t588 ON t1.x = t588.x JOIN (SELECT 1 AS x) t589 ON t1.x = t589.x JOIN (SELECT 1 AS x) t590 ON t1.x = t590.x JOIN (SELECT 1 AS x) t591 ON t1.x = t591.x JOIN (SELECT 1 AS x) t592 ON t1.x = t592.x JOIN (SELECT 1 AS x) t593 ON t1.x = t593.x JOIN (SELECT 1 AS x) t594 ON t1.x = t594.x JOIN (SELECT 1 AS x) t595 ON t1.x = t595.x JOIN (SELECT 1 AS x) t596 ON t1.x = t596.x JOIN (SELECT 1 AS x) t597 ON t1.x = t597.x JOIN (SELECT 1 AS x) t598 ON t1.x = t598.x JOIN (SELECT 1 AS x) t599 ON t1.x = t599.x JOIN (SELECT 1 AS x) t600 ON t1.x = t600.x JOIN (SELECT 1 AS x) t601 ON t1.x = t601.x JOIN (SELECT 1 AS x) t602 ON t1.x = t602.x JOIN (SELECT 1 AS x) t603 ON t1.x = t603.x JOIN (SELECT 1 AS x) t604 ON t1.x = t604.x JOIN (SELECT 1 AS x) t605 ON t1.x = t605.x JOIN (SELECT 1 AS x) t606 ON t1.x = t606.x JOIN (SELECT 1 AS x) t607 ON t1.x = t607.x JOIN (SELECT 1 AS x) t608 ON t1.x = t608.x JOIN (SELECT 1 AS x) t609 ON t1.x = t609.x JOIN (SELECT 1 AS x) t610 ON t1.x = t610.x JOIN (SELECT 1 AS x) t611 ON t1.x = t611.x JOIN (SELECT 1 AS x) t612 ON t1.x = t612.x JOIN (SELECT 1 AS x) t613 ON t1.x = t613.x JOIN (SELECT 1 AS x) t614 ON t1.x = t614.x JOIN (SELECT 1 AS x) t615 ON t1.x = t615.x JOIN (SELECT 1 AS x) t616 ON t1.x = t616.x JOIN (SELECT 1 AS x) t617 ON t1.x = t617.x JOIN (SELECT 1 AS x) t618 ON t1.x = t618.x JOIN (SELECT 1 AS x) t619 ON t1.x = t619.x JOIN (SELECT 1 AS x) t620 ON t1.x = t620.x JOIN (SELECT 1 AS x) t621 ON t1.x = t621.x JOIN (SELECT 1 AS x) t622 ON t1.x = t622.x JOIN (SELECT 1 AS x) t623 ON t1.x = t623.x JOIN (SELECT 1 AS x) t624 ON t1.x = t624.x JOIN (SELECT 1 AS x) t625 ON t1.x = t625.x JOIN (SELECT 1 AS x) t626 ON t1.x = t626.x JOIN (SELECT 1 AS x) t627 ON t1.x = t627.x JOIN (SELECT 1 AS x) t628 ON t1.x = t628.x JOIN (SELECT 1 AS x) t629 ON t1.x = t629.x JOIN (SELECT 1 AS x) t630 ON t1.x = t630.x JOIN (SELECT 1 AS x) t631 ON t1.x = t631.x JOIN (SELECT 1 AS x) t632 ON t1.x = t632.x JOIN (SELECT 1 AS x) t633 ON t1.x = t633.x JOIN (SELECT 1 AS x) t634 ON t1.x = t634.x JOIN (SELECT 1 AS x) t635 ON t1.x = t635.x JOIN (SELECT 1 AS x) t636 ON t1.x = t636.x JOIN (SELECT 1 AS x) t637 ON t1.x = t637.x JOIN (SELECT 1 AS x) t638 ON t1.x = t638.x JOIN (SELECT 1 AS x) t639 ON t1.x = t639.x JOIN (SELECT 1 AS x) t640 ON t1.x = t640.x JOIN (SELECT 1 AS x) t641 ON t1.x = t641.x JOIN (SELECT 1 AS x) t642 ON t1.x = t642.x JOIN (SELECT 1 AS x) t643 ON t1.x = t643.x JOIN (SELECT 1 AS x) t644 ON t1.x = t644.x JOIN (SELECT 1 AS x) t645 ON t1.x = t645.x JOIN (SELECT 1 AS x) t646 ON t1.x = t646.x JOIN (SELECT 1 AS x) t647 ON t1.x = t647.x JOIN (SELECT 1 AS x) t648 ON t1.x = t648.x JOIN (SELECT 1 AS x) t649 ON t1.x = t649.x JOIN (SELECT 1 AS x) t650 ON t1.x = t650.x JOIN (SELECT 1 AS x) t651 ON t1.x = t651.x JOIN (SELECT 1 AS x) t652 ON t1.x = t652.x JOIN (SELECT 1 AS x) t653 ON t1.x = t653.x JOIN (SELECT 1 AS x) t654 ON t1.x = t654.x JOIN (SELECT 1 AS x) t655 ON t1.x = t655.x JOIN (SELECT 1 AS x) t656 ON t1.x = t656.x JOIN (SELECT 1 AS x) t657 ON t1.x = t657.x JOIN (SELECT 1 AS x) t658 ON t1.x = t658.x JOIN (SELECT 1 AS x) t659 ON t1.x = t659.x JOIN (SELECT 1 AS x) t660 ON t1.x = t660.x JOIN (SELECT 1 AS x) t661 ON t1.x = t661.x JOIN (SELECT 1 AS x) t662 ON t1.x = t662.x JOIN (SELECT 1 AS x) t663 ON t1.x = t663.x JOIN (SELECT 1 AS x) t664 ON t1.x = t664.x JOIN (SELECT 1 AS x) t665 ON t1.x = t665.x JOIN (SELECT 1 AS x) t666 ON t1.x = t666.x + + diff --git a/tests/queries/0_stateless/03094_one_thousand_joins.reference b/tests/queries/0_stateless/03094_one_thousand_joins.reference new file mode 100644 index 00000000000..52176202f05 --- /dev/null +++ b/tests/queries/0_stateless/03094_one_thousand_joins.reference @@ -0,0 +1 @@ +1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 diff --git a/tests/queries/0_stateless/03094_one_thousand_joins.sql b/tests/queries/0_stateless/03094_one_thousand_joins.sql new file mode 100644 index 00000000000..f7ca2134d2e --- /dev/null +++ b/tests/queries/0_stateless/03094_one_thousand_joins.sql @@ -0,0 +1,4 @@ +-- Tags: no-fasttest, long + +-- Bug 33446, marked as 'long' because it still runs around 10 sec +SELECT * FROM (SELECT 1 AS x) t1 JOIN (SELECT 1 AS x) t2 ON t1.x = t2.x JOIN (SELECT 1 AS x) t3 ON t1.x = t3.x JOIN (SELECT 1 AS x) t4 ON t1.x = t4.x JOIN (SELECT 1 AS x) t5 ON t1.x = t5.x JOIN (SELECT 1 AS x) t6 ON t1.x = t6.x JOIN (SELECT 1 AS x) t7 ON t1.x = t7.x JOIN (SELECT 1 AS x) t8 ON t1.x = t8.x JOIN (SELECT 1 AS x) t9 ON t1.x = t9.x JOIN (SELECT 1 AS x) t10 ON t1.x = t10.x JOIN (SELECT 1 AS x) t11 ON t1.x = t11.x JOIN (SELECT 1 AS x) t12 ON t1.x = t12.x JOIN (SELECT 1 AS x) t13 ON t1.x = t13.x JOIN (SELECT 1 AS x) t14 ON t1.x = t14.x JOIN (SELECT 1 AS x) t15 ON t1.x = t15.x JOIN (SELECT 1 AS x) t16 ON t1.x = t16.x JOIN (SELECT 1 AS x) t17 ON t1.x = t17.x JOIN (SELECT 1 AS x) t18 ON t1.x = t18.x JOIN (SELECT 1 AS x) t19 ON t1.x = t19.x JOIN (SELECT 1 AS x) t20 ON t1.x = t20.x JOIN (SELECT 1 AS x) t21 ON t1.x = t21.x JOIN (SELECT 1 AS x) t22 ON t1.x = t22.x JOIN (SELECT 1 AS x) t23 ON t1.x = t23.x JOIN (SELECT 1 AS x) t24 ON t1.x = t24.x JOIN (SELECT 1 AS x) t25 ON t1.x = t25.x JOIN (SELECT 1 AS x) t26 ON t1.x = t26.x JOIN (SELECT 1 AS x) t27 ON t1.x = t27.x JOIN (SELECT 1 AS x) t28 ON t1.x = t28.x JOIN (SELECT 1 AS x) t29 ON t1.x = t29.x JOIN (SELECT 1 AS x) t30 ON t1.x = t30.x JOIN (SELECT 1 AS x) t31 ON t1.x = t31.x JOIN (SELECT 1 AS x) t32 ON t1.x = t32.x JOIN (SELECT 1 AS x) t33 ON t1.x = t33.x JOIN (SELECT 1 AS x) t34 ON t1.x = t34.x JOIN (SELECT 1 AS x) t35 ON t1.x = t35.x JOIN (SELECT 1 AS x) t36 ON t1.x = t36.x JOIN (SELECT 1 AS x) t37 ON t1.x = t37.x JOIN (SELECT 1 AS x) t38 ON t1.x = t38.x JOIN (SELECT 1 AS x) t39 ON t1.x = t39.x JOIN (SELECT 1 AS x) t40 ON t1.x = t40.x JOIN (SELECT 1 AS x) t41 ON t1.x = t41.x JOIN (SELECT 1 AS x) t42 ON t1.x = t42.x JOIN (SELECT 1 AS x) t43 ON t1.x = t43.x JOIN (SELECT 1 AS x) t44 ON t1.x = t44.x JOIN (SELECT 1 AS x) t45 ON t1.x = t45.x JOIN (SELECT 1 AS x) t46 ON t1.x = t46.x JOIN (SELECT 1 AS x) t47 ON t1.x = t47.x JOIN (SELECT 1 AS x) t48 ON t1.x = t48.x JOIN (SELECT 1 AS x) t49 ON t1.x = t49.x JOIN (SELECT 1 AS x) t50 ON t1.x = t50.x JOIN (SELECT 1 AS x) t51 ON t1.x = t51.x JOIN (SELECT 1 AS x) t52 ON t1.x = t52.x JOIN (SELECT 1 AS x) t53 ON t1.x = t53.x JOIN (SELECT 1 AS x) t54 ON t1.x = t54.x JOIN (SELECT 1 AS x) t55 ON t1.x = t55.x JOIN (SELECT 1 AS x) t56 ON t1.x = t56.x JOIN (SELECT 1 AS x) t57 ON t1.x = t57.x JOIN (SELECT 1 AS x) t58 ON t1.x = t58.x JOIN (SELECT 1 AS x) t59 ON t1.x = t59.x JOIN (SELECT 1 AS x) t60 ON t1.x = t60.x JOIN (SELECT 1 AS x) t61 ON t1.x = t61.x JOIN (SELECT 1 AS x) t62 ON t1.x = t62.x JOIN (SELECT 1 AS x) t63 ON t1.x = t63.x JOIN (SELECT 1 AS x) t64 ON t1.x = t64.x JOIN (SELECT 1 AS x) t65 ON t1.x = t65.x JOIN (SELECT 1 AS x) t66 ON t1.x = t66.x JOIN (SELECT 1 AS x) t67 ON t1.x = t67.x JOIN (SELECT 1 AS x) t68 ON t1.x = t68.x JOIN (SELECT 1 AS x) t69 ON t1.x = t69.x JOIN (SELECT 1 AS x) t70 ON t1.x = t70.x JOIN (SELECT 1 AS x) t71 ON t1.x = t71.x JOIN (SELECT 1 AS x) t72 ON t1.x = t72.x JOIN (SELECT 1 AS x) t73 ON t1.x = t73.x JOIN (SELECT 1 AS x) t74 ON t1.x = t74.x JOIN (SELECT 1 AS x) t75 ON t1.x = t75.x JOIN (SELECT 1 AS x) t76 ON t1.x = t76.x JOIN (SELECT 1 AS x) t77 ON t1.x = t77.x JOIN (SELECT 1 AS x) t78 ON t1.x = t78.x JOIN (SELECT 1 AS x) t79 ON t1.x = t79.x JOIN (SELECT 1 AS x) t80 ON t1.x = t80.x JOIN (SELECT 1 AS x) t81 ON t1.x = t81.x JOIN (SELECT 1 AS x) t82 ON t1.x = t82.x JOIN (SELECT 1 AS x) t83 ON t1.x = t83.x JOIN (SELECT 1 AS x) t84 ON t1.x = t84.x JOIN (SELECT 1 AS x) t85 ON t1.x = t85.x JOIN (SELECT 1 AS x) t86 ON t1.x = t86.x JOIN (SELECT 1 AS x) t87 ON t1.x = t87.x JOIN (SELECT 1 AS x) t88 ON t1.x = t88.x JOIN (SELECT 1 AS x) t89 ON t1.x = t89.x JOIN (SELECT 1 AS x) t90 ON t1.x = t90.x JOIN (SELECT 1 AS x) t91 ON t1.x = t91.x JOIN (SELECT 1 AS x) t92 ON t1.x = t92.x JOIN (SELECT 1 AS x) t93 ON t1.x = t93.x JOIN (SELECT 1 AS x) t94 ON t1.x = t94.x JOIN (SELECT 1 AS x) t95 ON t1.x = t95.x JOIN (SELECT 1 AS x) t96 ON t1.x = t96.x JOIN (SELECT 1 AS x) t97 ON t1.x = t97.x JOIN (SELECT 1 AS x) t98 ON t1.x = t98.x JOIN (SELECT 1 AS x) t99 ON t1.x = t99.x JOIN (SELECT 1 AS x) t100 ON t1.x = t100.x JOIN (SELECT 1 AS x) t101 ON t1.x = t101.x JOIN (SELECT 1 AS x) t102 ON t1.x = t102.x JOIN (SELECT 1 AS x) t103 ON t1.x = t103.x JOIN (SELECT 1 AS x) t104 ON t1.x = t104.x JOIN (SELECT 1 AS x) t105 ON t1.x = t105.x JOIN (SELECT 1 AS x) t106 ON t1.x = t106.x JOIN (SELECT 1 AS x) t107 ON t1.x = t107.x JOIN (SELECT 1 AS x) t108 ON t1.x = t108.x JOIN (SELECT 1 AS x) t109 ON t1.x = t109.x JOIN (SELECT 1 AS x) t110 ON t1.x = t110.x JOIN (SELECT 1 AS x) t111 ON t1.x = t111.x JOIN (SELECT 1 AS x) t112 ON t1.x = t112.x JOIN (SELECT 1 AS x) t113 ON t1.x = t113.x JOIN (SELECT 1 AS x) t114 ON t1.x = t114.x JOIN (SELECT 1 AS x) t115 ON t1.x = t115.x JOIN (SELECT 1 AS x) t116 ON t1.x = t116.x JOIN (SELECT 1 AS x) t117 ON t1.x = t117.x JOIN (SELECT 1 AS x) t118 ON t1.x = t118.x JOIN (SELECT 1 AS x) t119 ON t1.x = t119.x JOIN (SELECT 1 AS x) t120 ON t1.x = t120.x JOIN (SELECT 1 AS x) t121 ON t1.x = t121.x JOIN (SELECT 1 AS x) t122 ON t1.x = t122.x JOIN (SELECT 1 AS x) t123 ON t1.x = t123.x JOIN (SELECT 1 AS x) t124 ON t1.x = t124.x JOIN (SELECT 1 AS x) t125 ON t1.x = t125.x JOIN (SELECT 1 AS x) t126 ON t1.x = t126.x JOIN (SELECT 1 AS x) t127 ON t1.x = t127.x JOIN (SELECT 1 AS x) t128 ON t1.x = t128.x JOIN (SELECT 1 AS x) t129 ON t1.x = t129.x JOIN (SELECT 1 AS x) t130 ON t1.x = t130.x JOIN (SELECT 1 AS x) t131 ON t1.x = t131.x JOIN (SELECT 1 AS x) t132 ON t1.x = t132.x JOIN (SELECT 1 AS x) t133 ON t1.x = t133.x JOIN (SELECT 1 AS x) t134 ON t1.x = t134.x JOIN (SELECT 1 AS x) t135 ON t1.x = t135.x JOIN (SELECT 1 AS x) t136 ON t1.x = t136.x JOIN (SELECT 1 AS x) t137 ON t1.x = t137.x JOIN (SELECT 1 AS x) t138 ON t1.x = t138.x JOIN (SELECT 1 AS x) t139 ON t1.x = t139.x JOIN (SELECT 1 AS x) t140 ON t1.x = t140.x JOIN (SELECT 1 AS x) t141 ON t1.x = t141.x JOIN (SELECT 1 AS x) t142 ON t1.x = t142.x JOIN (SELECT 1 AS x) t143 ON t1.x = t143.x JOIN (SELECT 1 AS x) t144 ON t1.x = t144.x JOIN (SELECT 1 AS x) t145 ON t1.x = t145.x JOIN (SELECT 1 AS x) t146 ON t1.x = t146.x JOIN (SELECT 1 AS x) t147 ON t1.x = t147.x JOIN (SELECT 1 AS x) t148 ON t1.x = t148.x JOIN (SELECT 1 AS x) t149 ON t1.x = t149.x JOIN (SELECT 1 AS x) t150 ON t1.x = t150.x JOIN (SELECT 1 AS x) t151 ON t1.x = t151.x JOIN (SELECT 1 AS x) t152 ON t1.x = t152.x JOIN (SELECT 1 AS x) t153 ON t1.x = t153.x JOIN (SELECT 1 AS x) t154 ON t1.x = t154.x JOIN (SELECT 1 AS x) t155 ON t1.x = t155.x JOIN (SELECT 1 AS x) t156 ON t1.x = t156.x JOIN (SELECT 1 AS x) t157 ON t1.x = t157.x JOIN (SELECT 1 AS x) t158 ON t1.x = t158.x JOIN (SELECT 1 AS x) t159 ON t1.x = t159.x JOIN (SELECT 1 AS x) t160 ON t1.x = t160.x JOIN (SELECT 1 AS x) t161 ON t1.x = t161.x JOIN (SELECT 1 AS x) t162 ON t1.x = t162.x JOIN (SELECT 1 AS x) t163 ON t1.x = t163.x JOIN (SELECT 1 AS x) t164 ON t1.x = t164.x JOIN (SELECT 1 AS x) t165 ON t1.x = t165.x JOIN (SELECT 1 AS x) t166 ON t1.x = t166.x JOIN (SELECT 1 AS x) t167 ON t1.x = t167.x JOIN (SELECT 1 AS x) t168 ON t1.x = t168.x JOIN (SELECT 1 AS x) t169 ON t1.x = t169.x JOIN (SELECT 1 AS x) t170 ON t1.x = t170.x JOIN (SELECT 1 AS x) t171 ON t1.x = t171.x JOIN (SELECT 1 AS x) t172 ON t1.x = t172.x JOIN (SELECT 1 AS x) t173 ON t1.x = t173.x JOIN (SELECT 1 AS x) t174 ON t1.x = t174.x JOIN (SELECT 1 AS x) t175 ON t1.x = t175.x JOIN (SELECT 1 AS x) t176 ON t1.x = t176.x JOIN (SELECT 1 AS x) t177 ON t1.x = t177.x JOIN (SELECT 1 AS x) t178 ON t1.x = t178.x JOIN (SELECT 1 AS x) t179 ON t1.x = t179.x JOIN (SELECT 1 AS x) t180 ON t1.x = t180.x JOIN (SELECT 1 AS x) t181 ON t1.x = t181.x JOIN (SELECT 1 AS x) t182 ON t1.x = t182.x JOIN (SELECT 1 AS x) t183 ON t1.x = t183.x JOIN (SELECT 1 AS x) t184 ON t1.x = t184.x JOIN (SELECT 1 AS x) t185 ON t1.x = t185.x JOIN (SELECT 1 AS x) t186 ON t1.x = t186.x JOIN (SELECT 1 AS x) t187 ON t1.x = t187.x JOIN (SELECT 1 AS x) t188 ON t1.x = t188.x JOIN (SELECT 1 AS x) t189 ON t1.x = t189.x JOIN (SELECT 1 AS x) t190 ON t1.x = t190.x JOIN (SELECT 1 AS x) t191 ON t1.x = t191.x JOIN (SELECT 1 AS x) t192 ON t1.x = t192.x JOIN (SELECT 1 AS x) t193 ON t1.x = t193.x JOIN (SELECT 1 AS x) t194 ON t1.x = t194.x JOIN (SELECT 1 AS x) t195 ON t1.x = t195.x JOIN (SELECT 1 AS x) t196 ON t1.x = t196.x JOIN (SELECT 1 AS x) t197 ON t1.x = t197.x JOIN (SELECT 1 AS x) t198 ON t1.x = t198.x JOIN (SELECT 1 AS x) t199 ON t1.x = t199.x JOIN (SELECT 1 AS x) t200 ON t1.x = t200.x JOIN (SELECT 1 AS x) t201 ON t1.x = t201.x JOIN (SELECT 1 AS x) t202 ON t1.x = t202.x JOIN (SELECT 1 AS x) t203 ON t1.x = t203.x JOIN (SELECT 1 AS x) t204 ON t1.x = t204.x JOIN (SELECT 1 AS x) t205 ON t1.x = t205.x JOIN (SELECT 1 AS x) t206 ON t1.x = t206.x JOIN (SELECT 1 AS x) t207 ON t1.x = t207.x JOIN (SELECT 1 AS x) t208 ON t1.x = t208.x JOIN (SELECT 1 AS x) t209 ON t1.x = t209.x JOIN (SELECT 1 AS x) t210 ON t1.x = t210.x JOIN (SELECT 1 AS x) t211 ON t1.x = t211.x JOIN (SELECT 1 AS x) t212 ON t1.x = t212.x JOIN (SELECT 1 AS x) t213 ON t1.x = t213.x JOIN (SELECT 1 AS x) t214 ON t1.x = t214.x JOIN (SELECT 1 AS x) t215 ON t1.x = t215.x JOIN (SELECT 1 AS x) t216 ON t1.x = t216.x JOIN (SELECT 1 AS x) t217 ON t1.x = t217.x JOIN (SELECT 1 AS x) t218 ON t1.x = t218.x JOIN (SELECT 1 AS x) t219 ON t1.x = t219.x JOIN (SELECT 1 AS x) t220 ON t1.x = t220.x JOIN (SELECT 1 AS x) t221 ON t1.x = t221.x JOIN (SELECT 1 AS x) t222 ON t1.x = t222.x JOIN (SELECT 1 AS x) t223 ON t1.x = t223.x JOIN (SELECT 1 AS x) t224 ON t1.x = t224.x JOIN (SELECT 1 AS x) t225 ON t1.x = t225.x JOIN (SELECT 1 AS x) t226 ON t1.x = t226.x JOIN (SELECT 1 AS x) t227 ON t1.x = t227.x JOIN (SELECT 1 AS x) t228 ON t1.x = t228.x JOIN (SELECT 1 AS x) t229 ON t1.x = t229.x JOIN (SELECT 1 AS x) t230 ON t1.x = t230.x JOIN (SELECT 1 AS x) t231 ON t1.x = t231.x JOIN (SELECT 1 AS x) t232 ON t1.x = t232.x JOIN (SELECT 1 AS x) t233 ON t1.x = t233.x JOIN (SELECT 1 AS x) t234 ON t1.x = t234.x JOIN (SELECT 1 AS x) t235 ON t1.x = t235.x JOIN (SELECT 1 AS x) t236 ON t1.x = t236.x JOIN (SELECT 1 AS x) t237 ON t1.x = t237.x JOIN (SELECT 1 AS x) t238 ON t1.x = t238.x JOIN (SELECT 1 AS x) t239 ON t1.x = t239.x JOIN (SELECT 1 AS x) t240 ON t1.x = t240.x JOIN (SELECT 1 AS x) t241 ON t1.x = t241.x JOIN (SELECT 1 AS x) t242 ON t1.x = t242.x JOIN (SELECT 1 AS x) t243 ON t1.x = t243.x JOIN (SELECT 1 AS x) t244 ON t1.x = t244.x JOIN (SELECT 1 AS x) t245 ON t1.x = t245.x JOIN (SELECT 1 AS x) t246 ON t1.x = t246.x JOIN (SELECT 1 AS x) t247 ON t1.x = t247.x JOIN (SELECT 1 AS x) t248 ON t1.x = t248.x JOIN (SELECT 1 AS x) t249 ON t1.x = t249.x JOIN (SELECT 1 AS x) t250 ON t1.x = t250.x JOIN (SELECT 1 AS x) t251 ON t1.x = t251.x JOIN (SELECT 1 AS x) t252 ON t1.x = t252.x JOIN (SELECT 1 AS x) t253 ON t1.x = t253.x JOIN (SELECT 1 AS x) t254 ON t1.x = t254.x JOIN (SELECT 1 AS x) t255 ON t1.x = t255.x JOIN (SELECT 1 AS x) t256 ON t1.x = t256.x JOIN (SELECT 1 AS x) t257 ON t1.x = t257.x JOIN (SELECT 1 AS x) t258 ON t1.x = t258.x JOIN (SELECT 1 AS x) t259 ON t1.x = t259.x JOIN (SELECT 1 AS x) t260 ON t1.x = t260.x JOIN (SELECT 1 AS x) t261 ON t1.x = t261.x JOIN (SELECT 1 AS x) t262 ON t1.x = t262.x JOIN (SELECT 1 AS x) t263 ON t1.x = t263.x JOIN (SELECT 1 AS x) t264 ON t1.x = t264.x JOIN (SELECT 1 AS x) t265 ON t1.x = t265.x JOIN (SELECT 1 AS x) t266 ON t1.x = t266.x JOIN (SELECT 1 AS x) t267 ON t1.x = t267.x JOIN (SELECT 1 AS x) t268 ON t1.x = t268.x JOIN (SELECT 1 AS x) t269 ON t1.x = t269.x JOIN (SELECT 1 AS x) t270 ON t1.x = t270.x JOIN (SELECT 1 AS x) t271 ON t1.x = t271.x JOIN (SELECT 1 AS x) t272 ON t1.x = t272.x JOIN (SELECT 1 AS x) t273 ON t1.x = t273.x JOIN (SELECT 1 AS x) t274 ON t1.x = t274.x JOIN (SELECT 1 AS x) t275 ON t1.x = t275.x JOIN (SELECT 1 AS x) t276 ON t1.x = t276.x JOIN (SELECT 1 AS x) t277 ON t1.x = t277.x JOIN (SELECT 1 AS x) t278 ON t1.x = t278.x JOIN (SELECT 1 AS x) t279 ON t1.x = t279.x JOIN (SELECT 1 AS x) t280 ON t1.x = t280.x JOIN (SELECT 1 AS x) t281 ON t1.x = t281.x JOIN (SELECT 1 AS x) t282 ON t1.x = t282.x JOIN (SELECT 1 AS x) t283 ON t1.x = t283.x JOIN (SELECT 1 AS x) t284 ON t1.x = t284.x JOIN (SELECT 1 AS x) t285 ON t1.x = t285.x JOIN (SELECT 1 AS x) t286 ON t1.x = t286.x JOIN (SELECT 1 AS x) t287 ON t1.x = t287.x JOIN (SELECT 1 AS x) t288 ON t1.x = t288.x JOIN (SELECT 1 AS x) t289 ON t1.x = t289.x JOIN (SELECT 1 AS x) t290 ON t1.x = t290.x JOIN (SELECT 1 AS x) t291 ON t1.x = t291.x JOIN (SELECT 1 AS x) t292 ON t1.x = t292.x JOIN (SELECT 1 AS x) t293 ON t1.x = t293.x JOIN (SELECT 1 AS x) t294 ON t1.x = t294.x JOIN (SELECT 1 AS x) t295 ON t1.x = t295.x JOIN (SELECT 1 AS x) t296 ON t1.x = t296.x JOIN (SELECT 1 AS x) t297 ON t1.x = t297.x JOIN (SELECT 1 AS x) t298 ON t1.x = t298.x JOIN (SELECT 1 AS x) t299 ON t1.x = t299.x JOIN (SELECT 1 AS x) t300 ON t1.x = t300.x JOIN (SELECT 1 AS x) t301 ON t1.x = t301.x JOIN (SELECT 1 AS x) t302 ON t1.x = t302.x JOIN (SELECT 1 AS x) t303 ON t1.x = t303.x JOIN (SELECT 1 AS x) t304 ON t1.x = t304.x JOIN (SELECT 1 AS x) t305 ON t1.x = t305.x JOIN (SELECT 1 AS x) t306 ON t1.x = t306.x JOIN (SELECT 1 AS x) t307 ON t1.x = t307.x JOIN (SELECT 1 AS x) t308 ON t1.x = t308.x JOIN (SELECT 1 AS x) t309 ON t1.x = t309.x JOIN (SELECT 1 AS x) t310 ON t1.x = t310.x JOIN (SELECT 1 AS x) t311 ON t1.x = t311.x JOIN (SELECT 1 AS x) t312 ON t1.x = t312.x JOIN (SELECT 1 AS x) t313 ON t1.x = t313.x JOIN (SELECT 1 AS x) t314 ON t1.x = t314.x JOIN (SELECT 1 AS x) t315 ON t1.x = t315.x JOIN (SELECT 1 AS x) t316 ON t1.x = t316.x JOIN (SELECT 1 AS x) t317 ON t1.x = t317.x JOIN (SELECT 1 AS x) t318 ON t1.x = t318.x JOIN (SELECT 1 AS x) t319 ON t1.x = t319.x JOIN (SELECT 1 AS x) t320 ON t1.x = t320.x JOIN (SELECT 1 AS x) t321 ON t1.x = t321.x JOIN (SELECT 1 AS x) t322 ON t1.x = t322.x JOIN (SELECT 1 AS x) t323 ON t1.x = t323.x JOIN (SELECT 1 AS x) t324 ON t1.x = t324.x JOIN (SELECT 1 AS x) t325 ON t1.x = t325.x JOIN (SELECT 1 AS x) t326 ON t1.x = t326.x JOIN (SELECT 1 AS x) t327 ON t1.x = t327.x JOIN (SELECT 1 AS x) t328 ON t1.x = t328.x JOIN (SELECT 1 AS x) t329 ON t1.x = t329.x JOIN (SELECT 1 AS x) t330 ON t1.x = t330.x JOIN (SELECT 1 AS x) t331 ON t1.x = t331.x JOIN (SELECT 1 AS x) t332 ON t1.x = t332.x JOIN (SELECT 1 AS x) t333 ON t1.x = t333.x JOIN (SELECT 1 AS x) t334 ON t1.x = t334.x JOIN (SELECT 1 AS x) t335 ON t1.x = t335.x JOIN (SELECT 1 AS x) t336 ON t1.x = t336.x JOIN (SELECT 1 AS x) t337 ON t1.x = t337.x JOIN (SELECT 1 AS x) t338 ON t1.x = t338.x JOIN (SELECT 1 AS x) t339 ON t1.x = t339.x JOIN (SELECT 1 AS x) t340 ON t1.x = t340.x JOIN (SELECT 1 AS x) t341 ON t1.x = t341.x JOIN (SELECT 1 AS x) t342 ON t1.x = t342.x JOIN (SELECT 1 AS x) t343 ON t1.x = t343.x JOIN (SELECT 1 AS x) t344 ON t1.x = t344.x JOIN (SELECT 1 AS x) t345 ON t1.x = t345.x JOIN (SELECT 1 AS x) t346 ON t1.x = t346.x JOIN (SELECT 1 AS x) t347 ON t1.x = t347.x JOIN (SELECT 1 AS x) t348 ON t1.x = t348.x JOIN (SELECT 1 AS x) t349 ON t1.x = t349.x JOIN (SELECT 1 AS x) t350 ON t1.x = t350.x JOIN (SELECT 1 AS x) t351 ON t1.x = t351.x JOIN (SELECT 1 AS x) t352 ON t1.x = t352.x JOIN (SELECT 1 AS x) t353 ON t1.x = t353.x JOIN (SELECT 1 AS x) t354 ON t1.x = t354.x JOIN (SELECT 1 AS x) t355 ON t1.x = t355.x JOIN (SELECT 1 AS x) t356 ON t1.x = t356.x JOIN (SELECT 1 AS x) t357 ON t1.x = t357.x JOIN (SELECT 1 AS x) t358 ON t1.x = t358.x JOIN (SELECT 1 AS x) t359 ON t1.x = t359.x JOIN (SELECT 1 AS x) t360 ON t1.x = t360.x JOIN (SELECT 1 AS x) t361 ON t1.x = t361.x JOIN (SELECT 1 AS x) t362 ON t1.x = t362.x JOIN (SELECT 1 AS x) t363 ON t1.x = t363.x JOIN (SELECT 1 AS x) t364 ON t1.x = t364.x JOIN (SELECT 1 AS x) t365 ON t1.x = t365.x JOIN (SELECT 1 AS x) t366 ON t1.x = t366.x JOIN (SELECT 1 AS x) t367 ON t1.x = t367.x JOIN (SELECT 1 AS x) t368 ON t1.x = t368.x JOIN (SELECT 1 AS x) t369 ON t1.x = t369.x JOIN (SELECT 1 AS x) t370 ON t1.x = t370.x JOIN (SELECT 1 AS x) t371 ON t1.x = t371.x JOIN (SELECT 1 AS x) t372 ON t1.x = t372.x JOIN (SELECT 1 AS x) t373 ON t1.x = t373.x JOIN (SELECT 1 AS x) t374 ON t1.x = t374.x JOIN (SELECT 1 AS x) t375 ON t1.x = t375.x JOIN (SELECT 1 AS x) t376 ON t1.x = t376.x JOIN (SELECT 1 AS x) t377 ON t1.x = t377.x JOIN (SELECT 1 AS x) t378 ON t1.x = t378.x JOIN (SELECT 1 AS x) t379 ON t1.x = t379.x JOIN (SELECT 1 AS x) t380 ON t1.x = t380.x JOIN (SELECT 1 AS x) t381 ON t1.x = t381.x JOIN (SELECT 1 AS x) t382 ON t1.x = t382.x JOIN (SELECT 1 AS x) t383 ON t1.x = t383.x JOIN (SELECT 1 AS x) t384 ON t1.x = t384.x JOIN (SELECT 1 AS x) t385 ON t1.x = t385.x JOIN (SELECT 1 AS x) t386 ON t1.x = t386.x JOIN (SELECT 1 AS x) t387 ON t1.x = t387.x JOIN (SELECT 1 AS x) t388 ON t1.x = t388.x JOIN (SELECT 1 AS x) t389 ON t1.x = t389.x JOIN (SELECT 1 AS x) t390 ON t1.x = t390.x JOIN (SELECT 1 AS x) t391 ON t1.x = t391.x JOIN (SELECT 1 AS x) t392 ON t1.x = t392.x JOIN (SELECT 1 AS x) t393 ON t1.x = t393.x JOIN (SELECT 1 AS x) t394 ON t1.x = t394.x JOIN (SELECT 1 AS x) t395 ON t1.x = t395.x JOIN (SELECT 1 AS x) t396 ON t1.x = t396.x JOIN (SELECT 1 AS x) t397 ON t1.x = t397.x JOIN (SELECT 1 AS x) t398 ON t1.x = t398.x JOIN (SELECT 1 AS x) t399 ON t1.x = t399.x JOIN (SELECT 1 AS x) t400 ON t1.x = t400.x JOIN (SELECT 1 AS x) t401 ON t1.x = t401.x JOIN (SELECT 1 AS x) t402 ON t1.x = t402.x JOIN (SELECT 1 AS x) t403 ON t1.x = t403.x JOIN (SELECT 1 AS x) t404 ON t1.x = t404.x JOIN (SELECT 1 AS x) t405 ON t1.x = t405.x JOIN (SELECT 1 AS x) t406 ON t1.x = t406.x JOIN (SELECT 1 AS x) t407 ON t1.x = t407.x JOIN (SELECT 1 AS x) t408 ON t1.x = t408.x JOIN (SELECT 1 AS x) t409 ON t1.x = t409.x JOIN (SELECT 1 AS x) t410 ON t1.x = t410.x JOIN (SELECT 1 AS x) t411 ON t1.x = t411.x JOIN (SELECT 1 AS x) t412 ON t1.x = t412.x JOIN (SELECT 1 AS x) t413 ON t1.x = t413.x JOIN (SELECT 1 AS x) t414 ON t1.x = t414.x JOIN (SELECT 1 AS x) t415 ON t1.x = t415.x JOIN (SELECT 1 AS x) t416 ON t1.x = t416.x JOIN (SELECT 1 AS x) t417 ON t1.x = t417.x JOIN (SELECT 1 AS x) t418 ON t1.x = t418.x JOIN (SELECT 1 AS x) t419 ON t1.x = t419.x JOIN (SELECT 1 AS x) t420 ON t1.x = t420.x JOIN (SELECT 1 AS x) t421 ON t1.x = t421.x JOIN (SELECT 1 AS x) t422 ON t1.x = t422.x JOIN (SELECT 1 AS x) t423 ON t1.x = t423.x JOIN (SELECT 1 AS x) t424 ON t1.x = t424.x JOIN (SELECT 1 AS x) t425 ON t1.x = t425.x JOIN (SELECT 1 AS x) t426 ON t1.x = t426.x JOIN (SELECT 1 AS x) t427 ON t1.x = t427.x JOIN (SELECT 1 AS x) t428 ON t1.x = t428.x JOIN (SELECT 1 AS x) t429 ON t1.x = t429.x JOIN (SELECT 1 AS x) t430 ON t1.x = t430.x JOIN (SELECT 1 AS x) t431 ON t1.x = t431.x JOIN (SELECT 1 AS x) t432 ON t1.x = t432.x JOIN (SELECT 1 AS x) t433 ON t1.x = t433.x JOIN (SELECT 1 AS x) t434 ON t1.x = t434.x JOIN (SELECT 1 AS x) t435 ON t1.x = t435.x JOIN (SELECT 1 AS x) t436 ON t1.x = t436.x JOIN (SELECT 1 AS x) t437 ON t1.x = t437.x JOIN (SELECT 1 AS x) t438 ON t1.x = t438.x JOIN (SELECT 1 AS x) t439 ON t1.x = t439.x JOIN (SELECT 1 AS x) t440 ON t1.x = t440.x JOIN (SELECT 1 AS x) t441 ON t1.x = t441.x JOIN (SELECT 1 AS x) t442 ON t1.x = t442.x JOIN (SELECT 1 AS x) t443 ON t1.x = t443.x JOIN (SELECT 1 AS x) t444 ON t1.x = t444.x JOIN (SELECT 1 AS x) t445 ON t1.x = t445.x JOIN (SELECT 1 AS x) t446 ON t1.x = t446.x JOIN (SELECT 1 AS x) t447 ON t1.x = t447.x JOIN (SELECT 1 AS x) t448 ON t1.x = t448.x JOIN (SELECT 1 AS x) t449 ON t1.x = t449.x JOIN (SELECT 1 AS x) t450 ON t1.x = t450.x JOIN (SELECT 1 AS x) t451 ON t1.x = t451.x JOIN (SELECT 1 AS x) t452 ON t1.x = t452.x JOIN (SELECT 1 AS x) t453 ON t1.x = t453.x JOIN (SELECT 1 AS x) t454 ON t1.x = t454.x JOIN (SELECT 1 AS x) t455 ON t1.x = t455.x JOIN (SELECT 1 AS x) t456 ON t1.x = t456.x JOIN (SELECT 1 AS x) t457 ON t1.x = t457.x JOIN (SELECT 1 AS x) t458 ON t1.x = t458.x JOIN (SELECT 1 AS x) t459 ON t1.x = t459.x JOIN (SELECT 1 AS x) t460 ON t1.x = t460.x JOIN (SELECT 1 AS x) t461 ON t1.x = t461.x JOIN (SELECT 1 AS x) t462 ON t1.x = t462.x JOIN (SELECT 1 AS x) t463 ON t1.x = t463.x JOIN (SELECT 1 AS x) t464 ON t1.x = t464.x JOIN (SELECT 1 AS x) t465 ON t1.x = t465.x JOIN (SELECT 1 AS x) t466 ON t1.x = t466.x JOIN (SELECT 1 AS x) t467 ON t1.x = t467.x JOIN (SELECT 1 AS x) t468 ON t1.x = t468.x JOIN (SELECT 1 AS x) t469 ON t1.x = t469.x JOIN (SELECT 1 AS x) t470 ON t1.x = t470.x JOIN (SELECT 1 AS x) t471 ON t1.x = t471.x JOIN (SELECT 1 AS x) t472 ON t1.x = t472.x JOIN (SELECT 1 AS x) t473 ON t1.x = t473.x JOIN (SELECT 1 AS x) t474 ON t1.x = t474.x JOIN (SELECT 1 AS x) t475 ON t1.x = t475.x JOIN (SELECT 1 AS x) t476 ON t1.x = t476.x JOIN (SELECT 1 AS x) t477 ON t1.x = t477.x JOIN (SELECT 1 AS x) t478 ON t1.x = t478.x JOIN (SELECT 1 AS x) t479 ON t1.x = t479.x JOIN (SELECT 1 AS x) t480 ON t1.x = t480.x JOIN (SELECT 1 AS x) t481 ON t1.x = t481.x JOIN (SELECT 1 AS x) t482 ON t1.x = t482.x JOIN (SELECT 1 AS x) t483 ON t1.x = t483.x JOIN (SELECT 1 AS x) t484 ON t1.x = t484.x JOIN (SELECT 1 AS x) t485 ON t1.x = t485.x JOIN (SELECT 1 AS x) t486 ON t1.x = t486.x JOIN (SELECT 1 AS x) t487 ON t1.x = t487.x JOIN (SELECT 1 AS x) t488 ON t1.x = t488.x JOIN (SELECT 1 AS x) t489 ON t1.x = t489.x JOIN (SELECT 1 AS x) t490 ON t1.x = t490.x JOIN (SELECT 1 AS x) t491 ON t1.x = t491.x JOIN (SELECT 1 AS x) t492 ON t1.x = t492.x JOIN (SELECT 1 AS x) t493 ON t1.x = t493.x JOIN (SELECT 1 AS x) t494 ON t1.x = t494.x JOIN (SELECT 1 AS x) t495 ON t1.x = t495.x JOIN (SELECT 1 AS x) t496 ON t1.x = t496.x JOIN (SELECT 1 AS x) t497 ON t1.x = t497.x JOIN (SELECT 1 AS x) t498 ON t1.x = t498.x JOIN (SELECT 1 AS x) t499 ON t1.x = t499.x JOIN (SELECT 1 AS x) t500 ON t1.x = t500.x JOIN (SELECT 1 AS x) t501 ON t1.x = t501.x JOIN (SELECT 1 AS x) t502 ON t1.x = t502.x JOIN (SELECT 1 AS x) t503 ON t1.x = t503.x JOIN (SELECT 1 AS x) t504 ON t1.x = t504.x JOIN (SELECT 1 AS x) t505 ON t1.x = t505.x JOIN (SELECT 1 AS x) t506 ON t1.x = t506.x JOIN (SELECT 1 AS x) t507 ON t1.x = t507.x JOIN (SELECT 1 AS x) t508 ON t1.x = t508.x JOIN (SELECT 1 AS x) t509 ON t1.x = t509.x JOIN (SELECT 1 AS x) t510 ON t1.x = t510.x JOIN (SELECT 1 AS x) t511 ON t1.x = t511.x JOIN (SELECT 1 AS x) t512 ON t1.x = t512.x JOIN (SELECT 1 AS x) t513 ON t1.x = t513.x JOIN (SELECT 1 AS x) t514 ON t1.x = t514.x JOIN (SELECT 1 AS x) t515 ON t1.x = t515.x JOIN (SELECT 1 AS x) t516 ON t1.x = t516.x JOIN (SELECT 1 AS x) t517 ON t1.x = t517.x JOIN (SELECT 1 AS x) t518 ON t1.x = t518.x JOIN (SELECT 1 AS x) t519 ON t1.x = t519.x JOIN (SELECT 1 AS x) t520 ON t1.x = t520.x JOIN (SELECT 1 AS x) t521 ON t1.x = t521.x JOIN (SELECT 1 AS x) t522 ON t1.x = t522.x JOIN (SELECT 1 AS x) t523 ON t1.x = t523.x JOIN (SELECT 1 AS x) t524 ON t1.x = t524.x JOIN (SELECT 1 AS x) t525 ON t1.x = t525.x JOIN (SELECT 1 AS x) t526 ON t1.x = t526.x JOIN (SELECT 1 AS x) t527 ON t1.x = t527.x JOIN (SELECT 1 AS x) t528 ON t1.x = t528.x JOIN (SELECT 1 AS x) t529 ON t1.x = t529.x JOIN (SELECT 1 AS x) t530 ON t1.x = t530.x JOIN (SELECT 1 AS x) t531 ON t1.x = t531.x JOIN (SELECT 1 AS x) t532 ON t1.x = t532.x JOIN (SELECT 1 AS x) t533 ON t1.x = t533.x JOIN (SELECT 1 AS x) t534 ON t1.x = t534.x JOIN (SELECT 1 AS x) t535 ON t1.x = t535.x JOIN (SELECT 1 AS x) t536 ON t1.x = t536.x JOIN (SELECT 1 AS x) t537 ON t1.x = t537.x JOIN (SELECT 1 AS x) t538 ON t1.x = t538.x JOIN (SELECT 1 AS x) t539 ON t1.x = t539.x JOIN (SELECT 1 AS x) t540 ON t1.x = t540.x JOIN (SELECT 1 AS x) t541 ON t1.x = t541.x JOIN (SELECT 1 AS x) t542 ON t1.x = t542.x JOIN (SELECT 1 AS x) t543 ON t1.x = t543.x JOIN (SELECT 1 AS x) t544 ON t1.x = t544.x JOIN (SELECT 1 AS x) t545 ON t1.x = t545.x JOIN (SELECT 1 AS x) t546 ON t1.x = t546.x JOIN (SELECT 1 AS x) t547 ON t1.x = t547.x JOIN (SELECT 1 AS x) t548 ON t1.x = t548.x JOIN (SELECT 1 AS x) t549 ON t1.x = t549.x JOIN (SELECT 1 AS x) t550 ON t1.x = t550.x JOIN (SELECT 1 AS x) t551 ON t1.x = t551.x JOIN (SELECT 1 AS x) t552 ON t1.x = t552.x JOIN (SELECT 1 AS x) t553 ON t1.x = t553.x JOIN (SELECT 1 AS x) t554 ON t1.x = t554.x JOIN (SELECT 1 AS x) t555 ON t1.x = t555.x JOIN (SELECT 1 AS x) t556 ON t1.x = t556.x JOIN (SELECT 1 AS x) t557 ON t1.x = t557.x JOIN (SELECT 1 AS x) t558 ON t1.x = t558.x JOIN (SELECT 1 AS x) t559 ON t1.x = t559.x JOIN (SELECT 1 AS x) t560 ON t1.x = t560.x JOIN (SELECT 1 AS x) t561 ON t1.x = t561.x JOIN (SELECT 1 AS x) t562 ON t1.x = t562.x JOIN (SELECT 1 AS x) t563 ON t1.x = t563.x JOIN (SELECT 1 AS x) t564 ON t1.x = t564.x JOIN (SELECT 1 AS x) t565 ON t1.x = t565.x JOIN (SELECT 1 AS x) t566 ON t1.x = t566.x JOIN (SELECT 1 AS x) t567 ON t1.x = t567.x JOIN (SELECT 1 AS x) t568 ON t1.x = t568.x JOIN (SELECT 1 AS x) t569 ON t1.x = t569.x JOIN (SELECT 1 AS x) t570 ON t1.x = t570.x JOIN (SELECT 1 AS x) t571 ON t1.x = t571.x JOIN (SELECT 1 AS x) t572 ON t1.x = t572.x JOIN (SELECT 1 AS x) t573 ON t1.x = t573.x JOIN (SELECT 1 AS x) t574 ON t1.x = t574.x JOIN (SELECT 1 AS x) t575 ON t1.x = t575.x JOIN (SELECT 1 AS x) t576 ON t1.x = t576.x JOIN (SELECT 1 AS x) t577 ON t1.x = t577.x JOIN (SELECT 1 AS x) t578 ON t1.x = t578.x JOIN (SELECT 1 AS x) t579 ON t1.x = t579.x JOIN (SELECT 1 AS x) t580 ON t1.x = t580.x JOIN (SELECT 1 AS x) t581 ON t1.x = t581.x JOIN (SELECT 1 AS x) t582 ON t1.x = t582.x JOIN (SELECT 1 AS x) t583 ON t1.x = t583.x JOIN (SELECT 1 AS x) t584 ON t1.x = t584.x JOIN (SELECT 1 AS x) t585 ON t1.x = t585.x JOIN (SELECT 1 AS x) t586 ON t1.x = t586.x JOIN (SELECT 1 AS x) t587 ON t1.x = t587.x JOIN (SELECT 1 AS x) t588 ON t1.x = t588.x JOIN (SELECT 1 AS x) t589 ON t1.x = t589.x JOIN (SELECT 1 AS x) t590 ON t1.x = t590.x JOIN (SELECT 1 AS x) t591 ON t1.x = t591.x JOIN (SELECT 1 AS x) t592 ON t1.x = t592.x JOIN (SELECT 1 AS x) t593 ON t1.x = t593.x JOIN (SELECT 1 AS x) t594 ON t1.x = t594.x JOIN (SELECT 1 AS x) t595 ON t1.x = t595.x JOIN (SELECT 1 AS x) t596 ON t1.x = t596.x JOIN (SELECT 1 AS x) t597 ON t1.x = t597.x JOIN (SELECT 1 AS x) t598 ON t1.x = t598.x JOIN (SELECT 1 AS x) t599 ON t1.x = t599.x JOIN (SELECT 1 AS x) t600 ON t1.x = t600.x JOIN (SELECT 1 AS x) t601 ON t1.x = t601.x JOIN (SELECT 1 AS x) t602 ON t1.x = t602.x JOIN (SELECT 1 AS x) t603 ON t1.x = t603.x JOIN (SELECT 1 AS x) t604 ON t1.x = t604.x JOIN (SELECT 1 AS x) t605 ON t1.x = t605.x JOIN (SELECT 1 AS x) t606 ON t1.x = t606.x JOIN (SELECT 1 AS x) t607 ON t1.x = t607.x JOIN (SELECT 1 AS x) t608 ON t1.x = t608.x JOIN (SELECT 1 AS x) t609 ON t1.x = t609.x JOIN (SELECT 1 AS x) t610 ON t1.x = t610.x JOIN (SELECT 1 AS x) t611 ON t1.x = t611.x JOIN (SELECT 1 AS x) t612 ON t1.x = t612.x JOIN (SELECT 1 AS x) t613 ON t1.x = t613.x JOIN (SELECT 1 AS x) t614 ON t1.x = t614.x JOIN (SELECT 1 AS x) t615 ON t1.x = t615.x JOIN (SELECT 1 AS x) t616 ON t1.x = t616.x JOIN (SELECT 1 AS x) t617 ON t1.x = t617.x JOIN (SELECT 1 AS x) t618 ON t1.x = t618.x JOIN (SELECT 1 AS x) t619 ON t1.x = t619.x JOIN (SELECT 1 AS x) t620 ON t1.x = t620.x JOIN (SELECT 1 AS x) t621 ON t1.x = t621.x JOIN (SELECT 1 AS x) t622 ON t1.x = t622.x JOIN (SELECT 1 AS x) t623 ON t1.x = t623.x JOIN (SELECT 1 AS x) t624 ON t1.x = t624.x JOIN (SELECT 1 AS x) t625 ON t1.x = t625.x JOIN (SELECT 1 AS x) t626 ON t1.x = t626.x JOIN (SELECT 1 AS x) t627 ON t1.x = t627.x JOIN (SELECT 1 AS x) t628 ON t1.x = t628.x JOIN (SELECT 1 AS x) t629 ON t1.x = t629.x JOIN (SELECT 1 AS x) t630 ON t1.x = t630.x JOIN (SELECT 1 AS x) t631 ON t1.x = t631.x JOIN (SELECT 1 AS x) t632 ON t1.x = t632.x JOIN (SELECT 1 AS x) t633 ON t1.x = t633.x JOIN (SELECT 1 AS x) t634 ON t1.x = t634.x JOIN (SELECT 1 AS x) t635 ON t1.x = t635.x JOIN (SELECT 1 AS x) t636 ON t1.x = t636.x JOIN (SELECT 1 AS x) t637 ON t1.x = t637.x JOIN (SELECT 1 AS x) t638 ON t1.x = t638.x JOIN (SELECT 1 AS x) t639 ON t1.x = t639.x JOIN (SELECT 1 AS x) t640 ON t1.x = t640.x JOIN (SELECT 1 AS x) t641 ON t1.x = t641.x JOIN (SELECT 1 AS x) t642 ON t1.x = t642.x JOIN (SELECT 1 AS x) t643 ON t1.x = t643.x JOIN (SELECT 1 AS x) t644 ON t1.x = t644.x JOIN (SELECT 1 AS x) t645 ON t1.x = t645.x JOIN (SELECT 1 AS x) t646 ON t1.x = t646.x JOIN (SELECT 1 AS x) t647 ON t1.x = t647.x JOIN (SELECT 1 AS x) t648 ON t1.x = t648.x JOIN (SELECT 1 AS x) t649 ON t1.x = t649.x JOIN (SELECT 1 AS x) t650 ON t1.x = t650.x JOIN (SELECT 1 AS x) t651 ON t1.x = t651.x JOIN (SELECT 1 AS x) t652 ON t1.x = t652.x JOIN (SELECT 1 AS x) t653 ON t1.x = t653.x JOIN (SELECT 1 AS x) t654 ON t1.x = t654.x JOIN (SELECT 1 AS x) t655 ON t1.x = t655.x JOIN (SELECT 1 AS x) t656 ON t1.x = t656.x JOIN (SELECT 1 AS x) t657 ON t1.x = t657.x JOIN (SELECT 1 AS x) t658 ON t1.x = t658.x JOIN (SELECT 1 AS x) t659 ON t1.x = t659.x JOIN (SELECT 1 AS x) t660 ON t1.x = t660.x JOIN (SELECT 1 AS x) t661 ON t1.x = t661.x JOIN (SELECT 1 AS x) t662 ON t1.x = t662.x JOIN (SELECT 1 AS x) t663 ON t1.x = t663.x JOIN (SELECT 1 AS x) t664 ON t1.x = t664.x JOIN (SELECT 1 AS x) t665 ON t1.x = t665.x JOIN (SELECT 1 AS x) t666 ON t1.x = t666.x JOIN (SELECT 1 AS x) t667 ON t1.x = t667.x JOIN (SELECT 1 AS x) t668 ON t1.x = t668.x JOIN (SELECT 1 AS x) t669 ON t1.x = t669.x JOIN (SELECT 1 AS x) t670 ON t1.x = t670.x JOIN (SELECT 1 AS x) t671 ON t1.x = t671.x JOIN (SELECT 1 AS x) t672 ON t1.x = t672.x JOIN (SELECT 1 AS x) t673 ON t1.x = t673.x JOIN (SELECT 1 AS x) t674 ON t1.x = t674.x JOIN (SELECT 1 AS x) t675 ON t1.x = t675.x JOIN (SELECT 1 AS x) t676 ON t1.x = t676.x JOIN (SELECT 1 AS x) t677 ON t1.x = t677.x JOIN (SELECT 1 AS x) t678 ON t1.x = t678.x JOIN (SELECT 1 AS x) t679 ON t1.x = t679.x JOIN (SELECT 1 AS x) t680 ON t1.x = t680.x JOIN (SELECT 1 AS x) t681 ON t1.x = t681.x JOIN (SELECT 1 AS x) t682 ON t1.x = t682.x JOIN (SELECT 1 AS x) t683 ON t1.x = t683.x JOIN (SELECT 1 AS x) t684 ON t1.x = t684.x JOIN (SELECT 1 AS x) t685 ON t1.x = t685.x JOIN (SELECT 1 AS x) t686 ON t1.x = t686.x JOIN (SELECT 1 AS x) t687 ON t1.x = t687.x JOIN (SELECT 1 AS x) t688 ON t1.x = t688.x JOIN (SELECT 1 AS x) t689 ON t1.x = t689.x JOIN (SELECT 1 AS x) t690 ON t1.x = t690.x JOIN (SELECT 1 AS x) t691 ON t1.x = t691.x JOIN (SELECT 1 AS x) t692 ON t1.x = t692.x JOIN (SELECT 1 AS x) t693 ON t1.x = t693.x JOIN (SELECT 1 AS x) t694 ON t1.x = t694.x JOIN (SELECT 1 AS x) t695 ON t1.x = t695.x JOIN (SELECT 1 AS x) t696 ON t1.x = t696.x JOIN (SELECT 1 AS x) t697 ON t1.x = t697.x JOIN (SELECT 1 AS x) t698 ON t1.x = t698.x JOIN (SELECT 1 AS x) t699 ON t1.x = t699.x JOIN (SELECT 1 AS x) t700 ON t1.x = t700.x JOIN (SELECT 1 AS x) t701 ON t1.x = t701.x JOIN (SELECT 1 AS x) t702 ON t1.x = t702.x JOIN (SELECT 1 AS x) t703 ON t1.x = t703.x JOIN (SELECT 1 AS x) t704 ON t1.x = t704.x JOIN (SELECT 1 AS x) t705 ON t1.x = t705.x JOIN (SELECT 1 AS x) t706 ON t1.x = t706.x JOIN (SELECT 1 AS x) t707 ON t1.x = t707.x JOIN (SELECT 1 AS x) t708 ON t1.x = t708.x JOIN (SELECT 1 AS x) t709 ON t1.x = t709.x JOIN (SELECT 1 AS x) t710 ON t1.x = t710.x JOIN (SELECT 1 AS x) t711 ON t1.x = t711.x JOIN (SELECT 1 AS x) t712 ON t1.x = t712.x JOIN (SELECT 1 AS x) t713 ON t1.x = t713.x JOIN (SELECT 1 AS x) t714 ON t1.x = t714.x JOIN (SELECT 1 AS x) t715 ON t1.x = t715.x JOIN (SELECT 1 AS x) t716 ON t1.x = t716.x JOIN (SELECT 1 AS x) t717 ON t1.x = t717.x JOIN (SELECT 1 AS x) t718 ON t1.x = t718.x JOIN (SELECT 1 AS x) t719 ON t1.x = t719.x JOIN (SELECT 1 AS x) t720 ON t1.x = t720.x JOIN (SELECT 1 AS x) t721 ON t1.x = t721.x JOIN (SELECT 1 AS x) t722 ON t1.x = t722.x JOIN (SELECT 1 AS x) t723 ON t1.x = t723.x JOIN (SELECT 1 AS x) t724 ON t1.x = t724.x JOIN (SELECT 1 AS x) t725 ON t1.x = t725.x JOIN (SELECT 1 AS x) t726 ON t1.x = t726.x JOIN (SELECT 1 AS x) t727 ON t1.x = t727.x JOIN (SELECT 1 AS x) t728 ON t1.x = t728.x JOIN (SELECT 1 AS x) t729 ON t1.x = t729.x JOIN (SELECT 1 AS x) t730 ON t1.x = t730.x JOIN (SELECT 1 AS x) t731 ON t1.x = t731.x JOIN (SELECT 1 AS x) t732 ON t1.x = t732.x JOIN (SELECT 1 AS x) t733 ON t1.x = t733.x JOIN (SELECT 1 AS x) t734 ON t1.x = t734.x JOIN (SELECT 1 AS x) t735 ON t1.x = t735.x JOIN (SELECT 1 AS x) t736 ON t1.x = t736.x JOIN (SELECT 1 AS x) t737 ON t1.x = t737.x JOIN (SELECT 1 AS x) t738 ON t1.x = t738.x JOIN (SELECT 1 AS x) t739 ON t1.x = t739.x JOIN (SELECT 1 AS x) t740 ON t1.x = t740.x JOIN (SELECT 1 AS x) t741 ON t1.x = t741.x JOIN (SELECT 1 AS x) t742 ON t1.x = t742.x JOIN (SELECT 1 AS x) t743 ON t1.x = t743.x JOIN (SELECT 1 AS x) t744 ON t1.x = t744.x JOIN (SELECT 1 AS x) t745 ON t1.x = t745.x JOIN (SELECT 1 AS x) t746 ON t1.x = t746.x JOIN (SELECT 1 AS x) t747 ON t1.x = t747.x JOIN (SELECT 1 AS x) t748 ON t1.x = t748.x JOIN (SELECT 1 AS x) t749 ON t1.x = t749.x JOIN (SELECT 1 AS x) t750 ON t1.x = t750.x JOIN (SELECT 1 AS x) t751 ON t1.x = t751.x JOIN (SELECT 1 AS x) t752 ON t1.x = t752.x JOIN (SELECT 1 AS x) t753 ON t1.x = t753.x JOIN (SELECT 1 AS x) t754 ON t1.x = t754.x JOIN (SELECT 1 AS x) t755 ON t1.x = t755.x JOIN (SELECT 1 AS x) t756 ON t1.x = t756.x JOIN (SELECT 1 AS x) t757 ON t1.x = t757.x JOIN (SELECT 1 AS x) t758 ON t1.x = t758.x JOIN (SELECT 1 AS x) t759 ON t1.x = t759.x JOIN (SELECT 1 AS x) t760 ON t1.x = t760.x JOIN (SELECT 1 AS x) t761 ON t1.x = t761.x JOIN (SELECT 1 AS x) t762 ON t1.x = t762.x JOIN (SELECT 1 AS x) t763 ON t1.x = t763.x JOIN (SELECT 1 AS x) t764 ON t1.x = t764.x JOIN (SELECT 1 AS x) t765 ON t1.x = t765.x JOIN (SELECT 1 AS x) t766 ON t1.x = t766.x JOIN (SELECT 1 AS x) t767 ON t1.x = t767.x JOIN (SELECT 1 AS x) t768 ON t1.x = t768.x JOIN (SELECT 1 AS x) t769 ON t1.x = t769.x JOIN (SELECT 1 AS x) t770 ON t1.x = t770.x JOIN (SELECT 1 AS x) t771 ON t1.x = t771.x JOIN (SELECT 1 AS x) t772 ON t1.x = t772.x JOIN (SELECT 1 AS x) t773 ON t1.x = t773.x JOIN (SELECT 1 AS x) t774 ON t1.x = t774.x JOIN (SELECT 1 AS x) t775 ON t1.x = t775.x JOIN (SELECT 1 AS x) t776 ON t1.x = t776.x JOIN (SELECT 1 AS x) t777 ON t1.x = t777.x JOIN (SELECT 1 AS x) t778 ON t1.x = t778.x JOIN (SELECT 1 AS x) t779 ON t1.x = t779.x JOIN (SELECT 1 AS x) t780 ON t1.x = t780.x JOIN (SELECT 1 AS x) t781 ON t1.x = t781.x JOIN (SELECT 1 AS x) t782 ON t1.x = t782.x JOIN (SELECT 1 AS x) t783 ON t1.x = t783.x JOIN (SELECT 1 AS x) t784 ON t1.x = t784.x JOIN (SELECT 1 AS x) t785 ON t1.x = t785.x JOIN (SELECT 1 AS x) t786 ON t1.x = t786.x JOIN (SELECT 1 AS x) t787 ON t1.x = t787.x JOIN (SELECT 1 AS x) t788 ON t1.x = t788.x JOIN (SELECT 1 AS x) t789 ON t1.x = t789.x JOIN (SELECT 1 AS x) t790 ON t1.x = t790.x JOIN (SELECT 1 AS x) t791 ON t1.x = t791.x JOIN (SELECT 1 AS x) t792 ON t1.x = t792.x JOIN (SELECT 1 AS x) t793 ON t1.x = t793.x JOIN (SELECT 1 AS x) t794 ON t1.x = t794.x JOIN (SELECT 1 AS x) t795 ON t1.x = t795.x JOIN (SELECT 1 AS x) t796 ON t1.x = t796.x JOIN (SELECT 1 AS x) t797 ON t1.x = t797.x JOIN (SELECT 1 AS x) t798 ON t1.x = t798.x JOIN (SELECT 1 AS x) t799 ON t1.x = t799.x JOIN (SELECT 1 AS x) t800 ON t1.x = t800.x JOIN (SELECT 1 AS x) t801 ON t1.x = t801.x JOIN (SELECT 1 AS x) t802 ON t1.x = t802.x JOIN (SELECT 1 AS x) t803 ON t1.x = t803.x JOIN (SELECT 1 AS x) t804 ON t1.x = t804.x JOIN (SELECT 1 AS x) t805 ON t1.x = t805.x JOIN (SELECT 1 AS x) t806 ON t1.x = t806.x JOIN (SELECT 1 AS x) t807 ON t1.x = t807.x JOIN (SELECT 1 AS x) t808 ON t1.x = t808.x JOIN (SELECT 1 AS x) t809 ON t1.x = t809.x JOIN (SELECT 1 AS x) t810 ON t1.x = t810.x JOIN (SELECT 1 AS x) t811 ON t1.x = t811.x JOIN (SELECT 1 AS x) t812 ON t1.x = t812.x JOIN (SELECT 1 AS x) t813 ON t1.x = t813.x JOIN (SELECT 1 AS x) t814 ON t1.x = t814.x JOIN (SELECT 1 AS x) t815 ON t1.x = t815.x JOIN (SELECT 1 AS x) t816 ON t1.x = t816.x JOIN (SELECT 1 AS x) t817 ON t1.x = t817.x JOIN (SELECT 1 AS x) t818 ON t1.x = t818.x JOIN (SELECT 1 AS x) t819 ON t1.x = t819.x JOIN (SELECT 1 AS x) t820 ON t1.x = t820.x JOIN (SELECT 1 AS x) t821 ON t1.x = t821.x JOIN (SELECT 1 AS x) t822 ON t1.x = t822.x JOIN (SELECT 1 AS x) t823 ON t1.x = t823.x JOIN (SELECT 1 AS x) t824 ON t1.x = t824.x JOIN (SELECT 1 AS x) t825 ON t1.x = t825.x JOIN (SELECT 1 AS x) t826 ON t1.x = t826.x JOIN (SELECT 1 AS x) t827 ON t1.x = t827.x JOIN (SELECT 1 AS x) t828 ON t1.x = t828.x JOIN (SELECT 1 AS x) t829 ON t1.x = t829.x JOIN (SELECT 1 AS x) t830 ON t1.x = t830.x JOIN (SELECT 1 AS x) t831 ON t1.x = t831.x JOIN (SELECT 1 AS x) t832 ON t1.x = t832.x JOIN (SELECT 1 AS x) t833 ON t1.x = t833.x JOIN (SELECT 1 AS x) t834 ON t1.x = t834.x JOIN (SELECT 1 AS x) t835 ON t1.x = t835.x JOIN (SELECT 1 AS x) t836 ON t1.x = t836.x JOIN (SELECT 1 AS x) t837 ON t1.x = t837.x JOIN (SELECT 1 AS x) t838 ON t1.x = t838.x JOIN (SELECT 1 AS x) t839 ON t1.x = t839.x JOIN (SELECT 1 AS x) t840 ON t1.x = t840.x JOIN (SELECT 1 AS x) t841 ON t1.x = t841.x JOIN (SELECT 1 AS x) t842 ON t1.x = t842.x JOIN (SELECT 1 AS x) t843 ON t1.x = t843.x JOIN (SELECT 1 AS x) t844 ON t1.x = t844.x JOIN (SELECT 1 AS x) t845 ON t1.x = t845.x JOIN (SELECT 1 AS x) t846 ON t1.x = t846.x JOIN (SELECT 1 AS x) t847 ON t1.x = t847.x JOIN (SELECT 1 AS x) t848 ON t1.x = t848.x JOIN (SELECT 1 AS x) t849 ON t1.x = t849.x JOIN (SELECT 1 AS x) t850 ON t1.x = t850.x JOIN (SELECT 1 AS x) t851 ON t1.x = t851.x JOIN (SELECT 1 AS x) t852 ON t1.x = t852.x JOIN (SELECT 1 AS x) t853 ON t1.x = t853.x JOIN (SELECT 1 AS x) t854 ON t1.x = t854.x JOIN (SELECT 1 AS x) t855 ON t1.x = t855.x JOIN (SELECT 1 AS x) t856 ON t1.x = t856.x JOIN (SELECT 1 AS x) t857 ON t1.x = t857.x JOIN (SELECT 1 AS x) t858 ON t1.x = t858.x JOIN (SELECT 1 AS x) t859 ON t1.x = t859.x JOIN (SELECT 1 AS x) t860 ON t1.x = t860.x JOIN (SELECT 1 AS x) t861 ON t1.x = t861.x JOIN (SELECT 1 AS x) t862 ON t1.x = t862.x JOIN (SELECT 1 AS x) t863 ON t1.x = t863.x JOIN (SELECT 1 AS x) t864 ON t1.x = t864.x JOIN (SELECT 1 AS x) t865 ON t1.x = t865.x JOIN (SELECT 1 AS x) t866 ON t1.x = t866.x JOIN (SELECT 1 AS x) t867 ON t1.x = t867.x JOIN (SELECT 1 AS x) t868 ON t1.x = t868.x JOIN (SELECT 1 AS x) t869 ON t1.x = t869.x JOIN (SELECT 1 AS x) t870 ON t1.x = t870.x JOIN (SELECT 1 AS x) t871 ON t1.x = t871.x JOIN (SELECT 1 AS x) t872 ON t1.x = t872.x JOIN (SELECT 1 AS x) t873 ON t1.x = t873.x JOIN (SELECT 1 AS x) t874 ON t1.x = t874.x JOIN (SELECT 1 AS x) t875 ON t1.x = t875.x JOIN (SELECT 1 AS x) t876 ON t1.x = t876.x JOIN (SELECT 1 AS x) t877 ON t1.x = t877.x JOIN (SELECT 1 AS x) t878 ON t1.x = t878.x JOIN (SELECT 1 AS x) t879 ON t1.x = t879.x JOIN (SELECT 1 AS x) t880 ON t1.x = t880.x JOIN (SELECT 1 AS x) t881 ON t1.x = t881.x JOIN (SELECT 1 AS x) t882 ON t1.x = t882.x JOIN (SELECT 1 AS x) t883 ON t1.x = t883.x JOIN (SELECT 1 AS x) t884 ON t1.x = t884.x JOIN (SELECT 1 AS x) t885 ON t1.x = t885.x JOIN (SELECT 1 AS x) t886 ON t1.x = t886.x JOIN (SELECT 1 AS x) t887 ON t1.x = t887.x JOIN (SELECT 1 AS x) t888 ON t1.x = t888.x JOIN (SELECT 1 AS x) t889 ON t1.x = t889.x JOIN (SELECT 1 AS x) t890 ON t1.x = t890.x JOIN (SELECT 1 AS x) t891 ON t1.x = t891.x JOIN (SELECT 1 AS x) t892 ON t1.x = t892.x JOIN (SELECT 1 AS x) t893 ON t1.x = t893.x JOIN (SELECT 1 AS x) t894 ON t1.x = t894.x JOIN (SELECT 1 AS x) t895 ON t1.x = t895.x JOIN (SELECT 1 AS x) t896 ON t1.x = t896.x JOIN (SELECT 1 AS x) t897 ON t1.x = t897.x JOIN (SELECT 1 AS x) t898 ON t1.x = t898.x JOIN (SELECT 1 AS x) t899 ON t1.x = t899.x JOIN (SELECT 1 AS x) t900 ON t1.x = t900.x JOIN (SELECT 1 AS x) t901 ON t1.x = t901.x JOIN (SELECT 1 AS x) t902 ON t1.x = t902.x JOIN (SELECT 1 AS x) t903 ON t1.x = t903.x JOIN (SELECT 1 AS x) t904 ON t1.x = t904.x JOIN (SELECT 1 AS x) t905 ON t1.x = t905.x JOIN (SELECT 1 AS x) t906 ON t1.x = t906.x JOIN (SELECT 1 AS x) t907 ON t1.x = t907.x JOIN (SELECT 1 AS x) t908 ON t1.x = t908.x JOIN (SELECT 1 AS x) t909 ON t1.x = t909.x JOIN (SELECT 1 AS x) t910 ON t1.x = t910.x JOIN (SELECT 1 AS x) t911 ON t1.x = t911.x JOIN (SELECT 1 AS x) t912 ON t1.x = t912.x JOIN (SELECT 1 AS x) t913 ON t1.x = t913.x JOIN (SELECT 1 AS x) t914 ON t1.x = t914.x JOIN (SELECT 1 AS x) t915 ON t1.x = t915.x JOIN (SELECT 1 AS x) t916 ON t1.x = t916.x JOIN (SELECT 1 AS x) t917 ON t1.x = t917.x JOIN (SELECT 1 AS x) t918 ON t1.x = t918.x JOIN (SELECT 1 AS x) t919 ON t1.x = t919.x JOIN (SELECT 1 AS x) t920 ON t1.x = t920.x JOIN (SELECT 1 AS x) t921 ON t1.x = t921.x JOIN (SELECT 1 AS x) t922 ON t1.x = t922.x JOIN (SELECT 1 AS x) t923 ON t1.x = t923.x JOIN (SELECT 1 AS x) t924 ON t1.x = t924.x JOIN (SELECT 1 AS x) t925 ON t1.x = t925.x JOIN (SELECT 1 AS x) t926 ON t1.x = t926.x JOIN (SELECT 1 AS x) t927 ON t1.x = t927.x JOIN (SELECT 1 AS x) t928 ON t1.x = t928.x JOIN (SELECT 1 AS x) t929 ON t1.x = t929.x JOIN (SELECT 1 AS x) t930 ON t1.x = t930.x JOIN (SELECT 1 AS x) t931 ON t1.x = t931.x JOIN (SELECT 1 AS x) t932 ON t1.x = t932.x JOIN (SELECT 1 AS x) t933 ON t1.x = t933.x JOIN (SELECT 1 AS x) t934 ON t1.x = t934.x JOIN (SELECT 1 AS x) t935 ON t1.x = t935.x JOIN (SELECT 1 AS x) t936 ON t1.x = t936.x JOIN (SELECT 1 AS x) t937 ON t1.x = t937.x JOIN (SELECT 1 AS x) t938 ON t1.x = t938.x JOIN (SELECT 1 AS x) t939 ON t1.x = t939.x JOIN (SELECT 1 AS x) t940 ON t1.x = t940.x JOIN (SELECT 1 AS x) t941 ON t1.x = t941.x JOIN (SELECT 1 AS x) t942 ON t1.x = t942.x JOIN (SELECT 1 AS x) t943 ON t1.x = t943.x JOIN (SELECT 1 AS x) t944 ON t1.x = t944.x JOIN (SELECT 1 AS x) t945 ON t1.x = t945.x JOIN (SELECT 1 AS x) t946 ON t1.x = t946.x JOIN (SELECT 1 AS x) t947 ON t1.x = t947.x JOIN (SELECT 1 AS x) t948 ON t1.x = t948.x JOIN (SELECT 1 AS x) t949 ON t1.x = t949.x JOIN (SELECT 1 AS x) t950 ON t1.x = t950.x JOIN (SELECT 1 AS x) t951 ON t1.x = t951.x JOIN (SELECT 1 AS x) t952 ON t1.x = t952.x JOIN (SELECT 1 AS x) t953 ON t1.x = t953.x JOIN (SELECT 1 AS x) t954 ON t1.x = t954.x JOIN (SELECT 1 AS x) t955 ON t1.x = t955.x JOIN (SELECT 1 AS x) t956 ON t1.x = t956.x JOIN (SELECT 1 AS x) t957 ON t1.x = t957.x JOIN (SELECT 1 AS x) t958 ON t1.x = t958.x JOIN (SELECT 1 AS x) t959 ON t1.x = t959.x JOIN (SELECT 1 AS x) t960 ON t1.x = t960.x JOIN (SELECT 1 AS x) t961 ON t1.x = t961.x JOIN (SELECT 1 AS x) t962 ON t1.x = t962.x JOIN (SELECT 1 AS x) t963 ON t1.x = t963.x JOIN (SELECT 1 AS x) t964 ON t1.x = t964.x JOIN (SELECT 1 AS x) t965 ON t1.x = t965.x JOIN (SELECT 1 AS x) t966 ON t1.x = t966.x JOIN (SELECT 1 AS x) t967 ON t1.x = t967.x JOIN (SELECT 1 AS x) t968 ON t1.x = t968.x JOIN (SELECT 1 AS x) t969 ON t1.x = t969.x JOIN (SELECT 1 AS x) t970 ON t1.x = t970.x JOIN (SELECT 1 AS x) t971 ON t1.x = t971.x JOIN (SELECT 1 AS x) t972 ON t1.x = t972.x JOIN (SELECT 1 AS x) t973 ON t1.x = t973.x JOIN (SELECT 1 AS x) t974 ON t1.x = t974.x JOIN (SELECT 1 AS x) t975 ON t1.x = t975.x JOIN (SELECT 1 AS x) t976 ON t1.x = t976.x JOIN (SELECT 1 AS x) t977 ON t1.x = t977.x JOIN (SELECT 1 AS x) t978 ON t1.x = t978.x JOIN (SELECT 1 AS x) t979 ON t1.x = t979.x JOIN (SELECT 1 AS x) t980 ON t1.x = t980.x JOIN (SELECT 1 AS x) t981 ON t1.x = t981.x JOIN (SELECT 1 AS x) t982 ON t1.x = t982.x JOIN (SELECT 1 AS x) t983 ON t1.x = t983.x JOIN (SELECT 1 AS x) t984 ON t1.x = t984.x JOIN (SELECT 1 AS x) t985 ON t1.x = t985.x JOIN (SELECT 1 AS x) t986 ON t1.x = t986.x JOIN (SELECT 1 AS x) t987 ON t1.x = t987.x JOIN (SELECT 1 AS x) t988 ON t1.x = t988.x JOIN (SELECT 1 AS x) t989 ON t1.x = t989.x JOIN (SELECT 1 AS x) t990 ON t1.x = t990.x JOIN (SELECT 1 AS x) t991 ON t1.x = t991.x JOIN (SELECT 1 AS x) t992 ON t1.x = t992.x JOIN (SELECT 1 AS x) t993 ON t1.x = t993.x JOIN (SELECT 1 AS x) t994 ON t1.x = t994.x JOIN (SELECT 1 AS x) t995 ON t1.x = t995.x JOIN (SELECT 1 AS x) t996 ON t1.x = t996.x JOIN (SELECT 1 AS x) t997 ON t1.x = t997.x JOIN (SELECT 1 AS x) t998 ON t1.x = t998.x JOIN (SELECT 1 AS x) t999 ON t1.x = t999.x JOIN (SELECT 1 AS x) t1000 ON t1.x = t1000.x From ad442ce024acebe8c617388817622b42534e50b3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 17 Apr 2024 17:47:51 +0100 Subject: [PATCH 0203/1018] Test --- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 44 ++++++++++++----------- 1 file changed, 24 insertions(+), 20 deletions(-) diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index fc3dc1da8ad..902e54cba53 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -1158,29 +1158,33 @@ bool StorageRabbitMQ::tryStreamToViews() ++queue_empty; if (source->needChannelUpdate()) - source->updateChannel(*connection); - - /* false is returned by the sendAck function in only two cases: - * 1) if connection failed. In this case all channels will be closed and will be unable to send ack. Also ack is made based on - * delivery tags, which are unique to channels, so if channels fail, those delivery tags will become invalid and there is - * no way to send specific ack from a different channel. Actually once the server realises that it has messages in a queue - * waiting for confirm from a channel which suddenly closed, it will immediately make those messages accessible to other - * consumers. So in this case duplicates are inevitable. - * 2) size of the sent frame (libraries's internal request interface) exceeds max frame - internal library error. This is more - * common for message frames, but not likely to happen to ack frame I suppose. So I do not believe it is likely to happen. - * Also in this case if channel didn't get closed - it is ok if failed to send ack, because the next attempt to send ack on - * the same channel will also commit all previously not-committed messages. Anyway I do not think that for ack frame this - * will ever happen. - */ - if (write_failed ? source->sendNack() : source->sendAck()) { - /// Iterate loop to activate error callbacks if they happened - connection->getHandler().iterateLoop(); - if (!connection->isConnected()) - break; + source->updateChannel(*connection); } + else + { + /* false is returned by the sendAck function in only two cases: + * 1) if connection failed. In this case all channels will be closed and will be unable to send ack. Also ack is made based on + * delivery tags, which are unique to channels, so if channels fail, those delivery tags will become invalid and there is + * no way to send specific ack from a different channel. Actually once the server realises that it has messages in a queue + * waiting for confirm from a channel which suddenly closed, it will immediately make those messages accessible to other + * consumers. So in this case duplicates are inevitable. + * 2) size of the sent frame (libraries's internal request interface) exceeds max frame - internal library error. This is more + * common for message frames, but not likely to happen to ack frame I suppose. So I do not believe it is likely to happen. + * Also in this case if channel didn't get closed - it is ok if failed to send ack, because the next attempt to send ack on + * the same channel will also commit all previously not-committed messages. Anyway I do not think that for ack frame this + * will ever happen. + */ + if (write_failed ? source->sendNack() : source->sendAck()) + { + /// Iterate loop to activate error callbacks if they happened + connection->getHandler().iterateLoop(); + if (!connection->isConnected()) + break; + } - connection->getHandler().iterateLoop(); + connection->getHandler().iterateLoop(); + } } } From 1e43e3515ea889a55752bf8a1d9d401fdcdf4e0a Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 17 Apr 2024 16:47:52 +0000 Subject: [PATCH 0204/1018] Check table status for parallel replicas --- src/Client/Connection.cpp | 1 - src/Interpreters/ClusterProxy/executeQuery.cpp | 9 ++++++--- src/Interpreters/ClusterProxy/executeQuery.h | 4 +++- src/Planner/findParallelReplicasQuery.cpp | 11 +++-------- src/Processors/QueryPlan/ReadFromRemote.cpp | 5 +++-- src/Processors/QueryPlan/ReadFromRemote.h | 2 ++ .../RemoteQueryExecutorReadContext.h | 1 - src/Storages/StorageMergeTree.cpp | 11 +++-------- src/Storages/StorageReplicatedMergeTree.cpp | 15 +++++---------- src/Storages/StorageReplicatedMergeTree.h | 1 - 10 files changed, 25 insertions(+), 35 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 6a62a9eb296..e1bad42198f 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -617,7 +617,6 @@ TablesStatusResponse Connection::getTablesStatus(const ConnectionTimeouts & time out->next(); fiu_do_on(FailPoints::receive_timeout_on_table_status_response, { - sleepForSeconds(10); throw NetException(ErrorCodes::SOCKET_TIMEOUT, "Injected timeout exceeded while reading from socket ({}:{})", host, port); }); diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 07ef7aa6c96..35978f07c20 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -364,7 +364,9 @@ void executeQuery( void executeQueryWithParallelReplicas( QueryPlan & query_plan, - SelectStreamFactory & stream_factory, + const StorageID & storage_id, + const Block & header, + QueryProcessingStage::Enum processed_stage, const ASTPtr & query_ast, ContextPtr context, std::shared_ptr storage_limits) @@ -453,9 +455,10 @@ void executeQueryWithParallelReplicas( auto read_from_remote = std::make_unique( query_ast, new_cluster, + storage_id, std::move(coordinator), - stream_factory.header, - stream_factory.processed_stage, + header, + processed_stage, new_context, getThrottler(new_context), std::move(scalars), diff --git a/src/Interpreters/ClusterProxy/executeQuery.h b/src/Interpreters/ClusterProxy/executeQuery.h index 8f6f6300c7b..582f8d74fd5 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.h +++ b/src/Interpreters/ClusterProxy/executeQuery.h @@ -68,7 +68,9 @@ void executeQuery( void executeQueryWithParallelReplicas( QueryPlan & query_plan, - SelectStreamFactory & stream_factory, + const StorageID & storage_id, + const Block & header, + QueryProcessingStage::Enum processed_stage, const ASTPtr & query_ast, ContextPtr context, std::shared_ptr storage_limits); diff --git a/src/Planner/findParallelReplicasQuery.cpp b/src/Planner/findParallelReplicasQuery.cpp index ef640bcd42d..fbe81185239 100644 --- a/src/Planner/findParallelReplicasQuery.cpp +++ b/src/Planner/findParallelReplicasQuery.cpp @@ -412,17 +412,12 @@ JoinTreeQueryPlan buildQueryPlanForParallelReplicas( Block header = InterpreterSelectQueryAnalyzer::getSampleBlock( modified_query_tree, context, SelectQueryOptions(processed_stage).analyze()); - ClusterProxy::SelectStreamFactory select_stream_factory = - ClusterProxy::SelectStreamFactory( - header, - {}, - {}, - processed_stage); - QueryPlan query_plan; ClusterProxy::executeQueryWithParallelReplicas( query_plan, - select_stream_factory, + StorageID::createEmpty(), + header, + processed_stage, modified_query_ast, context, storage_limits); diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 119710d06d8..b4e35af85d6 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -360,6 +360,7 @@ void ReadFromRemote::initializePipeline(QueryPipelineBuilder & pipeline, const B ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( ASTPtr query_ast_, ClusterPtr cluster_, + const StorageID & storage_id_, ParallelReplicasReadingCoordinatorPtr coordinator_, Block header_, QueryProcessingStage::Enum stage_, @@ -372,6 +373,7 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( : ISourceStep(DataStream{.header = std::move(header_)}) , cluster(cluster_) , query_ast(query_ast_) + , storage_id(storage_id_) , coordinator(std::move(coordinator_)) , stage(std::move(stage_)) , context(context_) @@ -419,7 +421,6 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder all_replicas_count = shard.getAllNodeCount(); } - std::vector shuffled_pool; if (all_replicas_count < shard.getAllNodeCount()) { @@ -452,7 +453,6 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder processor->setStorageLimits(storage_limits); pipeline.init(std::move(pipe)); - } @@ -488,6 +488,7 @@ void ReadFromParallelRemoteReplicasStep::addPipeForSingeReplica( RemoteQueryExecutor::Extension{.parallel_reading_coordinator = coordinator, .replica_info = std::move(replica_info)}); remote_query_executor->setLogger(log); + remote_query_executor->setMainTable(storage_id); pipes.emplace_back(createRemoteSourcePipe(std::move(remote_query_executor), add_agg_info, add_totals, add_extremes, async_read, async_query_sending)); addConvertingActions(pipes.back(), output_stream->header); diff --git a/src/Processors/QueryPlan/ReadFromRemote.h b/src/Processors/QueryPlan/ReadFromRemote.h index 498d584e85a..eb15269155a 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.h +++ b/src/Processors/QueryPlan/ReadFromRemote.h @@ -69,6 +69,7 @@ public: ReadFromParallelRemoteReplicasStep( ASTPtr query_ast_, ClusterPtr cluster_, + const StorageID & storage_id_, ParallelReplicasReadingCoordinatorPtr coordinator_, Block header_, QueryProcessingStage::Enum stage_, @@ -91,6 +92,7 @@ private: ClusterPtr cluster; ASTPtr query_ast; + StorageID storage_id; ParallelReplicasReadingCoordinatorPtr coordinator; QueryProcessingStage::Enum stage; ContextMutablePtr context; diff --git a/src/QueryPipeline/RemoteQueryExecutorReadContext.h b/src/QueryPipeline/RemoteQueryExecutorReadContext.h index 4e62b42a067..b8aa8bb9111 100644 --- a/src/QueryPipeline/RemoteQueryExecutorReadContext.h +++ b/src/QueryPipeline/RemoteQueryExecutorReadContext.h @@ -2,7 +2,6 @@ #if defined(OS_LINUX) -#include #include #include #include diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 86af02be899..65218632287 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -235,16 +235,11 @@ void StorageMergeTree::read( = InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); } - ClusterProxy::SelectStreamFactory select_stream_factory = - ClusterProxy::SelectStreamFactory( - header, - {}, - storage_snapshot, - processed_stage); - ClusterProxy::executeQueryWithParallelReplicas( query_plan, - select_stream_factory, + getStorageID(), + header, + processed_stage, modified_query_ast, local_context, query_info.storage_limits); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index aa90fc43d52..c9b28c5405c 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5449,7 +5449,7 @@ void StorageReplicatedMergeTree::read( return readLocalSequentialConsistencyImpl(query_plan, column_names, storage_snapshot, query_info, local_context, max_block_size, num_streams); if (local_context->canUseParallelReplicasOnInitiator()) - return readParallelReplicasImpl(query_plan, column_names, storage_snapshot, query_info, local_context, processed_stage); + return readParallelReplicasImpl(query_plan, column_names, query_info, local_context, processed_stage); readLocalImpl(query_plan, column_names, storage_snapshot, query_info, local_context, max_block_size, num_streams); } @@ -5476,13 +5476,13 @@ void StorageReplicatedMergeTree::readLocalSequentialConsistencyImpl( void StorageReplicatedMergeTree::readParallelReplicasImpl( QueryPlan & query_plan, const Names & /*column_names*/, - const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, ContextPtr local_context, QueryProcessingStage::Enum processed_stage) { ASTPtr modified_query_ast; Block header; + const auto table_id = getStorageID(); if (local_context->getSettingsRef().allow_experimental_analyzer) { @@ -5496,22 +5496,17 @@ void StorageReplicatedMergeTree::readParallelReplicasImpl( } else { - const auto table_id = getStorageID(); modified_query_ast = ClusterProxy::rewriteSelectQuery(local_context, query_info.query, table_id.database_name, table_id.table_name, /*remote_table_function_ptr*/nullptr); header = InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); } - ClusterProxy::SelectStreamFactory select_stream_factory = ClusterProxy::SelectStreamFactory( - header, - {}, - storage_snapshot, - processed_stage); - ClusterProxy::executeQueryWithParallelReplicas( query_plan, - select_stream_factory, + table_id, + header, + processed_stage, modified_query_ast, local_context, query_info.storage_limits); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index c472c11e7f8..c131aa3fad3 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -567,7 +567,6 @@ private: void readParallelReplicasImpl( QueryPlan & query_plan, const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, ContextPtr local_context, QueryProcessingStage::Enum processed_stage); From 556932706b565924e311dedf1e6f5b6b82f1b49b Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Wed, 17 Apr 2024 19:06:33 +0200 Subject: [PATCH 0205/1018] Add SYSTEM UNLOAD PRIMARY KEY --- docs/en/sql-reference/statements/system.md | 12 +++++ src/Access/Common/AccessType.h | 1 + src/Interpreters/InterpreterSystemQuery.cpp | 47 +++++++++++++++++++ src/Interpreters/InterpreterSystemQuery.h | 1 + src/Parsers/ASTSystemQuery.cpp | 1 + src/Parsers/ASTSystemQuery.h | 1 + src/Parsers/ParserSystemQuery.cpp | 1 + src/Storages/MergeTree/IMergeTreeDataPart.cpp | 10 ++-- src/Storages/MergeTree/IMergeTreeDataPart.h | 1 + src/Storages/MergeTree/MergeTreeData.cpp | 8 ++++ src/Storages/MergeTree/MergeTreeData.h | 2 + .../03127_system_unload_primary_key.reference | 9 ++++ .../03127_system_unload_primary_key.sql | 25 ++++++++++ 13 files changed, 115 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/03127_system_unload_primary_key.reference create mode 100644 tests/queries/0_stateless/03127_system_unload_primary_key.sql diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index b35e9426297..9fec5420f97 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -532,3 +532,15 @@ If there's a refresh in progress for the given view, interrupt and cancel it. Ot ```sql SYSTEM CANCEL VIEW [db.]name ``` + +### SYSTEM UNLOAD PRIMARY KEY + +Unload the primary keys for the given table or for all tables. + +```sql +SYSTEM UNLOAD PRIMARY KEY [db.]name +``` + +```sql +SYSTEM UNLOAD PRIMARY KEY +``` \ No newline at end of file diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 83b50ce96c3..63de202c060 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -208,6 +208,7 @@ enum class AccessType M(SYSTEM_FAILPOINT, "SYSTEM ENABLE FAILPOINT, SYSTEM DISABLE FAILPOINT, SYSTEM WAIT FAILPOINT", GLOBAL, SYSTEM) \ M(SYSTEM_LISTEN, "SYSTEM START LISTEN, SYSTEM STOP LISTEN", GLOBAL, SYSTEM) \ M(SYSTEM_JEMALLOC, "SYSTEM JEMALLOC PURGE, SYSTEM JEMALLOC ENABLE PROFILE, SYSTEM JEMALLOC DISABLE PROFILE, SYSTEM JEMALLOC FLUSH PROFILE", GLOBAL, SYSTEM) \ + M(SYSTEM_UNLOAD_PRIMARY_KEY, "SYSTEM UNLOAD PRIMARY KEY", TABLE, SYSTEM) \ M(SYSTEM, "", GROUP, ALL) /* allows to execute SYSTEM {SHUTDOWN|RELOAD CONFIG|...} */ \ \ M(dictGet, "dictHas, dictGetHierarchy, dictIsIn", DICTIONARY, ALL) /* allows to execute functions dictGet(), dictHas(), dictGetHierarchy(), dictIsIn() */\ diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index f6db12e977c..6c112dbdc84 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -771,6 +771,11 @@ BlockIO InterpreterSystemQuery::execute() resetCoverage(); break; } + case Type::UNLOAD_PRIMARY_KEY: + { + unloadPrimaryKeys(); + break; + } #if USE_JEMALLOC case Type::JEMALLOC_PURGE: @@ -1148,6 +1153,40 @@ void InterpreterSystemQuery::waitLoadingParts() } } +void InterpreterSystemQuery::unloadPrimaryKeys() +{ + if (!table_id.empty()) + { + getContext()->checkAccess(AccessType::SYSTEM_UNLOAD_PRIMARY_KEY, table_id.database_name, table_id.table_name); + StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext()); + + if (auto * merge_tree = dynamic_cast(table.get())) + { + LOG_TRACE(log, "Unloading primary keys for table {}", table_id.getFullTableName()); + merge_tree->unloadPrimaryKeys(); + } + else + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Command UNLOAD PRIMARY KEY is supported only for MergeTree table, but got: {}", table->getName()); + } + } else { + getContext()->checkAccess(AccessType::SYSTEM_UNLOAD_PRIMARY_KEY); + LOG_TRACE(log, "Unloading primary keys for all tables"); + + for (auto & database : DatabaseCatalog::instance().getDatabases()) + { + for (auto it = database.second->getTablesIterator(getContext()); it->isValid(); it->next()) + { + if (auto * merge_tree = dynamic_cast(it->table().get())) + { + merge_tree->unloadPrimaryKeys(); + } + } + } + } +} + void InterpreterSystemQuery::syncReplicatedDatabase(ASTSystemQuery & query) { const auto database_name = query.getDatabase(); @@ -1461,6 +1500,14 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() required_access.emplace_back(AccessType::SYSTEM_JEMALLOC); break; } + case Type::UNLOAD_PRIMARY_KEY: + { + if (!query.table) + required_access.emplace_back(AccessType::SYSTEM_UNLOAD_PRIMARY_KEY); + else + required_access.emplace_back(AccessType::SYSTEM_UNLOAD_PRIMARY_KEY, query.getDatabase(), query.getTable()); + break; + } case Type::STOP_THREAD_FUZZER: case Type::START_THREAD_FUZZER: case Type::ENABLE_FAILPOINT: diff --git a/src/Interpreters/InterpreterSystemQuery.h b/src/Interpreters/InterpreterSystemQuery.h index 1419c430aca..776dd7915f0 100644 --- a/src/Interpreters/InterpreterSystemQuery.h +++ b/src/Interpreters/InterpreterSystemQuery.h @@ -60,6 +60,7 @@ private: void syncReplica(ASTSystemQuery & query); void setReplicaReadiness(bool ready); void waitLoadingParts(); + void unloadPrimaryKeys(); void syncReplicatedDatabase(ASTSystemQuery & query); diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index ed122b2b191..a730ea0ba3d 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -173,6 +173,7 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState & s case Type::START_PULLING_REPLICATION_LOG: case Type::STOP_CLEANUP: case Type::START_CLEANUP: + case Type::UNLOAD_PRIMARY_KEY: { if (table) { diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 65c3f0eb328..167e724dcee 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -101,6 +101,7 @@ public: STOP_VIEWS, CANCEL_VIEW, TEST_VIEW, + UNLOAD_PRIMARY_KEY, END }; diff --git a/src/Parsers/ParserSystemQuery.cpp b/src/Parsers/ParserSystemQuery.cpp index b660f947290..696cb65e5be 100644 --- a/src/Parsers/ParserSystemQuery.cpp +++ b/src/Parsers/ParserSystemQuery.cpp @@ -323,6 +323,7 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & /// START/STOP DISTRIBUTED SENDS does not require table case Type::STOP_DISTRIBUTED_SENDS: case Type::START_DISTRIBUTED_SENDS: + case Type::UNLOAD_PRIMARY_KEY: { if (!parseQueryWithOnClusterAndMaybeTable(res, pos, expected, /* require table = */ false, /* allow_string_literal = */ false)) return false; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 441437855ab..360af3291bf 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -358,6 +358,12 @@ void IMergeTreeDataPart::setIndex(Columns index_) index_loaded = true; } +void IMergeTreeDataPart::unloadIndex() +{ + std::scoped_lock lock(index_mutex); + index.clear(); + index_loaded = false; +} void IMergeTreeDataPart::setName(const String & new_name) { @@ -843,10 +849,6 @@ void IMergeTreeDataPart::loadIndex() const /// Memory for index must not be accounted as memory usage for query, because it belongs to a table. MemoryTrackerBlockerInThread temporarily_disable_memory_tracker; - /// It can be empty in case of mutations - if (!index_granularity.isInitialized()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Index granularity is not loaded before index loading"); - auto metadata_snapshot = storage.getInMemoryMetadataPtr(); if (parent_part) metadata_snapshot = metadata_snapshot->projections.get(name).metadata; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 7519980a7a3..cdee732b97f 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -369,6 +369,7 @@ public: const Index & getIndex() const; void setIndex(Columns index_); + void unloadIndex(); /// For data in RAM ('index') UInt64 getIndexSizeInBytes() const; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index fcab606130d..c1d7d138f4b 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8371,4 +8371,12 @@ bool MergeTreeData::initializeDiskOnConfigChange(const std::set & new_ad } return true; } + +void MergeTreeData::unloadPrimaryKeys() +{ + for (auto & part : getAllDataPartsVector()) + { + const_cast(*part).unloadIndex(); + } +} } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 3b7ff1454a5..166f4fbba91 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1090,6 +1090,8 @@ public: static VirtualColumnsDescription createVirtuals(const StorageInMemoryMetadata & metadata); + void unloadPrimaryKeys(); + protected: friend class IMergeTreeDataPart; friend class MergeTreeDataMergerMutator; diff --git a/tests/queries/0_stateless/03127_system_unload_primary_key.reference b/tests/queries/0_stateless/03127_system_unload_primary_key.reference new file mode 100644 index 00000000000..8b352f9c80e --- /dev/null +++ b/tests/queries/0_stateless/03127_system_unload_primary_key.reference @@ -0,0 +1,9 @@ +100000000 140000000 +0 0 +0 0 +100000000 140000000 +1 +100000000 100000000 +100000000 140000000 +0 0 +0 0 diff --git a/tests/queries/0_stateless/03127_system_unload_primary_key.sql b/tests/queries/0_stateless/03127_system_unload_primary_key.sql new file mode 100644 index 00000000000..811d62b1d48 --- /dev/null +++ b/tests/queries/0_stateless/03127_system_unload_primary_key.sql @@ -0,0 +1,25 @@ +DROP TABLE IF EXISTS test; +CREATE TABLE test (s String) ENGINE = MergeTree ORDER BY s SETTINGS index_granularity = 1; + +INSERT INTO test SELECT randomString(1000) FROM numbers(100000); + +SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) FROM system.parts WHERE database = currentDatabase() AND table = 'test'; + +SYSTEM UNLOAD PRIMARY KEY {CLICKHOUSE_DATABASE:Identifier}.test; + +SELECT primary_key_bytes_in_memory, primary_key_bytes_in_memory_allocated FROM system.parts WHERE database = currentDatabase() AND table = 'test'; + +DROP TABLE IF EXISTS test2; +CREATE TABLE test2 (s String) ENGINE = MergeTree ORDER BY s SETTINGS index_granularity = 1; + +INSERT INTO test2 SELECT randomString(1000) FROM numbers(100000); + +SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) FROM system.parts WHERE database = currentDatabase() AND table IN ('test', 'test2'); + +SELECT s != '' FROM test LIMIT 1; + +SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) FROM system.parts WHERE database = currentDatabase() AND table IN ('test', 'test2'); + +SYSTEM UNLOAD PRIMARY KEY; + +SELECT primary_key_bytes_in_memory, primary_key_bytes_in_memory_allocated FROM system.parts WHERE database = currentDatabase() AND table IN ('test', 'test2'); From d8baccaa494be142a87b14c3d493b7b1b2f3f76d Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 18 Apr 2024 07:09:22 +0200 Subject: [PATCH 0206/1018] Fix coding style --- src/Interpreters/InterpreterSystemQuery.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 6c112dbdc84..fb1dd88d7be 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -1170,7 +1170,9 @@ void InterpreterSystemQuery::unloadPrimaryKeys() throw Exception(ErrorCodes::BAD_ARGUMENTS, "Command UNLOAD PRIMARY KEY is supported only for MergeTree table, but got: {}", table->getName()); } - } else { + } + else + { getContext()->checkAccess(AccessType::SYSTEM_UNLOAD_PRIMARY_KEY); LOG_TRACE(log, "Unloading primary keys for all tables"); From 3d73759d02db2b6f6983f60b87fa0f1c2eceb919 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 18 Apr 2024 07:25:58 +0200 Subject: [PATCH 0207/1018] Fix 01271_show_privileges.reference with new entry --- tests/queries/0_stateless/01271_show_privileges.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index b1237189cb3..48d78c04dbc 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -158,6 +158,7 @@ SYSTEM UNFREEZE ['SYSTEM UNFREEZE'] GLOBAL SYSTEM SYSTEM FAILPOINT ['SYSTEM ENABLE FAILPOINT','SYSTEM DISABLE FAILPOINT','SYSTEM WAIT FAILPOINT'] GLOBAL SYSTEM SYSTEM LISTEN ['SYSTEM START LISTEN','SYSTEM STOP LISTEN'] GLOBAL SYSTEM SYSTEM JEMALLOC ['SYSTEM JEMALLOC PURGE','SYSTEM JEMALLOC ENABLE PROFILE','SYSTEM JEMALLOC DISABLE PROFILE','SYSTEM JEMALLOC FLUSH PROFILE'] GLOBAL SYSTEM +SYSTEM UNLOAD PRIMARY KEY ['SYSTEM UNLOAD PRIMARY KEY'] TABLE SYSTEM SYSTEM [] \N ALL dictGet ['dictHas','dictGetHierarchy','dictIsIn'] DICTIONARY ALL displaySecretsInShowAndSelect [] GLOBAL ALL From 210d20f44b46c20857ac94c5fe38108b2a0c9496 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 18 Apr 2024 08:58:05 +0200 Subject: [PATCH 0208/1018] Fix AccessRights.Union unit test with new entry --- src/Access/tests/gtest_access_rights_ops.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Access/tests/gtest_access_rights_ops.cpp b/src/Access/tests/gtest_access_rights_ops.cpp index 97acef1c9eb..7384a741699 100644 --- a/src/Access/tests/gtest_access_rights_ops.cpp +++ b/src/Access/tests/gtest_access_rights_ops.cpp @@ -53,7 +53,8 @@ TEST(AccessRights, Union) "SHOW ROW POLICIES, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, " "SYSTEM MOVES, SYSTEM PULLING REPLICATION LOG, SYSTEM CLEANUP, SYSTEM VIEWS, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM VIRTUAL PARTS UPDATE, " "SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, " - "SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM SYNC DATABASE REPLICA, SYSTEM FLUSH DISTRIBUTED, dictGet ON db1.*, " + "SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM SYNC DATABASE REPLICA, SYSTEM FLUSH DISTRIBUTED, " + "SYSTEM UNLOAD PRIMARY KEY, dictGet ON db1.*, " "GRANT SET DEFINER ON db1, GRANT NAMED COLLECTION ADMIN ON db1"); } From 506517e42b0073daac4510a107f3d6eba4147bb4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 18 Apr 2024 11:13:59 +0100 Subject: [PATCH 0209/1018] Test --- src/Storages/RabbitMQ/RabbitMQConsumer.cpp | 9 +++++++-- src/Storages/RabbitMQ/RabbitMQSource.h | 1 + src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 1 + 3 files changed, 9 insertions(+), 2 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQConsumer.cpp b/src/Storages/RabbitMQ/RabbitMQConsumer.cpp index 1d99581b4a5..c025fee8808 100644 --- a/src/Storages/RabbitMQ/RabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/RabbitMQConsumer.cpp @@ -200,8 +200,12 @@ void RabbitMQConsumer::updateChannel(RabbitMQConnection & connection) consumer_channel->onError([&](const char * message) { - LOG_ERROR(log, "Channel {} in in error state: {}", channel_id, message); - state = State::ERROR; + LOG_ERROR(log, "Channel {} received an error: {}", channel_id, message); + + if (!consumer_channel->usable() || !consumer_channel->connected()) + { + state = State::ERROR; + } }); } @@ -209,6 +213,7 @@ void RabbitMQConsumer::updateChannel(RabbitMQConnection & connection) bool RabbitMQConsumer::needChannelUpdate() { chassert(consumer_channel); + // chassert(state != State::ERROR || !consumer_channel->usable() || !consumer_channel->connected() || !consumer_channel->ready()); return state == State::ERROR; } diff --git a/src/Storages/RabbitMQ/RabbitMQSource.h b/src/Storages/RabbitMQ/RabbitMQSource.h index 8d939e5c57c..dc0af0043be 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.h +++ b/src/Storages/RabbitMQ/RabbitMQSource.h @@ -27,6 +27,7 @@ public: String getName() const override { return storage.getName(); } void updateChannel(RabbitMQConnection & connection) { consumer->updateChannel(connection); } + String getChannelID() const { return consumer->getChannelID(); } Chunk generate() override; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 902e54cba53..a73db9b0b15 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -1159,6 +1159,7 @@ bool StorageRabbitMQ::tryStreamToViews() if (source->needChannelUpdate()) { + LOG_TEST(log, "Channel {} is in error state, will update", source->getChannelID()); source->updateChannel(*connection); } else From aca46eecc90a56d28a52b45ca2302cc51816d755 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Thu, 18 Apr 2024 10:30:36 +0000 Subject: [PATCH 0210/1018] Block cannot allocate thread fault in noexcept functions in `MergeTreeTransaction` --- src/Interpreters/MergeTreeTransaction.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index 6b8e09a64f5..b389b6cbb4b 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -212,6 +212,7 @@ scope_guard MergeTreeTransaction::beforeCommit() void MergeTreeTransaction::afterCommit(CSN assigned_csn) noexcept { + auto blocker = CannotAllocateThreadFaultInjector::blockFaultInjections(); LockMemoryExceptionInThread memory_tracker_lock(VariableContext::Global); /// Write allocated CSN into version metadata, so we will know CSN without reading it from transaction log /// and we will be able to remove old entries from transaction log in ZK. @@ -248,6 +249,7 @@ void MergeTreeTransaction::afterCommit(CSN assigned_csn) noexcept bool MergeTreeTransaction::rollback() noexcept { + auto blocker = CannotAllocateThreadFaultInjector::blockFaultInjections(); LockMemoryExceptionInThread memory_tracker_lock(VariableContext::Global); CSN expected = Tx::UnknownCSN; bool need_rollback = csn.compare_exchange_strong(expected, Tx::RolledBackCSN); From 01b80e754abbde6e379064bdd01987b1990a2b42 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Thu, 18 Apr 2024 11:37:09 +0100 Subject: [PATCH 0211/1018] Log profile events send timings --- src/Server/TCPHandler.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 4e3d6ab69f6..4553d1992b2 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1233,6 +1233,7 @@ void TCPHandler::sendExtremes(const Block & extremes) void TCPHandler::sendProfileEvents() { + Stopwatch stopwatch; Block block; ProfileEvents::getProfileEvents(host_name, state.profile_queue, block, last_sent_snapshots); if (block.rows() != 0) @@ -1244,6 +1245,9 @@ void TCPHandler::sendProfileEvents() state.profile_events_block_out->write(block); out->next(); + + LOG_TRACE(log, "Sending profile events block with {} rows, {} bytes took {} milliseconds", + block.rows(), block.bytes(), stopwatch.elapsedMilliseconds()); } } From 6e5fe39e570fe92ed45907a955193a31133acab8 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 18 Apr 2024 13:13:28 +0200 Subject: [PATCH 0212/1018] Fix test_grant_and_revoke/test.py::test_grant_all_on_table with new entry --- tests/integration/test_grant_and_revoke/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_grant_and_revoke/test.py b/tests/integration/test_grant_and_revoke/test.py index 46d8d254a0a..18fbab1a0a1 100644 --- a/tests/integration/test_grant_and_revoke/test.py +++ b/tests/integration/test_grant_and_revoke/test.py @@ -186,7 +186,7 @@ def test_grant_all_on_table(): instance.query("GRANT ALL ON test.table TO B", user="A") assert ( instance.query("SHOW GRANTS FOR B") - == "GRANT SHOW TABLES, SHOW COLUMNS, SHOW DICTIONARIES, SELECT, INSERT, ALTER TABLE, ALTER VIEW, CREATE TABLE, CREATE VIEW, CREATE DICTIONARY, DROP TABLE, DROP VIEW, DROP DICTIONARY, UNDROP TABLE, TRUNCATE, OPTIMIZE, BACKUP, CREATE ROW POLICY, ALTER ROW POLICY, DROP ROW POLICY, SHOW ROW POLICIES, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, SYSTEM MOVES, SYSTEM PULLING REPLICATION LOG, SYSTEM CLEANUP, SYSTEM VIEWS, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM VIRTUAL PARTS UPDATE, SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM FLUSH DISTRIBUTED, dictGet ON test.`table` TO B\n" + == "GRANT SHOW TABLES, SHOW COLUMNS, SHOW DICTIONARIES, SELECT, INSERT, ALTER TABLE, ALTER VIEW, CREATE TABLE, CREATE VIEW, CREATE DICTIONARY, DROP TABLE, DROP VIEW, DROP DICTIONARY, UNDROP TABLE, TRUNCATE, OPTIMIZE, BACKUP, CREATE ROW POLICY, ALTER ROW POLICY, DROP ROW POLICY, SHOW ROW POLICIES, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, SYSTEM MOVES, SYSTEM PULLING REPLICATION LOG, SYSTEM CLEANUP, SYSTEM VIEWS, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM VIRTUAL PARTS UPDATE, SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM FLUSH DISTRIBUTED, SYSTEM UNLOAD PRIMARY KEY, dictGet ON test.`table` TO B" ) instance.query("REVOKE ALL ON test.table FROM B", user="A") assert instance.query("SHOW GRANTS FOR B") == "" From 11eab640c855938bd317348bdc4922753ecbc849 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 18 Apr 2024 13:14:41 +0100 Subject: [PATCH 0213/1018] Ping CI From 8de93c206c67dfaa4e369b1f5751c2ed18850539 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 18 Apr 2024 12:26:40 +0000 Subject: [PATCH 0214/1018] Reduce test size --- tests/queries/0_stateless/03094_one_thousand_joins.reference | 2 +- tests/queries/0_stateless/03094_one_thousand_joins.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03094_one_thousand_joins.reference b/tests/queries/0_stateless/03094_one_thousand_joins.reference index 52176202f05..dd729ab0220 100644 --- a/tests/queries/0_stateless/03094_one_thousand_joins.reference +++ b/tests/queries/0_stateless/03094_one_thousand_joins.reference @@ -1 +1 @@ -1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 +1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 diff --git a/tests/queries/0_stateless/03094_one_thousand_joins.sql b/tests/queries/0_stateless/03094_one_thousand_joins.sql index f7ca2134d2e..b6345595811 100644 --- a/tests/queries/0_stateless/03094_one_thousand_joins.sql +++ b/tests/queries/0_stateless/03094_one_thousand_joins.sql @@ -1,4 +1,4 @@ -- Tags: no-fasttest, long -- Bug 33446, marked as 'long' because it still runs around 10 sec -SELECT * FROM (SELECT 1 AS x) t1 JOIN (SELECT 1 AS x) t2 ON t1.x = t2.x JOIN (SELECT 1 AS x) t3 ON t1.x = t3.x JOIN (SELECT 1 AS x) t4 ON t1.x = t4.x JOIN (SELECT 1 AS x) t5 ON t1.x = t5.x JOIN (SELECT 1 AS x) t6 ON t1.x = t6.x JOIN (SELECT 1 AS x) t7 ON t1.x = t7.x JOIN (SELECT 1 AS x) t8 ON t1.x = t8.x JOIN (SELECT 1 AS x) t9 ON t1.x = t9.x JOIN (SELECT 1 AS x) t10 ON t1.x = t10.x JOIN (SELECT 1 AS x) t11 ON t1.x = t11.x JOIN (SELECT 1 AS x) t12 ON t1.x = t12.x JOIN (SELECT 1 AS x) t13 ON t1.x = t13.x JOIN (SELECT 1 AS x) t14 ON t1.x = t14.x JOIN (SELECT 1 AS x) t15 ON t1.x = t15.x JOIN (SELECT 1 AS x) t16 ON t1.x = t16.x JOIN (SELECT 1 AS x) t17 ON t1.x = t17.x JOIN (SELECT 1 AS x) t18 ON t1.x = t18.x JOIN (SELECT 1 AS x) t19 ON t1.x = t19.x JOIN (SELECT 1 AS x) t20 ON t1.x = t20.x JOIN (SELECT 1 AS x) t21 ON t1.x = t21.x JOIN (SELECT 1 AS x) t22 ON t1.x = t22.x JOIN (SELECT 1 AS x) t23 ON t1.x = t23.x JOIN (SELECT 1 AS x) t24 ON t1.x = t24.x JOIN (SELECT 1 AS x) t25 ON t1.x = t25.x JOIN (SELECT 1 AS x) t26 ON t1.x = t26.x JOIN (SELECT 1 AS x) t27 ON t1.x = t27.x JOIN (SELECT 1 AS x) t28 ON t1.x = t28.x JOIN (SELECT 1 AS x) t29 ON t1.x = t29.x JOIN (SELECT 1 AS x) t30 ON t1.x = t30.x JOIN (SELECT 1 AS x) t31 ON t1.x = t31.x JOIN (SELECT 1 AS x) t32 ON t1.x = t32.x JOIN (SELECT 1 AS x) t33 ON t1.x = t33.x JOIN (SELECT 1 AS x) t34 ON t1.x = t34.x JOIN (SELECT 1 AS x) t35 ON t1.x = t35.x JOIN (SELECT 1 AS x) t36 ON t1.x = t36.x JOIN (SELECT 1 AS x) t37 ON t1.x = t37.x JOIN (SELECT 1 AS x) t38 ON t1.x = t38.x JOIN (SELECT 1 AS x) t39 ON t1.x = t39.x JOIN (SELECT 1 AS x) t40 ON t1.x = t40.x JOIN (SELECT 1 AS x) t41 ON t1.x = t41.x JOIN (SELECT 1 AS x) t42 ON t1.x = t42.x JOIN (SELECT 1 AS x) t43 ON t1.x = t43.x JOIN (SELECT 1 AS x) t44 ON t1.x = t44.x JOIN (SELECT 1 AS x) t45 ON t1.x = t45.x JOIN (SELECT 1 AS x) t46 ON t1.x = t46.x JOIN (SELECT 1 AS x) t47 ON t1.x = t47.x JOIN (SELECT 1 AS x) t48 ON t1.x = t48.x JOIN (SELECT 1 AS x) t49 ON t1.x = t49.x JOIN (SELECT 1 AS x) t50 ON t1.x = t50.x JOIN (SELECT 1 AS x) t51 ON t1.x = t51.x JOIN (SELECT 1 AS x) t52 ON t1.x = t52.x JOIN (SELECT 1 AS x) t53 ON t1.x = t53.x JOIN (SELECT 1 AS x) t54 ON t1.x = t54.x JOIN (SELECT 1 AS x) t55 ON t1.x = t55.x JOIN (SELECT 1 AS x) t56 ON t1.x = t56.x JOIN (SELECT 1 AS x) t57 ON t1.x = t57.x JOIN (SELECT 1 AS x) t58 ON t1.x = t58.x JOIN (SELECT 1 AS x) t59 ON t1.x = t59.x JOIN (SELECT 1 AS x) t60 ON t1.x = t60.x JOIN (SELECT 1 AS x) t61 ON t1.x = t61.x JOIN (SELECT 1 AS x) t62 ON t1.x = t62.x JOIN (SELECT 1 AS x) t63 ON t1.x = t63.x JOIN (SELECT 1 AS x) t64 ON t1.x = t64.x JOIN (SELECT 1 AS x) t65 ON t1.x = t65.x JOIN (SELECT 1 AS x) t66 ON t1.x = t66.x JOIN (SELECT 1 AS x) t67 ON t1.x = t67.x JOIN (SELECT 1 AS x) t68 ON t1.x = t68.x JOIN (SELECT 1 AS x) t69 ON t1.x = t69.x JOIN (SELECT 1 AS x) t70 ON t1.x = t70.x JOIN (SELECT 1 AS x) t71 ON t1.x = t71.x JOIN (SELECT 1 AS x) t72 ON t1.x = t72.x JOIN (SELECT 1 AS x) t73 ON t1.x = t73.x JOIN (SELECT 1 AS x) t74 ON t1.x = t74.x JOIN (SELECT 1 AS x) t75 ON t1.x = t75.x JOIN (SELECT 1 AS x) t76 ON t1.x = t76.x JOIN (SELECT 1 AS x) t77 ON t1.x = t77.x JOIN (SELECT 1 AS x) t78 ON t1.x = t78.x JOIN (SELECT 1 AS x) t79 ON t1.x = t79.x JOIN (SELECT 1 AS x) t80 ON t1.x = t80.x JOIN (SELECT 1 AS x) t81 ON t1.x = t81.x JOIN (SELECT 1 AS x) t82 ON t1.x = t82.x JOIN (SELECT 1 AS x) t83 ON t1.x = t83.x JOIN (SELECT 1 AS x) t84 ON t1.x = t84.x JOIN (SELECT 1 AS x) t85 ON t1.x = t85.x JOIN (SELECT 1 AS x) t86 ON t1.x = t86.x JOIN (SELECT 1 AS x) t87 ON t1.x = t87.x JOIN (SELECT 1 AS x) t88 ON t1.x = t88.x JOIN (SELECT 1 AS x) t89 ON t1.x = t89.x JOIN (SELECT 1 AS x) t90 ON t1.x = t90.x JOIN (SELECT 1 AS x) t91 ON t1.x = t91.x JOIN (SELECT 1 AS x) t92 ON t1.x = t92.x JOIN (SELECT 1 AS x) t93 ON t1.x = t93.x JOIN (SELECT 1 AS x) t94 ON t1.x = t94.x JOIN (SELECT 1 AS x) t95 ON t1.x = t95.x JOIN (SELECT 1 AS x) t96 ON t1.x = t96.x JOIN (SELECT 1 AS x) t97 ON t1.x = t97.x JOIN (SELECT 1 AS x) t98 ON t1.x = t98.x JOIN (SELECT 1 AS x) t99 ON t1.x = t99.x JOIN (SELECT 1 AS x) t100 ON t1.x = t100.x JOIN (SELECT 1 AS x) t101 ON t1.x = t101.x JOIN (SELECT 1 AS x) t102 ON t1.x = t102.x JOIN (SELECT 1 AS x) t103 ON t1.x = t103.x JOIN (SELECT 1 AS x) t104 ON t1.x = t104.x JOIN (SELECT 1 AS x) t105 ON t1.x = t105.x JOIN (SELECT 1 AS x) t106 ON t1.x = t106.x JOIN (SELECT 1 AS x) t107 ON t1.x = t107.x JOIN (SELECT 1 AS x) t108 ON t1.x = t108.x JOIN (SELECT 1 AS x) t109 ON t1.x = t109.x JOIN (SELECT 1 AS x) t110 ON t1.x = t110.x JOIN (SELECT 1 AS x) t111 ON t1.x = t111.x JOIN (SELECT 1 AS x) t112 ON t1.x = t112.x JOIN (SELECT 1 AS x) t113 ON t1.x = t113.x JOIN (SELECT 1 AS x) t114 ON t1.x = t114.x JOIN (SELECT 1 AS x) t115 ON t1.x = t115.x JOIN (SELECT 1 AS x) t116 ON t1.x = t116.x JOIN (SELECT 1 AS x) t117 ON t1.x = t117.x JOIN (SELECT 1 AS x) t118 ON t1.x = t118.x JOIN (SELECT 1 AS x) t119 ON t1.x = t119.x JOIN (SELECT 1 AS x) t120 ON t1.x = t120.x JOIN (SELECT 1 AS x) t121 ON t1.x = t121.x JOIN (SELECT 1 AS x) t122 ON t1.x = t122.x JOIN (SELECT 1 AS x) t123 ON t1.x = t123.x JOIN (SELECT 1 AS x) t124 ON t1.x = t124.x JOIN (SELECT 1 AS x) t125 ON t1.x = t125.x JOIN (SELECT 1 AS x) t126 ON t1.x = t126.x JOIN (SELECT 1 AS x) t127 ON t1.x = t127.x JOIN (SELECT 1 AS x) t128 ON t1.x = t128.x JOIN (SELECT 1 AS x) t129 ON t1.x = t129.x JOIN (SELECT 1 AS x) t130 ON t1.x = t130.x JOIN (SELECT 1 AS x) t131 ON t1.x = t131.x JOIN (SELECT 1 AS x) t132 ON t1.x = t132.x JOIN (SELECT 1 AS x) t133 ON t1.x = t133.x JOIN (SELECT 1 AS x) t134 ON t1.x = t134.x JOIN (SELECT 1 AS x) t135 ON t1.x = t135.x JOIN (SELECT 1 AS x) t136 ON t1.x = t136.x JOIN (SELECT 1 AS x) t137 ON t1.x = t137.x JOIN (SELECT 1 AS x) t138 ON t1.x = t138.x JOIN (SELECT 1 AS x) t139 ON t1.x = t139.x JOIN (SELECT 1 AS x) t140 ON t1.x = t140.x JOIN (SELECT 1 AS x) t141 ON t1.x = t141.x JOIN (SELECT 1 AS x) t142 ON t1.x = t142.x JOIN (SELECT 1 AS x) t143 ON t1.x = t143.x JOIN (SELECT 1 AS x) t144 ON t1.x = t144.x JOIN (SELECT 1 AS x) t145 ON t1.x = t145.x JOIN (SELECT 1 AS x) t146 ON t1.x = t146.x JOIN (SELECT 1 AS x) t147 ON t1.x = t147.x JOIN (SELECT 1 AS x) t148 ON t1.x = t148.x JOIN (SELECT 1 AS x) t149 ON t1.x = t149.x JOIN (SELECT 1 AS x) t150 ON t1.x = t150.x JOIN (SELECT 1 AS x) t151 ON t1.x = t151.x JOIN (SELECT 1 AS x) t152 ON t1.x = t152.x JOIN (SELECT 1 AS x) t153 ON t1.x = t153.x JOIN (SELECT 1 AS x) t154 ON t1.x = t154.x JOIN (SELECT 1 AS x) t155 ON t1.x = t155.x JOIN (SELECT 1 AS x) t156 ON t1.x = t156.x JOIN (SELECT 1 AS x) t157 ON t1.x = t157.x JOIN (SELECT 1 AS x) t158 ON t1.x = t158.x JOIN (SELECT 1 AS x) t159 ON t1.x = t159.x JOIN (SELECT 1 AS x) t160 ON t1.x = t160.x JOIN (SELECT 1 AS x) t161 ON t1.x = t161.x JOIN (SELECT 1 AS x) t162 ON t1.x = t162.x JOIN (SELECT 1 AS x) t163 ON t1.x = t163.x JOIN (SELECT 1 AS x) t164 ON t1.x = t164.x JOIN (SELECT 1 AS x) t165 ON t1.x = t165.x JOIN (SELECT 1 AS x) t166 ON t1.x = t166.x JOIN (SELECT 1 AS x) t167 ON t1.x = t167.x JOIN (SELECT 1 AS x) t168 ON t1.x = t168.x JOIN (SELECT 1 AS x) t169 ON t1.x = t169.x JOIN (SELECT 1 AS x) t170 ON t1.x = t170.x JOIN (SELECT 1 AS x) t171 ON t1.x = t171.x JOIN (SELECT 1 AS x) t172 ON t1.x = t172.x JOIN (SELECT 1 AS x) t173 ON t1.x = t173.x JOIN (SELECT 1 AS x) t174 ON t1.x = t174.x JOIN (SELECT 1 AS x) t175 ON t1.x = t175.x JOIN (SELECT 1 AS x) t176 ON t1.x = t176.x JOIN (SELECT 1 AS x) t177 ON t1.x = t177.x JOIN (SELECT 1 AS x) t178 ON t1.x = t178.x JOIN (SELECT 1 AS x) t179 ON t1.x = t179.x JOIN (SELECT 1 AS x) t180 ON t1.x = t180.x JOIN (SELECT 1 AS x) t181 ON t1.x = t181.x JOIN (SELECT 1 AS x) t182 ON t1.x = t182.x JOIN (SELECT 1 AS x) t183 ON t1.x = t183.x JOIN (SELECT 1 AS x) t184 ON t1.x = t184.x JOIN (SELECT 1 AS x) t185 ON t1.x = t185.x JOIN (SELECT 1 AS x) t186 ON t1.x = t186.x JOIN (SELECT 1 AS x) t187 ON t1.x = t187.x JOIN (SELECT 1 AS x) t188 ON t1.x = t188.x JOIN (SELECT 1 AS x) t189 ON t1.x = t189.x JOIN (SELECT 1 AS x) t190 ON t1.x = t190.x JOIN (SELECT 1 AS x) t191 ON t1.x = t191.x JOIN (SELECT 1 AS x) t192 ON t1.x = t192.x JOIN (SELECT 1 AS x) t193 ON t1.x = t193.x JOIN (SELECT 1 AS x) t194 ON t1.x = t194.x JOIN (SELECT 1 AS x) t195 ON t1.x = t195.x JOIN (SELECT 1 AS x) t196 ON t1.x = t196.x JOIN (SELECT 1 AS x) t197 ON t1.x = t197.x JOIN (SELECT 1 AS x) t198 ON t1.x = t198.x JOIN (SELECT 1 AS x) t199 ON t1.x = t199.x JOIN (SELECT 1 AS x) t200 ON t1.x = t200.x JOIN (SELECT 1 AS x) t201 ON t1.x = t201.x JOIN (SELECT 1 AS x) t202 ON t1.x = t202.x JOIN (SELECT 1 AS x) t203 ON t1.x = t203.x JOIN (SELECT 1 AS x) t204 ON t1.x = t204.x JOIN (SELECT 1 AS x) t205 ON t1.x = t205.x JOIN (SELECT 1 AS x) t206 ON t1.x = t206.x JOIN (SELECT 1 AS x) t207 ON t1.x = t207.x JOIN (SELECT 1 AS x) t208 ON t1.x = t208.x JOIN (SELECT 1 AS x) t209 ON t1.x = t209.x JOIN (SELECT 1 AS x) t210 ON t1.x = t210.x JOIN (SELECT 1 AS x) t211 ON t1.x = t211.x JOIN (SELECT 1 AS x) t212 ON t1.x = t212.x JOIN (SELECT 1 AS x) t213 ON t1.x = t213.x JOIN (SELECT 1 AS x) t214 ON t1.x = t214.x JOIN (SELECT 1 AS x) t215 ON t1.x = t215.x JOIN (SELECT 1 AS x) t216 ON t1.x = t216.x JOIN (SELECT 1 AS x) t217 ON t1.x = t217.x JOIN (SELECT 1 AS x) t218 ON t1.x = t218.x JOIN (SELECT 1 AS x) t219 ON t1.x = t219.x JOIN (SELECT 1 AS x) t220 ON t1.x = t220.x JOIN (SELECT 1 AS x) t221 ON t1.x = t221.x JOIN (SELECT 1 AS x) t222 ON t1.x = t222.x JOIN (SELECT 1 AS x) t223 ON t1.x = t223.x JOIN (SELECT 1 AS x) t224 ON t1.x = t224.x JOIN (SELECT 1 AS x) t225 ON t1.x = t225.x JOIN (SELECT 1 AS x) t226 ON t1.x = t226.x JOIN (SELECT 1 AS x) t227 ON t1.x = t227.x JOIN (SELECT 1 AS x) t228 ON t1.x = t228.x JOIN (SELECT 1 AS x) t229 ON t1.x = t229.x JOIN (SELECT 1 AS x) t230 ON t1.x = t230.x JOIN (SELECT 1 AS x) t231 ON t1.x = t231.x JOIN (SELECT 1 AS x) t232 ON t1.x = t232.x JOIN (SELECT 1 AS x) t233 ON t1.x = t233.x JOIN (SELECT 1 AS x) t234 ON t1.x = t234.x JOIN (SELECT 1 AS x) t235 ON t1.x = t235.x JOIN (SELECT 1 AS x) t236 ON t1.x = t236.x JOIN (SELECT 1 AS x) t237 ON t1.x = t237.x JOIN (SELECT 1 AS x) t238 ON t1.x = t238.x JOIN (SELECT 1 AS x) t239 ON t1.x = t239.x JOIN (SELECT 1 AS x) t240 ON t1.x = t240.x JOIN (SELECT 1 AS x) t241 ON t1.x = t241.x JOIN (SELECT 1 AS x) t242 ON t1.x = t242.x JOIN (SELECT 1 AS x) t243 ON t1.x = t243.x JOIN (SELECT 1 AS x) t244 ON t1.x = t244.x JOIN (SELECT 1 AS x) t245 ON t1.x = t245.x JOIN (SELECT 1 AS x) t246 ON t1.x = t246.x JOIN (SELECT 1 AS x) t247 ON t1.x = t247.x JOIN (SELECT 1 AS x) t248 ON t1.x = t248.x JOIN (SELECT 1 AS x) t249 ON t1.x = t249.x JOIN (SELECT 1 AS x) t250 ON t1.x = t250.x JOIN (SELECT 1 AS x) t251 ON t1.x = t251.x JOIN (SELECT 1 AS x) t252 ON t1.x = t252.x JOIN (SELECT 1 AS x) t253 ON t1.x = t253.x JOIN (SELECT 1 AS x) t254 ON t1.x = t254.x JOIN (SELECT 1 AS x) t255 ON t1.x = t255.x JOIN (SELECT 1 AS x) t256 ON t1.x = t256.x JOIN (SELECT 1 AS x) t257 ON t1.x = t257.x JOIN (SELECT 1 AS x) t258 ON t1.x = t258.x JOIN (SELECT 1 AS x) t259 ON t1.x = t259.x JOIN (SELECT 1 AS x) t260 ON t1.x = t260.x JOIN (SELECT 1 AS x) t261 ON t1.x = t261.x JOIN (SELECT 1 AS x) t262 ON t1.x = t262.x JOIN (SELECT 1 AS x) t263 ON t1.x = t263.x JOIN (SELECT 1 AS x) t264 ON t1.x = t264.x JOIN (SELECT 1 AS x) t265 ON t1.x = t265.x JOIN (SELECT 1 AS x) t266 ON t1.x = t266.x JOIN (SELECT 1 AS x) t267 ON t1.x = t267.x JOIN (SELECT 1 AS x) t268 ON t1.x = t268.x JOIN (SELECT 1 AS x) t269 ON t1.x = t269.x JOIN (SELECT 1 AS x) t270 ON t1.x = t270.x JOIN (SELECT 1 AS x) t271 ON t1.x = t271.x JOIN (SELECT 1 AS x) t272 ON t1.x = t272.x JOIN (SELECT 1 AS x) t273 ON t1.x = t273.x JOIN (SELECT 1 AS x) t274 ON t1.x = t274.x JOIN (SELECT 1 AS x) t275 ON t1.x = t275.x JOIN (SELECT 1 AS x) t276 ON t1.x = t276.x JOIN (SELECT 1 AS x) t277 ON t1.x = t277.x JOIN (SELECT 1 AS x) t278 ON t1.x = t278.x JOIN (SELECT 1 AS x) t279 ON t1.x = t279.x JOIN (SELECT 1 AS x) t280 ON t1.x = t280.x JOIN (SELECT 1 AS x) t281 ON t1.x = t281.x JOIN (SELECT 1 AS x) t282 ON t1.x = t282.x JOIN (SELECT 1 AS x) t283 ON t1.x = t283.x JOIN (SELECT 1 AS x) t284 ON t1.x = t284.x JOIN (SELECT 1 AS x) t285 ON t1.x = t285.x JOIN (SELECT 1 AS x) t286 ON t1.x = t286.x JOIN (SELECT 1 AS x) t287 ON t1.x = t287.x JOIN (SELECT 1 AS x) t288 ON t1.x = t288.x JOIN (SELECT 1 AS x) t289 ON t1.x = t289.x JOIN (SELECT 1 AS x) t290 ON t1.x = t290.x JOIN (SELECT 1 AS x) t291 ON t1.x = t291.x JOIN (SELECT 1 AS x) t292 ON t1.x = t292.x JOIN (SELECT 1 AS x) t293 ON t1.x = t293.x JOIN (SELECT 1 AS x) t294 ON t1.x = t294.x JOIN (SELECT 1 AS x) t295 ON t1.x = t295.x JOIN (SELECT 1 AS x) t296 ON t1.x = t296.x JOIN (SELECT 1 AS x) t297 ON t1.x = t297.x JOIN (SELECT 1 AS x) t298 ON t1.x = t298.x JOIN (SELECT 1 AS x) t299 ON t1.x = t299.x JOIN (SELECT 1 AS x) t300 ON t1.x = t300.x JOIN (SELECT 1 AS x) t301 ON t1.x = t301.x JOIN (SELECT 1 AS x) t302 ON t1.x = t302.x JOIN (SELECT 1 AS x) t303 ON t1.x = t303.x JOIN (SELECT 1 AS x) t304 ON t1.x = t304.x JOIN (SELECT 1 AS x) t305 ON t1.x = t305.x JOIN (SELECT 1 AS x) t306 ON t1.x = t306.x JOIN (SELECT 1 AS x) t307 ON t1.x = t307.x JOIN (SELECT 1 AS x) t308 ON t1.x = t308.x JOIN (SELECT 1 AS x) t309 ON t1.x = t309.x JOIN (SELECT 1 AS x) t310 ON t1.x = t310.x JOIN (SELECT 1 AS x) t311 ON t1.x = t311.x JOIN (SELECT 1 AS x) t312 ON t1.x = t312.x JOIN (SELECT 1 AS x) t313 ON t1.x = t313.x JOIN (SELECT 1 AS x) t314 ON t1.x = t314.x JOIN (SELECT 1 AS x) t315 ON t1.x = t315.x JOIN (SELECT 1 AS x) t316 ON t1.x = t316.x JOIN (SELECT 1 AS x) t317 ON t1.x = t317.x JOIN (SELECT 1 AS x) t318 ON t1.x = t318.x JOIN (SELECT 1 AS x) t319 ON t1.x = t319.x JOIN (SELECT 1 AS x) t320 ON t1.x = t320.x JOIN (SELECT 1 AS x) t321 ON t1.x = t321.x JOIN (SELECT 1 AS x) t322 ON t1.x = t322.x JOIN (SELECT 1 AS x) t323 ON t1.x = t323.x JOIN (SELECT 1 AS x) t324 ON t1.x = t324.x JOIN (SELECT 1 AS x) t325 ON t1.x = t325.x JOIN (SELECT 1 AS x) t326 ON t1.x = t326.x JOIN (SELECT 1 AS x) t327 ON t1.x = t327.x JOIN (SELECT 1 AS x) t328 ON t1.x = t328.x JOIN (SELECT 1 AS x) t329 ON t1.x = t329.x JOIN (SELECT 1 AS x) t330 ON t1.x = t330.x JOIN (SELECT 1 AS x) t331 ON t1.x = t331.x JOIN (SELECT 1 AS x) t332 ON t1.x = t332.x JOIN (SELECT 1 AS x) t333 ON t1.x = t333.x JOIN (SELECT 1 AS x) t334 ON t1.x = t334.x JOIN (SELECT 1 AS x) t335 ON t1.x = t335.x JOIN (SELECT 1 AS x) t336 ON t1.x = t336.x JOIN (SELECT 1 AS x) t337 ON t1.x = t337.x JOIN (SELECT 1 AS x) t338 ON t1.x = t338.x JOIN (SELECT 1 AS x) t339 ON t1.x = t339.x JOIN (SELECT 1 AS x) t340 ON t1.x = t340.x JOIN (SELECT 1 AS x) t341 ON t1.x = t341.x JOIN (SELECT 1 AS x) t342 ON t1.x = t342.x JOIN (SELECT 1 AS x) t343 ON t1.x = t343.x JOIN (SELECT 1 AS x) t344 ON t1.x = t344.x JOIN (SELECT 1 AS x) t345 ON t1.x = t345.x JOIN (SELECT 1 AS x) t346 ON t1.x = t346.x JOIN (SELECT 1 AS x) t347 ON t1.x = t347.x JOIN (SELECT 1 AS x) t348 ON t1.x = t348.x JOIN (SELECT 1 AS x) t349 ON t1.x = t349.x JOIN (SELECT 1 AS x) t350 ON t1.x = t350.x JOIN (SELECT 1 AS x) t351 ON t1.x = t351.x JOIN (SELECT 1 AS x) t352 ON t1.x = t352.x JOIN (SELECT 1 AS x) t353 ON t1.x = t353.x JOIN (SELECT 1 AS x) t354 ON t1.x = t354.x JOIN (SELECT 1 AS x) t355 ON t1.x = t355.x JOIN (SELECT 1 AS x) t356 ON t1.x = t356.x JOIN (SELECT 1 AS x) t357 ON t1.x = t357.x JOIN (SELECT 1 AS x) t358 ON t1.x = t358.x JOIN (SELECT 1 AS x) t359 ON t1.x = t359.x JOIN (SELECT 1 AS x) t360 ON t1.x = t360.x JOIN (SELECT 1 AS x) t361 ON t1.x = t361.x JOIN (SELECT 1 AS x) t362 ON t1.x = t362.x JOIN (SELECT 1 AS x) t363 ON t1.x = t363.x JOIN (SELECT 1 AS x) t364 ON t1.x = t364.x JOIN (SELECT 1 AS x) t365 ON t1.x = t365.x JOIN (SELECT 1 AS x) t366 ON t1.x = t366.x JOIN (SELECT 1 AS x) t367 ON t1.x = t367.x JOIN (SELECT 1 AS x) t368 ON t1.x = t368.x JOIN (SELECT 1 AS x) t369 ON t1.x = t369.x JOIN (SELECT 1 AS x) t370 ON t1.x = t370.x JOIN (SELECT 1 AS x) t371 ON t1.x = t371.x JOIN (SELECT 1 AS x) t372 ON t1.x = t372.x JOIN (SELECT 1 AS x) t373 ON t1.x = t373.x JOIN (SELECT 1 AS x) t374 ON t1.x = t374.x JOIN (SELECT 1 AS x) t375 ON t1.x = t375.x JOIN (SELECT 1 AS x) t376 ON t1.x = t376.x JOIN (SELECT 1 AS x) t377 ON t1.x = t377.x JOIN (SELECT 1 AS x) t378 ON t1.x = t378.x JOIN (SELECT 1 AS x) t379 ON t1.x = t379.x JOIN (SELECT 1 AS x) t380 ON t1.x = t380.x JOIN (SELECT 1 AS x) t381 ON t1.x = t381.x JOIN (SELECT 1 AS x) t382 ON t1.x = t382.x JOIN (SELECT 1 AS x) t383 ON t1.x = t383.x JOIN (SELECT 1 AS x) t384 ON t1.x = t384.x JOIN (SELECT 1 AS x) t385 ON t1.x = t385.x JOIN (SELECT 1 AS x) t386 ON t1.x = t386.x JOIN (SELECT 1 AS x) t387 ON t1.x = t387.x JOIN (SELECT 1 AS x) t388 ON t1.x = t388.x JOIN (SELECT 1 AS x) t389 ON t1.x = t389.x JOIN (SELECT 1 AS x) t390 ON t1.x = t390.x JOIN (SELECT 1 AS x) t391 ON t1.x = t391.x JOIN (SELECT 1 AS x) t392 ON t1.x = t392.x JOIN (SELECT 1 AS x) t393 ON t1.x = t393.x JOIN (SELECT 1 AS x) t394 ON t1.x = t394.x JOIN (SELECT 1 AS x) t395 ON t1.x = t395.x JOIN (SELECT 1 AS x) t396 ON t1.x = t396.x JOIN (SELECT 1 AS x) t397 ON t1.x = t397.x JOIN (SELECT 1 AS x) t398 ON t1.x = t398.x JOIN (SELECT 1 AS x) t399 ON t1.x = t399.x JOIN (SELECT 1 AS x) t400 ON t1.x = t400.x JOIN (SELECT 1 AS x) t401 ON t1.x = t401.x JOIN (SELECT 1 AS x) t402 ON t1.x = t402.x JOIN (SELECT 1 AS x) t403 ON t1.x = t403.x JOIN (SELECT 1 AS x) t404 ON t1.x = t404.x JOIN (SELECT 1 AS x) t405 ON t1.x = t405.x JOIN (SELECT 1 AS x) t406 ON t1.x = t406.x JOIN (SELECT 1 AS x) t407 ON t1.x = t407.x JOIN (SELECT 1 AS x) t408 ON t1.x = t408.x JOIN (SELECT 1 AS x) t409 ON t1.x = t409.x JOIN (SELECT 1 AS x) t410 ON t1.x = t410.x JOIN (SELECT 1 AS x) t411 ON t1.x = t411.x JOIN (SELECT 1 AS x) t412 ON t1.x = t412.x JOIN (SELECT 1 AS x) t413 ON t1.x = t413.x JOIN (SELECT 1 AS x) t414 ON t1.x = t414.x JOIN (SELECT 1 AS x) t415 ON t1.x = t415.x JOIN (SELECT 1 AS x) t416 ON t1.x = t416.x JOIN (SELECT 1 AS x) t417 ON t1.x = t417.x JOIN (SELECT 1 AS x) t418 ON t1.x = t418.x JOIN (SELECT 1 AS x) t419 ON t1.x = t419.x JOIN (SELECT 1 AS x) t420 ON t1.x = t420.x JOIN (SELECT 1 AS x) t421 ON t1.x = t421.x JOIN (SELECT 1 AS x) t422 ON t1.x = t422.x JOIN (SELECT 1 AS x) t423 ON t1.x = t423.x JOIN (SELECT 1 AS x) t424 ON t1.x = t424.x JOIN (SELECT 1 AS x) t425 ON t1.x = t425.x JOIN (SELECT 1 AS x) t426 ON t1.x = t426.x JOIN (SELECT 1 AS x) t427 ON t1.x = t427.x JOIN (SELECT 1 AS x) t428 ON t1.x = t428.x JOIN (SELECT 1 AS x) t429 ON t1.x = t429.x JOIN (SELECT 1 AS x) t430 ON t1.x = t430.x JOIN (SELECT 1 AS x) t431 ON t1.x = t431.x JOIN (SELECT 1 AS x) t432 ON t1.x = t432.x JOIN (SELECT 1 AS x) t433 ON t1.x = t433.x JOIN (SELECT 1 AS x) t434 ON t1.x = t434.x JOIN (SELECT 1 AS x) t435 ON t1.x = t435.x JOIN (SELECT 1 AS x) t436 ON t1.x = t436.x JOIN (SELECT 1 AS x) t437 ON t1.x = t437.x JOIN (SELECT 1 AS x) t438 ON t1.x = t438.x JOIN (SELECT 1 AS x) t439 ON t1.x = t439.x JOIN (SELECT 1 AS x) t440 ON t1.x = t440.x JOIN (SELECT 1 AS x) t441 ON t1.x = t441.x JOIN (SELECT 1 AS x) t442 ON t1.x = t442.x JOIN (SELECT 1 AS x) t443 ON t1.x = t443.x JOIN (SELECT 1 AS x) t444 ON t1.x = t444.x JOIN (SELECT 1 AS x) t445 ON t1.x = t445.x JOIN (SELECT 1 AS x) t446 ON t1.x = t446.x JOIN (SELECT 1 AS x) t447 ON t1.x = t447.x JOIN (SELECT 1 AS x) t448 ON t1.x = t448.x JOIN (SELECT 1 AS x) t449 ON t1.x = t449.x JOIN (SELECT 1 AS x) t450 ON t1.x = t450.x JOIN (SELECT 1 AS x) t451 ON t1.x = t451.x JOIN (SELECT 1 AS x) t452 ON t1.x = t452.x JOIN (SELECT 1 AS x) t453 ON t1.x = t453.x JOIN (SELECT 1 AS x) t454 ON t1.x = t454.x JOIN (SELECT 1 AS x) t455 ON t1.x = t455.x JOIN (SELECT 1 AS x) t456 ON t1.x = t456.x JOIN (SELECT 1 AS x) t457 ON t1.x = t457.x JOIN (SELECT 1 AS x) t458 ON t1.x = t458.x JOIN (SELECT 1 AS x) t459 ON t1.x = t459.x JOIN (SELECT 1 AS x) t460 ON t1.x = t460.x JOIN (SELECT 1 AS x) t461 ON t1.x = t461.x JOIN (SELECT 1 AS x) t462 ON t1.x = t462.x JOIN (SELECT 1 AS x) t463 ON t1.x = t463.x JOIN (SELECT 1 AS x) t464 ON t1.x = t464.x JOIN (SELECT 1 AS x) t465 ON t1.x = t465.x JOIN (SELECT 1 AS x) t466 ON t1.x = t466.x JOIN (SELECT 1 AS x) t467 ON t1.x = t467.x JOIN (SELECT 1 AS x) t468 ON t1.x = t468.x JOIN (SELECT 1 AS x) t469 ON t1.x = t469.x JOIN (SELECT 1 AS x) t470 ON t1.x = t470.x JOIN (SELECT 1 AS x) t471 ON t1.x = t471.x JOIN (SELECT 1 AS x) t472 ON t1.x = t472.x JOIN (SELECT 1 AS x) t473 ON t1.x = t473.x JOIN (SELECT 1 AS x) t474 ON t1.x = t474.x JOIN (SELECT 1 AS x) t475 ON t1.x = t475.x JOIN (SELECT 1 AS x) t476 ON t1.x = t476.x JOIN (SELECT 1 AS x) t477 ON t1.x = t477.x JOIN (SELECT 1 AS x) t478 ON t1.x = t478.x JOIN (SELECT 1 AS x) t479 ON t1.x = t479.x JOIN (SELECT 1 AS x) t480 ON t1.x = t480.x JOIN (SELECT 1 AS x) t481 ON t1.x = t481.x JOIN (SELECT 1 AS x) t482 ON t1.x = t482.x JOIN (SELECT 1 AS x) t483 ON t1.x = t483.x JOIN (SELECT 1 AS x) t484 ON t1.x = t484.x JOIN (SELECT 1 AS x) t485 ON t1.x = t485.x JOIN (SELECT 1 AS x) t486 ON t1.x = t486.x JOIN (SELECT 1 AS x) t487 ON t1.x = t487.x JOIN (SELECT 1 AS x) t488 ON t1.x = t488.x JOIN (SELECT 1 AS x) t489 ON t1.x = t489.x JOIN (SELECT 1 AS x) t490 ON t1.x = t490.x JOIN (SELECT 1 AS x) t491 ON t1.x = t491.x JOIN (SELECT 1 AS x) t492 ON t1.x = t492.x JOIN (SELECT 1 AS x) t493 ON t1.x = t493.x JOIN (SELECT 1 AS x) t494 ON t1.x = t494.x JOIN (SELECT 1 AS x) t495 ON t1.x = t495.x JOIN (SELECT 1 AS x) t496 ON t1.x = t496.x JOIN (SELECT 1 AS x) t497 ON t1.x = t497.x JOIN (SELECT 1 AS x) t498 ON t1.x = t498.x JOIN (SELECT 1 AS x) t499 ON t1.x = t499.x JOIN (SELECT 1 AS x) t500 ON t1.x = t500.x JOIN (SELECT 1 AS x) t501 ON t1.x = t501.x JOIN (SELECT 1 AS x) t502 ON t1.x = t502.x JOIN (SELECT 1 AS x) t503 ON t1.x = t503.x JOIN (SELECT 1 AS x) t504 ON t1.x = t504.x JOIN (SELECT 1 AS x) t505 ON t1.x = t505.x JOIN (SELECT 1 AS x) t506 ON t1.x = t506.x JOIN (SELECT 1 AS x) t507 ON t1.x = t507.x JOIN (SELECT 1 AS x) t508 ON t1.x = t508.x JOIN (SELECT 1 AS x) t509 ON t1.x = t509.x JOIN (SELECT 1 AS x) t510 ON t1.x = t510.x JOIN (SELECT 1 AS x) t511 ON t1.x = t511.x JOIN (SELECT 1 AS x) t512 ON t1.x = t512.x JOIN (SELECT 1 AS x) t513 ON t1.x = t513.x JOIN (SELECT 1 AS x) t514 ON t1.x = t514.x JOIN (SELECT 1 AS x) t515 ON t1.x = t515.x JOIN (SELECT 1 AS x) t516 ON t1.x = t516.x JOIN (SELECT 1 AS x) t517 ON t1.x = t517.x JOIN (SELECT 1 AS x) t518 ON t1.x = t518.x JOIN (SELECT 1 AS x) t519 ON t1.x = t519.x JOIN (SELECT 1 AS x) t520 ON t1.x = t520.x JOIN (SELECT 1 AS x) t521 ON t1.x = t521.x JOIN (SELECT 1 AS x) t522 ON t1.x = t522.x JOIN (SELECT 1 AS x) t523 ON t1.x = t523.x JOIN (SELECT 1 AS x) t524 ON t1.x = t524.x JOIN (SELECT 1 AS x) t525 ON t1.x = t525.x JOIN (SELECT 1 AS x) t526 ON t1.x = t526.x JOIN (SELECT 1 AS x) t527 ON t1.x = t527.x JOIN (SELECT 1 AS x) t528 ON t1.x = t528.x JOIN (SELECT 1 AS x) t529 ON t1.x = t529.x JOIN (SELECT 1 AS x) t530 ON t1.x = t530.x JOIN (SELECT 1 AS x) t531 ON t1.x = t531.x JOIN (SELECT 1 AS x) t532 ON t1.x = t532.x JOIN (SELECT 1 AS x) t533 ON t1.x = t533.x JOIN (SELECT 1 AS x) t534 ON t1.x = t534.x JOIN (SELECT 1 AS x) t535 ON t1.x = t535.x JOIN (SELECT 1 AS x) t536 ON t1.x = t536.x JOIN (SELECT 1 AS x) t537 ON t1.x = t537.x JOIN (SELECT 1 AS x) t538 ON t1.x = t538.x JOIN (SELECT 1 AS x) t539 ON t1.x = t539.x JOIN (SELECT 1 AS x) t540 ON t1.x = t540.x JOIN (SELECT 1 AS x) t541 ON t1.x = t541.x JOIN (SELECT 1 AS x) t542 ON t1.x = t542.x JOIN (SELECT 1 AS x) t543 ON t1.x = t543.x JOIN (SELECT 1 AS x) t544 ON t1.x = t544.x JOIN (SELECT 1 AS x) t545 ON t1.x = t545.x JOIN (SELECT 1 AS x) t546 ON t1.x = t546.x JOIN (SELECT 1 AS x) t547 ON t1.x = t547.x JOIN (SELECT 1 AS x) t548 ON t1.x = t548.x JOIN (SELECT 1 AS x) t549 ON t1.x = t549.x JOIN (SELECT 1 AS x) t550 ON t1.x = t550.x JOIN (SELECT 1 AS x) t551 ON t1.x = t551.x JOIN (SELECT 1 AS x) t552 ON t1.x = t552.x JOIN (SELECT 1 AS x) t553 ON t1.x = t553.x JOIN (SELECT 1 AS x) t554 ON t1.x = t554.x JOIN (SELECT 1 AS x) t555 ON t1.x = t555.x JOIN (SELECT 1 AS x) t556 ON t1.x = t556.x JOIN (SELECT 1 AS x) t557 ON t1.x = t557.x JOIN (SELECT 1 AS x) t558 ON t1.x = t558.x JOIN (SELECT 1 AS x) t559 ON t1.x = t559.x JOIN (SELECT 1 AS x) t560 ON t1.x = t560.x JOIN (SELECT 1 AS x) t561 ON t1.x = t561.x JOIN (SELECT 1 AS x) t562 ON t1.x = t562.x JOIN (SELECT 1 AS x) t563 ON t1.x = t563.x JOIN (SELECT 1 AS x) t564 ON t1.x = t564.x JOIN (SELECT 1 AS x) t565 ON t1.x = t565.x JOIN (SELECT 1 AS x) t566 ON t1.x = t566.x JOIN (SELECT 1 AS x) t567 ON t1.x = t567.x JOIN (SELECT 1 AS x) t568 ON t1.x = t568.x JOIN (SELECT 1 AS x) t569 ON t1.x = t569.x JOIN (SELECT 1 AS x) t570 ON t1.x = t570.x JOIN (SELECT 1 AS x) t571 ON t1.x = t571.x JOIN (SELECT 1 AS x) t572 ON t1.x = t572.x JOIN (SELECT 1 AS x) t573 ON t1.x = t573.x JOIN (SELECT 1 AS x) t574 ON t1.x = t574.x JOIN (SELECT 1 AS x) t575 ON t1.x = t575.x JOIN (SELECT 1 AS x) t576 ON t1.x = t576.x JOIN (SELECT 1 AS x) t577 ON t1.x = t577.x JOIN (SELECT 1 AS x) t578 ON t1.x = t578.x JOIN (SELECT 1 AS x) t579 ON t1.x = t579.x JOIN (SELECT 1 AS x) t580 ON t1.x = t580.x JOIN (SELECT 1 AS x) t581 ON t1.x = t581.x JOIN (SELECT 1 AS x) t582 ON t1.x = t582.x JOIN (SELECT 1 AS x) t583 ON t1.x = t583.x JOIN (SELECT 1 AS x) t584 ON t1.x = t584.x JOIN (SELECT 1 AS x) t585 ON t1.x = t585.x JOIN (SELECT 1 AS x) t586 ON t1.x = t586.x JOIN (SELECT 1 AS x) t587 ON t1.x = t587.x JOIN (SELECT 1 AS x) t588 ON t1.x = t588.x JOIN (SELECT 1 AS x) t589 ON t1.x = t589.x JOIN (SELECT 1 AS x) t590 ON t1.x = t590.x JOIN (SELECT 1 AS x) t591 ON t1.x = t591.x JOIN (SELECT 1 AS x) t592 ON t1.x = t592.x JOIN (SELECT 1 AS x) t593 ON t1.x = t593.x JOIN (SELECT 1 AS x) t594 ON t1.x = t594.x JOIN (SELECT 1 AS x) t595 ON t1.x = t595.x JOIN (SELECT 1 AS x) t596 ON t1.x = t596.x JOIN (SELECT 1 AS x) t597 ON t1.x = t597.x JOIN (SELECT 1 AS x) t598 ON t1.x = t598.x JOIN (SELECT 1 AS x) t599 ON t1.x = t599.x JOIN (SELECT 1 AS x) t600 ON t1.x = t600.x JOIN (SELECT 1 AS x) t601 ON t1.x = t601.x JOIN (SELECT 1 AS x) t602 ON t1.x = t602.x JOIN (SELECT 1 AS x) t603 ON t1.x = t603.x JOIN (SELECT 1 AS x) t604 ON t1.x = t604.x JOIN (SELECT 1 AS x) t605 ON t1.x = t605.x JOIN (SELECT 1 AS x) t606 ON t1.x = t606.x JOIN (SELECT 1 AS x) t607 ON t1.x = t607.x JOIN (SELECT 1 AS x) t608 ON t1.x = t608.x JOIN (SELECT 1 AS x) t609 ON t1.x = t609.x JOIN (SELECT 1 AS x) t610 ON t1.x = t610.x JOIN (SELECT 1 AS x) t611 ON t1.x = t611.x JOIN (SELECT 1 AS x) t612 ON t1.x = t612.x JOIN (SELECT 1 AS x) t613 ON t1.x = t613.x JOIN (SELECT 1 AS x) t614 ON t1.x = t614.x JOIN (SELECT 1 AS x) t615 ON t1.x = t615.x JOIN (SELECT 1 AS x) t616 ON t1.x = t616.x JOIN (SELECT 1 AS x) t617 ON t1.x = t617.x JOIN (SELECT 1 AS x) t618 ON t1.x = t618.x JOIN (SELECT 1 AS x) t619 ON t1.x = t619.x JOIN (SELECT 1 AS x) t620 ON t1.x = t620.x JOIN (SELECT 1 AS x) t621 ON t1.x = t621.x JOIN (SELECT 1 AS x) t622 ON t1.x = t622.x JOIN (SELECT 1 AS x) t623 ON t1.x = t623.x JOIN (SELECT 1 AS x) t624 ON t1.x = t624.x JOIN (SELECT 1 AS x) t625 ON t1.x = t625.x JOIN (SELECT 1 AS x) t626 ON t1.x = t626.x JOIN (SELECT 1 AS x) t627 ON t1.x = t627.x JOIN (SELECT 1 AS x) t628 ON t1.x = t628.x JOIN (SELECT 1 AS x) t629 ON t1.x = t629.x JOIN (SELECT 1 AS x) t630 ON t1.x = t630.x JOIN (SELECT 1 AS x) t631 ON t1.x = t631.x JOIN (SELECT 1 AS x) t632 ON t1.x = t632.x JOIN (SELECT 1 AS x) t633 ON t1.x = t633.x JOIN (SELECT 1 AS x) t634 ON t1.x = t634.x JOIN (SELECT 1 AS x) t635 ON t1.x = t635.x JOIN (SELECT 1 AS x) t636 ON t1.x = t636.x JOIN (SELECT 1 AS x) t637 ON t1.x = t637.x JOIN (SELECT 1 AS x) t638 ON t1.x = t638.x JOIN (SELECT 1 AS x) t639 ON t1.x = t639.x JOIN (SELECT 1 AS x) t640 ON t1.x = t640.x JOIN (SELECT 1 AS x) t641 ON t1.x = t641.x JOIN (SELECT 1 AS x) t642 ON t1.x = t642.x JOIN (SELECT 1 AS x) t643 ON t1.x = t643.x JOIN (SELECT 1 AS x) t644 ON t1.x = t644.x JOIN (SELECT 1 AS x) t645 ON t1.x = t645.x JOIN (SELECT 1 AS x) t646 ON t1.x = t646.x JOIN (SELECT 1 AS x) t647 ON t1.x = t647.x JOIN (SELECT 1 AS x) t648 ON t1.x = t648.x JOIN (SELECT 1 AS x) t649 ON t1.x = t649.x JOIN (SELECT 1 AS x) t650 ON t1.x = t650.x JOIN (SELECT 1 AS x) t651 ON t1.x = t651.x JOIN (SELECT 1 AS x) t652 ON t1.x = t652.x JOIN (SELECT 1 AS x) t653 ON t1.x = t653.x JOIN (SELECT 1 AS x) t654 ON t1.x = t654.x JOIN (SELECT 1 AS x) t655 ON t1.x = t655.x JOIN (SELECT 1 AS x) t656 ON t1.x = t656.x JOIN (SELECT 1 AS x) t657 ON t1.x = t657.x JOIN (SELECT 1 AS x) t658 ON t1.x = t658.x JOIN (SELECT 1 AS x) t659 ON t1.x = t659.x JOIN (SELECT 1 AS x) t660 ON t1.x = t660.x JOIN (SELECT 1 AS x) t661 ON t1.x = t661.x JOIN (SELECT 1 AS x) t662 ON t1.x = t662.x JOIN (SELECT 1 AS x) t663 ON t1.x = t663.x JOIN (SELECT 1 AS x) t664 ON t1.x = t664.x JOIN (SELECT 1 AS x) t665 ON t1.x = t665.x JOIN (SELECT 1 AS x) t666 ON t1.x = t666.x JOIN (SELECT 1 AS x) t667 ON t1.x = t667.x JOIN (SELECT 1 AS x) t668 ON t1.x = t668.x JOIN (SELECT 1 AS x) t669 ON t1.x = t669.x JOIN (SELECT 1 AS x) t670 ON t1.x = t670.x JOIN (SELECT 1 AS x) t671 ON t1.x = t671.x JOIN (SELECT 1 AS x) t672 ON t1.x = t672.x JOIN (SELECT 1 AS x) t673 ON t1.x = t673.x JOIN (SELECT 1 AS x) t674 ON t1.x = t674.x JOIN (SELECT 1 AS x) t675 ON t1.x = t675.x JOIN (SELECT 1 AS x) t676 ON t1.x = t676.x JOIN (SELECT 1 AS x) t677 ON t1.x = t677.x JOIN (SELECT 1 AS x) t678 ON t1.x = t678.x JOIN (SELECT 1 AS x) t679 ON t1.x = t679.x JOIN (SELECT 1 AS x) t680 ON t1.x = t680.x JOIN (SELECT 1 AS x) t681 ON t1.x = t681.x JOIN (SELECT 1 AS x) t682 ON t1.x = t682.x JOIN (SELECT 1 AS x) t683 ON t1.x = t683.x JOIN (SELECT 1 AS x) t684 ON t1.x = t684.x JOIN (SELECT 1 AS x) t685 ON t1.x = t685.x JOIN (SELECT 1 AS x) t686 ON t1.x = t686.x JOIN (SELECT 1 AS x) t687 ON t1.x = t687.x JOIN (SELECT 1 AS x) t688 ON t1.x = t688.x JOIN (SELECT 1 AS x) t689 ON t1.x = t689.x JOIN (SELECT 1 AS x) t690 ON t1.x = t690.x JOIN (SELECT 1 AS x) t691 ON t1.x = t691.x JOIN (SELECT 1 AS x) t692 ON t1.x = t692.x JOIN (SELECT 1 AS x) t693 ON t1.x = t693.x JOIN (SELECT 1 AS x) t694 ON t1.x = t694.x JOIN (SELECT 1 AS x) t695 ON t1.x = t695.x JOIN (SELECT 1 AS x) t696 ON t1.x = t696.x JOIN (SELECT 1 AS x) t697 ON t1.x = t697.x JOIN (SELECT 1 AS x) t698 ON t1.x = t698.x JOIN (SELECT 1 AS x) t699 ON t1.x = t699.x JOIN (SELECT 1 AS x) t700 ON t1.x = t700.x JOIN (SELECT 1 AS x) t701 ON t1.x = t701.x JOIN (SELECT 1 AS x) t702 ON t1.x = t702.x JOIN (SELECT 1 AS x) t703 ON t1.x = t703.x JOIN (SELECT 1 AS x) t704 ON t1.x = t704.x JOIN (SELECT 1 AS x) t705 ON t1.x = t705.x JOIN (SELECT 1 AS x) t706 ON t1.x = t706.x JOIN (SELECT 1 AS x) t707 ON t1.x = t707.x JOIN (SELECT 1 AS x) t708 ON t1.x = t708.x JOIN (SELECT 1 AS x) t709 ON t1.x = t709.x JOIN (SELECT 1 AS x) t710 ON t1.x = t710.x JOIN (SELECT 1 AS x) t711 ON t1.x = t711.x JOIN (SELECT 1 AS x) t712 ON t1.x = t712.x JOIN (SELECT 1 AS x) t713 ON t1.x = t713.x JOIN (SELECT 1 AS x) t714 ON t1.x = t714.x JOIN (SELECT 1 AS x) t715 ON t1.x = t715.x JOIN (SELECT 1 AS x) t716 ON t1.x = t716.x JOIN (SELECT 1 AS x) t717 ON t1.x = t717.x JOIN (SELECT 1 AS x) t718 ON t1.x = t718.x JOIN (SELECT 1 AS x) t719 ON t1.x = t719.x JOIN (SELECT 1 AS x) t720 ON t1.x = t720.x JOIN (SELECT 1 AS x) t721 ON t1.x = t721.x JOIN (SELECT 1 AS x) t722 ON t1.x = t722.x JOIN (SELECT 1 AS x) t723 ON t1.x = t723.x JOIN (SELECT 1 AS x) t724 ON t1.x = t724.x JOIN (SELECT 1 AS x) t725 ON t1.x = t725.x JOIN (SELECT 1 AS x) t726 ON t1.x = t726.x JOIN (SELECT 1 AS x) t727 ON t1.x = t727.x JOIN (SELECT 1 AS x) t728 ON t1.x = t728.x JOIN (SELECT 1 AS x) t729 ON t1.x = t729.x JOIN (SELECT 1 AS x) t730 ON t1.x = t730.x JOIN (SELECT 1 AS x) t731 ON t1.x = t731.x JOIN (SELECT 1 AS x) t732 ON t1.x = t732.x JOIN (SELECT 1 AS x) t733 ON t1.x = t733.x JOIN (SELECT 1 AS x) t734 ON t1.x = t734.x JOIN (SELECT 1 AS x) t735 ON t1.x = t735.x JOIN (SELECT 1 AS x) t736 ON t1.x = t736.x JOIN (SELECT 1 AS x) t737 ON t1.x = t737.x JOIN (SELECT 1 AS x) t738 ON t1.x = t738.x JOIN (SELECT 1 AS x) t739 ON t1.x = t739.x JOIN (SELECT 1 AS x) t740 ON t1.x = t740.x JOIN (SELECT 1 AS x) t741 ON t1.x = t741.x JOIN (SELECT 1 AS x) t742 ON t1.x = t742.x JOIN (SELECT 1 AS x) t743 ON t1.x = t743.x JOIN (SELECT 1 AS x) t744 ON t1.x = t744.x JOIN (SELECT 1 AS x) t745 ON t1.x = t745.x JOIN (SELECT 1 AS x) t746 ON t1.x = t746.x JOIN (SELECT 1 AS x) t747 ON t1.x = t747.x JOIN (SELECT 1 AS x) t748 ON t1.x = t748.x JOIN (SELECT 1 AS x) t749 ON t1.x = t749.x JOIN (SELECT 1 AS x) t750 ON t1.x = t750.x JOIN (SELECT 1 AS x) t751 ON t1.x = t751.x JOIN (SELECT 1 AS x) t752 ON t1.x = t752.x JOIN (SELECT 1 AS x) t753 ON t1.x = t753.x JOIN (SELECT 1 AS x) t754 ON t1.x = t754.x JOIN (SELECT 1 AS x) t755 ON t1.x = t755.x JOIN (SELECT 1 AS x) t756 ON t1.x = t756.x JOIN (SELECT 1 AS x) t757 ON t1.x = t757.x JOIN (SELECT 1 AS x) t758 ON t1.x = t758.x JOIN (SELECT 1 AS x) t759 ON t1.x = t759.x JOIN (SELECT 1 AS x) t760 ON t1.x = t760.x JOIN (SELECT 1 AS x) t761 ON t1.x = t761.x JOIN (SELECT 1 AS x) t762 ON t1.x = t762.x JOIN (SELECT 1 AS x) t763 ON t1.x = t763.x JOIN (SELECT 1 AS x) t764 ON t1.x = t764.x JOIN (SELECT 1 AS x) t765 ON t1.x = t765.x JOIN (SELECT 1 AS x) t766 ON t1.x = t766.x JOIN (SELECT 1 AS x) t767 ON t1.x = t767.x JOIN (SELECT 1 AS x) t768 ON t1.x = t768.x JOIN (SELECT 1 AS x) t769 ON t1.x = t769.x JOIN (SELECT 1 AS x) t770 ON t1.x = t770.x JOIN (SELECT 1 AS x) t771 ON t1.x = t771.x JOIN (SELECT 1 AS x) t772 ON t1.x = t772.x JOIN (SELECT 1 AS x) t773 ON t1.x = t773.x JOIN (SELECT 1 AS x) t774 ON t1.x = t774.x JOIN (SELECT 1 AS x) t775 ON t1.x = t775.x JOIN (SELECT 1 AS x) t776 ON t1.x = t776.x JOIN (SELECT 1 AS x) t777 ON t1.x = t777.x JOIN (SELECT 1 AS x) t778 ON t1.x = t778.x JOIN (SELECT 1 AS x) t779 ON t1.x = t779.x JOIN (SELECT 1 AS x) t780 ON t1.x = t780.x JOIN (SELECT 1 AS x) t781 ON t1.x = t781.x JOIN (SELECT 1 AS x) t782 ON t1.x = t782.x JOIN (SELECT 1 AS x) t783 ON t1.x = t783.x JOIN (SELECT 1 AS x) t784 ON t1.x = t784.x JOIN (SELECT 1 AS x) t785 ON t1.x = t785.x JOIN (SELECT 1 AS x) t786 ON t1.x = t786.x JOIN (SELECT 1 AS x) t787 ON t1.x = t787.x JOIN (SELECT 1 AS x) t788 ON t1.x = t788.x JOIN (SELECT 1 AS x) t789 ON t1.x = t789.x JOIN (SELECT 1 AS x) t790 ON t1.x = t790.x JOIN (SELECT 1 AS x) t791 ON t1.x = t791.x JOIN (SELECT 1 AS x) t792 ON t1.x = t792.x JOIN (SELECT 1 AS x) t793 ON t1.x = t793.x JOIN (SELECT 1 AS x) t794 ON t1.x = t794.x JOIN (SELECT 1 AS x) t795 ON t1.x = t795.x JOIN (SELECT 1 AS x) t796 ON t1.x = t796.x JOIN (SELECT 1 AS x) t797 ON t1.x = t797.x JOIN (SELECT 1 AS x) t798 ON t1.x = t798.x JOIN (SELECT 1 AS x) t799 ON t1.x = t799.x JOIN (SELECT 1 AS x) t800 ON t1.x = t800.x JOIN (SELECT 1 AS x) t801 ON t1.x = t801.x JOIN (SELECT 1 AS x) t802 ON t1.x = t802.x JOIN (SELECT 1 AS x) t803 ON t1.x = t803.x JOIN (SELECT 1 AS x) t804 ON t1.x = t804.x JOIN (SELECT 1 AS x) t805 ON t1.x = t805.x JOIN (SELECT 1 AS x) t806 ON t1.x = t806.x JOIN (SELECT 1 AS x) t807 ON t1.x = t807.x JOIN (SELECT 1 AS x) t808 ON t1.x = t808.x JOIN (SELECT 1 AS x) t809 ON t1.x = t809.x JOIN (SELECT 1 AS x) t810 ON t1.x = t810.x JOIN (SELECT 1 AS x) t811 ON t1.x = t811.x JOIN (SELECT 1 AS x) t812 ON t1.x = t812.x JOIN (SELECT 1 AS x) t813 ON t1.x = t813.x JOIN (SELECT 1 AS x) t814 ON t1.x = t814.x JOIN (SELECT 1 AS x) t815 ON t1.x = t815.x JOIN (SELECT 1 AS x) t816 ON t1.x = t816.x JOIN (SELECT 1 AS x) t817 ON t1.x = t817.x JOIN (SELECT 1 AS x) t818 ON t1.x = t818.x JOIN (SELECT 1 AS x) t819 ON t1.x = t819.x JOIN (SELECT 1 AS x) t820 ON t1.x = t820.x JOIN (SELECT 1 AS x) t821 ON t1.x = t821.x JOIN (SELECT 1 AS x) t822 ON t1.x = t822.x JOIN (SELECT 1 AS x) t823 ON t1.x = t823.x JOIN (SELECT 1 AS x) t824 ON t1.x = t824.x JOIN (SELECT 1 AS x) t825 ON t1.x = t825.x JOIN (SELECT 1 AS x) t826 ON t1.x = t826.x JOIN (SELECT 1 AS x) t827 ON t1.x = t827.x JOIN (SELECT 1 AS x) t828 ON t1.x = t828.x JOIN (SELECT 1 AS x) t829 ON t1.x = t829.x JOIN (SELECT 1 AS x) t830 ON t1.x = t830.x JOIN (SELECT 1 AS x) t831 ON t1.x = t831.x JOIN (SELECT 1 AS x) t832 ON t1.x = t832.x JOIN (SELECT 1 AS x) t833 ON t1.x = t833.x JOIN (SELECT 1 AS x) t834 ON t1.x = t834.x JOIN (SELECT 1 AS x) t835 ON t1.x = t835.x JOIN (SELECT 1 AS x) t836 ON t1.x = t836.x JOIN (SELECT 1 AS x) t837 ON t1.x = t837.x JOIN (SELECT 1 AS x) t838 ON t1.x = t838.x JOIN (SELECT 1 AS x) t839 ON t1.x = t839.x JOIN (SELECT 1 AS x) t840 ON t1.x = t840.x JOIN (SELECT 1 AS x) t841 ON t1.x = t841.x JOIN (SELECT 1 AS x) t842 ON t1.x = t842.x JOIN (SELECT 1 AS x) t843 ON t1.x = t843.x JOIN (SELECT 1 AS x) t844 ON t1.x = t844.x JOIN (SELECT 1 AS x) t845 ON t1.x = t845.x JOIN (SELECT 1 AS x) t846 ON t1.x = t846.x JOIN (SELECT 1 AS x) t847 ON t1.x = t847.x JOIN (SELECT 1 AS x) t848 ON t1.x = t848.x JOIN (SELECT 1 AS x) t849 ON t1.x = t849.x JOIN (SELECT 1 AS x) t850 ON t1.x = t850.x JOIN (SELECT 1 AS x) t851 ON t1.x = t851.x JOIN (SELECT 1 AS x) t852 ON t1.x = t852.x JOIN (SELECT 1 AS x) t853 ON t1.x = t853.x JOIN (SELECT 1 AS x) t854 ON t1.x = t854.x JOIN (SELECT 1 AS x) t855 ON t1.x = t855.x JOIN (SELECT 1 AS x) t856 ON t1.x = t856.x JOIN (SELECT 1 AS x) t857 ON t1.x = t857.x JOIN (SELECT 1 AS x) t858 ON t1.x = t858.x JOIN (SELECT 1 AS x) t859 ON t1.x = t859.x JOIN (SELECT 1 AS x) t860 ON t1.x = t860.x JOIN (SELECT 1 AS x) t861 ON t1.x = t861.x JOIN (SELECT 1 AS x) t862 ON t1.x = t862.x JOIN (SELECT 1 AS x) t863 ON t1.x = t863.x JOIN (SELECT 1 AS x) t864 ON t1.x = t864.x JOIN (SELECT 1 AS x) t865 ON t1.x = t865.x JOIN (SELECT 1 AS x) t866 ON t1.x = t866.x JOIN (SELECT 1 AS x) t867 ON t1.x = t867.x JOIN (SELECT 1 AS x) t868 ON t1.x = t868.x JOIN (SELECT 1 AS x) t869 ON t1.x = t869.x JOIN (SELECT 1 AS x) t870 ON t1.x = t870.x JOIN (SELECT 1 AS x) t871 ON t1.x = t871.x JOIN (SELECT 1 AS x) t872 ON t1.x = t872.x JOIN (SELECT 1 AS x) t873 ON t1.x = t873.x JOIN (SELECT 1 AS x) t874 ON t1.x = t874.x JOIN (SELECT 1 AS x) t875 ON t1.x = t875.x JOIN (SELECT 1 AS x) t876 ON t1.x = t876.x JOIN (SELECT 1 AS x) t877 ON t1.x = t877.x JOIN (SELECT 1 AS x) t878 ON t1.x = t878.x JOIN (SELECT 1 AS x) t879 ON t1.x = t879.x JOIN (SELECT 1 AS x) t880 ON t1.x = t880.x JOIN (SELECT 1 AS x) t881 ON t1.x = t881.x JOIN (SELECT 1 AS x) t882 ON t1.x = t882.x JOIN (SELECT 1 AS x) t883 ON t1.x = t883.x JOIN (SELECT 1 AS x) t884 ON t1.x = t884.x JOIN (SELECT 1 AS x) t885 ON t1.x = t885.x JOIN (SELECT 1 AS x) t886 ON t1.x = t886.x JOIN (SELECT 1 AS x) t887 ON t1.x = t887.x JOIN (SELECT 1 AS x) t888 ON t1.x = t888.x JOIN (SELECT 1 AS x) t889 ON t1.x = t889.x JOIN (SELECT 1 AS x) t890 ON t1.x = t890.x JOIN (SELECT 1 AS x) t891 ON t1.x = t891.x JOIN (SELECT 1 AS x) t892 ON t1.x = t892.x JOIN (SELECT 1 AS x) t893 ON t1.x = t893.x JOIN (SELECT 1 AS x) t894 ON t1.x = t894.x JOIN (SELECT 1 AS x) t895 ON t1.x = t895.x JOIN (SELECT 1 AS x) t896 ON t1.x = t896.x JOIN (SELECT 1 AS x) t897 ON t1.x = t897.x JOIN (SELECT 1 AS x) t898 ON t1.x = t898.x JOIN (SELECT 1 AS x) t899 ON t1.x = t899.x JOIN (SELECT 1 AS x) t900 ON t1.x = t900.x JOIN (SELECT 1 AS x) t901 ON t1.x = t901.x JOIN (SELECT 1 AS x) t902 ON t1.x = t902.x JOIN (SELECT 1 AS x) t903 ON t1.x = t903.x JOIN (SELECT 1 AS x) t904 ON t1.x = t904.x JOIN (SELECT 1 AS x) t905 ON t1.x = t905.x JOIN (SELECT 1 AS x) t906 ON t1.x = t906.x JOIN (SELECT 1 AS x) t907 ON t1.x = t907.x JOIN (SELECT 1 AS x) t908 ON t1.x = t908.x JOIN (SELECT 1 AS x) t909 ON t1.x = t909.x JOIN (SELECT 1 AS x) t910 ON t1.x = t910.x JOIN (SELECT 1 AS x) t911 ON t1.x = t911.x JOIN (SELECT 1 AS x) t912 ON t1.x = t912.x JOIN (SELECT 1 AS x) t913 ON t1.x = t913.x JOIN (SELECT 1 AS x) t914 ON t1.x = t914.x JOIN (SELECT 1 AS x) t915 ON t1.x = t915.x JOIN (SELECT 1 AS x) t916 ON t1.x = t916.x JOIN (SELECT 1 AS x) t917 ON t1.x = t917.x JOIN (SELECT 1 AS x) t918 ON t1.x = t918.x JOIN (SELECT 1 AS x) t919 ON t1.x = t919.x JOIN (SELECT 1 AS x) t920 ON t1.x = t920.x JOIN (SELECT 1 AS x) t921 ON t1.x = t921.x JOIN (SELECT 1 AS x) t922 ON t1.x = t922.x JOIN (SELECT 1 AS x) t923 ON t1.x = t923.x JOIN (SELECT 1 AS x) t924 ON t1.x = t924.x JOIN (SELECT 1 AS x) t925 ON t1.x = t925.x JOIN (SELECT 1 AS x) t926 ON t1.x = t926.x JOIN (SELECT 1 AS x) t927 ON t1.x = t927.x JOIN (SELECT 1 AS x) t928 ON t1.x = t928.x JOIN (SELECT 1 AS x) t929 ON t1.x = t929.x JOIN (SELECT 1 AS x) t930 ON t1.x = t930.x JOIN (SELECT 1 AS x) t931 ON t1.x = t931.x JOIN (SELECT 1 AS x) t932 ON t1.x = t932.x JOIN (SELECT 1 AS x) t933 ON t1.x = t933.x JOIN (SELECT 1 AS x) t934 ON t1.x = t934.x JOIN (SELECT 1 AS x) t935 ON t1.x = t935.x JOIN (SELECT 1 AS x) t936 ON t1.x = t936.x JOIN (SELECT 1 AS x) t937 ON t1.x = t937.x JOIN (SELECT 1 AS x) t938 ON t1.x = t938.x JOIN (SELECT 1 AS x) t939 ON t1.x = t939.x JOIN (SELECT 1 AS x) t940 ON t1.x = t940.x JOIN (SELECT 1 AS x) t941 ON t1.x = t941.x JOIN (SELECT 1 AS x) t942 ON t1.x = t942.x JOIN (SELECT 1 AS x) t943 ON t1.x = t943.x JOIN (SELECT 1 AS x) t944 ON t1.x = t944.x JOIN (SELECT 1 AS x) t945 ON t1.x = t945.x JOIN (SELECT 1 AS x) t946 ON t1.x = t946.x JOIN (SELECT 1 AS x) t947 ON t1.x = t947.x JOIN (SELECT 1 AS x) t948 ON t1.x = t948.x JOIN (SELECT 1 AS x) t949 ON t1.x = t949.x JOIN (SELECT 1 AS x) t950 ON t1.x = t950.x JOIN (SELECT 1 AS x) t951 ON t1.x = t951.x JOIN (SELECT 1 AS x) t952 ON t1.x = t952.x JOIN (SELECT 1 AS x) t953 ON t1.x = t953.x JOIN (SELECT 1 AS x) t954 ON t1.x = t954.x JOIN (SELECT 1 AS x) t955 ON t1.x = t955.x JOIN (SELECT 1 AS x) t956 ON t1.x = t956.x JOIN (SELECT 1 AS x) t957 ON t1.x = t957.x JOIN (SELECT 1 AS x) t958 ON t1.x = t958.x JOIN (SELECT 1 AS x) t959 ON t1.x = t959.x JOIN (SELECT 1 AS x) t960 ON t1.x = t960.x JOIN (SELECT 1 AS x) t961 ON t1.x = t961.x JOIN (SELECT 1 AS x) t962 ON t1.x = t962.x JOIN (SELECT 1 AS x) t963 ON t1.x = t963.x JOIN (SELECT 1 AS x) t964 ON t1.x = t964.x JOIN (SELECT 1 AS x) t965 ON t1.x = t965.x JOIN (SELECT 1 AS x) t966 ON t1.x = t966.x JOIN (SELECT 1 AS x) t967 ON t1.x = t967.x JOIN (SELECT 1 AS x) t968 ON t1.x = t968.x JOIN (SELECT 1 AS x) t969 ON t1.x = t969.x JOIN (SELECT 1 AS x) t970 ON t1.x = t970.x JOIN (SELECT 1 AS x) t971 ON t1.x = t971.x JOIN (SELECT 1 AS x) t972 ON t1.x = t972.x JOIN (SELECT 1 AS x) t973 ON t1.x = t973.x JOIN (SELECT 1 AS x) t974 ON t1.x = t974.x JOIN (SELECT 1 AS x) t975 ON t1.x = t975.x JOIN (SELECT 1 AS x) t976 ON t1.x = t976.x JOIN (SELECT 1 AS x) t977 ON t1.x = t977.x JOIN (SELECT 1 AS x) t978 ON t1.x = t978.x JOIN (SELECT 1 AS x) t979 ON t1.x = t979.x JOIN (SELECT 1 AS x) t980 ON t1.x = t980.x JOIN (SELECT 1 AS x) t981 ON t1.x = t981.x JOIN (SELECT 1 AS x) t982 ON t1.x = t982.x JOIN (SELECT 1 AS x) t983 ON t1.x = t983.x JOIN (SELECT 1 AS x) t984 ON t1.x = t984.x JOIN (SELECT 1 AS x) t985 ON t1.x = t985.x JOIN (SELECT 1 AS x) t986 ON t1.x = t986.x JOIN (SELECT 1 AS x) t987 ON t1.x = t987.x JOIN (SELECT 1 AS x) t988 ON t1.x = t988.x JOIN (SELECT 1 AS x) t989 ON t1.x = t989.x JOIN (SELECT 1 AS x) t990 ON t1.x = t990.x JOIN (SELECT 1 AS x) t991 ON t1.x = t991.x JOIN (SELECT 1 AS x) t992 ON t1.x = t992.x JOIN (SELECT 1 AS x) t993 ON t1.x = t993.x JOIN (SELECT 1 AS x) t994 ON t1.x = t994.x JOIN (SELECT 1 AS x) t995 ON t1.x = t995.x JOIN (SELECT 1 AS x) t996 ON t1.x = t996.x JOIN (SELECT 1 AS x) t997 ON t1.x = t997.x JOIN (SELECT 1 AS x) t998 ON t1.x = t998.x JOIN (SELECT 1 AS x) t999 ON t1.x = t999.x JOIN (SELECT 1 AS x) t1000 ON t1.x = t1000.x +SELECT * FROM (SELECT 1 AS x) t1 JOIN (SELECT 1 AS x) t2 ON t1.x = t2.x JOIN (SELECT 1 AS x) t3 ON t1.x = t3.x JOIN (SELECT 1 AS x) t4 ON t1.x = t4.x JOIN (SELECT 1 AS x) t5 ON t1.x = t5.x JOIN (SELECT 1 AS x) t6 ON t1.x = t6.x JOIN (SELECT 1 AS x) t7 ON t1.x = t7.x JOIN (SELECT 1 AS x) t8 ON t1.x = t8.x JOIN (SELECT 1 AS x) t9 ON t1.x = t9.x JOIN (SELECT 1 AS x) t10 ON t1.x = t10.x JOIN (SELECT 1 AS x) t11 ON t1.x = t11.x JOIN (SELECT 1 AS x) t12 ON t1.x = t12.x JOIN (SELECT 1 AS x) t13 ON t1.x = t13.x JOIN (SELECT 1 AS x) t14 ON t1.x = t14.x JOIN (SELECT 1 AS x) t15 ON t1.x = t15.x JOIN (SELECT 1 AS x) t16 ON t1.x = t16.x JOIN (SELECT 1 AS x) t17 ON t1.x = t17.x JOIN (SELECT 1 AS x) t18 ON t1.x = t18.x JOIN (SELECT 1 AS x) t19 ON t1.x = t19.x JOIN (SELECT 1 AS x) t20 ON t1.x = t20.x JOIN (SELECT 1 AS x) t21 ON t1.x = t21.x JOIN (SELECT 1 AS x) t22 ON t1.x = t22.x JOIN (SELECT 1 AS x) t23 ON t1.x = t23.x JOIN (SELECT 1 AS x) t24 ON t1.x = t24.x JOIN (SELECT 1 AS x) t25 ON t1.x = t25.x JOIN (SELECT 1 AS x) t26 ON t1.x = t26.x JOIN (SELECT 1 AS x) t27 ON t1.x = t27.x JOIN (SELECT 1 AS x) t28 ON t1.x = t28.x JOIN (SELECT 1 AS x) t29 ON t1.x = t29.x JOIN (SELECT 1 AS x) t30 ON t1.x = t30.x JOIN (SELECT 1 AS x) t31 ON t1.x = t31.x JOIN (SELECT 1 AS x) t32 ON t1.x = t32.x JOIN (SELECT 1 AS x) t33 ON t1.x = t33.x JOIN (SELECT 1 AS x) t34 ON t1.x = t34.x JOIN (SELECT 1 AS x) t35 ON t1.x = t35.x JOIN (SELECT 1 AS x) t36 ON t1.x = t36.x JOIN (SELECT 1 AS x) t37 ON t1.x = t37.x JOIN (SELECT 1 AS x) t38 ON t1.x = t38.x JOIN (SELECT 1 AS x) t39 ON t1.x = t39.x JOIN (SELECT 1 AS x) t40 ON t1.x = t40.x JOIN (SELECT 1 AS x) t41 ON t1.x = t41.x JOIN (SELECT 1 AS x) t42 ON t1.x = t42.x JOIN (SELECT 1 AS x) t43 ON t1.x = t43.x JOIN (SELECT 1 AS x) t44 ON t1.x = t44.x JOIN (SELECT 1 AS x) t45 ON t1.x = t45.x JOIN (SELECT 1 AS x) t46 ON t1.x = t46.x JOIN (SELECT 1 AS x) t47 ON t1.x = t47.x JOIN (SELECT 1 AS x) t48 ON t1.x = t48.x JOIN (SELECT 1 AS x) t49 ON t1.x = t49.x JOIN (SELECT 1 AS x) t50 ON t1.x = t50.x JOIN (SELECT 1 AS x) t51 ON t1.x = t51.x JOIN (SELECT 1 AS x) t52 ON t1.x = t52.x JOIN (SELECT 1 AS x) t53 ON t1.x = t53.x JOIN (SELECT 1 AS x) t54 ON t1.x = t54.x JOIN (SELECT 1 AS x) t55 ON t1.x = t55.x JOIN (SELECT 1 AS x) t56 ON t1.x = t56.x JOIN (SELECT 1 AS x) t57 ON t1.x = t57.x JOIN (SELECT 1 AS x) t58 ON t1.x = t58.x JOIN (SELECT 1 AS x) t59 ON t1.x = t59.x JOIN (SELECT 1 AS x) t60 ON t1.x = t60.x JOIN (SELECT 1 AS x) t61 ON t1.x = t61.x JOIN (SELECT 1 AS x) t62 ON t1.x = t62.x JOIN (SELECT 1 AS x) t63 ON t1.x = t63.x JOIN (SELECT 1 AS x) t64 ON t1.x = t64.x JOIN (SELECT 1 AS x) t65 ON t1.x = t65.x JOIN (SELECT 1 AS x) t66 ON t1.x = t66.x JOIN (SELECT 1 AS x) t67 ON t1.x = t67.x JOIN (SELECT 1 AS x) t68 ON t1.x = t68.x JOIN (SELECT 1 AS x) t69 ON t1.x = t69.x JOIN (SELECT 1 AS x) t70 ON t1.x = t70.x JOIN (SELECT 1 AS x) t71 ON t1.x = t71.x JOIN (SELECT 1 AS x) t72 ON t1.x = t72.x JOIN (SELECT 1 AS x) t73 ON t1.x = t73.x JOIN (SELECT 1 AS x) t74 ON t1.x = t74.x JOIN (SELECT 1 AS x) t75 ON t1.x = t75.x JOIN (SELECT 1 AS x) t76 ON t1.x = t76.x JOIN (SELECT 1 AS x) t77 ON t1.x = t77.x JOIN (SELECT 1 AS x) t78 ON t1.x = t78.x JOIN (SELECT 1 AS x) t79 ON t1.x = t79.x JOIN (SELECT 1 AS x) t80 ON t1.x = t80.x JOIN (SELECT 1 AS x) t81 ON t1.x = t81.x JOIN (SELECT 1 AS x) t82 ON t1.x = t82.x JOIN (SELECT 1 AS x) t83 ON t1.x = t83.x JOIN (SELECT 1 AS x) t84 ON t1.x = t84.x JOIN (SELECT 1 AS x) t85 ON t1.x = t85.x JOIN (SELECT 1 AS x) t86 ON t1.x = t86.x JOIN (SELECT 1 AS x) t87 ON t1.x = t87.x JOIN (SELECT 1 AS x) t88 ON t1.x = t88.x JOIN (SELECT 1 AS x) t89 ON t1.x = t89.x JOIN (SELECT 1 AS x) t90 ON t1.x = t90.x JOIN (SELECT 1 AS x) t91 ON t1.x = t91.x JOIN (SELECT 1 AS x) t92 ON t1.x = t92.x JOIN (SELECT 1 AS x) t93 ON t1.x = t93.x JOIN (SELECT 1 AS x) t94 ON t1.x = t94.x JOIN (SELECT 1 AS x) t95 ON t1.x = t95.x JOIN (SELECT 1 AS x) t96 ON t1.x = t96.x JOIN (SELECT 1 AS x) t97 ON t1.x = t97.x JOIN (SELECT 1 AS x) t98 ON t1.x = t98.x JOIN (SELECT 1 AS x) t99 ON t1.x = t99.x JOIN (SELECT 1 AS x) t100 ON t1.x = t100.x JOIN (SELECT 1 AS x) t101 ON t1.x = t101.x JOIN (SELECT 1 AS x) t102 ON t1.x = t102.x JOIN (SELECT 1 AS x) t103 ON t1.x = t103.x JOIN (SELECT 1 AS x) t104 ON t1.x = t104.x JOIN (SELECT 1 AS x) t105 ON t1.x = t105.x JOIN (SELECT 1 AS x) t106 ON t1.x = t106.x JOIN (SELECT 1 AS x) t107 ON t1.x = t107.x JOIN (SELECT 1 AS x) t108 ON t1.x = t108.x JOIN (SELECT 1 AS x) t109 ON t1.x = t109.x JOIN (SELECT 1 AS x) t110 ON t1.x = t110.x JOIN (SELECT 1 AS x) t111 ON t1.x = t111.x JOIN (SELECT 1 AS x) t112 ON t1.x = t112.x JOIN (SELECT 1 AS x) t113 ON t1.x = t113.x JOIN (SELECT 1 AS x) t114 ON t1.x = t114.x JOIN (SELECT 1 AS x) t115 ON t1.x = t115.x JOIN (SELECT 1 AS x) t116 ON t1.x = t116.x JOIN (SELECT 1 AS x) t117 ON t1.x = t117.x JOIN (SELECT 1 AS x) t118 ON t1.x = t118.x JOIN (SELECT 1 AS x) t119 ON t1.x = t119.x JOIN (SELECT 1 AS x) t120 ON t1.x = t120.x JOIN (SELECT 1 AS x) t121 ON t1.x = t121.x JOIN (SELECT 1 AS x) t122 ON t1.x = t122.x JOIN (SELECT 1 AS x) t123 ON t1.x = t123.x JOIN (SELECT 1 AS x) t124 ON t1.x = t124.x JOIN (SELECT 1 AS x) t125 ON t1.x = t125.x JOIN (SELECT 1 AS x) t126 ON t1.x = t126.x JOIN (SELECT 1 AS x) t127 ON t1.x = t127.x JOIN (SELECT 1 AS x) t128 ON t1.x = t128.x JOIN (SELECT 1 AS x) t129 ON t1.x = t129.x JOIN (SELECT 1 AS x) t130 ON t1.x = t130.x JOIN (SELECT 1 AS x) t131 ON t1.x = t131.x JOIN (SELECT 1 AS x) t132 ON t1.x = t132.x JOIN (SELECT 1 AS x) t133 ON t1.x = t133.x JOIN (SELECT 1 AS x) t134 ON t1.x = t134.x JOIN (SELECT 1 AS x) t135 ON t1.x = t135.x JOIN (SELECT 1 AS x) t136 ON t1.x = t136.x JOIN (SELECT 1 AS x) t137 ON t1.x = t137.x JOIN (SELECT 1 AS x) t138 ON t1.x = t138.x JOIN (SELECT 1 AS x) t139 ON t1.x = t139.x JOIN (SELECT 1 AS x) t140 ON t1.x = t140.x JOIN (SELECT 1 AS x) t141 ON t1.x = t141.x JOIN (SELECT 1 AS x) t142 ON t1.x = t142.x JOIN (SELECT 1 AS x) t143 ON t1.x = t143.x JOIN (SELECT 1 AS x) t144 ON t1.x = t144.x JOIN (SELECT 1 AS x) t145 ON t1.x = t145.x JOIN (SELECT 1 AS x) t146 ON t1.x = t146.x JOIN (SELECT 1 AS x) t147 ON t1.x = t147.x JOIN (SELECT 1 AS x) t148 ON t1.x = t148.x JOIN (SELECT 1 AS x) t149 ON t1.x = t149.x JOIN (SELECT 1 AS x) t150 ON t1.x = t150.x JOIN (SELECT 1 AS x) t151 ON t1.x = t151.x JOIN (SELECT 1 AS x) t152 ON t1.x = t152.x JOIN (SELECT 1 AS x) t153 ON t1.x = t153.x JOIN (SELECT 1 AS x) t154 ON t1.x = t154.x JOIN (SELECT 1 AS x) t155 ON t1.x = t155.x JOIN (SELECT 1 AS x) t156 ON t1.x = t156.x JOIN (SELECT 1 AS x) t157 ON t1.x = t157.x JOIN (SELECT 1 AS x) t158 ON t1.x = t158.x JOIN (SELECT 1 AS x) t159 ON t1.x = t159.x JOIN (SELECT 1 AS x) t160 ON t1.x = t160.x JOIN (SELECT 1 AS x) t161 ON t1.x = t161.x JOIN (SELECT 1 AS x) t162 ON t1.x = t162.x JOIN (SELECT 1 AS x) t163 ON t1.x = t163.x JOIN (SELECT 1 AS x) t164 ON t1.x = t164.x JOIN (SELECT 1 AS x) t165 ON t1.x = t165.x JOIN (SELECT 1 AS x) t166 ON t1.x = t166.x JOIN (SELECT 1 AS x) t167 ON t1.x = t167.x JOIN (SELECT 1 AS x) t168 ON t1.x = t168.x JOIN (SELECT 1 AS x) t169 ON t1.x = t169.x JOIN (SELECT 1 AS x) t170 ON t1.x = t170.x JOIN (SELECT 1 AS x) t171 ON t1.x = t171.x JOIN (SELECT 1 AS x) t172 ON t1.x = t172.x JOIN (SELECT 1 AS x) t173 ON t1.x = t173.x JOIN (SELECT 1 AS x) t174 ON t1.x = t174.x JOIN (SELECT 1 AS x) t175 ON t1.x = t175.x JOIN (SELECT 1 AS x) t176 ON t1.x = t176.x JOIN (SELECT 1 AS x) t177 ON t1.x = t177.x JOIN (SELECT 1 AS x) t178 ON t1.x = t178.x JOIN (SELECT 1 AS x) t179 ON t1.x = t179.x JOIN (SELECT 1 AS x) t180 ON t1.x = t180.x JOIN (SELECT 1 AS x) t181 ON t1.x = t181.x JOIN (SELECT 1 AS x) t182 ON t1.x = t182.x JOIN (SELECT 1 AS x) t183 ON t1.x = t183.x JOIN (SELECT 1 AS x) t184 ON t1.x = t184.x JOIN (SELECT 1 AS x) t185 ON t1.x = t185.x JOIN (SELECT 1 AS x) t186 ON t1.x = t186.x JOIN (SELECT 1 AS x) t187 ON t1.x = t187.x JOIN (SELECT 1 AS x) t188 ON t1.x = t188.x JOIN (SELECT 1 AS x) t189 ON t1.x = t189.x JOIN (SELECT 1 AS x) t190 ON t1.x = t190.x JOIN (SELECT 1 AS x) t191 ON t1.x = t191.x JOIN (SELECT 1 AS x) t192 ON t1.x = t192.x JOIN (SELECT 1 AS x) t193 ON t1.x = t193.x JOIN (SELECT 1 AS x) t194 ON t1.x = t194.x JOIN (SELECT 1 AS x) t195 ON t1.x = t195.x JOIN (SELECT 1 AS x) t196 ON t1.x = t196.x JOIN (SELECT 1 AS x) t197 ON t1.x = t197.x JOIN (SELECT 1 AS x) t198 ON t1.x = t198.x JOIN (SELECT 1 AS x) t199 ON t1.x = t199.x JOIN (SELECT 1 AS x) t200 ON t1.x = t200.x JOIN (SELECT 1 AS x) t201 ON t1.x = t201.x JOIN (SELECT 1 AS x) t202 ON t1.x = t202.x JOIN (SELECT 1 AS x) t203 ON t1.x = t203.x JOIN (SELECT 1 AS x) t204 ON t1.x = t204.x JOIN (SELECT 1 AS x) t205 ON t1.x = t205.x JOIN (SELECT 1 AS x) t206 ON t1.x = t206.x JOIN (SELECT 1 AS x) t207 ON t1.x = t207.x JOIN (SELECT 1 AS x) t208 ON t1.x = t208.x JOIN (SELECT 1 AS x) t209 ON t1.x = t209.x JOIN (SELECT 1 AS x) t210 ON t1.x = t210.x JOIN (SELECT 1 AS x) t211 ON t1.x = t211.x JOIN (SELECT 1 AS x) t212 ON t1.x = t212.x JOIN (SELECT 1 AS x) t213 ON t1.x = t213.x JOIN (SELECT 1 AS x) t214 ON t1.x = t214.x JOIN (SELECT 1 AS x) t215 ON t1.x = t215.x JOIN (SELECT 1 AS x) t216 ON t1.x = t216.x JOIN (SELECT 1 AS x) t217 ON t1.x = t217.x JOIN (SELECT 1 AS x) t218 ON t1.x = t218.x JOIN (SELECT 1 AS x) t219 ON t1.x = t219.x JOIN (SELECT 1 AS x) t220 ON t1.x = t220.x JOIN (SELECT 1 AS x) t221 ON t1.x = t221.x JOIN (SELECT 1 AS x) t222 ON t1.x = t222.x JOIN (SELECT 1 AS x) t223 ON t1.x = t223.x JOIN (SELECT 1 AS x) t224 ON t1.x = t224.x JOIN (SELECT 1 AS x) t225 ON t1.x = t225.x JOIN (SELECT 1 AS x) t226 ON t1.x = t226.x JOIN (SELECT 1 AS x) t227 ON t1.x = t227.x JOIN (SELECT 1 AS x) t228 ON t1.x = t228.x JOIN (SELECT 1 AS x) t229 ON t1.x = t229.x JOIN (SELECT 1 AS x) t230 ON t1.x = t230.x JOIN (SELECT 1 AS x) t231 ON t1.x = t231.x JOIN (SELECT 1 AS x) t232 ON t1.x = t232.x JOIN (SELECT 1 AS x) t233 ON t1.x = t233.x JOIN (SELECT 1 AS x) t234 ON t1.x = t234.x JOIN (SELECT 1 AS x) t235 ON t1.x = t235.x JOIN (SELECT 1 AS x) t236 ON t1.x = t236.x JOIN (SELECT 1 AS x) t237 ON t1.x = t237.x JOIN (SELECT 1 AS x) t238 ON t1.x = t238.x JOIN (SELECT 1 AS x) t239 ON t1.x = t239.x JOIN (SELECT 1 AS x) t240 ON t1.x = t240.x JOIN (SELECT 1 AS x) t241 ON t1.x = t241.x JOIN (SELECT 1 AS x) t242 ON t1.x = t242.x JOIN (SELECT 1 AS x) t243 ON t1.x = t243.x JOIN (SELECT 1 AS x) t244 ON t1.x = t244.x JOIN (SELECT 1 AS x) t245 ON t1.x = t245.x JOIN (SELECT 1 AS x) t246 ON t1.x = t246.x JOIN (SELECT 1 AS x) t247 ON t1.x = t247.x JOIN (SELECT 1 AS x) t248 ON t1.x = t248.x JOIN (SELECT 1 AS x) t249 ON t1.x = t249.x JOIN (SELECT 1 AS x) t250 ON t1.x = t250.x JOIN (SELECT 1 AS x) t251 ON t1.x = t251.x JOIN (SELECT 1 AS x) t252 ON t1.x = t252.x JOIN (SELECT 1 AS x) t253 ON t1.x = t253.x JOIN (SELECT 1 AS x) t254 ON t1.x = t254.x JOIN (SELECT 1 AS x) t255 ON t1.x = t255.x JOIN (SELECT 1 AS x) t256 ON t1.x = t256.x JOIN (SELECT 1 AS x) t257 ON t1.x = t257.x JOIN (SELECT 1 AS x) t258 ON t1.x = t258.x JOIN (SELECT 1 AS x) t259 ON t1.x = t259.x JOIN (SELECT 1 AS x) t260 ON t1.x = t260.x JOIN (SELECT 1 AS x) t261 ON t1.x = t261.x JOIN (SELECT 1 AS x) t262 ON t1.x = t262.x JOIN (SELECT 1 AS x) t263 ON t1.x = t263.x JOIN (SELECT 1 AS x) t264 ON t1.x = t264.x JOIN (SELECT 1 AS x) t265 ON t1.x = t265.x JOIN (SELECT 1 AS x) t266 ON t1.x = t266.x JOIN (SELECT 1 AS x) t267 ON t1.x = t267.x JOIN (SELECT 1 AS x) t268 ON t1.x = t268.x JOIN (SELECT 1 AS x) t269 ON t1.x = t269.x JOIN (SELECT 1 AS x) t270 ON t1.x = t270.x JOIN (SELECT 1 AS x) t271 ON t1.x = t271.x JOIN (SELECT 1 AS x) t272 ON t1.x = t272.x JOIN (SELECT 1 AS x) t273 ON t1.x = t273.x JOIN (SELECT 1 AS x) t274 ON t1.x = t274.x JOIN (SELECT 1 AS x) t275 ON t1.x = t275.x JOIN (SELECT 1 AS x) t276 ON t1.x = t276.x JOIN (SELECT 1 AS x) t277 ON t1.x = t277.x JOIN (SELECT 1 AS x) t278 ON t1.x = t278.x JOIN (SELECT 1 AS x) t279 ON t1.x = t279.x JOIN (SELECT 1 AS x) t280 ON t1.x = t280.x JOIN (SELECT 1 AS x) t281 ON t1.x = t281.x JOIN (SELECT 1 AS x) t282 ON t1.x = t282.x JOIN (SELECT 1 AS x) t283 ON t1.x = t283.x JOIN (SELECT 1 AS x) t284 ON t1.x = t284.x JOIN (SELECT 1 AS x) t285 ON t1.x = t285.x JOIN (SELECT 1 AS x) t286 ON t1.x = t286.x JOIN (SELECT 1 AS x) t287 ON t1.x = t287.x JOIN (SELECT 1 AS x) t288 ON t1.x = t288.x JOIN (SELECT 1 AS x) t289 ON t1.x = t289.x JOIN (SELECT 1 AS x) t290 ON t1.x = t290.x JOIN (SELECT 1 AS x) t291 ON t1.x = t291.x JOIN (SELECT 1 AS x) t292 ON t1.x = t292.x JOIN (SELECT 1 AS x) t293 ON t1.x = t293.x JOIN (SELECT 1 AS x) t294 ON t1.x = t294.x JOIN (SELECT 1 AS x) t295 ON t1.x = t295.x JOIN (SELECT 1 AS x) t296 ON t1.x = t296.x JOIN (SELECT 1 AS x) t297 ON t1.x = t297.x JOIN (SELECT 1 AS x) t298 ON t1.x = t298.x JOIN (SELECT 1 AS x) t299 ON t1.x = t299.x JOIN (SELECT 1 AS x) t300 ON t1.x = t300.x JOIN (SELECT 1 AS x) t301 ON t1.x = t301.x JOIN (SELECT 1 AS x) t302 ON t1.x = t302.x JOIN (SELECT 1 AS x) t303 ON t1.x = t303.x JOIN (SELECT 1 AS x) t304 ON t1.x = t304.x JOIN (SELECT 1 AS x) t305 ON t1.x = t305.x JOIN (SELECT 1 AS x) t306 ON t1.x = t306.x JOIN (SELECT 1 AS x) t307 ON t1.x = t307.x JOIN (SELECT 1 AS x) t308 ON t1.x = t308.x JOIN (SELECT 1 AS x) t309 ON t1.x = t309.x JOIN (SELECT 1 AS x) t310 ON t1.x = t310.x JOIN (SELECT 1 AS x) t311 ON t1.x = t311.x JOIN (SELECT 1 AS x) t312 ON t1.x = t312.x JOIN (SELECT 1 AS x) t313 ON t1.x = t313.x JOIN (SELECT 1 AS x) t314 ON t1.x = t314.x JOIN (SELECT 1 AS x) t315 ON t1.x = t315.x JOIN (SELECT 1 AS x) t316 ON t1.x = t316.x JOIN (SELECT 1 AS x) t317 ON t1.x = t317.x JOIN (SELECT 1 AS x) t318 ON t1.x = t318.x JOIN (SELECT 1 AS x) t319 ON t1.x = t319.x JOIN (SELECT 1 AS x) t320 ON t1.x = t320.x JOIN (SELECT 1 AS x) t321 ON t1.x = t321.x JOIN (SELECT 1 AS x) t322 ON t1.x = t322.x JOIN (SELECT 1 AS x) t323 ON t1.x = t323.x JOIN (SELECT 1 AS x) t324 ON t1.x = t324.x JOIN (SELECT 1 AS x) t325 ON t1.x = t325.x JOIN (SELECT 1 AS x) t326 ON t1.x = t326.x JOIN (SELECT 1 AS x) t327 ON t1.x = t327.x JOIN (SELECT 1 AS x) t328 ON t1.x = t328.x JOIN (SELECT 1 AS x) t329 ON t1.x = t329.x JOIN (SELECT 1 AS x) t330 ON t1.x = t330.x JOIN (SELECT 1 AS x) t331 ON t1.x = t331.x JOIN (SELECT 1 AS x) t332 ON t1.x = t332.x JOIN (SELECT 1 AS x) t333 ON t1.x = t333.x JOIN (SELECT 1 AS x) t334 ON t1.x = t334.x JOIN (SELECT 1 AS x) t335 ON t1.x = t335.x JOIN (SELECT 1 AS x) t336 ON t1.x = t336.x JOIN (SELECT 1 AS x) t337 ON t1.x = t337.x JOIN (SELECT 1 AS x) t338 ON t1.x = t338.x JOIN (SELECT 1 AS x) t339 ON t1.x = t339.x JOIN (SELECT 1 AS x) t340 ON t1.x = t340.x JOIN (SELECT 1 AS x) t341 ON t1.x = t341.x JOIN (SELECT 1 AS x) t342 ON t1.x = t342.x JOIN (SELECT 1 AS x) t343 ON t1.x = t343.x JOIN (SELECT 1 AS x) t344 ON t1.x = t344.x JOIN (SELECT 1 AS x) t345 ON t1.x = t345.x JOIN (SELECT 1 AS x) t346 ON t1.x = t346.x JOIN (SELECT 1 AS x) t347 ON t1.x = t347.x JOIN (SELECT 1 AS x) t348 ON t1.x = t348.x JOIN (SELECT 1 AS x) t349 ON t1.x = t349.x JOIN (SELECT 1 AS x) t350 ON t1.x = t350.x JOIN (SELECT 1 AS x) t351 ON t1.x = t351.x JOIN (SELECT 1 AS x) t352 ON t1.x = t352.x JOIN (SELECT 1 AS x) t353 ON t1.x = t353.x JOIN (SELECT 1 AS x) t354 ON t1.x = t354.x JOIN (SELECT 1 AS x) t355 ON t1.x = t355.x JOIN (SELECT 1 AS x) t356 ON t1.x = t356.x JOIN (SELECT 1 AS x) t357 ON t1.x = t357.x JOIN (SELECT 1 AS x) t358 ON t1.x = t358.x JOIN (SELECT 1 AS x) t359 ON t1.x = t359.x JOIN (SELECT 1 AS x) t360 ON t1.x = t360.x JOIN (SELECT 1 AS x) t361 ON t1.x = t361.x JOIN (SELECT 1 AS x) t362 ON t1.x = t362.x JOIN (SELECT 1 AS x) t363 ON t1.x = t363.x JOIN (SELECT 1 AS x) t364 ON t1.x = t364.x JOIN (SELECT 1 AS x) t365 ON t1.x = t365.x JOIN (SELECT 1 AS x) t366 ON t1.x = t366.x JOIN (SELECT 1 AS x) t367 ON t1.x = t367.x JOIN (SELECT 1 AS x) t368 ON t1.x = t368.x JOIN (SELECT 1 AS x) t369 ON t1.x = t369.x JOIN (SELECT 1 AS x) t370 ON t1.x = t370.x JOIN (SELECT 1 AS x) t371 ON t1.x = t371.x JOIN (SELECT 1 AS x) t372 ON t1.x = t372.x JOIN (SELECT 1 AS x) t373 ON t1.x = t373.x JOIN (SELECT 1 AS x) t374 ON t1.x = t374.x JOIN (SELECT 1 AS x) t375 ON t1.x = t375.x JOIN (SELECT 1 AS x) t376 ON t1.x = t376.x JOIN (SELECT 1 AS x) t377 ON t1.x = t377.x JOIN (SELECT 1 AS x) t378 ON t1.x = t378.x JOIN (SELECT 1 AS x) t379 ON t1.x = t379.x JOIN (SELECT 1 AS x) t380 ON t1.x = t380.x JOIN (SELECT 1 AS x) t381 ON t1.x = t381.x JOIN (SELECT 1 AS x) t382 ON t1.x = t382.x JOIN (SELECT 1 AS x) t383 ON t1.x = t383.x JOIN (SELECT 1 AS x) t384 ON t1.x = t384.x JOIN (SELECT 1 AS x) t385 ON t1.x = t385.x JOIN (SELECT 1 AS x) t386 ON t1.x = t386.x JOIN (SELECT 1 AS x) t387 ON t1.x = t387.x JOIN (SELECT 1 AS x) t388 ON t1.x = t388.x JOIN (SELECT 1 AS x) t389 ON t1.x = t389.x JOIN (SELECT 1 AS x) t390 ON t1.x = t390.x JOIN (SELECT 1 AS x) t391 ON t1.x = t391.x JOIN (SELECT 1 AS x) t392 ON t1.x = t392.x JOIN (SELECT 1 AS x) t393 ON t1.x = t393.x JOIN (SELECT 1 AS x) t394 ON t1.x = t394.x JOIN (SELECT 1 AS x) t395 ON t1.x = t395.x JOIN (SELECT 1 AS x) t396 ON t1.x = t396.x JOIN (SELECT 1 AS x) t397 ON t1.x = t397.x JOIN (SELECT 1 AS x) t398 ON t1.x = t398.x JOIN (SELECT 1 AS x) t399 ON t1.x = t399.x JOIN (SELECT 1 AS x) t400 ON t1.x = t400.x JOIN (SELECT 1 AS x) t401 ON t1.x = t401.x JOIN (SELECT 1 AS x) t402 ON t1.x = t402.x JOIN (SELECT 1 AS x) t403 ON t1.x = t403.x JOIN (SELECT 1 AS x) t404 ON t1.x = t404.x JOIN (SELECT 1 AS x) t405 ON t1.x = t405.x JOIN (SELECT 1 AS x) t406 ON t1.x = t406.x JOIN (SELECT 1 AS x) t407 ON t1.x = t407.x JOIN (SELECT 1 AS x) t408 ON t1.x = t408.x JOIN (SELECT 1 AS x) t409 ON t1.x = t409.x JOIN (SELECT 1 AS x) t410 ON t1.x = t410.x JOIN (SELECT 1 AS x) t411 ON t1.x = t411.x JOIN (SELECT 1 AS x) t412 ON t1.x = t412.x JOIN (SELECT 1 AS x) t413 ON t1.x = t413.x JOIN (SELECT 1 AS x) t414 ON t1.x = t414.x JOIN (SELECT 1 AS x) t415 ON t1.x = t415.x JOIN (SELECT 1 AS x) t416 ON t1.x = t416.x JOIN (SELECT 1 AS x) t417 ON t1.x = t417.x JOIN (SELECT 1 AS x) t418 ON t1.x = t418.x JOIN (SELECT 1 AS x) t419 ON t1.x = t419.x JOIN (SELECT 1 AS x) t420 ON t1.x = t420.x JOIN (SELECT 1 AS x) t421 ON t1.x = t421.x JOIN (SELECT 1 AS x) t422 ON t1.x = t422.x JOIN (SELECT 1 AS x) t423 ON t1.x = t423.x JOIN (SELECT 1 AS x) t424 ON t1.x = t424.x JOIN (SELECT 1 AS x) t425 ON t1.x = t425.x JOIN (SELECT 1 AS x) t426 ON t1.x = t426.x JOIN (SELECT 1 AS x) t427 ON t1.x = t427.x JOIN (SELECT 1 AS x) t428 ON t1.x = t428.x JOIN (SELECT 1 AS x) t429 ON t1.x = t429.x JOIN (SELECT 1 AS x) t430 ON t1.x = t430.x JOIN (SELECT 1 AS x) t431 ON t1.x = t431.x JOIN (SELECT 1 AS x) t432 ON t1.x = t432.x JOIN (SELECT 1 AS x) t433 ON t1.x = t433.x JOIN (SELECT 1 AS x) t434 ON t1.x = t434.x JOIN (SELECT 1 AS x) t435 ON t1.x = t435.x JOIN (SELECT 1 AS x) t436 ON t1.x = t436.x JOIN (SELECT 1 AS x) t437 ON t1.x = t437.x JOIN (SELECT 1 AS x) t438 ON t1.x = t438.x JOIN (SELECT 1 AS x) t439 ON t1.x = t439.x JOIN (SELECT 1 AS x) t440 ON t1.x = t440.x JOIN (SELECT 1 AS x) t441 ON t1.x = t441.x JOIN (SELECT 1 AS x) t442 ON t1.x = t442.x JOIN (SELECT 1 AS x) t443 ON t1.x = t443.x JOIN (SELECT 1 AS x) t444 ON t1.x = t444.x JOIN (SELECT 1 AS x) t445 ON t1.x = t445.x JOIN (SELECT 1 AS x) t446 ON t1.x = t446.x JOIN (SELECT 1 AS x) t447 ON t1.x = t447.x JOIN (SELECT 1 AS x) t448 ON t1.x = t448.x JOIN (SELECT 1 AS x) t449 ON t1.x = t449.x JOIN (SELECT 1 AS x) t450 ON t1.x = t450.x JOIN (SELECT 1 AS x) t451 ON t1.x = t451.x JOIN (SELECT 1 AS x) t452 ON t1.x = t452.x JOIN (SELECT 1 AS x) t453 ON t1.x = t453.x JOIN (SELECT 1 AS x) t454 ON t1.x = t454.x JOIN (SELECT 1 AS x) t455 ON t1.x = t455.x JOIN (SELECT 1 AS x) t456 ON t1.x = t456.x JOIN (SELECT 1 AS x) t457 ON t1.x = t457.x JOIN (SELECT 1 AS x) t458 ON t1.x = t458.x JOIN (SELECT 1 AS x) t459 ON t1.x = t459.x JOIN (SELECT 1 AS x) t460 ON t1.x = t460.x JOIN (SELECT 1 AS x) t461 ON t1.x = t461.x JOIN (SELECT 1 AS x) t462 ON t1.x = t462.x JOIN (SELECT 1 AS x) t463 ON t1.x = t463.x JOIN (SELECT 1 AS x) t464 ON t1.x = t464.x JOIN (SELECT 1 AS x) t465 ON t1.x = t465.x JOIN (SELECT 1 AS x) t466 ON t1.x = t466.x JOIN (SELECT 1 AS x) t467 ON t1.x = t467.x JOIN (SELECT 1 AS x) t468 ON t1.x = t468.x JOIN (SELECT 1 AS x) t469 ON t1.x = t469.x JOIN (SELECT 1 AS x) t470 ON t1.x = t470.x JOIN (SELECT 1 AS x) t471 ON t1.x = t471.x JOIN (SELECT 1 AS x) t472 ON t1.x = t472.x JOIN (SELECT 1 AS x) t473 ON t1.x = t473.x JOIN (SELECT 1 AS x) t474 ON t1.x = t474.x JOIN (SELECT 1 AS x) t475 ON t1.x = t475.x JOIN (SELECT 1 AS x) t476 ON t1.x = t476.x JOIN (SELECT 1 AS x) t477 ON t1.x = t477.x JOIN (SELECT 1 AS x) t478 ON t1.x = t478.x JOIN (SELECT 1 AS x) t479 ON t1.x = t479.x JOIN (SELECT 1 AS x) t480 ON t1.x = t480.x JOIN (SELECT 1 AS x) t481 ON t1.x = t481.x JOIN (SELECT 1 AS x) t482 ON t1.x = t482.x JOIN (SELECT 1 AS x) t483 ON t1.x = t483.x JOIN (SELECT 1 AS x) t484 ON t1.x = t484.x JOIN (SELECT 1 AS x) t485 ON t1.x = t485.x JOIN (SELECT 1 AS x) t486 ON t1.x = t486.x JOIN (SELECT 1 AS x) t487 ON t1.x = t487.x JOIN (SELECT 1 AS x) t488 ON t1.x = t488.x JOIN (SELECT 1 AS x) t489 ON t1.x = t489.x JOIN (SELECT 1 AS x) t490 ON t1.x = t490.x JOIN (SELECT 1 AS x) t491 ON t1.x = t491.x JOIN (SELECT 1 AS x) t492 ON t1.x = t492.x JOIN (SELECT 1 AS x) t493 ON t1.x = t493.x JOIN (SELECT 1 AS x) t494 ON t1.x = t494.x JOIN (SELECT 1 AS x) t495 ON t1.x = t495.x JOIN (SELECT 1 AS x) t496 ON t1.x = t496.x JOIN (SELECT 1 AS x) t497 ON t1.x = t497.x JOIN (SELECT 1 AS x) t498 ON t1.x = t498.x JOIN (SELECT 1 AS x) t499 ON t1.x = t499.x JOIN (SELECT 1 AS x) t500 ON t1.x = t500.x JOIN (SELECT 1 AS x) t501 ON t1.x = t501.x JOIN (SELECT 1 AS x) t502 ON t1.x = t502.x JOIN (SELECT 1 AS x) t503 ON t1.x = t503.x JOIN (SELECT 1 AS x) t504 ON t1.x = t504.x JOIN (SELECT 1 AS x) t505 ON t1.x = t505.x JOIN (SELECT 1 AS x) t506 ON t1.x = t506.x JOIN (SELECT 1 AS x) t507 ON t1.x = t507.x JOIN (SELECT 1 AS x) t508 ON t1.x = t508.x JOIN (SELECT 1 AS x) t509 ON t1.x = t509.x JOIN (SELECT 1 AS x) t510 ON t1.x = t510.x JOIN (SELECT 1 AS x) t511 ON t1.x = t511.x JOIN (SELECT 1 AS x) t512 ON t1.x = t512.x JOIN (SELECT 1 AS x) t513 ON t1.x = t513.x JOIN (SELECT 1 AS x) t514 ON t1.x = t514.x JOIN (SELECT 1 AS x) t515 ON t1.x = t515.x JOIN (SELECT 1 AS x) t516 ON t1.x = t516.x JOIN (SELECT 1 AS x) t517 ON t1.x = t517.x JOIN (SELECT 1 AS x) t518 ON t1.x = t518.x JOIN (SELECT 1 AS x) t519 ON t1.x = t519.x JOIN (SELECT 1 AS x) t520 ON t1.x = t520.x JOIN (SELECT 1 AS x) t521 ON t1.x = t521.x JOIN (SELECT 1 AS x) t522 ON t1.x = t522.x JOIN (SELECT 1 AS x) t523 ON t1.x = t523.x JOIN (SELECT 1 AS x) t524 ON t1.x = t524.x JOIN (SELECT 1 AS x) t525 ON t1.x = t525.x JOIN (SELECT 1 AS x) t526 ON t1.x = t526.x JOIN (SELECT 1 AS x) t527 ON t1.x = t527.x JOIN (SELECT 1 AS x) t528 ON t1.x = t528.x JOIN (SELECT 1 AS x) t529 ON t1.x = t529.x JOIN (SELECT 1 AS x) t530 ON t1.x = t530.x JOIN (SELECT 1 AS x) t531 ON t1.x = t531.x JOIN (SELECT 1 AS x) t532 ON t1.x = t532.x JOIN (SELECT 1 AS x) t533 ON t1.x = t533.x JOIN (SELECT 1 AS x) t534 ON t1.x = t534.x JOIN (SELECT 1 AS x) t535 ON t1.x = t535.x JOIN (SELECT 1 AS x) t536 ON t1.x = t536.x JOIN (SELECT 1 AS x) t537 ON t1.x = t537.x JOIN (SELECT 1 AS x) t538 ON t1.x = t538.x JOIN (SELECT 1 AS x) t539 ON t1.x = t539.x JOIN (SELECT 1 AS x) t540 ON t1.x = t540.x JOIN (SELECT 1 AS x) t541 ON t1.x = t541.x JOIN (SELECT 1 AS x) t542 ON t1.x = t542.x JOIN (SELECT 1 AS x) t543 ON t1.x = t543.x JOIN (SELECT 1 AS x) t544 ON t1.x = t544.x JOIN (SELECT 1 AS x) t545 ON t1.x = t545.x JOIN (SELECT 1 AS x) t546 ON t1.x = t546.x JOIN (SELECT 1 AS x) t547 ON t1.x = t547.x JOIN (SELECT 1 AS x) t548 ON t1.x = t548.x JOIN (SELECT 1 AS x) t549 ON t1.x = t549.x JOIN (SELECT 1 AS x) t550 ON t1.x = t550.x JOIN (SELECT 1 AS x) t551 ON t1.x = t551.x JOIN (SELECT 1 AS x) t552 ON t1.x = t552.x JOIN (SELECT 1 AS x) t553 ON t1.x = t553.x JOIN (SELECT 1 AS x) t554 ON t1.x = t554.x JOIN (SELECT 1 AS x) t555 ON t1.x = t555.x JOIN (SELECT 1 AS x) t556 ON t1.x = t556.x JOIN (SELECT 1 AS x) t557 ON t1.x = t557.x JOIN (SELECT 1 AS x) t558 ON t1.x = t558.x JOIN (SELECT 1 AS x) t559 ON t1.x = t559.x JOIN (SELECT 1 AS x) t560 ON t1.x = t560.x JOIN (SELECT 1 AS x) t561 ON t1.x = t561.x JOIN (SELECT 1 AS x) t562 ON t1.x = t562.x JOIN (SELECT 1 AS x) t563 ON t1.x = t563.x JOIN (SELECT 1 AS x) t564 ON t1.x = t564.x JOIN (SELECT 1 AS x) t565 ON t1.x = t565.x JOIN (SELECT 1 AS x) t566 ON t1.x = t566.x JOIN (SELECT 1 AS x) t567 ON t1.x = t567.x JOIN (SELECT 1 AS x) t568 ON t1.x = t568.x JOIN (SELECT 1 AS x) t569 ON t1.x = t569.x JOIN (SELECT 1 AS x) t570 ON t1.x = t570.x JOIN (SELECT 1 AS x) t571 ON t1.x = t571.x JOIN (SELECT 1 AS x) t572 ON t1.x = t572.x JOIN (SELECT 1 AS x) t573 ON t1.x = t573.x JOIN (SELECT 1 AS x) t574 ON t1.x = t574.x JOIN (SELECT 1 AS x) t575 ON t1.x = t575.x JOIN (SELECT 1 AS x) t576 ON t1.x = t576.x JOIN (SELECT 1 AS x) t577 ON t1.x = t577.x JOIN (SELECT 1 AS x) t578 ON t1.x = t578.x JOIN (SELECT 1 AS x) t579 ON t1.x = t579.x JOIN (SELECT 1 AS x) t580 ON t1.x = t580.x JOIN (SELECT 1 AS x) t581 ON t1.x = t581.x JOIN (SELECT 1 AS x) t582 ON t1.x = t582.x JOIN (SELECT 1 AS x) t583 ON t1.x = t583.x JOIN (SELECT 1 AS x) t584 ON t1.x = t584.x JOIN (SELECT 1 AS x) t585 ON t1.x = t585.x JOIN (SELECT 1 AS x) t586 ON t1.x = t586.x JOIN (SELECT 1 AS x) t587 ON t1.x = t587.x JOIN (SELECT 1 AS x) t588 ON t1.x = t588.x JOIN (SELECT 1 AS x) t589 ON t1.x = t589.x JOIN (SELECT 1 AS x) t590 ON t1.x = t590.x JOIN (SELECT 1 AS x) t591 ON t1.x = t591.x JOIN (SELECT 1 AS x) t592 ON t1.x = t592.x JOIN (SELECT 1 AS x) t593 ON t1.x = t593.x JOIN (SELECT 1 AS x) t594 ON t1.x = t594.x JOIN (SELECT 1 AS x) t595 ON t1.x = t595.x JOIN (SELECT 1 AS x) t596 ON t1.x = t596.x JOIN (SELECT 1 AS x) t597 ON t1.x = t597.x JOIN (SELECT 1 AS x) t598 ON t1.x = t598.x JOIN (SELECT 1 AS x) t599 ON t1.x = t599.x JOIN (SELECT 1 AS x) t600 ON t1.x = t600.x JOIN (SELECT 1 AS x) t601 ON t1.x = t601.x JOIN (SELECT 1 AS x) t602 ON t1.x = t602.x JOIN (SELECT 1 AS x) t603 ON t1.x = t603.x JOIN (SELECT 1 AS x) t604 ON t1.x = t604.x JOIN (SELECT 1 AS x) t605 ON t1.x = t605.x JOIN (SELECT 1 AS x) t606 ON t1.x = t606.x JOIN (SELECT 1 AS x) t607 ON t1.x = t607.x JOIN (SELECT 1 AS x) t608 ON t1.x = t608.x JOIN (SELECT 1 AS x) t609 ON t1.x = t609.x JOIN (SELECT 1 AS x) t610 ON t1.x = t610.x JOIN (SELECT 1 AS x) t611 ON t1.x = t611.x JOIN (SELECT 1 AS x) t612 ON t1.x = t612.x JOIN (SELECT 1 AS x) t613 ON t1.x = t613.x JOIN (SELECT 1 AS x) t614 ON t1.x = t614.x JOIN (SELECT 1 AS x) t615 ON t1.x = t615.x JOIN (SELECT 1 AS x) t616 ON t1.x = t616.x JOIN (SELECT 1 AS x) t617 ON t1.x = t617.x JOIN (SELECT 1 AS x) t618 ON t1.x = t618.x JOIN (SELECT 1 AS x) t619 ON t1.x = t619.x JOIN (SELECT 1 AS x) t620 ON t1.x = t620.x JOIN (SELECT 1 AS x) t621 ON t1.x = t621.x JOIN (SELECT 1 AS x) t622 ON t1.x = t622.x JOIN (SELECT 1 AS x) t623 ON t1.x = t623.x JOIN (SELECT 1 AS x) t624 ON t1.x = t624.x JOIN (SELECT 1 AS x) t625 ON t1.x = t625.x JOIN (SELECT 1 AS x) t626 ON t1.x = t626.x JOIN (SELECT 1 AS x) t627 ON t1.x = t627.x JOIN (SELECT 1 AS x) t628 ON t1.x = t628.x JOIN (SELECT 1 AS x) t629 ON t1.x = t629.x JOIN (SELECT 1 AS x) t630 ON t1.x = t630.x JOIN (SELECT 1 AS x) t631 ON t1.x = t631.x JOIN (SELECT 1 AS x) t632 ON t1.x = t632.x JOIN (SELECT 1 AS x) t633 ON t1.x = t633.x JOIN (SELECT 1 AS x) t634 ON t1.x = t634.x JOIN (SELECT 1 AS x) t635 ON t1.x = t635.x JOIN (SELECT 1 AS x) t636 ON t1.x = t636.x JOIN (SELECT 1 AS x) t637 ON t1.x = t637.x JOIN (SELECT 1 AS x) t638 ON t1.x = t638.x JOIN (SELECT 1 AS x) t639 ON t1.x = t639.x JOIN (SELECT 1 AS x) t640 ON t1.x = t640.x JOIN (SELECT 1 AS x) t641 ON t1.x = t641.x JOIN (SELECT 1 AS x) t642 ON t1.x = t642.x JOIN (SELECT 1 AS x) t643 ON t1.x = t643.x JOIN (SELECT 1 AS x) t644 ON t1.x = t644.x JOIN (SELECT 1 AS x) t645 ON t1.x = t645.x JOIN (SELECT 1 AS x) t646 ON t1.x = t646.x JOIN (SELECT 1 AS x) t647 ON t1.x = t647.x JOIN (SELECT 1 AS x) t648 ON t1.x = t648.x JOIN (SELECT 1 AS x) t649 ON t1.x = t649.x JOIN (SELECT 1 AS x) t650 ON t1.x = t650.x JOIN (SELECT 1 AS x) t651 ON t1.x = t651.x JOIN (SELECT 1 AS x) t652 ON t1.x = t652.x JOIN (SELECT 1 AS x) t653 ON t1.x = t653.x JOIN (SELECT 1 AS x) t654 ON t1.x = t654.x JOIN (SELECT 1 AS x) t655 ON t1.x = t655.x JOIN (SELECT 1 AS x) t656 ON t1.x = t656.x JOIN (SELECT 1 AS x) t657 ON t1.x = t657.x JOIN (SELECT 1 AS x) t658 ON t1.x = t658.x JOIN (SELECT 1 AS x) t659 ON t1.x = t659.x JOIN (SELECT 1 AS x) t660 ON t1.x = t660.x JOIN (SELECT 1 AS x) t661 ON t1.x = t661.x JOIN (SELECT 1 AS x) t662 ON t1.x = t662.x JOIN (SELECT 1 AS x) t663 ON t1.x = t663.x JOIN (SELECT 1 AS x) t664 ON t1.x = t664.x JOIN (SELECT 1 AS x) t665 ON t1.x = t665.x JOIN (SELECT 1 AS x) t666 ON t1.x = t666.x From 8d4d5a84f2c0ba9286c030bad8ba879d6e8553a0 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 18 Apr 2024 15:12:11 +0200 Subject: [PATCH 0215/1018] fixup! Fix test_grant_and_revoke/test.py::test_grant_all_on_table with new entry --- tests/integration/test_grant_and_revoke/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_grant_and_revoke/test.py b/tests/integration/test_grant_and_revoke/test.py index 18fbab1a0a1..a7beb97f113 100644 --- a/tests/integration/test_grant_and_revoke/test.py +++ b/tests/integration/test_grant_and_revoke/test.py @@ -186,7 +186,7 @@ def test_grant_all_on_table(): instance.query("GRANT ALL ON test.table TO B", user="A") assert ( instance.query("SHOW GRANTS FOR B") - == "GRANT SHOW TABLES, SHOW COLUMNS, SHOW DICTIONARIES, SELECT, INSERT, ALTER TABLE, ALTER VIEW, CREATE TABLE, CREATE VIEW, CREATE DICTIONARY, DROP TABLE, DROP VIEW, DROP DICTIONARY, UNDROP TABLE, TRUNCATE, OPTIMIZE, BACKUP, CREATE ROW POLICY, ALTER ROW POLICY, DROP ROW POLICY, SHOW ROW POLICIES, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, SYSTEM MOVES, SYSTEM PULLING REPLICATION LOG, SYSTEM CLEANUP, SYSTEM VIEWS, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM VIRTUAL PARTS UPDATE, SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM FLUSH DISTRIBUTED, SYSTEM UNLOAD PRIMARY KEY, dictGet ON test.`table` TO B" + == "GRANT SHOW TABLES, SHOW COLUMNS, SHOW DICTIONARIES, SELECT, INSERT, ALTER TABLE, ALTER VIEW, CREATE TABLE, CREATE VIEW, CREATE DICTIONARY, DROP TABLE, DROP VIEW, DROP DICTIONARY, UNDROP TABLE, TRUNCATE, OPTIMIZE, BACKUP, CREATE ROW POLICY, ALTER ROW POLICY, DROP ROW POLICY, SHOW ROW POLICIES, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, SYSTEM MOVES, SYSTEM PULLING REPLICATION LOG, SYSTEM CLEANUP, SYSTEM VIEWS, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM VIRTUAL PARTS UPDATE, SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM FLUSH DISTRIBUTED, SYSTEM UNLOAD PRIMARY KEY, dictGet ON test.`table` TO B\n" ) instance.query("REVOKE ALL ON test.table FROM B", user="A") assert instance.query("SHOW GRANTS FOR B") == "" From 25cfd48f87317992d9aeb678182619935a354436 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Thu, 18 Apr 2024 14:39:23 +0100 Subject: [PATCH 0216/1018] Only log if it took longer than 100 ms --- src/Server/TCPHandler.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 4553d1992b2..7bcce58f3c9 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1246,8 +1246,10 @@ void TCPHandler::sendProfileEvents() state.profile_events_block_out->write(block); out->next(); - LOG_TRACE(log, "Sending profile events block with {} rows, {} bytes took {} milliseconds", - block.rows(), block.bytes(), stopwatch.elapsedMilliseconds()); + auto elapsed_milliseconds = stopwatch.elapsedMilliseconds(); + if (elapsed_milliseconds > 100) + LOG_DEBUG(log, "Sending profile events block with {} rows, {} bytes took {} milliseconds", + block.rows(), block.bytes(), elapsed_milliseconds); } } From 9048098107bc27900e5bb9e9b3b932f3c1743343 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 18 Apr 2024 18:02:53 +0200 Subject: [PATCH 0217/1018] Simplify expectations in tests to avoid flakyness The CI is injecting random values to detect flaky tests. Whenever the index_granularity was changed, the result did not have the right expectations. --- contrib/azure | 2 +- .../0_stateless/03127_system_unload_primary_key.reference | 8 ++++---- .../0_stateless/03127_system_unload_primary_key.sql | 6 +++--- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/contrib/azure b/contrib/azure index b90fd3c6ef3..ad2d3d42356 160000 --- a/contrib/azure +++ b/contrib/azure @@ -1 +1 @@ -Subproject commit b90fd3c6ef3185f5be3408056567bca0854129b6 +Subproject commit ad2d3d423565b8a8e7b7ec863eae9318a8283878 diff --git a/tests/queries/0_stateless/03127_system_unload_primary_key.reference b/tests/queries/0_stateless/03127_system_unload_primary_key.reference index 8b352f9c80e..92315c2c431 100644 --- a/tests/queries/0_stateless/03127_system_unload_primary_key.reference +++ b/tests/queries/0_stateless/03127_system_unload_primary_key.reference @@ -1,9 +1,9 @@ -100000000 140000000 +1 1 0 0 0 0 -100000000 140000000 +1 1 1 -100000000 100000000 -100000000 140000000 +1 1 +1 1 0 0 0 0 diff --git a/tests/queries/0_stateless/03127_system_unload_primary_key.sql b/tests/queries/0_stateless/03127_system_unload_primary_key.sql index 811d62b1d48..f687b27f45f 100644 --- a/tests/queries/0_stateless/03127_system_unload_primary_key.sql +++ b/tests/queries/0_stateless/03127_system_unload_primary_key.sql @@ -3,7 +3,7 @@ CREATE TABLE test (s String) ENGINE = MergeTree ORDER BY s SETTINGS index_granul INSERT INTO test SELECT randomString(1000) FROM numbers(100000); -SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) FROM system.parts WHERE database = currentDatabase() AND table = 'test'; +SELECT primary_key_bytes_in_memory > 0, primary_key_bytes_in_memory_allocated > 0 FROM system.parts WHERE database = currentDatabase() AND table = 'test'; SYSTEM UNLOAD PRIMARY KEY {CLICKHOUSE_DATABASE:Identifier}.test; @@ -14,11 +14,11 @@ CREATE TABLE test2 (s String) ENGINE = MergeTree ORDER BY s SETTINGS index_granu INSERT INTO test2 SELECT randomString(1000) FROM numbers(100000); -SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) FROM system.parts WHERE database = currentDatabase() AND table IN ('test', 'test2'); +SELECT primary_key_bytes_in_memory > 0, primary_key_bytes_in_memory_allocated > 0 FROM system.parts WHERE database = currentDatabase() AND table IN ('test', 'test2'); SELECT s != '' FROM test LIMIT 1; -SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) FROM system.parts WHERE database = currentDatabase() AND table IN ('test', 'test2'); +SELECT primary_key_bytes_in_memory > 0, primary_key_bytes_in_memory_allocated > 0 FROM system.parts WHERE database = currentDatabase() AND table IN ('test', 'test2'); SYSTEM UNLOAD PRIMARY KEY; From f95894c66c7e55b5361f702b5477c48e18db255e Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 18 Apr 2024 18:15:01 +0200 Subject: [PATCH 0218/1018] Fix AccessRights.Union unit test after merge --- src/Access/tests/gtest_access_rights_ops.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Access/tests/gtest_access_rights_ops.cpp b/src/Access/tests/gtest_access_rights_ops.cpp index 7384a741699..9fe9a3d8506 100644 --- a/src/Access/tests/gtest_access_rights_ops.cpp +++ b/src/Access/tests/gtest_access_rights_ops.cpp @@ -54,7 +54,7 @@ TEST(AccessRights, Union) "SYSTEM MOVES, SYSTEM PULLING REPLICATION LOG, SYSTEM CLEANUP, SYSTEM VIEWS, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM VIRTUAL PARTS UPDATE, " "SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, " "SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM SYNC DATABASE REPLICA, SYSTEM FLUSH DISTRIBUTED, " - "SYSTEM UNLOAD PRIMARY KEY, dictGet ON db1.*, " + "SYSTEM UNLOAD PRIMARY KEY, dictGet ON db1.*, GRANT TABLE ENGINE ON db1, " "GRANT SET DEFINER ON db1, GRANT NAMED COLLECTION ADMIN ON db1"); } From 4cab58d4afa7bfc19b0c5c329d495fc8ef13b1c9 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 18 Apr 2024 18:21:27 +0200 Subject: [PATCH 0219/1018] Use master's azure submodule version --- contrib/azure | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/azure b/contrib/azure index ad2d3d42356..b90fd3c6ef3 160000 --- a/contrib/azure +++ b/contrib/azure @@ -1 +1 @@ -Subproject commit ad2d3d423565b8a8e7b7ec863eae9318a8283878 +Subproject commit b90fd3c6ef3185f5be3408056567bca0854129b6 From 613dd1a6c040c1399a9027652d73321c71fbeeb7 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 18 Apr 2024 18:26:07 +0200 Subject: [PATCH 0220/1018] Updated to use workload identity for backups & disk --- .../registerBackupEngineAzureBlobStorage.cpp | 6 +++ .../AzureBlobStorage/AzureBlobStorageAuth.cpp | 13 ++++--- src/Storages/StorageAzureBlob.cpp | 38 ++++++++++++++++--- 3 files changed, 46 insertions(+), 11 deletions(-) diff --git a/src/Backups/registerBackupEngineAzureBlobStorage.cpp b/src/Backups/registerBackupEngineAzureBlobStorage.cpp index dbe5b555c31..1b9545fc455 100644 --- a/src/Backups/registerBackupEngineAzureBlobStorage.cpp +++ b/src/Backups/registerBackupEngineAzureBlobStorage.cpp @@ -72,6 +72,12 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) configuration.container = config.getString(config_prefix + ".container"); configuration.account_name = config.getString(config_prefix + ".account_name"); configuration.account_key = config.getString(config_prefix + ".account_key"); + + if (config.has(config_prefix + ".account_name") && config.has(config_prefix + ".account_key")) + { + configuration.account_name = config.getString(config_prefix + ".account_name"); + configuration.account_key = config.getString(config_prefix + ".account_key"); + } } if (args.size() > 1) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp index 3ce2c57d46e..ea8018f358d 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -161,12 +162,6 @@ template std::unique_ptr getAzureBlobStorageClientWithAuth( const String & url, const String & container_name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix) { - if (config.has(config_prefix + ".use_workload_identity_for_azure")) - { - auto workload_identity_credential = std::make_shared(); - return std::make_unique(url, workload_identity_credential); - } - std::string connection_str; if (config.has(config_prefix + ".connection_string")) connection_str = config.getString(config_prefix + ".connection_string"); @@ -183,6 +178,12 @@ std::unique_ptr getAzureBlobStorageClientWithAuth( return std::make_unique(url, storage_shared_key_credential); } + if (config.getBool(config_prefix + ".use_workload_identity", false)) + { + auto workload_identity_credential = std::make_shared(); + return std::make_unique(url, workload_identity_credential); + } + auto managed_identity_credential = std::make_shared(); return std::make_unique(url, managed_identity_credential); } diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index c39e3e7339f..f2e2833dad4 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -17,6 +17,7 @@ #include #include +#include #include #include #include @@ -383,6 +384,7 @@ AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration co } std::unique_ptr blob_service_client; + size_t pos = configuration.connection_url.find('?'); std::shared_ptr managed_identity_credential; if (storage_shared_key_credential) { @@ -390,12 +392,20 @@ AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration co } else { - managed_identity_credential = std::make_shared(); - blob_service_client = std::make_unique(configuration.connection_url, managed_identity_credential); + /// If conneciton_url does not have '?', then its not SAS + if (pos == std::string::npos) + { + auto workload_identity_credential = std::make_shared(); + blob_service_client = std::make_unique(configuration.connection_url, workload_identity_credential); + } + else + { + managed_identity_credential = std::make_shared(); + blob_service_client = std::make_unique(configuration.connection_url, managed_identity_credential); + } } std::string final_url; - size_t pos = configuration.connection_url.find('?'); if (pos != std::string::npos) { auto url_without_sas = configuration.connection_url.substr(0, pos); @@ -420,7 +430,16 @@ 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, managed_identity_credential); + { + /// If conneciton_url does not have '?', then its not SAS + if (pos == std::string::npos) + { + auto workload_identity_credential = std::make_shared(); + result = std::make_unique(final_url, workload_identity_credential); + } + else + result = std::make_unique(final_url, managed_identity_credential); + } } else { @@ -441,7 +460,16 @@ 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, managed_identity_credential); + { + /// If conneciton_url does not have '?', then its not SAS + if (pos == std::string::npos) + { + auto workload_identity_credential = std::make_shared(); + result = std::make_unique(final_url, workload_identity_credential); + } + else + result = std::make_unique(final_url, managed_identity_credential); + } } else { From a2fc46ad7cc74770585140e512a1eb1d2ad1f390 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 18 Apr 2024 17:50:18 +0100 Subject: [PATCH 0221/1018] More logging --- src/Storages/RabbitMQ/RabbitMQConsumer.cpp | 4 +++- src/Storages/RabbitMQ/RabbitMQSource.cpp | 2 +- src/Storages/RabbitMQ/StorageRabbitMQ.h | 3 ++- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQConsumer.cpp b/src/Storages/RabbitMQ/RabbitMQConsumer.cpp index c025fee8808..945e0124420 100644 --- a/src/Storages/RabbitMQ/RabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/RabbitMQConsumer.cpp @@ -200,7 +200,9 @@ void RabbitMQConsumer::updateChannel(RabbitMQConnection & connection) consumer_channel->onError([&](const char * message) { - LOG_ERROR(log, "Channel {} received an error: {}", channel_id, message); + LOG_ERROR( + log, "Channel {} received an error: {} (usable: {}, connected: {})", + channel_id, message, consumer_channel->usable(), consumer_channel->connected()); if (!consumer_channel->usable() || !consumer_channel->connected()) { diff --git a/src/Storages/RabbitMQ/RabbitMQSource.cpp b/src/Storages/RabbitMQ/RabbitMQSource.cpp index b69a533fd92..e41edb60a63 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSource.cpp @@ -84,7 +84,7 @@ RabbitMQSource::RabbitMQSource( , nack_broken_messages(nack_broken_messages_) , non_virtual_header(std::move(headers.first)) , virtual_header(std::move(headers.second)) - , log(getLogger("RabbitMQSource")) + , log(storage.log) , max_execution_time_ms(max_execution_time_) { storage.incrementReader(); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index afef2920b07..0350cd05af7 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -77,6 +77,8 @@ public: void incrementReader(); void decrementReader(); + LoggerPtr log; + private: ContextMutablePtr rabbitmq_context; std::unique_ptr rabbitmq_settings; @@ -102,7 +104,6 @@ private: bool use_user_setup; bool hash_exchange; - LoggerPtr log; RabbitMQConnectionPtr connection; /// Connection for all consumers RabbitMQConfiguration configuration; From 520aa02059bae3393ca1602666eb465bea7703a7 Mon Sep 17 00:00:00 2001 From: Eliot Hautefeuille Date: Thu, 18 Apr 2024 23:40:27 +0200 Subject: [PATCH 0222/1018] fix: update: nested materialized columns: size check fixes --- src/Interpreters/MutationsInterpreter.cpp | 10 ++++- ...nested_materialized_column_check.reference | 3 ++ ...pdate_nested_materialized_column_check.sql | 44 +++++++++++++++++++ 3 files changed, 56 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03129_update_nested_materialized_column_check.reference create mode 100644 tests/queries/0_stateless/03129_update_nested_materialized_column_check.sql diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 35fd549559b..65379a82445 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -508,6 +508,7 @@ static void validateUpdateColumns( /// because their sizes couldn't change, since sizes of all nested subcolumns must be consistent. static std::optional> getExpressionsOfUpdatedNestedSubcolumns( const String & column_name, + NameSet affected_materialized, const NamesAndTypesList & all_columns, const std::unordered_map & column_to_update_expression) { @@ -520,6 +521,10 @@ static std::optional> getExpressionsOfUpdatedNestedSubcolumn auto split = Nested::splitName(column.name); if (isArray(column.type) && split.first == source_name && !split.second.empty()) { + // Materialized nested columns shall never be part of the update expression + if (affected_materialized.contains(column.name)) + continue ; + auto it = column_to_update_expression.find(column.name); if (it == column_to_update_expression.end()) return {}; @@ -655,7 +660,10 @@ void MutationsInterpreter::prepare(bool dry_run) if (materialized_it != column_to_affected_materialized.end()) for (const auto & mat_column : materialized_it->second) affected_materialized.emplace(mat_column); + } + for (const auto & [column_name, update_expr] : command.column_to_update_expression) + { /// When doing UPDATE column = expression WHERE condition /// we will replace column to the result of the following expression: /// @@ -689,7 +697,7 @@ void MutationsInterpreter::prepare(bool dry_run) { std::shared_ptr function = nullptr; - auto nested_update_exprs = getExpressionsOfUpdatedNestedSubcolumns(column_name, all_columns, command.column_to_update_expression); + auto nested_update_exprs = getExpressionsOfUpdatedNestedSubcolumns(column_name, affected_materialized, all_columns, command.column_to_update_expression); if (!nested_update_exprs) { function = makeASTFunction("validateNestedArraySizes", diff --git a/tests/queries/0_stateless/03129_update_nested_materialized_column_check.reference b/tests/queries/0_stateless/03129_update_nested_materialized_column_check.reference new file mode 100644 index 00000000000..5aac4321953 --- /dev/null +++ b/tests/queries/0_stateless/03129_update_nested_materialized_column_check.reference @@ -0,0 +1,3 @@ +5555 ['moto','hello'] ['chocolatine','croissant'] [3159487597665552601,10142490492830962361] [17245759883905314919,3957103312270590367] +5555 ['hello'] ['croissant'] [10142490492830962361] [3957103312270590367] +5555 ['hello'] ['croissant au chocolat'] [10142490492830962361] [6230099869648002788] diff --git a/tests/queries/0_stateless/03129_update_nested_materialized_column_check.sql b/tests/queries/0_stateless/03129_update_nested_materialized_column_check.sql new file mode 100644 index 00000000000..210b49b78d4 --- /dev/null +++ b/tests/queries/0_stateless/03129_update_nested_materialized_column_check.sql @@ -0,0 +1,44 @@ +SET asterisk_include_materialized_columns = 1 ; + +CREATE TABLE elements +( + `id` UInt32, + `nested.key` Array(String), + `nested.value` Array(String), + `nested.key_hashed` Array(UInt64) MATERIALIZED arrayMap(x -> sipHash64(x), nested.key), + `nested.val_hashed` Array(UInt64) MATERIALIZED arrayMap(x -> sipHash64(x), nested.value), +) + ENGINE = Memory ; + + +INSERT INTO elements (id,`nested.key`,`nested.value`) VALUES (5555, ['moto', 'hello'],['chocolatine', 'croissant']); + +SELECT * FROM elements ; + +ALTER TABLE elements +UPDATE + + `nested.key` = arrayFilter( + (c, k, v) -> NOT (match(k, '.*') AND match(v, 'chocolatine')), + `nested.key`, `nested.key`, `nested.value` + ), + + `nested.value` = arrayFilter( + (c, k, v) -> NOT (match(k, '.*') AND match(v, 'chocolatine')), + `nested.value`, `nested.key`, `nested.value` + ) + +WHERE id = 5555 AND 1=1 + SETTINGS mutations_sync = 1 ; + +SELECT * FROM elements ; + +ALTER TABLE elements +UPDATE + + `nested.value` = arrayMap(x -> concat(x, ' au chocolat'), `nested.value`) + +WHERE id = 5555 AND 1=1 + SETTINGS mutations_sync = 1 ; + +SELECT * FROM elements ; From 2bfe78455e94890355f6a3a5ec1d619935ae5689 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Fri, 19 Apr 2024 08:10:20 +0200 Subject: [PATCH 0223/1018] Revert "Simplify expectations in tests to avoid flakyness" This reverts commit 9048098107bc27900e5bb9e9b3b932f3c1743343. --- .../0_stateless/03127_system_unload_primary_key.reference | 8 ++++---- .../0_stateless/03127_system_unload_primary_key.sql | 6 +++--- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/03127_system_unload_primary_key.reference b/tests/queries/0_stateless/03127_system_unload_primary_key.reference index 92315c2c431..8b352f9c80e 100644 --- a/tests/queries/0_stateless/03127_system_unload_primary_key.reference +++ b/tests/queries/0_stateless/03127_system_unload_primary_key.reference @@ -1,9 +1,9 @@ -1 1 +100000000 140000000 0 0 0 0 -1 1 +100000000 140000000 1 -1 1 -1 1 +100000000 100000000 +100000000 140000000 0 0 0 0 diff --git a/tests/queries/0_stateless/03127_system_unload_primary_key.sql b/tests/queries/0_stateless/03127_system_unload_primary_key.sql index f687b27f45f..811d62b1d48 100644 --- a/tests/queries/0_stateless/03127_system_unload_primary_key.sql +++ b/tests/queries/0_stateless/03127_system_unload_primary_key.sql @@ -3,7 +3,7 @@ CREATE TABLE test (s String) ENGINE = MergeTree ORDER BY s SETTINGS index_granul INSERT INTO test SELECT randomString(1000) FROM numbers(100000); -SELECT primary_key_bytes_in_memory > 0, primary_key_bytes_in_memory_allocated > 0 FROM system.parts WHERE database = currentDatabase() AND table = 'test'; +SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) FROM system.parts WHERE database = currentDatabase() AND table = 'test'; SYSTEM UNLOAD PRIMARY KEY {CLICKHOUSE_DATABASE:Identifier}.test; @@ -14,11 +14,11 @@ CREATE TABLE test2 (s String) ENGINE = MergeTree ORDER BY s SETTINGS index_granu INSERT INTO test2 SELECT randomString(1000) FROM numbers(100000); -SELECT primary_key_bytes_in_memory > 0, primary_key_bytes_in_memory_allocated > 0 FROM system.parts WHERE database = currentDatabase() AND table IN ('test', 'test2'); +SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) FROM system.parts WHERE database = currentDatabase() AND table IN ('test', 'test2'); SELECT s != '' FROM test LIMIT 1; -SELECT primary_key_bytes_in_memory > 0, primary_key_bytes_in_memory_allocated > 0 FROM system.parts WHERE database = currentDatabase() AND table IN ('test', 'test2'); +SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) FROM system.parts WHERE database = currentDatabase() AND table IN ('test', 'test2'); SYSTEM UNLOAD PRIMARY KEY; From 71bcc46e8c2bfec81d6316dfa746b7fda0ff4a82 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Fri, 19 Apr 2024 08:30:16 +0200 Subject: [PATCH 0224/1018] Fix tests to run sequentially the one that unloads all partition keys --- ...system_unload_primary_key_table.reference} | 10 +++++-- ...03127_system_unload_primary_key_table.sql} | 28 +++++++++---------- .../03128_system_unload_primary_key.reference | 4 +++ .../03128_system_unload_primary_key.sql | 15 ++++++++++ 4 files changed, 39 insertions(+), 18 deletions(-) rename tests/queries/0_stateless/{03127_system_unload_primary_key.reference => 03127_system_unload_primary_key_table.reference} (61%) rename tests/queries/0_stateless/{03127_system_unload_primary_key.sql => 03127_system_unload_primary_key_table.sql} (58%) create mode 100644 tests/queries/0_stateless/03128_system_unload_primary_key.reference create mode 100644 tests/queries/0_stateless/03128_system_unload_primary_key.sql diff --git a/tests/queries/0_stateless/03127_system_unload_primary_key.reference b/tests/queries/0_stateless/03127_system_unload_primary_key_table.reference similarity index 61% rename from tests/queries/0_stateless/03127_system_unload_primary_key.reference rename to tests/queries/0_stateless/03127_system_unload_primary_key_table.reference index 8b352f9c80e..3ac6127fb21 100644 --- a/tests/queries/0_stateless/03127_system_unload_primary_key.reference +++ b/tests/queries/0_stateless/03127_system_unload_primary_key_table.reference @@ -1,9 +1,13 @@ 100000000 140000000 +100000000 140000000 +100000000 140000000 +0 0 +100000000 140000000 0 0 0 0 -100000000 140000000 1 100000000 100000000 -100000000 140000000 -0 0 0 0 +1 +100000000 100000000 +100000000 100000000 diff --git a/tests/queries/0_stateless/03127_system_unload_primary_key.sql b/tests/queries/0_stateless/03127_system_unload_primary_key_table.sql similarity index 58% rename from tests/queries/0_stateless/03127_system_unload_primary_key.sql rename to tests/queries/0_stateless/03127_system_unload_primary_key_table.sql index 811d62b1d48..de99464a9e6 100644 --- a/tests/queries/0_stateless/03127_system_unload_primary_key.sql +++ b/tests/queries/0_stateless/03127_system_unload_primary_key_table.sql @@ -1,25 +1,23 @@ DROP TABLE IF EXISTS test; -CREATE TABLE test (s String) ENGINE = MergeTree ORDER BY s SETTINGS index_granularity = 1; - -INSERT INTO test SELECT randomString(1000) FROM numbers(100000); - -SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) FROM system.parts WHERE database = currentDatabase() AND table = 'test'; - -SYSTEM UNLOAD PRIMARY KEY {CLICKHOUSE_DATABASE:Identifier}.test; - -SELECT primary_key_bytes_in_memory, primary_key_bytes_in_memory_allocated FROM system.parts WHERE database = currentDatabase() AND table = 'test'; - DROP TABLE IF EXISTS test2; + +CREATE TABLE test (s String) ENGINE = MergeTree ORDER BY s SETTINGS index_granularity = 1; CREATE TABLE test2 (s String) ENGINE = MergeTree ORDER BY s SETTINGS index_granularity = 1; -INSERT INTO test2 SELECT randomString(1000) FROM numbers(100000); +INSERT INTO test SELECT randomString(1000) FROM numbers(100000); +SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) FROM system.parts WHERE database = currentDatabase() AND table IN ('test', 'test2'); +INSERT INTO test2 SELECT randomString(1000) FROM numbers(100000); +SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) FROM system.parts WHERE database = currentDatabase() AND table IN ('test', 'test2'); + +SYSTEM UNLOAD PRIMARY KEY {CLICKHOUSE_DATABASE:Identifier}.test; +SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) FROM system.parts WHERE database = currentDatabase() AND table IN ('test', 'test2'); + +SYSTEM UNLOAD PRIMARY KEY {CLICKHOUSE_DATABASE:Identifier}.test2; SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) FROM system.parts WHERE database = currentDatabase() AND table IN ('test', 'test2'); SELECT s != '' FROM test LIMIT 1; - SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) FROM system.parts WHERE database = currentDatabase() AND table IN ('test', 'test2'); -SYSTEM UNLOAD PRIMARY KEY; - -SELECT primary_key_bytes_in_memory, primary_key_bytes_in_memory_allocated FROM system.parts WHERE database = currentDatabase() AND table IN ('test', 'test2'); +SELECT s != '' FROM test2 LIMIT 1; +SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) FROM system.parts WHERE database = currentDatabase() AND table IN ('test', 'test2'); diff --git a/tests/queries/0_stateless/03128_system_unload_primary_key.reference b/tests/queries/0_stateless/03128_system_unload_primary_key.reference new file mode 100644 index 00000000000..c7b40ae5b06 --- /dev/null +++ b/tests/queries/0_stateless/03128_system_unload_primary_key.reference @@ -0,0 +1,4 @@ +100000000 140000000 +100000000 140000000 +0 0 +0 0 diff --git a/tests/queries/0_stateless/03128_system_unload_primary_key.sql b/tests/queries/0_stateless/03128_system_unload_primary_key.sql new file mode 100644 index 00000000000..907afb331c5 --- /dev/null +++ b/tests/queries/0_stateless/03128_system_unload_primary_key.sql @@ -0,0 +1,15 @@ +-- Tags: no-parallel +DROP TABLE IF EXISTS test; +DROP TABLE IF EXISTS test2; + +CREATE TABLE test (s String) ENGINE = MergeTree ORDER BY s SETTINGS index_granularity = 1; +CREATE TABLE test2 (s String) ENGINE = MergeTree ORDER BY s SETTINGS index_granularity = 1; + +INSERT INTO test SELECT randomString(1000) FROM numbers(100000); +INSERT INTO test2 SELECT randomString(1000) FROM numbers(100000); + +SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) FROM system.parts WHERE database = currentDatabase() AND table IN ('test', 'test2'); + +SYSTEM UNLOAD PRIMARY KEY; + +SELECT primary_key_bytes_in_memory, primary_key_bytes_in_memory_allocated FROM system.parts WHERE database = currentDatabase() AND table IN ('test', 'test2') \ No newline at end of file From af22bec7164beabc5b57967585e3e15d9df63719 Mon Sep 17 00:00:00 2001 From: Eliot Hautefeuille Date: Fri, 19 Apr 2024 11:03:30 +0200 Subject: [PATCH 0225/1018] fix: update: nested materialized columns: size check --- ...pdate_nested_materialized_column_check.sql | 23 +++++-------------- 1 file changed, 6 insertions(+), 17 deletions(-) diff --git a/tests/queries/0_stateless/03129_update_nested_materialized_column_check.sql b/tests/queries/0_stateless/03129_update_nested_materialized_column_check.sql index 210b49b78d4..85a0b9903ef 100644 --- a/tests/queries/0_stateless/03129_update_nested_materialized_column_check.sql +++ b/tests/queries/0_stateless/03129_update_nested_materialized_column_check.sql @@ -17,28 +17,17 @@ SELECT * FROM elements ; ALTER TABLE elements UPDATE - - `nested.key` = arrayFilter( - (c, k, v) -> NOT (match(k, '.*') AND match(v, 'chocolatine')), - `nested.key`, `nested.key`, `nested.value` - ), - - `nested.value` = arrayFilter( - (c, k, v) -> NOT (match(k, '.*') AND match(v, 'chocolatine')), - `nested.value`, `nested.key`, `nested.value` - ) - -WHERE id = 5555 AND 1=1 - SETTINGS mutations_sync = 1 ; + `nested.key` = arrayFilter((x, v) -> NOT (match(v, 'chocolatine')), `nested.key`, `nested.value` ), + `nested.value` = arrayFilter((x, v) -> NOT (match(v, 'chocolatine')), `nested.value`, `nested.value`) +WHERE id = 5555 +SETTINGS mutations_sync = 1 ; SELECT * FROM elements ; ALTER TABLE elements UPDATE - `nested.value` = arrayMap(x -> concat(x, ' au chocolat'), `nested.value`) - -WHERE id = 5555 AND 1=1 - SETTINGS mutations_sync = 1 ; +WHERE id = 5555 +SETTINGS mutations_sync = 1 ; SELECT * FROM elements ; From ff0376fde24c58ebd65ec0fa200db1036ebb2fcf Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 19 Apr 2024 12:30:13 +0200 Subject: [PATCH 0226/1018] Update azure --- contrib/azure | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/azure b/contrib/azure index e71395e44f3..b90fd3c6ef3 160000 --- a/contrib/azure +++ b/contrib/azure @@ -1 +1 @@ -Subproject commit e71395e44f309f97b5a486f5c2c59b82f85dd2d2 +Subproject commit b90fd3c6ef3185f5be3408056567bca0854129b6 From 4e38ccefdd2332eeb96fa7d56ce726f440cae3a3 Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Fri, 19 Apr 2024 13:34:26 -0700 Subject: [PATCH 0227/1018] [Docs] Add examples for FINAL --- .../sql-reference/statements/select/from.md | 40 +++++++++++++++---- 1 file changed, 33 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/statements/select/from.md b/docs/en/sql-reference/statements/select/from.md index 06742ff74e2..934dfd738ae 100644 --- a/docs/en/sql-reference/statements/select/from.md +++ b/docs/en/sql-reference/statements/select/from.md @@ -19,25 +19,51 @@ Subquery is another `SELECT` query that may be specified in parenthesis inside ` ## FINAL Modifier -When `FINAL` is specified, ClickHouse fully merges the data before returning the result and thus performs all data transformations that happen during merges for the given table engine. +When `FINAL` is specified, ClickHouse fully merges the data before returning the result. This also performs all data transformations that happen during merges for the given table engine. -It is applicable when selecting data from ReplacingMergeTree, SummingMergeTree, AggregatingMergeTree, CollapsingMergeTree and VersionedCollapsingMergeTree tables. +It is applicable when selecting data from from tables using the following table engines: +- `ReplacingMergeTree` +- `SummingMergeTree` +- `AggregatingMergeTree` +- `CollapsingMergeTree` +- `VersionedCollapsingMergeTree` `SELECT` queries with `FINAL` are executed in parallel. The [max_final_threads](../../../operations/settings/settings.md#max-final-threads) setting limits the number of threads used. -There are drawbacks to using `FINAL` (see below). - ### Drawbacks -Queries that use `FINAL` are executed slightly slower than similar queries that do not, because: +Queries that use `FINAL` execute slightly slower than similar queries that do not use `FINAL` because: - Data is merged during query execution. -- Queries with `FINAL` read primary key columns in addition to the columns specified in the query. +- Queries with `FINAL` may read primary key columns in addition to the columns specified in the query. -`FINAL` requires additional compute and memory resources, as the processing that normally would occur at merge time must occur in memory at the time of the query. However, using FINAL is sometimes necessary in order to produce accurate results, and is less expensive than running `OPTIMIZE` to force a merge. It is also sometimes possible to use different queries that assume the background processes of the `MergeTree` engine haven’t happened yet and deal with it by applying aggregation (for example, to discard duplicates). If you need to use FINAL in your queries in order to get the required results, then it is okay to do so but be aware of the additional processing required. +`FINAL` requires additional compute and memory resources because the processing that normally would occur at merge time must occur in memory at the time of the query. However, using FINAL is sometimes necessary in order to produce accurate results (as data may not yet be fully merged). It is less expensive than running `OPTIMIZE` to force a merge. + +As an alternative to using `FINAL`, it is sometimes possible to use different queries that assume the background processes of the `MergeTree` engine have not yet occurred and deal with it by applying an aggregation (for example, to discard duplicates). If you need to use `FINAL` in your queries in order to get the required results, it is okay to do so but be aware of the additional processing required. `FINAL` can be applied automatically using [FINAL](../../../operations/settings/settings.md#final) setting to all tables in a query using a session or a user profile. +### Example Usage + +**Using the `FINAL` keyword** + +```sql +SELECT x, y FROM mytable FINAL WHERE x > 1; +``` + +**Using `FINAL` as a query-level setting** + +```sql +SELECT x, y FROM mytable WHERE x > 1 SETTINGS final = 1; +``` + +**Using `FINAL` as a session-level setting** + +```sql +SET final = 1; +SELECT x, y FROM mytable WHERE x > 1 SETTINGS final = 1; +``` + ## Implementation Details If the `FROM` clause is omitted, data will be read from the `system.one` table. From 98fef9f0366bc16acafa8e77f422a2eb065fcd9f Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Fri, 19 Apr 2024 13:36:41 -0700 Subject: [PATCH 0228/1018] [Docs] Fix query --- docs/en/sql-reference/statements/select/from.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/select/from.md b/docs/en/sql-reference/statements/select/from.md index 934dfd738ae..7a6e2ab054c 100644 --- a/docs/en/sql-reference/statements/select/from.md +++ b/docs/en/sql-reference/statements/select/from.md @@ -61,7 +61,7 @@ SELECT x, y FROM mytable WHERE x > 1 SETTINGS final = 1; ```sql SET final = 1; -SELECT x, y FROM mytable WHERE x > 1 SETTINGS final = 1; +SELECT x, y FROM mytable WHERE x > 1; ``` ## Implementation Details From de5b0dab3272c554e61783491cfb630a634731af Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Sat, 20 Apr 2024 02:18:19 +0000 Subject: [PATCH 0229/1018] Fix FINAL modifier is not respected in CTE with analyzer Signed-off-by: Duc Canh Le --- src/Analyzer/IdentifierNode.cpp | 4 +++- .../03129_cte_with_final.reference | 24 +++++++++++++++++++ .../0_stateless/03129_cte_with_final.sql | 22 +++++++++++++++++ 3 files changed, 49 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03129_cte_with_final.reference create mode 100644 tests/queries/0_stateless/03129_cte_with_final.sql diff --git a/src/Analyzer/IdentifierNode.cpp b/src/Analyzer/IdentifierNode.cpp index 181e75a57fd..f17f3b3d305 100644 --- a/src/Analyzer/IdentifierNode.cpp +++ b/src/Analyzer/IdentifierNode.cpp @@ -56,7 +56,9 @@ void IdentifierNode::updateTreeHashImpl(HashState & state, CompareOptions) const QueryTreeNodePtr IdentifierNode::cloneImpl() const { - return std::make_shared(identifier); + auto clone_identifier_node = std::make_shared(identifier); + clone_identifier_node->table_expression_modifiers = table_expression_modifiers; + return clone_identifier_node; } ASTPtr IdentifierNode::toASTImpl(const ConvertToASTOptions & /* options */) const diff --git a/tests/queries/0_stateless/03129_cte_with_final.reference b/tests/queries/0_stateless/03129_cte_with_final.reference new file mode 100644 index 00000000000..af1053c2d47 --- /dev/null +++ b/tests/queries/0_stateless/03129_cte_with_final.reference @@ -0,0 +1,24 @@ +QUERY id: 0 + PROJECTION COLUMNS + key Int64 + someCol String + eventTime DateTime + PROJECTION + LIST id: 1, nodes: 3 + COLUMN id: 2, column_name: key, result_type: Int64, source_id: 3 + COLUMN id: 4, column_name: someCol, result_type: String, source_id: 3 + COLUMN id: 5, column_name: eventTime, result_type: DateTime, source_id: 3 + JOIN TREE + QUERY id: 3, alias: __table1, is_subquery: 1, is_cte: 1, cte_name: merged_test + PROJECTION COLUMNS + key Int64 + someCol String + eventTime DateTime + PROJECTION + LIST id: 6, nodes: 3 + COLUMN id: 7, column_name: key, result_type: Int64, source_id: 8 + COLUMN id: 9, column_name: someCol, result_type: String, source_id: 8 + COLUMN id: 10, column_name: eventTime, result_type: DateTime, source_id: 8 + JOIN TREE + TABLE id: 8, alias: __table2, table_name: default.test, final: 1 +1 first 2024-04-19 01:01:01 diff --git a/tests/queries/0_stateless/03129_cte_with_final.sql b/tests/queries/0_stateless/03129_cte_with_final.sql new file mode 100644 index 00000000000..fc211f54384 --- /dev/null +++ b/tests/queries/0_stateless/03129_cte_with_final.sql @@ -0,0 +1,22 @@ +CREATE OR REPLACE TABLE test +( + `key` Int64, + `someCol` String, + `eventTime` DateTime +) +ENGINE = ReplacingMergeTree(eventTime) +ORDER BY key; + +INSERT INTO test Values (1, 'first', '2024-04-19 01:01:01'); +INSERT INTO test Values (1, 'first', '2024-04-19 01:01:01'); + +EXPLAIN QUERY TREE passes=1 +WITH merged_test AS( + SELECT * FROM test Final +) +SELECT * FROM merged_test; + +WITH merged_test AS( + SELECT * FROM test Final +) +SELECT * FROM merged_test; From 5ebdcf06ca97543c9262c4271b48bfe3e0321062 Mon Sep 17 00:00:00 2001 From: Artur Malchanau Date: Sat, 20 Apr 2024 12:33:52 +0300 Subject: [PATCH 0230/1018] Fix taking full part if part contains less than 'limit' rows Do not take full part if part contains less than 'limit' rows. --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 6bdd060513c..662e833f090 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -987,7 +987,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( /// We take full part if it contains enough marks or /// if we know limit and part contains less than 'limit' rows. - bool take_full_part = marks_in_part <= need_marks || (input_order_info->limit && input_order_info->limit < part.getRowsCount()); + bool take_full_part = marks_in_part <= need_marks || (input_order_info->limit && input_order_info->limit > part.getRowsCount()); /// We take the whole part if it is small enough. if (take_full_part) From 0575d02b81de8f4efb59e9ddca88ac91833d96b9 Mon Sep 17 00:00:00 2001 From: Artur Malchanau Date: Sat, 20 Apr 2024 23:20:20 +0300 Subject: [PATCH 0231/1018] Fix 02499_monotonicity_toUnixTimestamp64 test Fix 02499_monotonicity_toUnixTimestamp64 test by using max_threads=1 to make read_rows more predictable. --- .../0_stateless/02499_monotonicity_toUnixTimestamp64.sh | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02499_monotonicity_toUnixTimestamp64.sh b/tests/queries/0_stateless/02499_monotonicity_toUnixTimestamp64.sh index 59b6e2abb06..aa1f635f380 100755 --- a/tests/queries/0_stateless/02499_monotonicity_toUnixTimestamp64.sh +++ b/tests/queries/0_stateless/02499_monotonicity_toUnixTimestamp64.sh @@ -13,8 +13,7 @@ $CLICKHOUSE_CLIENT -q "create table t(ts DateTime64) engine=MergeTree order by t max_block_size=8192 query_id="${CLICKHOUSE_DATABASE}_02499_$RANDOM$RANDOM" -$CLICKHOUSE_CLIENT --query_id="$query_id" -q "select ts from t order by toUnixTimestamp64Nano(ts) limit 10 format Null settings max_block_size = $max_block_size, optimize_read_in_order = 1;" +$CLICKHOUSE_CLIENT --query_id="$query_id" -q "select ts from t order by toUnixTimestamp64Nano(ts) limit 10 format Null settings max_block_size = $max_block_size, optimize_read_in_order = 1, max_threads = 1;" $CLICKHOUSE_CLIENT -q "system flush logs;" $CLICKHOUSE_CLIENT --param_query_id="$query_id" -q "select read_rows <= $max_block_size from system.query_log where event_date >= yesterday() and current_database = '$CLICKHOUSE_DATABASE' and query_id = {query_id:String} and type = 'QueryFinish';" - From 112fcaf21fe5f1f04287d9a437412545235df57d Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Sun, 21 Apr 2024 20:37:45 +0200 Subject: [PATCH 0232/1018] Update aspell-dict.txt Remove 'countSubstringsCaseInsensitiveUTF8' from list --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index ca67cef4711..def6fe2a2b7 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1372,7 +1372,6 @@ countMatches countMatchesCaseInsensitive countSubstrings countSubstringsCaseInsensitive -countSubstringsCaseInsensitiveUTF8 covarPop covarSamp covariates From 026027e51cce087c5135bc6d27e9c841c2adbff6 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Sun, 21 Apr 2024 20:47:11 +0200 Subject: [PATCH 0233/1018] Update aspell-dict.txt Add 'countSubstringsCaseInsensitiveUTF' --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index def6fe2a2b7..aca41b00ccb 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1372,6 +1372,7 @@ countMatches countMatchesCaseInsensitive countSubstrings countSubstringsCaseInsensitive +countSubstringsCaseInsensitiveUTF covarPop covarSamp covariates From 41a54daec982fe39dddaad67e327092fc9769642 Mon Sep 17 00:00:00 2001 From: Blargian Date: Sun, 21 Apr 2024 20:51:42 +0200 Subject: [PATCH 0234/1018] Fix incorrect formatting of existing examples --- .../functions/rounding-functions.md | 16 ++++++---------- 1 file changed, 6 insertions(+), 10 deletions(-) diff --git a/docs/en/sql-reference/functions/rounding-functions.md b/docs/en/sql-reference/functions/rounding-functions.md index 1ffe7807e4f..6695234a166 100644 --- a/docs/en/sql-reference/functions/rounding-functions.md +++ b/docs/en/sql-reference/functions/rounding-functions.md @@ -79,9 +79,9 @@ round(expression [, decimal_places]) The rounded number of the same type as the input number. -### Examples +**Examples** -**Example of use with Float** +Example of usage with Float: ``` sql SELECT number / 2 AS x, round(x) FROM system.numbers LIMIT 3; @@ -95,7 +95,7 @@ SELECT number / 2 AS x, round(x) FROM system.numbers LIMIT 3; └─────┴──────────────────────────┘ ``` -**Example of use with Decimal** +Example of usage with Decimal: ``` sql SELECT cast(number / 2 AS Decimal(10,4)) AS x, round(x) FROM system.numbers LIMIT 3; @@ -124,9 +124,7 @@ SELECT cast(number / 2 AS Decimal(10,4)) AS x, round(x) FROM system.numbers LIM └────────┴──────────────────────────────────────────────────┘ ``` -**Examples of rounding** - -Rounding to the nearest number. +Examples of rounding to the nearest number: ``` text round(3.2, 0) = 3 @@ -183,9 +181,7 @@ roundBankers(expression [, decimal_places]) A value rounded by the banker’s rounding method. -### Examples - -**Example of use** +**Examples** Query: @@ -210,7 +206,7 @@ Result: └─────┴───┘ ``` -**Examples of Banker’s rounding** +Examples of Banker’s rounding: ``` text roundBankers(0.4) = 0 From 2fea683c0560dfd993b14439c45d6120b3680aab Mon Sep 17 00:00:00 2001 From: Blargian Date: Sun, 21 Apr 2024 21:31:14 +0200 Subject: [PATCH 0235/1018] Add roundAge --- .../functions/rounding-functions.md | 51 +++++++++++++++---- 1 file changed, 42 insertions(+), 9 deletions(-) diff --git a/docs/en/sql-reference/functions/rounding-functions.md b/docs/en/sql-reference/functions/rounding-functions.md index 6695234a166..d079cab5cab 100644 --- a/docs/en/sql-reference/functions/rounding-functions.md +++ b/docs/en/sql-reference/functions/rounding-functions.md @@ -230,16 +230,49 @@ Accepts a number. If the number is less than one, it returns 0. Otherwise, it ro Accepts a number. If the number is less than one, it returns 0. Otherwise, it rounds the number down to numbers from the set: 1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000. -## roundAge(num) +## roundAge -Accepts a number. If the number is -- smaller than 1, it returns 0, -- between 1 and 17, it returns 17, -- between 18 and 24, it returns 18, -- between 25 and 34, it returns 25, -- between 35 and 44, it returns 35, -- between 45 and 54, it returns 45, -- larger than 55, it returns 55. +Accepts a number within various commonly used ranges of human age and returns either a maximum or a minimum within that range. + +**Syntax** + +```sql +roundAge(num) +``` + +**Parameters** + +- `age`: A number representing an age in years. [Numeric](../data-types/int-uint.md). + +**Returned value** + +- Returns `0`, for $age \lt 1$. +- Returns `17`, for $1 \leq age \leq 17$. +- Returns `18`, for $18 \leq age \leq 24$. +- Returns `25`, for $25 \leq age \leq 34$. +- Returns `35`, for $35 \leq age \leq 44$. +- Returns `45`, for $45 \leq age \leq 54$. +- Returns `55`, for $age \geq 55$. + +**Example** + +Query: + +```sql +SELECT *, roundAge(*) FROM system.numbers WHERE number IN (0, 5, 20, 31, 37, 54, 72); +``` + +```response +┌─number─┬─roundAge(number)─┐ +│ 0 │ 0 │ +│ 5 │ 17 │ +│ 20 │ 18 │ +│ 31 │ 25 │ +│ 37 │ 35 │ +│ 54 │ 45 │ +│ 72 │ 55 │ +└────────┴──────────────────┘ +``` ## roundDown(num, arr) From 2c986d7d17c1a0bcab9ff919fdee38342b2d4277 Mon Sep 17 00:00:00 2001 From: Blargian Date: Sun, 21 Apr 2024 21:43:04 +0200 Subject: [PATCH 0236/1018] Add roundDown --- .../functions/rounding-functions.md | 40 ++++++++++++++++++- 1 file changed, 39 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/rounding-functions.md b/docs/en/sql-reference/functions/rounding-functions.md index d079cab5cab..24a795b7d36 100644 --- a/docs/en/sql-reference/functions/rounding-functions.md +++ b/docs/en/sql-reference/functions/rounding-functions.md @@ -262,6 +262,8 @@ Query: SELECT *, roundAge(*) FROM system.numbers WHERE number IN (0, 5, 20, 31, 37, 54, 72); ``` +Result: + ```response ┌─number─┬─roundAge(number)─┐ │ 0 │ 0 │ @@ -274,6 +276,42 @@ SELECT *, roundAge(*) FROM system.numbers WHERE number IN (0, 5, 20, 31, 37, 54, └────────┴──────────────────┘ ``` -## roundDown(num, arr) +## roundDown Accepts a number and rounds it down to an element in the specified array. If the value is less than the lowest bound, the lowest bound is returned. + +**Syntax** + +```sql +roundDown(num, arr) +``` + +**Parameters** + +- `age`: A number representing an age in years. [Numeric](../data-types/int-uint.md). +- `arr`: Array of elements to round `age` down to. [Array](../data-types/array.md)([Numeric](../data-types/int-uint.md)) + +**Returned value** + +- Number rounded down to an element in `arr`. If the value is less than the lowest bound, the lowest bound is returned. [Numeric](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +SELECT *, roundDown(*, [3, 4, 5]) FROM system.numbers WHERE number IN (0, 1, 2, 3, 4, 5) +``` + +Result: + +```response +┌─number─┬─roundDown(number, [3, 4, 5])─┐ +│ 0 │ 3 │ +│ 1 │ 3 │ +│ 2 │ 3 │ +│ 3 │ 3 │ +│ 4 │ 4 │ +│ 5 │ 5 │ +└────────┴──────────────────────────────┘ +``` From 9c6ff535cdadbc18daafa48dcc4cd743f6ac6a47 Mon Sep 17 00:00:00 2001 From: Blargian Date: Sun, 21 Apr 2024 22:01:13 +0200 Subject: [PATCH 0237/1018] Add roundDuration --- .../functions/rounding-functions.md | 48 ++++++++++++++++++- 1 file changed, 46 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/rounding-functions.md b/docs/en/sql-reference/functions/rounding-functions.md index 24a795b7d36..f90a35922b0 100644 --- a/docs/en/sql-reference/functions/rounding-functions.md +++ b/docs/en/sql-reference/functions/rounding-functions.md @@ -226,9 +226,53 @@ roundBankers(10.755, 2) = 10.76 Accepts a number. If the number is less than one, it returns 0. Otherwise, it rounds the number down to the nearest (whole non-negative) degree of two. -## roundDuration(num) +## roundDuration -Accepts a number. If the number is less than one, it returns 0. Otherwise, it rounds the number down to numbers from the set: 1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000. +Accepts a number. If the number is less than one, it returns `0`. Otherwise, it rounds the number down to numbers from the set: $\set{1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000}$. + +**Syntax** + +```sql +roundDuration(num) +``` + +**Parameters** + +- `age`: A number representing an age in years. [Numeric](../data-types/int-uint.md). + +**Returned value** + +- `0`, for `num` $\lt 1$. +- One of $\set{1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000}$, otherwise. [UInt8](../data-types/int-uint.md). + +Query: + +```sql +SELECT *, roundDuration(*) FROM system.numbers WHERE number IN (0, 9, 19, 47, 101, 149, 205, 271, 421, 789, 1423, 2345, 4567, 9876, 24680, 42573) +``` + +Result: + +```response +┌─number─┬─roundDuration(number)─┐ +│ 0 │ 0 │ +│ 9 │ 1 │ +│ 19 │ 10 │ +│ 47 │ 30 │ +│ 101 │ 60 │ +│ 149 │ 120 │ +│ 205 │ 180 │ +│ 271 │ 240 │ +│ 421 │ 300 │ +│ 789 │ 600 │ +│ 1423 │ 1200 │ +│ 2345 │ 1800 │ +│ 4567 │ 3600 │ +│ 9876 │ 7200 │ +│ 24680 │ 18000 │ +│ 42573 │ 36000 │ +└────────┴───────────────────────┘ +``` ## roundAge From 298206c0af5cd12c1cadfb620576a2ad71ae6857 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 22 Apr 2024 01:27:44 +0000 Subject: [PATCH 0238/1018] minor fix in test Signed-off-by: Duc Canh Le --- .../0_stateless/03129_cte_with_final.reference | 2 +- tests/queries/0_stateless/03129_cte_with_final.sql | 14 +++++++++----- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/03129_cte_with_final.reference b/tests/queries/0_stateless/03129_cte_with_final.reference index af1053c2d47..b80fa9ec780 100644 --- a/tests/queries/0_stateless/03129_cte_with_final.reference +++ b/tests/queries/0_stateless/03129_cte_with_final.reference @@ -20,5 +20,5 @@ QUERY id: 0 COLUMN id: 9, column_name: someCol, result_type: String, source_id: 8 COLUMN id: 10, column_name: eventTime, result_type: DateTime, source_id: 8 JOIN TREE - TABLE id: 8, alias: __table2, table_name: default.test, final: 1 + TABLE id: 8, alias: __table2, table_name: default.t, final: 1 1 first 2024-04-19 01:01:01 diff --git a/tests/queries/0_stateless/03129_cte_with_final.sql b/tests/queries/0_stateless/03129_cte_with_final.sql index fc211f54384..b53c85e77e1 100644 --- a/tests/queries/0_stateless/03129_cte_with_final.sql +++ b/tests/queries/0_stateless/03129_cte_with_final.sql @@ -1,4 +1,6 @@ -CREATE OR REPLACE TABLE test +DROP TABLE IF EXISTS t; + +CREATE TABLE t ( `key` Int64, `someCol` String, @@ -7,16 +9,18 @@ CREATE OR REPLACE TABLE test ENGINE = ReplacingMergeTree(eventTime) ORDER BY key; -INSERT INTO test Values (1, 'first', '2024-04-19 01:01:01'); -INSERT INTO test Values (1, 'first', '2024-04-19 01:01:01'); +INSERT INTO t Values (1, 'first', '2024-04-19 01:01:01'); +INSERT INTO t Values (1, 'first', '2024-04-19 01:01:01'); EXPLAIN QUERY TREE passes=1 WITH merged_test AS( - SELECT * FROM test Final + SELECT * FROM t Final ) SELECT * FROM merged_test; WITH merged_test AS( - SELECT * FROM test Final + SELECT * FROM t Final ) SELECT * FROM merged_test; + +DROP TABLE t; From 233c6534e3271ccb766fa33fdc366d25f6b3378c Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 22 Apr 2024 03:48:44 +0000 Subject: [PATCH 0239/1018] explicitly enable analyzer Signed-off-by: Duc Canh Le --- tests/queries/0_stateless/03129_cte_with_final.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03129_cte_with_final.sql b/tests/queries/0_stateless/03129_cte_with_final.sql index b53c85e77e1..01e1ca6dfb0 100644 --- a/tests/queries/0_stateless/03129_cte_with_final.sql +++ b/tests/queries/0_stateless/03129_cte_with_final.sql @@ -12,6 +12,8 @@ ORDER BY key; INSERT INTO t Values (1, 'first', '2024-04-19 01:01:01'); INSERT INTO t Values (1, 'first', '2024-04-19 01:01:01'); +SET allow_experimental_analyzer = 1; + EXPLAIN QUERY TREE passes=1 WITH merged_test AS( SELECT * FROM t Final From f1660fa8bb62456f796f2ac541e0302318a07e3c Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 13 Apr 2024 18:46:47 +0300 Subject: [PATCH 0240/1018] Analyzer support QUALIFY clause --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 6 +++++ src/Analyzer/QueryNode.cpp | 9 +++++++ src/Analyzer/QueryNode.h | 33 ++++++++++++++++++----- src/Analyzer/QueryTreeBuilder.cpp | 4 +++ src/Analyzer/ValidationUtils.cpp | 3 +++ src/Client/Suggest.cpp | 2 +- src/Parsers/ASTSelectQuery.cpp | 6 +++++ src/Parsers/ASTSelectQuery.h | 7 ++++- src/Parsers/CommonParsers.h | 1 + src/Parsers/ExpressionElementParsers.cpp | 1 + src/Parsers/ParserSelectQuery.cpp | 10 +++++++ src/Planner/Planner.cpp | 13 +++++++++ src/Planner/PlannerExpressionAnalysis.cpp | 22 +++++++++++++-- src/Planner/PlannerExpressionAnalysis.h | 16 +++++++++++ 14 files changed, 122 insertions(+), 11 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 5faf8dd97c0..9b83d5660f7 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -7919,6 +7919,9 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (query_node_typed.hasWindow()) visitor.visit(query_node_typed.getWindowNode()); + if (query_node_typed.hasQualify()) + visitor.visit(query_node_typed.getQualify()); + if (query_node_typed.hasOrderBy()) visitor.visit(query_node_typed.getOrderByNode()); @@ -8067,6 +8070,9 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (query_node_typed.hasWindow()) resolveWindowNodeList(query_node_typed.getWindowNode(), scope); + if (query_node_typed.hasQualify()) + resolveExpressionNode(query_node_typed.getQualify(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + if (query_node_typed.hasOrderBy()) { replaceNodesWithPositionalArguments(query_node_typed.getOrderByNode(), query_node_typed.getProjection().getNodes(), scope); diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index f1361c328db..c0659e1998b 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -197,6 +197,12 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s getWindow().dumpTreeImpl(buffer, format_state, indent + 4); } + if (hasQualify()) + { + buffer << '\n' << std::string(indent + 2, ' ') << "QUALIFY\n"; + getQualify()->dumpTreeImpl(buffer, format_state, indent + 4); + } + if (hasOrderBy()) { buffer << '\n' << std::string(indent + 2, ' ') << "ORDER BY\n"; @@ -381,6 +387,9 @@ ASTPtr QueryNode::toASTImpl(const ConvertToASTOptions & options) const if (hasWindow()) select_query->setExpression(ASTSelectQuery::Expression::WINDOW, getWindow().toAST(options)); + if (hasQualify()) + select_query->setExpression(ASTSelectQuery::Expression::QUALIFY, getQualify()->toAST(options)); + if (hasOrderBy()) select_query->setExpression(ASTSelectQuery::Expression::ORDER_BY, getOrderBy().toAST(options)); diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h index af187df72a8..efabf604730 100644 --- a/src/Analyzer/QueryNode.h +++ b/src/Analyzer/QueryNode.h @@ -416,6 +416,24 @@ public: return children[window_child_index]; } + /// Returns true if query node QUALIFY section is not empty, false otherwise + bool hasQualify() const + { + return getQualify() != nullptr; + } + + /// Get QUALIFY section node + const QueryTreeNodePtr & getQualify() const + { + return children[qualify_child_index]; + } + + /// Get QUALIFY section node + QueryTreeNodePtr & getQualify() + { + return children[qualify_child_index]; + } + /// Returns true if query node ORDER BY section is not empty, false otherwise bool hasOrderBy() const { @@ -622,13 +640,14 @@ private: static constexpr size_t group_by_child_index = 5; static constexpr size_t having_child_index = 6; static constexpr size_t window_child_index = 7; - static constexpr size_t order_by_child_index = 8; - static constexpr size_t interpolate_child_index = 9; - static constexpr size_t limit_by_limit_child_index = 10; - static constexpr size_t limit_by_offset_child_index = 11; - static constexpr size_t limit_by_child_index = 12; - static constexpr size_t limit_child_index = 13; - static constexpr size_t offset_child_index = 14; + static constexpr size_t qualify_child_index = 8; + static constexpr size_t order_by_child_index = 9; + static constexpr size_t interpolate_child_index = 10; + static constexpr size_t limit_by_limit_child_index = 11; + static constexpr size_t limit_by_offset_child_index = 12; + static constexpr size_t limit_by_child_index = 13; + static constexpr size_t limit_child_index = 14; + static constexpr size_t offset_child_index = 15; static constexpr size_t children_size = offset_child_index + 1; }; diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index a4f20472432..876e583d393 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -330,6 +330,10 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q if (window_list) current_query_tree->getWindowNode() = buildWindowList(window_list, current_context); + auto qualify_expression = select_query_typed.qualify(); + if (qualify_expression) + current_query_tree->getQualify() = buildExpression(qualify_expression, current_context); + auto select_order_by_list = select_query_typed.orderBy(); if (select_order_by_list) current_query_tree->getOrderByNode() = buildSortList(select_order_by_list, current_context); diff --git a/src/Analyzer/ValidationUtils.cpp b/src/Analyzer/ValidationUtils.cpp index af35632ab81..60cc1dd521f 100644 --- a/src/Analyzer/ValidationUtils.cpp +++ b/src/Analyzer/ValidationUtils.cpp @@ -56,6 +56,9 @@ void validateFilters(const QueryTreeNodePtr & query_node) if (query_node_typed.hasHaving()) validateFilter(query_node_typed.getHaving(), "HAVING", query_node); + + if (query_node_typed.hasQualify()) + validateFilter(query_node_typed.getQualify(), "QUALIFY", query_node); } namespace diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index f63dbc64d27..ebfa2e89ea1 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -47,7 +47,7 @@ Suggest::Suggest() "GRANT", "REVOKE", "OPTION", "ADMIN", "EXCEPT", "REPLACE", "IDENTIFIED", "HOST", "NAME", "READONLY", "WRITABLE", "PERMISSIVE", "FOR", "RESTRICTIVE", "RANDOMIZED", "INTERVAL", "LIMITS", "ONLY", "TRACKING", "IP", "REGEXP", "ILIKE", "CLEANUP", "APPEND", - "IGNORE NULLS", "RESPECT NULLS", "OVER", "PASTE"}); + "IGNORE NULLS", "RESPECT NULLS", "OVER", "PASTE", "WINDOW", "QUALIFY"}); } static String getLoadSuggestionQuery(Int32 suggestion_limit, bool basic_suggestion) diff --git a/src/Parsers/ASTSelectQuery.cpp b/src/Parsers/ASTSelectQuery.cpp index 586477e1cfd..f99933b7969 100644 --- a/src/Parsers/ASTSelectQuery.cpp +++ b/src/Parsers/ASTSelectQuery.cpp @@ -144,6 +144,12 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F window()->as().formatImplMultiline(s, state, frame); } + if (qualify()) + { + s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "QUALIFY " << (s.hilite ? hilite_none : ""); + qualify()->formatImpl(s, state, frame); + } + if (!order_by_all && orderBy()) { s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "ORDER BY" << (s.hilite ? hilite_none : ""); diff --git a/src/Parsers/ASTSelectQuery.h b/src/Parsers/ASTSelectQuery.h index eb171dc00ee..9fd6becbaaa 100644 --- a/src/Parsers/ASTSelectQuery.h +++ b/src/Parsers/ASTSelectQuery.h @@ -25,6 +25,7 @@ public: GROUP_BY, HAVING, WINDOW, + QUALIFY, ORDER_BY, LIMIT_BY_OFFSET, LIMIT_BY_LENGTH, @@ -55,6 +56,8 @@ public: return "HAVING"; case Expression::WINDOW: return "WINDOW"; + case Expression::QUALIFY: + return "QUALIFY"; case Expression::ORDER_BY: return "ORDER BY"; case Expression::LIMIT_BY_OFFSET: @@ -95,6 +98,7 @@ public: ASTPtr & refPrewhere() { return getExpression(Expression::PREWHERE); } ASTPtr & refWhere() { return getExpression(Expression::WHERE); } ASTPtr & refHaving() { return getExpression(Expression::HAVING); } + ASTPtr & refQualify() { return getExpression(Expression::QUALIFY); } ASTPtr with() const { return getExpression(Expression::WITH); } ASTPtr select() const { return getExpression(Expression::SELECT); } @@ -104,6 +108,7 @@ public: ASTPtr groupBy() const { return getExpression(Expression::GROUP_BY); } ASTPtr having() const { return getExpression(Expression::HAVING); } ASTPtr window() const { return getExpression(Expression::WINDOW); } + ASTPtr qualify() const { return getExpression(Expression::QUALIFY); } ASTPtr orderBy() const { return getExpression(Expression::ORDER_BY); } ASTPtr limitByOffset() const { return getExpression(Expression::LIMIT_BY_OFFSET); } ASTPtr limitByLength() const { return getExpression(Expression::LIMIT_BY_LENGTH); } @@ -113,7 +118,7 @@ public: ASTPtr settings() const { return getExpression(Expression::SETTINGS); } ASTPtr interpolate() const { return getExpression(Expression::INTERPOLATE); } - bool hasFiltration() const { return where() || prewhere() || having(); } + bool hasFiltration() const { return where() || prewhere() || having() || qualify(); } /// Set/Reset/Remove expression. void setExpression(Expression expr, ASTPtr && ast); diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index 2277e348b0f..12c452d38c4 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -507,6 +507,7 @@ namespace DB MR_MACROS(WHEN, "WHEN") \ MR_MACROS(WHERE, "WHERE") \ MR_MACROS(WINDOW, "WINDOW") \ + MR_MACROS(QUALIFY, "QUALIFY") \ MR_MACROS(WITH_ADMIN_OPTION, "WITH ADMIN OPTION") \ MR_MACROS(WITH_CHECK, "WITH CHECK") \ MR_MACROS(WITH_FILL, "WITH FILL") \ diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index dce0bc62d5b..a483ac92f39 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1481,6 +1481,7 @@ const char * ParserAlias::restricted_keywords[] = "USING", "WHERE", "WINDOW", + "QUALIFY", "WITH", "INTERSECT", "EXCEPT", diff --git a/src/Parsers/ParserSelectQuery.cpp b/src/Parsers/ParserSelectQuery.cpp index ce15c7b03fd..e1ded0ab902 100644 --- a/src/Parsers/ParserSelectQuery.cpp +++ b/src/Parsers/ParserSelectQuery.cpp @@ -49,6 +49,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_totals(Keyword::TOTALS); ParserKeyword s_having(Keyword::HAVING); ParserKeyword s_window(Keyword::WINDOW); + ParserKeyword s_qualify(Keyword::QUALIFY); ParserKeyword s_order_by(Keyword::ORDER_BY); ParserKeyword s_limit(Keyword::LIMIT); ParserKeyword s_settings(Keyword::SETTINGS); @@ -86,6 +87,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ASTPtr group_expression_list; ASTPtr having_expression; ASTPtr window_list; + ASTPtr qualify_expression; ASTPtr order_expression_list; ASTPtr interpolate_expression_list; ASTPtr limit_by_length; @@ -266,6 +268,13 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } } + /// QUALIFY expr + if (s_qualify.ignore(pos, expected)) + { + if (!exp_elem.parse(pos, qualify_expression, expected)) + return false; + } + /// ORDER BY expr ASC|DESC COLLATE 'locale' list if (s_order_by.ignore(pos, expected)) { @@ -489,6 +498,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, std::move(group_expression_list)); select_query->setExpression(ASTSelectQuery::Expression::HAVING, std::move(having_expression)); select_query->setExpression(ASTSelectQuery::Expression::WINDOW, std::move(window_list)); + select_query->setExpression(ASTSelectQuery::Expression::QUALIFY, std::move(qualify_expression)); select_query->setExpression(ASTSelectQuery::Expression::ORDER_BY, std::move(order_expression_list)); select_query->setExpression(ASTSelectQuery::Expression::LIMIT_BY_OFFSET, std::move(limit_by_offset)); select_query->setExpression(ASTSelectQuery::Expression::LIMIT_BY_LENGTH, std::move(limit_by_length)); diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index d75573c8d99..b21dd5203e8 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1367,6 +1367,16 @@ void Planner::buildPlanForQueryNode() select_query_info.has_aggregates = hasAggregateFunctionNodes(query_tree); select_query_info.need_aggregate = query_node.hasGroupBy() || select_query_info.has_aggregates; + if (!select_query_info.has_window && query_node.hasQualify()) + { + if (query_node.hasHaving()) + query_node.getHaving() = mergeConditionNodes({query_node.getHaving(), query_node.getQualify()}, query_context); + else + query_node.getHaving() = query_node.getQualify(); + + query_node.getQualify() = {}; + } + if (!select_query_info.need_aggregate && query_node.hasHaving()) { if (query_node.hasWhere()) @@ -1636,6 +1646,9 @@ void Planner::buildPlanForQueryNode() addWindowSteps(query_plan, planner_context, window_analysis_result); } + if (expression_analysis_result.hasQualify()) + addFilterStep(query_plan, expression_analysis_result.getQualify(), "QUALIFY", result_actions_to_execute); + const auto & projection_analysis_result = expression_analysis_result.getProjection(); addExpressionStep(query_plan, projection_analysis_result.projection_actions, "Projection", result_actions_to_execute); diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index dd3769ee10b..ad8db83d66c 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -513,6 +513,16 @@ PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(const QueryTreeNo if (window_analysis_result_optional) current_output_columns = actions_chain.getLastStepAvailableOutputColumns(); + std::optional qualify_analysis_result_optional; + std::optional qualify_action_step_index_optional; + + if (query_node.hasQualify()) + { + qualify_analysis_result_optional = analyzeFilter(query_node.getQualify(), current_output_columns, planner_context, actions_chain); + qualify_action_step_index_optional = actions_chain.getLastStepIndex(); + current_output_columns = actions_chain.getLastStepAvailableOutputColumns(); + } + auto projection_analysis_result = analyzeProjection(query_node, current_output_columns, planner_context, actions_chain); current_output_columns = actions_chain.getLastStepAvailableOutputColumns(); @@ -604,7 +614,7 @@ PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(const QueryTreeNo PlannerExpressionsAnalysisResult expressions_analysis_result(std::move(projection_analysis_result)); - if (where_action_step_index_optional && where_analysis_result_optional) + if (where_analysis_result_optional && where_action_step_index_optional) { auto & where_analysis_result = *where_analysis_result_optional; auto & where_actions_chain_node = actions_chain.at(*where_action_step_index_optional); @@ -615,7 +625,7 @@ PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(const QueryTreeNo if (aggregation_analysis_result_optional) expressions_analysis_result.addAggregation(std::move(*aggregation_analysis_result_optional)); - if (having_action_step_index_optional && having_analysis_result_optional) + if (having_analysis_result_optional && having_action_step_index_optional) { auto & having_analysis_result = *having_analysis_result_optional; auto & having_actions_chain_node = actions_chain.at(*having_action_step_index_optional); @@ -626,6 +636,14 @@ PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(const QueryTreeNo if (window_analysis_result_optional) expressions_analysis_result.addWindow(std::move(*window_analysis_result_optional)); + if (qualify_analysis_result_optional && qualify_action_step_index_optional) + { + auto & qualify_analysis_result = *qualify_analysis_result_optional; + auto & qualify_actions_chain_node = actions_chain.at(*qualify_action_step_index_optional); + qualify_analysis_result.remove_filter_column = !qualify_actions_chain_node->getChildRequiredOutputColumnsNames().contains(qualify_analysis_result.filter_column_name); + expressions_analysis_result.addQualify(std::move(qualify_analysis_result)); + } + if (sort_analysis_result_optional) expressions_analysis_result.addSort(std::move(*sort_analysis_result_optional)); diff --git a/src/Planner/PlannerExpressionAnalysis.h b/src/Planner/PlannerExpressionAnalysis.h index 792cfdec2ff..0773272e49a 100644 --- a/src/Planner/PlannerExpressionAnalysis.h +++ b/src/Planner/PlannerExpressionAnalysis.h @@ -129,6 +129,21 @@ public: window_analysis_result = std::move(window_analysis_result_); } + bool hasQualify() const + { + return qualify_analysis_result.filter_actions != nullptr; + } + + const FilterAnalysisResult & getQualify() const + { + return qualify_analysis_result; + } + + void addQualify(FilterAnalysisResult qualify_analysis_result_) + { + qualify_analysis_result = std::move(qualify_analysis_result_); + } + bool hasSort() const { return sort_analysis_result.before_order_by_actions != nullptr; @@ -165,6 +180,7 @@ private: AggregationAnalysisResult aggregation_analysis_result; FilterAnalysisResult having_analysis_result; WindowAnalysisResult window_analysis_result; + FilterAnalysisResult qualify_analysis_result; SortAnalysisResult sort_analysis_result; LimitByAnalysisResult limit_by_analysis_result; }; From c91fff60f438f3a888ed8e38c44e15faeb6bf9bc Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 13 Apr 2024 18:47:40 +0300 Subject: [PATCH 0241/1018] Added tests --- .../03095_window_functions_qualify.reference | 18 ++++++++++++++++++ .../03095_window_functions_qualify.sql | 19 +++++++++++++++++++ 2 files changed, 37 insertions(+) create mode 100644 tests/queries/0_stateless/03095_window_functions_qualify.reference create mode 100644 tests/queries/0_stateless/03095_window_functions_qualify.sql diff --git a/tests/queries/0_stateless/03095_window_functions_qualify.reference b/tests/queries/0_stateless/03095_window_functions_qualify.reference new file mode 100644 index 00000000000..32bb9b5748c --- /dev/null +++ b/tests/queries/0_stateless/03095_window_functions_qualify.reference @@ -0,0 +1,18 @@ +0 4 +3 4 +6 4 +9 4 +-- +0 +3 +6 +9 +-- +6 +7 +8 +9 +-- +0 5 +-- +0 5 diff --git a/tests/queries/0_stateless/03095_window_functions_qualify.sql b/tests/queries/0_stateless/03095_window_functions_qualify.sql new file mode 100644 index 00000000000..e5ef78a1746 --- /dev/null +++ b/tests/queries/0_stateless/03095_window_functions_qualify.sql @@ -0,0 +1,19 @@ +SET allow_experimental_analyzer = 1; + +SELECT number, COUNT() OVER (PARTITION BY number % 3) AS partition_count FROM numbers(10) QUALIFY partition_count = 4 ORDER BY number; + +SELECT '--'; + +SELECT number FROM numbers(10) QUALIFY (COUNT() OVER (PARTITION BY number % 3) AS partition_count) = 4 ORDER BY number; + +SELECT '--'; + +SELECT number FROM numbers(10) QUALIFY number > 5 ORDER BY number; + +SELECT '--'; + +SELECT (number % 2) AS key, count() FROM numbers(10) GROUP BY key HAVING key = 0 QUALIFY key == 0; + +SELECT '--'; + +SELECT (number % 2) AS key, count() FROM numbers(10) GROUP BY key QUALIFY key == 0; From 0ad8c6ef78bb4a2ab1ddad3aa91d094bba7442fc Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 13 Apr 2024 18:48:38 +0300 Subject: [PATCH 0242/1018] Added docs --- docs/en/sql-reference/statements/select/index.md | 3 +++ docs/en/sql-reference/statements/select/qualify.md | 14 ++++++++++++++ 2 files changed, 17 insertions(+) create mode 100644 docs/en/sql-reference/statements/select/qualify.md diff --git a/docs/en/sql-reference/statements/select/index.md b/docs/en/sql-reference/statements/select/index.md index 86a4e9639f5..1fea29aa40e 100644 --- a/docs/en/sql-reference/statements/select/index.md +++ b/docs/en/sql-reference/statements/select/index.md @@ -21,6 +21,8 @@ SELECT [DISTINCT [ON (column1, column2, ...)]] expr_list [WHERE expr] [GROUP BY expr_list] [WITH ROLLUP|WITH CUBE] [WITH TOTALS] [HAVING expr] +[WINDOW window_expr_list] +[QUALIFY expr] [ORDER BY expr_list] [WITH FILL] [FROM expr] [TO expr] [STEP expr] [INTERPOLATE [(expr_list)]] [LIMIT [offset_value, ]n BY columns] [LIMIT [n, ]m] [WITH TIES] @@ -45,6 +47,7 @@ Specifics of each optional clause are covered in separate sections, which are li - [GROUP BY clause](../../../sql-reference/statements/select/group-by.md) - [LIMIT BY clause](../../../sql-reference/statements/select/limit-by.md) - [HAVING clause](../../../sql-reference/statements/select/having.md) +- [QUALIFY clause](../../../sql-reference/statements/select/qualify.md) - [LIMIT clause](../../../sql-reference/statements/select/limit.md) - [OFFSET clause](../../../sql-reference/statements/select/offset.md) - [UNION clause](../../../sql-reference/statements/select/union.md) diff --git a/docs/en/sql-reference/statements/select/qualify.md b/docs/en/sql-reference/statements/select/qualify.md new file mode 100644 index 00000000000..928bcbfc965 --- /dev/null +++ b/docs/en/sql-reference/statements/select/qualify.md @@ -0,0 +1,14 @@ +--- +slug: /en/sql-reference/statements/select/qualify +sidebar_label: QUALIFY +--- + +# QUALIFY Clause + +Allows filtering window functions results. It is similar to the [WHERE](../../../sql-reference/statements/select/where.md) clause, but the difference is that `WHERE` is performed before window functions evaluation, while `QUALIFY` is performed after it. + +It is possible to reference window functions results from `SELECT` clause in `QUALIFY` clause by their alias. Alternatively, `QUALIFY` clause can filter on results of additional window functions that are not returned in query results. + +## Limitations + +`QUALIFY` can’t be used if there are no window functions to evaluate. Use `WHERE` instead. From 580a581415a2e47027ac27cd4b852acf54f871b0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 22 Apr 2024 09:56:46 +0000 Subject: [PATCH 0243/1018] Fix typo --- src/Common/CgroupsMemoryUsageObserver.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/CgroupsMemoryUsageObserver.cpp b/src/Common/CgroupsMemoryUsageObserver.cpp index dd68bd0da64..8a4792f0a5a 100644 --- a/src/Common/CgroupsMemoryUsageObserver.cpp +++ b/src/Common/CgroupsMemoryUsageObserver.cpp @@ -77,7 +77,7 @@ void CgroupsMemoryUsageObserver::setMemoryUsageLimits(uint64_t hard_limit_, uint { if (up) { - LOG_WARNING(log, "Exceeded sort memory limit ({})", ReadableSize(soft_limit_)); + LOG_WARNING(log, "Exceeded soft memory limit ({})", ReadableSize(soft_limit_)); #if USE_JEMALLOC LOG_INFO(log, "Purging jemalloc arenas"); From a39baf84cf367054bf841a6c64b4ca22400996fd Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 22 Apr 2024 11:26:15 +0200 Subject: [PATCH 0244/1018] Get rid of merge_commit in style check autofix --- tests/ci/style_check.py | 44 +++++++++++++++++++++++++++++------------ 1 file changed, 31 insertions(+), 13 deletions(-) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index d49cd283e9f..a8f712e356f 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -77,23 +77,35 @@ def commit_push_staged(pr_info: PRInfo) -> None: return git_staged = git_runner("git diff --cached --name-only") if not git_staged: + logging.info("No fixes are staged") return - remote_url = pr_info.event["pull_request"]["base"]["repo"]["ssh_url"] - head = git_runner("git rev-parse HEAD^{}") - git_runner(f"{GIT_PREFIX} commit -m 'Automatic style fix'") - # The fetch to avoid issue 'pushed branch tip is behind its remote' - fetch_cmd = ( - f"{GIT_PREFIX} fetch {remote_url} --no-recurse-submodules --depth=2 {head}" - ) - push_cmd = f"{GIT_PREFIX} push {remote_url} HEAD:{pr_info.head_ref}" + + def push_fix() -> None: + """ + Stash staged changes to commit them on the top of the PR's head. + `pull_request` event runs on top of a temporary merge_commit, we need to avoid + including it in the autofix + """ + remote_url = pr_info.event["pull_request"]["base"]["repo"]["ssh_url"] + head = pr_info.sha + git_runner(f"{GIT_PREFIX} commit -m 'Automatic style fix'") + fix_commit = git_runner("git rev-parse HEAD") + logging.info( + "Fetching PR's head, check it out and cherry-pick autofix: %s", head + ) + git_runner( + f"{GIT_PREFIX} fetch {remote_url} --no-recurse-submodules --depth=1 {head}" + ) + git_runner(f"git reset --hard {head}") + git_runner(f"{GIT_PREFIX} cherry-pick {fix_commit}") + git_runner(f"{GIT_PREFIX} push {remote_url} HEAD:{pr_info.head_ref}") + if os.getenv("ROBOT_CLICKHOUSE_SSH_KEY", ""): with SSHKey("ROBOT_CLICKHOUSE_SSH_KEY"): - git_runner(fetch_cmd) - git_runner(push_cmd) + push_fix() return - git_runner(fetch_cmd) - git_runner(push_cmd) + push_fix() def _check_mime(file: Union[Path, str], mime: str) -> bool: @@ -180,7 +192,13 @@ def main(): _ = future.result() if args.push: - commit_push_staged(pr_info) + try: + commit_push_staged(pr_info) + except subprocess.SubprocessError: + # do not fail the whole script if the autofix didn't work out + logging.error( + "Unable to push the autofix. Continue." + ) subprocess.check_call( f"python3 ../../utils/check-style/process_style_check_result.py --in-results-dir {temp_path} " From e5339413f3ac2186a85ad780c6fb64862e1ac5a4 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Mon, 22 Apr 2024 12:24:26 +0200 Subject: [PATCH 0245/1018] Correct `tanh` return type Change return type from incorrect Float64 to correct Float32 --- docs/en/sql-reference/functions/math-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/math-functions.md b/docs/en/sql-reference/functions/math-functions.md index e59878f901b..2d0ae28aefc 100644 --- a/docs/en/sql-reference/functions/math-functions.md +++ b/docs/en/sql-reference/functions/math-functions.md @@ -575,7 +575,7 @@ tanh(x) - Values from the interval: `-1 < tanh(x) < 1`. -Type: [Float64](../../sql-reference/data-types/float.md#float32-float64). +Type: [Float32](../../sql-reference/data-types/float.md#float32-float64). **Example** From 0811124d7d7ff53149a397b285c7fd5c60851a0b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 22 Apr 2024 12:12:53 +0200 Subject: [PATCH 0246/1018] Add description to the commit status on failed autofix --- tests/ci/style_check.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index a8f712e356f..e09924048d7 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -191,14 +191,14 @@ def main(): future = executor.submit(subprocess.run, cmd_shell, shell=True) _ = future.result() + autofix_description = "" if args.push: try: commit_push_staged(pr_info) except subprocess.SubprocessError: # do not fail the whole script if the autofix didn't work out - logging.error( - "Unable to push the autofix. Continue." - ) + logging.error("Unable to push the autofix. Continue.") + autofix_description = ( "Failed to push autofix to the PR. " ) subprocess.check_call( f"python3 ../../utils/check-style/process_style_check_result.py --in-results-dir {temp_path} " @@ -210,7 +210,7 @@ def main(): state, description, test_results, additional_files = process_result(temp_path) JobReport( - description=description, + description=f"{autofix_description}{description}", test_results=test_results, status=state, start_time=stopwatch.start_time_str, From d3d0d4f6c02a79f40da6c4ae31c23ae7892a0e89 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 22 Apr 2024 11:36:27 +0100 Subject: [PATCH 0247/1018] empty commit From 19dbc8d2e27e9a9f03163499ce4a2214e0174455 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 22 Apr 2024 10:39:46 +0000 Subject: [PATCH 0248/1018] Automatic style fix --- tests/ci/style_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index e09924048d7..dc5d974daf0 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -198,7 +198,7 @@ def main(): except subprocess.SubprocessError: # do not fail the whole script if the autofix didn't work out logging.error("Unable to push the autofix. Continue.") - autofix_description = ( "Failed to push autofix to the PR. " ) + autofix_description = "Failed to push autofix to the PR. " subprocess.check_call( f"python3 ../../utils/check-style/process_style_check_result.py --in-results-dir {temp_path} " From 015d47ff4bad9d9ea49be8f8e2d824bb19a72e7b Mon Sep 17 00:00:00 2001 From: Blargian Date: Mon, 22 Apr 2024 13:00:08 +0200 Subject: [PATCH 0249/1018] Update return types of all added functions --- .../functions/rounding-functions.md | 58 ++++++++++++++++--- 1 file changed, 49 insertions(+), 9 deletions(-) diff --git a/docs/en/sql-reference/functions/rounding-functions.md b/docs/en/sql-reference/functions/rounding-functions.md index f90a35922b0..2625bd5af40 100644 --- a/docs/en/sql-reference/functions/rounding-functions.md +++ b/docs/en/sql-reference/functions/rounding-functions.md @@ -222,13 +222,49 @@ roundBankers(10.755, 2) = 10.76 - [round](#rounding_functions-round) -## roundToExp2(num) +## roundToExp2 -Accepts a number. If the number is less than one, it returns 0. Otherwise, it rounds the number down to the nearest (whole non-negative) degree of two. +Accepts a number. If the number is less than one, it returns `0`. Otherwise, it rounds the number down to the nearest (whole non-negative) degree of two. + +**Syntax** + +```sql +roundToExp2(num) +``` + +**Parameters** + +- `num`: A number representing an age in years. [UInt](../data-types/int-uint.md)/[Float](../data-types/float.md). + +**Returned value** + +- `0`, for `num` $\lt 1$. [UInt8](../data-types/int-uint.md). +- `num` rounded down to the nearest (whole non-negative) degree of two. [UInt](../data-types/int-uint.md)/[Float](../data-types/float.md) equivalent to the input type. + +**Example** + +Query: + +```sql +SELECT *, roundToExp2(*) FROM system.numbers WHERE number IN (0, 2, 5, 10, 19, 50) +``` + +Result: + +```response +┌─number─┬─roundToExp2(number)─┐ +│ 0 │ 0 │ +│ 2 │ 2 │ +│ 5 │ 4 │ +│ 10 │ 8 │ +│ 19 │ 16 │ +│ 50 │ 32 │ +└────────┴─────────────────────┘ +``` ## roundDuration -Accepts a number. If the number is less than one, it returns `0`. Otherwise, it rounds the number down to numbers from the set: $\set{1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000}$. +Accepts a number. If the number is less than one, it returns `0`. Otherwise, it rounds the number down to numbers from the set of commonly used durations: `1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000`. **Syntax** @@ -238,12 +274,14 @@ roundDuration(num) **Parameters** -- `age`: A number representing an age in years. [Numeric](../data-types/int-uint.md). +- `num`: A number to round to one of the numbers in the set of common durations. [UInt](../data-types/int-uint.md)/[Float](../data-types/float.md). **Returned value** - `0`, for `num` $\lt 1$. -- One of $\set{1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000}$, otherwise. [UInt8](../data-types/int-uint.md). +- Otherwise, one of: `1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000`. [UInt16](../data-types/int-uint.md). + +**Example** Query: @@ -286,7 +324,7 @@ roundAge(num) **Parameters** -- `age`: A number representing an age in years. [Numeric](../data-types/int-uint.md). +- `age`: A number representing an age in years. [UInt](../data-types/int-uint.md)/[Float](../data-types/float.md). **Returned value** @@ -298,6 +336,8 @@ roundAge(num) - Returns `45`, for $45 \leq age \leq 54$. - Returns `55`, for $age \geq 55$. +Type: [UInt8](../data-types/int-uint.md). + **Example** Query: @@ -332,12 +372,12 @@ roundDown(num, arr) **Parameters** -- `age`: A number representing an age in years. [Numeric](../data-types/int-uint.md). -- `arr`: Array of elements to round `age` down to. [Array](../data-types/array.md)([Numeric](../data-types/int-uint.md)) +- `num`: A number to round down. [Numeric](../data-types/int-uint.md). +- `arr`: Array of elements to round `age` down to. [Array](../data-types/array.md) of [UInt](../data-types/int-uint.md)/[Float](../data-types/float.md) type. **Returned value** -- Number rounded down to an element in `arr`. If the value is less than the lowest bound, the lowest bound is returned. [Numeric](../data-types/int-uint.md). +- Number rounded down to an element in `arr`. If the value is less than the lowest bound, the lowest bound is returned. [UInt](../data-types/int-uint.md)/[Float](../data-types/float.md) type deduced from the type of `arr`. **Example** From 59125601a1a23446a2db99e970bd8849013fa4c9 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 22 Apr 2024 11:36:45 +0000 Subject: [PATCH 0250/1018] fix crash in function formatRow --- src/Functions/formatRow.cpp | 8 +++++++- .../03129_format_row_json_http.reference | 15 +++++++++++++++ .../0_stateless/03129_format_row_json_http.sh | 7 +++++++ 3 files changed, 29 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03129_format_row_json_http.reference create mode 100755 tests/queries/0_stateless/03129_format_row_json_http.sh diff --git a/src/Functions/formatRow.cpp b/src/Functions/formatRow.cpp index 1ac6becfb15..5a67ce158a7 100644 --- a/src/Functions/formatRow.cpp +++ b/src/Functions/formatRow.cpp @@ -38,8 +38,14 @@ public: : format_name(std::move(format_name_)) , arguments_column_names(std::move(arguments_column_names_)) , context(std::move(context_)) + , format_settings(getFormatSettings(context)) { FormatFactory::instance().checkFormatName(format_name); + + /// We don't need handling exceptions while formatting as a row. + /// But it can be enabled in query sent via http interface. + format_settings.json.valid_output_on_exception = false; + format_settings.xml.valid_output_on_exception = false; } String getName() const override { return name; } @@ -68,7 +74,6 @@ public: } materializeBlockInplace(arg_columns); - auto format_settings = getFormatSettings(context); auto out = FormatFactory::instance().getOutputFormat(format_name, buffer, arg_columns, context, format_settings); /// This function make sense only for row output formats. @@ -104,6 +109,7 @@ private: String format_name; Names arguments_column_names; ContextPtr context; + FormatSettings format_settings; }; template diff --git a/tests/queries/0_stateless/03129_format_row_json_http.reference b/tests/queries/0_stateless/03129_format_row_json_http.reference new file mode 100644 index 00000000000..02f4f51d500 --- /dev/null +++ b/tests/queries/0_stateless/03129_format_row_json_http.reference @@ -0,0 +1,15 @@ +{"number":"0"}\n +{"number":"1"}\n +{"number":"2"}\n +{"number":"3"}\n +{"number":"4"}\n +{"number":"5"}\n +{"number":"6"}\n +{"number":"7"}\n +{"number":"8"}\n +{"number":"9"}\n +{"number":"10"}\n +{"number":"11"}\n +{"number":"12"}\n +{"number":"13"}\n +{"number":"14"}\n diff --git a/tests/queries/0_stateless/03129_format_row_json_http.sh b/tests/queries/0_stateless/03129_format_row_json_http.sh new file mode 100755 index 00000000000..729a5ef563f --- /dev/null +++ b/tests/queries/0_stateless/03129_format_row_json_http.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" --data-binary "SELECT formatRow('JSONEachRow', number) as test FROM (SELECT number FROM numbers(15))" From 0e28d2079137b2617a8a78d7ba38663e514415b8 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 22 Apr 2024 14:37:04 +0300 Subject: [PATCH 0251/1018] Fixed code review issues --- .../statements/select/qualify.md | 20 +++++++ src/Analyzer/Passes/QueryAnalysisPass.cpp | 3 + .../03095_window_functions_qualify.reference | 56 +++++++++++++++++++ .../03095_window_functions_qualify.sql | 17 ++++++ 4 files changed, 96 insertions(+) diff --git a/docs/en/sql-reference/statements/select/qualify.md b/docs/en/sql-reference/statements/select/qualify.md index 928bcbfc965..ccb3fce5efa 100644 --- a/docs/en/sql-reference/statements/select/qualify.md +++ b/docs/en/sql-reference/statements/select/qualify.md @@ -12,3 +12,23 @@ It is possible to reference window functions results from `SELECT` clause in `QU ## Limitations `QUALIFY` can’t be used if there are no window functions to evaluate. Use `WHERE` instead. + +## Examples + +Example: + +``` sql +SELECT number, COUNT() OVER (PARTITION BY number % 3) AS partition_count +FROM numbers(10) +QUALIFY partition_count = 4 +ORDER BY number; +``` + +``` text +┌─number─┬─partition_count─┐ +│ 0 │ 4 │ +│ 3 │ 4 │ +│ 6 │ 4 │ +│ 9 │ 4 │ +└────────┴─────────────────┘ +``` diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 9b83d5660f7..a3c60c76132 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -7895,6 +7895,9 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (query_node_typed.hasHaving() && query_node_typed.isGroupByWithTotals() && is_rollup_or_cube) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "WITH TOTALS and WITH ROLLUP or CUBE are not supported together in presence of HAVING"); + if (query_node_typed.hasQualify() && query_node_typed.isGroupByWithTotals() && is_rollup_or_cube) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "WITH TOTALS and WITH ROLLUP or CUBE are not supported together in presence of QUALIFY"); + /// Initialize aliases in query node scope QueryExpressionsAliasVisitor visitor(scope); diff --git a/tests/queries/0_stateless/03095_window_functions_qualify.reference b/tests/queries/0_stateless/03095_window_functions_qualify.reference index 32bb9b5748c..c74a212428b 100644 --- a/tests/queries/0_stateless/03095_window_functions_qualify.reference +++ b/tests/queries/0_stateless/03095_window_functions_qualify.reference @@ -16,3 +16,59 @@ 0 5 -- 0 5 +-- +0 4 +3 4 +6 4 +9 4 +-- +Expression (Project names) +Header: number UInt64 + partition_count UInt64 +Actions: INPUT : 0 -> __table1.number UInt64 : 0 + INPUT : 1 -> count() OVER (PARTITION BY modulo(__table1.number, 3_UInt8)) UInt64 : 1 + ALIAS __table1.number :: 0 -> number UInt64 : 2 + ALIAS count() OVER (PARTITION BY modulo(__table1.number, 3_UInt8)) :: 1 -> partition_count UInt64 : 0 +Positions: 2 0 + Sorting (Sorting for ORDER BY) + Header: __table1.number UInt64 + count() OVER (PARTITION BY modulo(__table1.number, 3_UInt8)) UInt64 + Sort description: __table1.number ASC + Expression ((Before ORDER BY + Projection)) + Header: __table1.number UInt64 + count() OVER (PARTITION BY modulo(__table1.number, 3_UInt8)) UInt64 + Actions: INPUT :: 0 -> __table1.number UInt64 : 0 + INPUT :: 1 -> count() OVER (PARTITION BY modulo(__table1.number, 3_UInt8)) UInt64 : 1 + Positions: 0 1 + Filter (QUALIFY) + Header: __table1.number UInt64 + count() OVER (PARTITION BY modulo(__table1.number, 3_UInt8)) UInt64 + Filter column: equals(count() OVER (PARTITION BY modulo(__table1.number, 3_UInt8)), 4_UInt8) (removed) + Actions: INPUT :: 0 -> __table1.number UInt64 : 0 + INPUT :: 1 -> count() OVER (PARTITION BY modulo(__table1.number, 3_UInt8)) UInt64 : 1 + INPUT : 2 -> count() OVER (PARTITION BY modulo(__table1.number, 3_UInt8)) UInt64 : 2 + COLUMN Const(UInt8) -> 4_UInt8 UInt8 : 3 + FUNCTION equals(count() OVER (PARTITION BY modulo(__table1.number, 3_UInt8)) :: 2, 4_UInt8 :: 3) -> equals(count() OVER (PARTITION BY modulo(__table1.number, 3_UInt8)), 4_UInt8) UInt8 : 4 + Positions: 4 0 1 + Window (Window step for window \'PARTITION BY modulo(__table1.number, 3_UInt8)\') + Header: modulo(__table1.number, 3_UInt8) UInt8 + __table1.number UInt64 + count() OVER (PARTITION BY modulo(__table1.number, 3_UInt8)) UInt64 + count() OVER (PARTITION BY modulo(__table1.number, 3_UInt8)) UInt64 + Window: (PARTITION BY modulo(__table1.number, 3_UInt8)) + Functions: count() OVER (PARTITION BY modulo(__table1.number, 3_UInt8)) + count() OVER (PARTITION BY modulo(__table1.number, 3_UInt8)) + Sorting (Sorting for window \'PARTITION BY modulo(__table1.number, 3_UInt8)\') + Header: modulo(__table1.number, 3_UInt8) UInt8 + __table1.number UInt64 + Sort description: modulo(__table1.number, 3_UInt8) ASC + Expression ((Before WINDOW + Change column names to column identifiers)) + Header: modulo(__table1.number, 3_UInt8) UInt8 + __table1.number UInt64 + Actions: INPUT : 0 -> number UInt64 : 0 + COLUMN Const(UInt8) -> 3_UInt8 UInt8 : 1 + ALIAS number :: 0 -> __table1.number UInt64 : 2 + FUNCTION modulo(__table1.number : 2, 3_UInt8 :: 1) -> modulo(__table1.number, 3_UInt8) UInt8 : 0 + Positions: 0 2 + ReadFromSystemNumbers + Header: number UInt64 diff --git a/tests/queries/0_stateless/03095_window_functions_qualify.sql b/tests/queries/0_stateless/03095_window_functions_qualify.sql index e5ef78a1746..35e203a2ffc 100644 --- a/tests/queries/0_stateless/03095_window_functions_qualify.sql +++ b/tests/queries/0_stateless/03095_window_functions_qualify.sql @@ -17,3 +17,20 @@ SELECT (number % 2) AS key, count() FROM numbers(10) GROUP BY key HAVING key = 0 SELECT '--'; SELECT (number % 2) AS key, count() FROM numbers(10) GROUP BY key QUALIFY key == 0; + +SELECT '--'; + +SELECT number, COUNT() OVER (PARTITION BY number % 3) AS partition_count FROM numbers(10) QUALIFY COUNT() OVER (PARTITION BY number % 3) = 4 ORDER BY number; + +SELECT '--'; + +EXPLAIN header = 1, actions = 1 +SELECT number, COUNT() OVER (PARTITION BY number % 3) AS partition_count FROM numbers(10) QUALIFY COUNT() OVER (PARTITION BY number % 3) = 4 ORDER BY number; + +SELECT number % toUInt256(2) AS key, count() FROM numbers(10) GROUP BY key WITH CUBE WITH TOTALS QUALIFY key = toNullable(toNullable(0)); -- { serverError 48 } + +SELECT number % 2 AS key, count(materialize(5)) IGNORE NULLS FROM numbers(10) WHERE toLowCardinality(toLowCardinality(materialize(2))) GROUP BY key WITH CUBE WITH TOTALS QUALIFY key = 0; -- { serverError 48 } + +SELECT 4, count(4) IGNORE NULLS, number % 2 AS key FROM numbers(10) GROUP BY key WITH ROLLUP WITH TOTALS QUALIFY key = materialize(0); -- { serverError 48 } + +SELECT 3, number % toLowCardinality(2) AS key, count() IGNORE NULLS FROM numbers(10) GROUP BY key WITH ROLLUP WITH TOTALS QUALIFY key = 0; -- { serverError 48 } From ae8b22b6e0c00700dd57475260a05f5133caf1f7 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Mon, 22 Apr 2024 13:48:28 +0200 Subject: [PATCH 0252/1018] Update math-functions argument types. Many functions have argument type listed as `Float64` type when it can be `Float*`,`(U)Int*` or in some cases `Decimal*` --- .../sql-reference/functions/math-functions.md | 68 ++++++++++--------- 1 file changed, 35 insertions(+), 33 deletions(-) diff --git a/docs/en/sql-reference/functions/math-functions.md b/docs/en/sql-reference/functions/math-functions.md index 2d0ae28aefc..5ebc6191010 100644 --- a/docs/en/sql-reference/functions/math-functions.md +++ b/docs/en/sql-reference/functions/math-functions.md @@ -8,7 +8,7 @@ sidebar_label: Mathematical ## e -Returns e ([Euler's constant](https://en.wikipedia.org/wiki/Euler%27s_constant)) +Returns e ([Euler's constant](https://en.wikipedia.org/wiki/Euler%27s_constant)). **Syntax** @@ -45,7 +45,7 @@ exp(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -65,7 +65,7 @@ Alias: `ln(x)` **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -83,7 +83,7 @@ exp2(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -111,7 +111,7 @@ log2(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -129,7 +129,7 @@ exp10(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -157,7 +157,7 @@ log10(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -173,7 +173,7 @@ sqrt(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -189,7 +189,7 @@ cbrt(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -207,7 +207,7 @@ erf(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -239,7 +239,7 @@ erfc(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -257,7 +257,7 @@ lgamma(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -275,7 +275,7 @@ gamma(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -293,7 +293,7 @@ sin(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -323,7 +323,7 @@ cos(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -341,7 +341,7 @@ tan(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -359,7 +359,7 @@ asin(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -377,7 +377,7 @@ acos(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -395,7 +395,7 @@ atan(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -434,7 +434,7 @@ cosh(x) **Arguments** -- `x` — The angle, in radians. Values from the interval: `-∞ < x < +∞`. [Float64](../../sql-reference/data-types/float.md#float32-float64). +- `x` — The angle, in radians. Values from the interval: `-∞ < x < +∞`. [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -468,7 +468,7 @@ acosh(x) **Arguments** -- `x` — Hyperbolic cosine of angle. Values from the interval: `1 <= x < +∞`. [Float64](../../sql-reference/data-types/float.md#float32-float64). +- `x` — Hyperbolic cosine of angle. Values from the interval: `1 <= x < +∞`. [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -502,7 +502,7 @@ sinh(x) **Arguments** -- `x` — The angle, in radians. Values from the interval: `-∞ < x < +∞`. [Float64](../../sql-reference/data-types/float.md#float32-float64). +- `x` — The angle, in radians. Values from the interval: `-∞ < x < +∞`. [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -536,7 +536,7 @@ asinh(x) **Arguments** -- `x` — Hyperbolic sine of angle. Values from the interval: `-∞ < x < +∞`. [Float64](../../sql-reference/data-types/float.md#float32-float64). +- `x` — Hyperbolic sine of angle. Values from the interval: `-∞ < x < +∞`. [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -569,13 +569,13 @@ tanh(x) **Arguments** -- `x` — The angle, in radians. Values from the interval: `-∞ < x < +∞`. [Float64](../../sql-reference/data-types/float.md#float32-float64). +- `x` — The angle, in radians. Values from the interval: `-∞ < x < +∞`. [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** - Values from the interval: `-1 < tanh(x) < 1`. -Type: [Float32](../../sql-reference/data-types/float.md#float32-float64). +Type: [Float*](../../sql-reference/data-types/float.md#float32-float64). **Example** @@ -601,7 +601,7 @@ atanh(x) **Arguments** -- `x` — Hyperbolic tangent of angle. Values from the interval: `–1 < x < 1`. [Float64](../../sql-reference/data-types/float.md#float32-float64). +- `x` — Hyperbolic tangent of angle. Values from the interval: `–1 < x < 1`. [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -635,8 +635,8 @@ atan2(y, x) **Arguments** -- `y` — y-coordinate of the point through which the ray passes. [Float64](../../sql-reference/data-types/float.md#float32-float64). -- `x` — x-coordinate of the point through which the ray passes. [Float64](../../sql-reference/data-types/float.md#float32-float64). +- `y` — y-coordinate of the point through which the ray passes. [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md). +- `x` — x-coordinate of the point through which the ray passes. [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md). **Returned value** @@ -670,8 +670,8 @@ hypot(x, y) **Arguments** -- `x` — The first cathetus of a right-angle triangle. [Float64](../../sql-reference/data-types/float.md#float32-float64). -- `y` — The second cathetus of a right-angle triangle. [Float64](../../sql-reference/data-types/float.md#float32-float64). +- `x` — The first cathetus of a right-angle triangle. [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md). +- `y` — The second cathetus of a right-angle triangle. [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md). **Returned value** @@ -705,7 +705,7 @@ log1p(x) **Arguments** -- `x` — Values from the interval: `-1 < x < +∞`. [Float64](../../sql-reference/data-types/float.md#float32-float64). +- `x` — Values from the interval: `-1 < x < +∞`. [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -747,6 +747,8 @@ sign(x) - 0 for `x = 0` - 1 for `x > 0` +Type: [Int8](../../sql-reference/data-types/int-uint.md). + **Examples** Sign for the zero value: @@ -803,7 +805,7 @@ degrees(x) **Arguments** -- `x` — Input in radians. [Float64](../../sql-reference/data-types/float.md#float32-float64). +- `x` — Input in radians. [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -837,7 +839,7 @@ radians(x) **Arguments** -- `x` — Input in degrees. [Float64](../../sql-reference/data-types/float.md#float32-float64). +- `x` — Input in degrees. [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** From 4a68b2a877bc30322552031535ede4247e08385b Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 22 Apr 2024 13:19:45 +0100 Subject: [PATCH 0253/1018] Allow quota key with different auth scheme --- src/Server/HTTPHandler.cpp | 17 ++++++++++++----- .../01399_http_request_headers.reference | 1 + .../0_stateless/01399_http_request_headers.sh | 1 + 3 files changed, 14 insertions(+), 5 deletions(-) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 81db7c04eb3..3fb44863425 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -364,16 +364,26 @@ bool HTTPHandler::authenticateUser( /// The header 'X-ClickHouse-SSL-Certificate-Auth: on' enables checking the common name /// extracted from the SSL certificate used for this connection instead of checking password. bool has_ssl_certificate_auth = (request.get("X-ClickHouse-SSL-Certificate-Auth", "") == "on"); - bool has_auth_headers = !user.empty() || !password.empty() || !quota_key.empty() || has_ssl_certificate_auth; + bool has_auth_headers = !user.empty() || !password.empty() || has_ssl_certificate_auth; /// User name and password can be passed using HTTP Basic auth or query parameters /// (both methods are insecure). bool has_http_credentials = request.hasCredentials(); - bool has_credentials_in_query_params = params.has("user") || params.has("password") || params.has("quota_key"); + bool has_credentials_in_query_params = params.has("user") || params.has("password"); std::string spnego_challenge; std::string certificate_common_name; + if (params.has("quota_key")) + { + if (!quota_key.empty()) + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, + "Invalid authentication: it is not allowed " + "to use quota key as HTTP header and quota key as parameter simultaneously"); + + quota_key = params.get("quota_key"); + } + if (has_auth_headers) { /// It is prohibited to mix different authorization schemes. @@ -435,15 +445,12 @@ bool HTTPHandler::authenticateUser( { throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: '{}' HTTP Authorization scheme is not supported", scheme); } - - quota_key = params.get("quota_key", ""); } else { /// If the user name is not set we assume it's the 'default' user. user = params.get("user", "default"); password = params.get("password", ""); - quota_key = params.get("quota_key", ""); } if (!certificate_common_name.empty()) diff --git a/tests/queries/0_stateless/01399_http_request_headers.reference b/tests/queries/0_stateless/01399_http_request_headers.reference index 92ea6606a12..80f52aa671b 100644 --- a/tests/queries/0_stateless/01399_http_request_headers.reference +++ b/tests/queries/0_stateless/01399_http_request_headers.reference @@ -10,3 +10,4 @@ processes Code: 81 [1] Code: 73 +Code: 516 diff --git a/tests/queries/0_stateless/01399_http_request_headers.sh b/tests/queries/0_stateless/01399_http_request_headers.sh index f06e7ffc32b..01600ecbe55 100755 --- a/tests/queries/0_stateless/01399_http_request_headers.sh +++ b/tests/queries/0_stateless/01399_http_request_headers.sh @@ -15,3 +15,4 @@ ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -H 'X-ClickHouse-Database: system' -d ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -H 'X-ClickHouse-Database: header_test' -d 'SHOW TABLES' | grep -o 'Code: 81' ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -H 'X-ClickHouse-Format: JSONCompactEachRow' -d 'SELECT 1' | grep -o '\[1\]' ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -H 'X-ClickHouse-Format: header_test' -d 'SELECT 1' | grep -o 'Code: 73' +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}"a_key=pingpong" -H 'Authorization: Basic dHQ6cGFzc3dvcmQ=' -d 'SELECT 1' | grep -o 'Code: 516' From 730995f2a4df4f73a7d467e16fafe655a286ebbb Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 22 Apr 2024 13:42:33 +0100 Subject: [PATCH 0254/1018] Review suggestion --- src/Server/HTTPHandler.cpp | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 3fb44863425..4d29e4d6a87 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -374,16 +374,6 @@ bool HTTPHandler::authenticateUser( std::string spnego_challenge; std::string certificate_common_name; - if (params.has("quota_key")) - { - if (!quota_key.empty()) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: it is not allowed " - "to use quota key as HTTP header and quota key as parameter simultaneously"); - - quota_key = params.get("quota_key"); - } - if (has_auth_headers) { /// It is prohibited to mix different authorization schemes. @@ -502,6 +492,16 @@ bool HTTPHandler::authenticateUser( basic_credentials->setPassword(password); } + if (params.has("quota_key")) + { + if (!quota_key.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Invalid authentication: it is not allowed " + "to use quota key as HTTP header and as parameter simultaneously"); + + quota_key = params.get("quota_key"); + } + /// Set client info. It will be used for quota accounting parameters in 'setUser' method. session->setHTTPClientInfo(request); From ed3f60599ecfff23d2caa972888b1d0646e40b8c Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 22 Apr 2024 13:53:42 +0100 Subject: [PATCH 0255/1018] Fix test --- tests/queries/0_stateless/01399_http_request_headers.reference | 2 +- tests/queries/0_stateless/01399_http_request_headers.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01399_http_request_headers.reference b/tests/queries/0_stateless/01399_http_request_headers.reference index 80f52aa671b..d13b6400092 100644 --- a/tests/queries/0_stateless/01399_http_request_headers.reference +++ b/tests/queries/0_stateless/01399_http_request_headers.reference @@ -10,4 +10,4 @@ processes Code: 81 [1] Code: 73 -Code: 516 +1 diff --git a/tests/queries/0_stateless/01399_http_request_headers.sh b/tests/queries/0_stateless/01399_http_request_headers.sh index 01600ecbe55..8576f374f7e 100755 --- a/tests/queries/0_stateless/01399_http_request_headers.sh +++ b/tests/queries/0_stateless/01399_http_request_headers.sh @@ -15,4 +15,4 @@ ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -H 'X-ClickHouse-Database: system' -d ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -H 'X-ClickHouse-Database: header_test' -d 'SHOW TABLES' | grep -o 'Code: 81' ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -H 'X-ClickHouse-Format: JSONCompactEachRow' -d 'SELECT 1' | grep -o '\[1\]' ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -H 'X-ClickHouse-Format: header_test' -d 'SELECT 1' | grep -o 'Code: 73' -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}"a_key=pingpong" -H 'Authorization: Basic dHQ6cGFzc3dvcmQ=' -d 'SELECT 1' | grep -o 'Code: 516' +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}"a_key=pingpong" -H 'X-ClickHouse-User: default' -d 'SELECT 1' From 7c0c3168c18f91bd800f8540f63f5d3590ea4501 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 22 Apr 2024 16:13:13 +0200 Subject: [PATCH 0256/1018] Revert "Add test for bug #33446" --- tests/performance/one_thousand_joins.xml | 6 ------ .../queries/0_stateless/03094_one_thousand_joins.reference | 1 - tests/queries/0_stateless/03094_one_thousand_joins.sql | 4 ---- 3 files changed, 11 deletions(-) delete mode 100644 tests/performance/one_thousand_joins.xml delete mode 100644 tests/queries/0_stateless/03094_one_thousand_joins.reference delete mode 100644 tests/queries/0_stateless/03094_one_thousand_joins.sql diff --git a/tests/performance/one_thousand_joins.xml b/tests/performance/one_thousand_joins.xml deleted file mode 100644 index d7e66bb1c67..00000000000 --- a/tests/performance/one_thousand_joins.xml +++ /dev/null @@ -1,6 +0,0 @@ - - - -SELECT * FROM (SELECT 1 AS x) t1 JOIN (SELECT 1 AS x) t2 ON t1.x = t2.x JOIN (SELECT 1 AS x) t3 ON t1.x = t3.x JOIN (SELECT 1 AS x) t4 ON t1.x = t4.x JOIN (SELECT 1 AS x) t5 ON t1.x = t5.x JOIN (SELECT 1 AS x) t6 ON t1.x = t6.x JOIN (SELECT 1 AS x) t7 ON t1.x = t7.x JOIN (SELECT 1 AS x) t8 ON t1.x = t8.x JOIN (SELECT 1 AS x) t9 ON t1.x = t9.x JOIN (SELECT 1 AS x) t10 ON t1.x = t10.x JOIN (SELECT 1 AS x) t11 ON t1.x = t11.x JOIN (SELECT 1 AS x) t12 ON t1.x = t12.x JOIN (SELECT 1 AS x) t13 ON t1.x = t13.x JOIN (SELECT 1 AS x) t14 ON t1.x = t14.x JOIN (SELECT 1 AS x) t15 ON t1.x = t15.x JOIN (SELECT 1 AS x) t16 ON t1.x = t16.x JOIN (SELECT 1 AS x) t17 ON t1.x = t17.x JOIN (SELECT 1 AS x) t18 ON t1.x = t18.x JOIN (SELECT 1 AS x) t19 ON t1.x = t19.x JOIN (SELECT 1 AS x) t20 ON t1.x = t20.x JOIN (SELECT 1 AS x) t21 ON t1.x = t21.x JOIN (SELECT 1 AS x) t22 ON t1.x = t22.x JOIN (SELECT 1 AS x) t23 ON t1.x = t23.x JOIN (SELECT 1 AS x) t24 ON t1.x = t24.x JOIN (SELECT 1 AS x) t25 ON t1.x = t25.x JOIN (SELECT 1 AS x) t26 ON t1.x = t26.x JOIN (SELECT 1 AS x) t27 ON t1.x = t27.x JOIN (SELECT 1 AS x) t28 ON t1.x = t28.x JOIN (SELECT 1 AS x) t29 ON t1.x = t29.x JOIN (SELECT 1 AS x) t30 ON t1.x = t30.x JOIN (SELECT 1 AS x) t31 ON t1.x = t31.x JOIN (SELECT 1 AS x) t32 ON t1.x = t32.x JOIN (SELECT 1 AS x) t33 ON t1.x = t33.x JOIN (SELECT 1 AS x) t34 ON t1.x = t34.x JOIN (SELECT 1 AS x) t35 ON t1.x = t35.x JOIN (SELECT 1 AS x) t36 ON t1.x = t36.x JOIN (SELECT 1 AS x) t37 ON t1.x = t37.x JOIN (SELECT 1 AS x) t38 ON t1.x = t38.x JOIN (SELECT 1 AS x) t39 ON t1.x = t39.x JOIN (SELECT 1 AS x) t40 ON t1.x = t40.x JOIN (SELECT 1 AS x) t41 ON t1.x = t41.x JOIN (SELECT 1 AS x) t42 ON t1.x = t42.x JOIN (SELECT 1 AS x) t43 ON t1.x = t43.x JOIN (SELECT 1 AS x) t44 ON t1.x = t44.x JOIN (SELECT 1 AS x) t45 ON t1.x = t45.x JOIN (SELECT 1 AS x) t46 ON t1.x = t46.x JOIN (SELECT 1 AS x) t47 ON t1.x = t47.x JOIN (SELECT 1 AS x) t48 ON t1.x = t48.x JOIN (SELECT 1 AS x) t49 ON t1.x = t49.x JOIN (SELECT 1 AS x) t50 ON t1.x = t50.x JOIN (SELECT 1 AS x) t51 ON t1.x = t51.x JOIN (SELECT 1 AS x) t52 ON t1.x = t52.x JOIN (SELECT 1 AS x) t53 ON t1.x = t53.x JOIN (SELECT 1 AS x) t54 ON t1.x = t54.x JOIN (SELECT 1 AS x) t55 ON t1.x = t55.x JOIN (SELECT 1 AS x) t56 ON t1.x = t56.x JOIN (SELECT 1 AS x) t57 ON t1.x = t57.x JOIN (SELECT 1 AS x) t58 ON t1.x = t58.x JOIN (SELECT 1 AS x) t59 ON t1.x = t59.x JOIN (SELECT 1 AS x) t60 ON t1.x = t60.x JOIN (SELECT 1 AS x) t61 ON t1.x = t61.x JOIN (SELECT 1 AS x) t62 ON t1.x = t62.x JOIN (SELECT 1 AS x) t63 ON t1.x = t63.x JOIN (SELECT 1 AS x) t64 ON t1.x = t64.x JOIN (SELECT 1 AS x) t65 ON t1.x = t65.x JOIN (SELECT 1 AS x) t66 ON t1.x = t66.x JOIN (SELECT 1 AS x) t67 ON t1.x = t67.x JOIN (SELECT 1 AS x) t68 ON t1.x = t68.x JOIN (SELECT 1 AS x) t69 ON t1.x = t69.x JOIN (SELECT 1 AS x) t70 ON t1.x = t70.x JOIN (SELECT 1 AS x) t71 ON t1.x = t71.x JOIN (SELECT 1 AS x) t72 ON t1.x = t72.x JOIN (SELECT 1 AS x) t73 ON t1.x = t73.x JOIN (SELECT 1 AS x) t74 ON t1.x = t74.x JOIN (SELECT 1 AS x) t75 ON t1.x = t75.x JOIN (SELECT 1 AS x) t76 ON t1.x = t76.x JOIN (SELECT 1 AS x) t77 ON t1.x = t77.x JOIN (SELECT 1 AS x) t78 ON t1.x = t78.x JOIN (SELECT 1 AS x) t79 ON t1.x = t79.x JOIN (SELECT 1 AS x) t80 ON t1.x = t80.x JOIN (SELECT 1 AS x) t81 ON t1.x = t81.x JOIN (SELECT 1 AS x) t82 ON t1.x = t82.x JOIN (SELECT 1 AS x) t83 ON t1.x = t83.x JOIN (SELECT 1 AS x) t84 ON t1.x = t84.x JOIN (SELECT 1 AS x) t85 ON t1.x = t85.x JOIN (SELECT 1 AS x) t86 ON t1.x = t86.x JOIN (SELECT 1 AS x) t87 ON t1.x = t87.x JOIN (SELECT 1 AS x) t88 ON t1.x = t88.x JOIN (SELECT 1 AS x) t89 ON t1.x = t89.x JOIN (SELECT 1 AS x) t90 ON t1.x = t90.x JOIN (SELECT 1 AS x) t91 ON t1.x = t91.x JOIN (SELECT 1 AS x) t92 ON t1.x = t92.x JOIN (SELECT 1 AS x) t93 ON t1.x = t93.x JOIN (SELECT 1 AS x) t94 ON t1.x = t94.x JOIN (SELECT 1 AS x) t95 ON t1.x = t95.x JOIN (SELECT 1 AS x) t96 ON t1.x = t96.x JOIN (SELECT 1 AS x) t97 ON t1.x = t97.x JOIN (SELECT 1 AS x) t98 ON t1.x = t98.x JOIN (SELECT 1 AS x) t99 ON t1.x = t99.x JOIN (SELECT 1 AS x) t100 ON t1.x = t100.x JOIN (SELECT 1 AS x) t101 ON t1.x = t101.x JOIN (SELECT 1 AS x) t102 ON t1.x = t102.x JOIN (SELECT 1 AS x) t103 ON t1.x = t103.x JOIN (SELECT 1 AS x) t104 ON t1.x = t104.x JOIN (SELECT 1 AS x) t105 ON t1.x = t105.x JOIN (SELECT 1 AS x) t106 ON t1.x = t106.x JOIN (SELECT 1 AS x) t107 ON t1.x = t107.x JOIN (SELECT 1 AS x) t108 ON t1.x = t108.x JOIN (SELECT 1 AS x) t109 ON t1.x = t109.x JOIN (SELECT 1 AS x) t110 ON t1.x = t110.x JOIN (SELECT 1 AS x) t111 ON t1.x = t111.x JOIN (SELECT 1 AS x) t112 ON t1.x = t112.x JOIN (SELECT 1 AS x) t113 ON t1.x = t113.x JOIN (SELECT 1 AS x) t114 ON t1.x = t114.x JOIN (SELECT 1 AS x) t115 ON t1.x = t115.x JOIN (SELECT 1 AS x) t116 ON t1.x = t116.x JOIN (SELECT 1 AS x) t117 ON t1.x = t117.x JOIN (SELECT 1 AS x) t118 ON t1.x = t118.x JOIN (SELECT 1 AS x) t119 ON t1.x = t119.x JOIN (SELECT 1 AS x) t120 ON t1.x = t120.x JOIN (SELECT 1 AS x) t121 ON t1.x = t121.x JOIN (SELECT 1 AS x) t122 ON t1.x = t122.x JOIN (SELECT 1 AS x) t123 ON t1.x = t123.x JOIN (SELECT 1 AS x) t124 ON t1.x = t124.x JOIN (SELECT 1 AS x) t125 ON t1.x = t125.x JOIN (SELECT 1 AS x) t126 ON t1.x = t126.x JOIN (SELECT 1 AS x) t127 ON t1.x = t127.x JOIN (SELECT 1 AS x) t128 ON t1.x = t128.x JOIN (SELECT 1 AS x) t129 ON t1.x = t129.x JOIN (SELECT 1 AS x) t130 ON t1.x = t130.x JOIN (SELECT 1 AS x) t131 ON t1.x = t131.x JOIN (SELECT 1 AS x) t132 ON t1.x = t132.x JOIN (SELECT 1 AS x) t133 ON t1.x = t133.x JOIN (SELECT 1 AS x) t134 ON t1.x = t134.x JOIN (SELECT 1 AS x) t135 ON t1.x = t135.x JOIN (SELECT 1 AS x) t136 ON t1.x = t136.x JOIN (SELECT 1 AS x) t137 ON t1.x = t137.x JOIN (SELECT 1 AS x) t138 ON t1.x = t138.x JOIN (SELECT 1 AS x) t139 ON t1.x = t139.x JOIN (SELECT 1 AS x) t140 ON t1.x = t140.x JOIN (SELECT 1 AS x) t141 ON t1.x = t141.x JOIN (SELECT 1 AS x) t142 ON t1.x = t142.x JOIN (SELECT 1 AS x) t143 ON t1.x = t143.x JOIN (SELECT 1 AS x) t144 ON t1.x = t144.x JOIN (SELECT 1 AS x) t145 ON t1.x = t145.x JOIN (SELECT 1 AS x) t146 ON t1.x = t146.x JOIN (SELECT 1 AS x) t147 ON t1.x = t147.x JOIN (SELECT 1 AS x) t148 ON t1.x = t148.x JOIN (SELECT 1 AS x) t149 ON t1.x = t149.x JOIN (SELECT 1 AS x) t150 ON t1.x = t150.x JOIN (SELECT 1 AS x) t151 ON t1.x = t151.x JOIN (SELECT 1 AS x) t152 ON t1.x = t152.x JOIN (SELECT 1 AS x) t153 ON t1.x = t153.x JOIN (SELECT 1 AS x) t154 ON t1.x = t154.x JOIN (SELECT 1 AS x) t155 ON t1.x = t155.x JOIN (SELECT 1 AS x) t156 ON t1.x = t156.x JOIN (SELECT 1 AS x) t157 ON t1.x = t157.x JOIN (SELECT 1 AS x) t158 ON t1.x = t158.x JOIN (SELECT 1 AS x) t159 ON t1.x = t159.x JOIN (SELECT 1 AS x) t160 ON t1.x = t160.x JOIN (SELECT 1 AS x) t161 ON t1.x = t161.x JOIN (SELECT 1 AS x) t162 ON t1.x = t162.x JOIN (SELECT 1 AS x) t163 ON t1.x = t163.x JOIN (SELECT 1 AS x) t164 ON t1.x = t164.x JOIN (SELECT 1 AS x) t165 ON t1.x = t165.x JOIN (SELECT 1 AS x) t166 ON t1.x = t166.x JOIN (SELECT 1 AS x) t167 ON t1.x = t167.x JOIN (SELECT 1 AS x) t168 ON t1.x = t168.x JOIN (SELECT 1 AS x) t169 ON t1.x = t169.x JOIN (SELECT 1 AS x) t170 ON t1.x = t170.x JOIN (SELECT 1 AS x) t171 ON t1.x = t171.x JOIN (SELECT 1 AS x) t172 ON t1.x = t172.x JOIN (SELECT 1 AS x) t173 ON t1.x = t173.x JOIN (SELECT 1 AS x) t174 ON t1.x = t174.x JOIN (SELECT 1 AS x) t175 ON t1.x = t175.x JOIN (SELECT 1 AS x) t176 ON t1.x = t176.x JOIN (SELECT 1 AS x) t177 ON t1.x = t177.x JOIN (SELECT 1 AS x) t178 ON t1.x = t178.x JOIN (SELECT 1 AS x) t179 ON t1.x = t179.x JOIN (SELECT 1 AS x) t180 ON t1.x = t180.x JOIN (SELECT 1 AS x) t181 ON t1.x = t181.x JOIN (SELECT 1 AS x) t182 ON t1.x = t182.x JOIN (SELECT 1 AS x) t183 ON t1.x = t183.x JOIN (SELECT 1 AS x) t184 ON t1.x = t184.x JOIN (SELECT 1 AS x) t185 ON t1.x = t185.x JOIN (SELECT 1 AS x) t186 ON t1.x = t186.x JOIN (SELECT 1 AS x) t187 ON t1.x = t187.x JOIN (SELECT 1 AS x) t188 ON t1.x = t188.x JOIN (SELECT 1 AS x) t189 ON t1.x = t189.x JOIN (SELECT 1 AS x) t190 ON t1.x = t190.x JOIN (SELECT 1 AS x) t191 ON t1.x = t191.x JOIN (SELECT 1 AS x) t192 ON t1.x = t192.x JOIN (SELECT 1 AS x) t193 ON t1.x = t193.x JOIN (SELECT 1 AS x) t194 ON t1.x = t194.x JOIN (SELECT 1 AS x) t195 ON t1.x = t195.x JOIN (SELECT 1 AS x) t196 ON t1.x = t196.x JOIN (SELECT 1 AS x) t197 ON t1.x = t197.x JOIN (SELECT 1 AS x) t198 ON t1.x = t198.x JOIN (SELECT 1 AS x) t199 ON t1.x = t199.x JOIN (SELECT 1 AS x) t200 ON t1.x = t200.x JOIN (SELECT 1 AS x) t201 ON t1.x = t201.x JOIN (SELECT 1 AS x) t202 ON t1.x = t202.x JOIN (SELECT 1 AS x) t203 ON t1.x = t203.x JOIN (SELECT 1 AS x) t204 ON t1.x = t204.x JOIN (SELECT 1 AS x) t205 ON t1.x = t205.x JOIN (SELECT 1 AS x) t206 ON t1.x = t206.x JOIN (SELECT 1 AS x) t207 ON t1.x = t207.x JOIN (SELECT 1 AS x) t208 ON t1.x = t208.x JOIN (SELECT 1 AS x) t209 ON t1.x = t209.x JOIN (SELECT 1 AS x) t210 ON t1.x = t210.x JOIN (SELECT 1 AS x) t211 ON t1.x = t211.x JOIN (SELECT 1 AS x) t212 ON t1.x = t212.x JOIN (SELECT 1 AS x) t213 ON t1.x = t213.x JOIN (SELECT 1 AS x) t214 ON t1.x = t214.x JOIN (SELECT 1 AS x) t215 ON t1.x = t215.x JOIN (SELECT 1 AS x) t216 ON t1.x = t216.x JOIN (SELECT 1 AS x) t217 ON t1.x = t217.x JOIN (SELECT 1 AS x) t218 ON t1.x = t218.x JOIN (SELECT 1 AS x) t219 ON t1.x = t219.x JOIN (SELECT 1 AS x) t220 ON t1.x = t220.x JOIN (SELECT 1 AS x) t221 ON t1.x = t221.x JOIN (SELECT 1 AS x) t222 ON t1.x = t222.x JOIN (SELECT 1 AS x) t223 ON t1.x = t223.x JOIN (SELECT 1 AS x) t224 ON t1.x = t224.x JOIN (SELECT 1 AS x) t225 ON t1.x = t225.x JOIN (SELECT 1 AS x) t226 ON t1.x = t226.x JOIN (SELECT 1 AS x) t227 ON t1.x = t227.x JOIN (SELECT 1 AS x) t228 ON t1.x = t228.x JOIN (SELECT 1 AS x) t229 ON t1.x = t229.x JOIN (SELECT 1 AS x) t230 ON t1.x = t230.x JOIN (SELECT 1 AS x) t231 ON t1.x = t231.x JOIN (SELECT 1 AS x) t232 ON t1.x = t232.x JOIN (SELECT 1 AS x) t233 ON t1.x = t233.x JOIN (SELECT 1 AS x) t234 ON t1.x = t234.x JOIN (SELECT 1 AS x) t235 ON t1.x = t235.x JOIN (SELECT 1 AS x) t236 ON t1.x = t236.x JOIN (SELECT 1 AS x) t237 ON t1.x = t237.x JOIN (SELECT 1 AS x) t238 ON t1.x = t238.x JOIN (SELECT 1 AS x) t239 ON t1.x = t239.x JOIN (SELECT 1 AS x) t240 ON t1.x = t240.x JOIN (SELECT 1 AS x) t241 ON t1.x = t241.x JOIN (SELECT 1 AS x) t242 ON t1.x = t242.x JOIN (SELECT 1 AS x) t243 ON t1.x = t243.x JOIN (SELECT 1 AS x) t244 ON t1.x = t244.x JOIN (SELECT 1 AS x) t245 ON t1.x = t245.x JOIN (SELECT 1 AS x) t246 ON t1.x = t246.x JOIN (SELECT 1 AS x) t247 ON t1.x = t247.x JOIN (SELECT 1 AS x) t248 ON t1.x = t248.x JOIN (SELECT 1 AS x) t249 ON t1.x = t249.x JOIN (SELECT 1 AS x) t250 ON t1.x = t250.x JOIN (SELECT 1 AS x) t251 ON t1.x = t251.x JOIN (SELECT 1 AS x) t252 ON t1.x = t252.x JOIN (SELECT 1 AS x) t253 ON t1.x = t253.x JOIN (SELECT 1 AS x) t254 ON t1.x = t254.x JOIN (SELECT 1 AS x) t255 ON t1.x = t255.x JOIN (SELECT 1 AS x) t256 ON t1.x = t256.x JOIN (SELECT 1 AS x) t257 ON t1.x = t257.x JOIN (SELECT 1 AS x) t258 ON t1.x = t258.x JOIN (SELECT 1 AS x) t259 ON t1.x = t259.x JOIN (SELECT 1 AS x) t260 ON t1.x = t260.x JOIN (SELECT 1 AS x) t261 ON t1.x = t261.x JOIN (SELECT 1 AS x) t262 ON t1.x = t262.x JOIN (SELECT 1 AS x) t263 ON t1.x = t263.x JOIN (SELECT 1 AS x) t264 ON t1.x = t264.x JOIN (SELECT 1 AS x) t265 ON t1.x = t265.x JOIN (SELECT 1 AS x) t266 ON t1.x = t266.x JOIN (SELECT 1 AS x) t267 ON t1.x = t267.x JOIN (SELECT 1 AS x) t268 ON t1.x = t268.x JOIN (SELECT 1 AS x) t269 ON t1.x = t269.x JOIN (SELECT 1 AS x) t270 ON t1.x = t270.x JOIN (SELECT 1 AS x) t271 ON t1.x = t271.x JOIN (SELECT 1 AS x) t272 ON t1.x = t272.x JOIN (SELECT 1 AS x) t273 ON t1.x = t273.x JOIN (SELECT 1 AS x) t274 ON t1.x = t274.x JOIN (SELECT 1 AS x) t275 ON t1.x = t275.x JOIN (SELECT 1 AS x) t276 ON t1.x = t276.x JOIN (SELECT 1 AS x) t277 ON t1.x = t277.x JOIN (SELECT 1 AS x) t278 ON t1.x = t278.x JOIN (SELECT 1 AS x) t279 ON t1.x = t279.x JOIN (SELECT 1 AS x) t280 ON t1.x = t280.x JOIN (SELECT 1 AS x) t281 ON t1.x = t281.x JOIN (SELECT 1 AS x) t282 ON t1.x = t282.x JOIN (SELECT 1 AS x) t283 ON t1.x = t283.x JOIN (SELECT 1 AS x) t284 ON t1.x = t284.x JOIN (SELECT 1 AS x) t285 ON t1.x = t285.x JOIN (SELECT 1 AS x) t286 ON t1.x = t286.x JOIN (SELECT 1 AS x) t287 ON t1.x = t287.x JOIN (SELECT 1 AS x) t288 ON t1.x = t288.x JOIN (SELECT 1 AS x) t289 ON t1.x = t289.x JOIN (SELECT 1 AS x) t290 ON t1.x = t290.x JOIN (SELECT 1 AS x) t291 ON t1.x = t291.x JOIN (SELECT 1 AS x) t292 ON t1.x = t292.x JOIN (SELECT 1 AS x) t293 ON t1.x = t293.x JOIN (SELECT 1 AS x) t294 ON t1.x = t294.x JOIN (SELECT 1 AS x) t295 ON t1.x = t295.x JOIN (SELECT 1 AS x) t296 ON t1.x = t296.x JOIN (SELECT 1 AS x) t297 ON t1.x = t297.x JOIN (SELECT 1 AS x) t298 ON t1.x = t298.x JOIN (SELECT 1 AS x) t299 ON t1.x = t299.x JOIN (SELECT 1 AS x) t300 ON t1.x = t300.x JOIN (SELECT 1 AS x) t301 ON t1.x = t301.x JOIN (SELECT 1 AS x) t302 ON t1.x = t302.x JOIN (SELECT 1 AS x) t303 ON t1.x = t303.x JOIN (SELECT 1 AS x) t304 ON t1.x = t304.x JOIN (SELECT 1 AS x) t305 ON t1.x = t305.x JOIN (SELECT 1 AS x) t306 ON t1.x = t306.x JOIN (SELECT 1 AS x) t307 ON t1.x = t307.x JOIN (SELECT 1 AS x) t308 ON t1.x = t308.x JOIN (SELECT 1 AS x) t309 ON t1.x = t309.x JOIN (SELECT 1 AS x) t310 ON t1.x = t310.x JOIN (SELECT 1 AS x) t311 ON t1.x = t311.x JOIN (SELECT 1 AS x) t312 ON t1.x = t312.x JOIN (SELECT 1 AS x) t313 ON t1.x = t313.x JOIN (SELECT 1 AS x) t314 ON t1.x = t314.x JOIN (SELECT 1 AS x) t315 ON t1.x = t315.x JOIN (SELECT 1 AS x) t316 ON t1.x = t316.x JOIN (SELECT 1 AS x) t317 ON t1.x = t317.x JOIN (SELECT 1 AS x) t318 ON t1.x = t318.x JOIN (SELECT 1 AS x) t319 ON t1.x = t319.x JOIN (SELECT 1 AS x) t320 ON t1.x = t320.x JOIN (SELECT 1 AS x) t321 ON t1.x = t321.x JOIN (SELECT 1 AS x) t322 ON t1.x = t322.x JOIN (SELECT 1 AS x) t323 ON t1.x = t323.x JOIN (SELECT 1 AS x) t324 ON t1.x = t324.x JOIN (SELECT 1 AS x) t325 ON t1.x = t325.x JOIN (SELECT 1 AS x) t326 ON t1.x = t326.x JOIN (SELECT 1 AS x) t327 ON t1.x = t327.x JOIN (SELECT 1 AS x) t328 ON t1.x = t328.x JOIN (SELECT 1 AS x) t329 ON t1.x = t329.x JOIN (SELECT 1 AS x) t330 ON t1.x = t330.x JOIN (SELECT 1 AS x) t331 ON t1.x = t331.x JOIN (SELECT 1 AS x) t332 ON t1.x = t332.x JOIN (SELECT 1 AS x) t333 ON t1.x = t333.x JOIN (SELECT 1 AS x) t334 ON t1.x = t334.x JOIN (SELECT 1 AS x) t335 ON t1.x = t335.x JOIN (SELECT 1 AS x) t336 ON t1.x = t336.x JOIN (SELECT 1 AS x) t337 ON t1.x = t337.x JOIN (SELECT 1 AS x) t338 ON t1.x = t338.x JOIN (SELECT 1 AS x) t339 ON t1.x = t339.x JOIN (SELECT 1 AS x) t340 ON t1.x = t340.x JOIN (SELECT 1 AS x) t341 ON t1.x = t341.x JOIN (SELECT 1 AS x) t342 ON t1.x = t342.x JOIN (SELECT 1 AS x) t343 ON t1.x = t343.x JOIN (SELECT 1 AS x) t344 ON t1.x = t344.x JOIN (SELECT 1 AS x) t345 ON t1.x = t345.x JOIN (SELECT 1 AS x) t346 ON t1.x = t346.x JOIN (SELECT 1 AS x) t347 ON t1.x = t347.x JOIN (SELECT 1 AS x) t348 ON t1.x = t348.x JOIN (SELECT 1 AS x) t349 ON t1.x = t349.x JOIN (SELECT 1 AS x) t350 ON t1.x = t350.x JOIN (SELECT 1 AS x) t351 ON t1.x = t351.x JOIN (SELECT 1 AS x) t352 ON t1.x = t352.x JOIN (SELECT 1 AS x) t353 ON t1.x = t353.x JOIN (SELECT 1 AS x) t354 ON t1.x = t354.x JOIN (SELECT 1 AS x) t355 ON t1.x = t355.x JOIN (SELECT 1 AS x) t356 ON t1.x = t356.x JOIN (SELECT 1 AS x) t357 ON t1.x = t357.x JOIN (SELECT 1 AS x) t358 ON t1.x = t358.x JOIN (SELECT 1 AS x) t359 ON t1.x = t359.x JOIN (SELECT 1 AS x) t360 ON t1.x = t360.x JOIN (SELECT 1 AS x) t361 ON t1.x = t361.x JOIN (SELECT 1 AS x) t362 ON t1.x = t362.x JOIN (SELECT 1 AS x) t363 ON t1.x = t363.x JOIN (SELECT 1 AS x) t364 ON t1.x = t364.x JOIN (SELECT 1 AS x) t365 ON t1.x = t365.x JOIN (SELECT 1 AS x) t366 ON t1.x = t366.x JOIN (SELECT 1 AS x) t367 ON t1.x = t367.x JOIN (SELECT 1 AS x) t368 ON t1.x = t368.x JOIN (SELECT 1 AS x) t369 ON t1.x = t369.x JOIN (SELECT 1 AS x) t370 ON t1.x = t370.x JOIN (SELECT 1 AS x) t371 ON t1.x = t371.x JOIN (SELECT 1 AS x) t372 ON t1.x = t372.x JOIN (SELECT 1 AS x) t373 ON t1.x = t373.x JOIN (SELECT 1 AS x) t374 ON t1.x = t374.x JOIN (SELECT 1 AS x) t375 ON t1.x = t375.x JOIN (SELECT 1 AS x) t376 ON t1.x = t376.x JOIN (SELECT 1 AS x) t377 ON t1.x = t377.x JOIN (SELECT 1 AS x) t378 ON t1.x = t378.x JOIN (SELECT 1 AS x) t379 ON t1.x = t379.x JOIN (SELECT 1 AS x) t380 ON t1.x = t380.x JOIN (SELECT 1 AS x) t381 ON t1.x = t381.x JOIN (SELECT 1 AS x) t382 ON t1.x = t382.x JOIN (SELECT 1 AS x) t383 ON t1.x = t383.x JOIN (SELECT 1 AS x) t384 ON t1.x = t384.x JOIN (SELECT 1 AS x) t385 ON t1.x = t385.x JOIN (SELECT 1 AS x) t386 ON t1.x = t386.x JOIN (SELECT 1 AS x) t387 ON t1.x = t387.x JOIN (SELECT 1 AS x) t388 ON t1.x = t388.x JOIN (SELECT 1 AS x) t389 ON t1.x = t389.x JOIN (SELECT 1 AS x) t390 ON t1.x = t390.x JOIN (SELECT 1 AS x) t391 ON t1.x = t391.x JOIN (SELECT 1 AS x) t392 ON t1.x = t392.x JOIN (SELECT 1 AS x) t393 ON t1.x = t393.x JOIN (SELECT 1 AS x) t394 ON t1.x = t394.x JOIN (SELECT 1 AS x) t395 ON t1.x = t395.x JOIN (SELECT 1 AS x) t396 ON t1.x = t396.x JOIN (SELECT 1 AS x) t397 ON t1.x = t397.x JOIN (SELECT 1 AS x) t398 ON t1.x = t398.x JOIN (SELECT 1 AS x) t399 ON t1.x = t399.x JOIN (SELECT 1 AS x) t400 ON t1.x = t400.x JOIN (SELECT 1 AS x) t401 ON t1.x = t401.x JOIN (SELECT 1 AS x) t402 ON t1.x = t402.x JOIN (SELECT 1 AS x) t403 ON t1.x = t403.x JOIN (SELECT 1 AS x) t404 ON t1.x = t404.x JOIN (SELECT 1 AS x) t405 ON t1.x = t405.x JOIN (SELECT 1 AS x) t406 ON t1.x = t406.x JOIN (SELECT 1 AS x) t407 ON t1.x = t407.x JOIN (SELECT 1 AS x) t408 ON t1.x = t408.x JOIN (SELECT 1 AS x) t409 ON t1.x = t409.x JOIN (SELECT 1 AS x) t410 ON t1.x = t410.x JOIN (SELECT 1 AS x) t411 ON t1.x = t411.x JOIN (SELECT 1 AS x) t412 ON t1.x = t412.x JOIN (SELECT 1 AS x) t413 ON t1.x = t413.x JOIN (SELECT 1 AS x) t414 ON t1.x = t414.x JOIN (SELECT 1 AS x) t415 ON t1.x = t415.x JOIN (SELECT 1 AS x) t416 ON t1.x = t416.x JOIN (SELECT 1 AS x) t417 ON t1.x = t417.x JOIN (SELECT 1 AS x) t418 ON t1.x = t418.x JOIN (SELECT 1 AS x) t419 ON t1.x = t419.x JOIN (SELECT 1 AS x) t420 ON t1.x = t420.x JOIN (SELECT 1 AS x) t421 ON t1.x = t421.x JOIN (SELECT 1 AS x) t422 ON t1.x = t422.x JOIN (SELECT 1 AS x) t423 ON t1.x = t423.x JOIN (SELECT 1 AS x) t424 ON t1.x = t424.x JOIN (SELECT 1 AS x) t425 ON t1.x = t425.x JOIN (SELECT 1 AS x) t426 ON t1.x = t426.x JOIN (SELECT 1 AS x) t427 ON t1.x = t427.x JOIN (SELECT 1 AS x) t428 ON t1.x = t428.x JOIN (SELECT 1 AS x) t429 ON t1.x = t429.x JOIN (SELECT 1 AS x) t430 ON t1.x = t430.x JOIN (SELECT 1 AS x) t431 ON t1.x = t431.x JOIN (SELECT 1 AS x) t432 ON t1.x = t432.x JOIN (SELECT 1 AS x) t433 ON t1.x = t433.x JOIN (SELECT 1 AS x) t434 ON t1.x = t434.x JOIN (SELECT 1 AS x) t435 ON t1.x = t435.x JOIN (SELECT 1 AS x) t436 ON t1.x = t436.x JOIN (SELECT 1 AS x) t437 ON t1.x = t437.x JOIN (SELECT 1 AS x) t438 ON t1.x = t438.x JOIN (SELECT 1 AS x) t439 ON t1.x = t439.x JOIN (SELECT 1 AS x) t440 ON t1.x = t440.x JOIN (SELECT 1 AS x) t441 ON t1.x = t441.x JOIN (SELECT 1 AS x) t442 ON t1.x = t442.x JOIN (SELECT 1 AS x) t443 ON t1.x = t443.x JOIN (SELECT 1 AS x) t444 ON t1.x = t444.x JOIN (SELECT 1 AS x) t445 ON t1.x = t445.x JOIN (SELECT 1 AS x) t446 ON t1.x = t446.x JOIN (SELECT 1 AS x) t447 ON t1.x = t447.x JOIN (SELECT 1 AS x) t448 ON t1.x = t448.x JOIN (SELECT 1 AS x) t449 ON t1.x = t449.x JOIN (SELECT 1 AS x) t450 ON t1.x = t450.x JOIN (SELECT 1 AS x) t451 ON t1.x = t451.x JOIN (SELECT 1 AS x) t452 ON t1.x = t452.x JOIN (SELECT 1 AS x) t453 ON t1.x = t453.x JOIN (SELECT 1 AS x) t454 ON t1.x = t454.x JOIN (SELECT 1 AS x) t455 ON t1.x = t455.x JOIN (SELECT 1 AS x) t456 ON t1.x = t456.x JOIN (SELECT 1 AS x) t457 ON t1.x = t457.x JOIN (SELECT 1 AS x) t458 ON t1.x = t458.x JOIN (SELECT 1 AS x) t459 ON t1.x = t459.x JOIN (SELECT 1 AS x) t460 ON t1.x = t460.x JOIN (SELECT 1 AS x) t461 ON t1.x = t461.x JOIN (SELECT 1 AS x) t462 ON t1.x = t462.x JOIN (SELECT 1 AS x) t463 ON t1.x = t463.x JOIN (SELECT 1 AS x) t464 ON t1.x = t464.x JOIN (SELECT 1 AS x) t465 ON t1.x = t465.x JOIN (SELECT 1 AS x) t466 ON t1.x = t466.x JOIN (SELECT 1 AS x) t467 ON t1.x = t467.x JOIN (SELECT 1 AS x) t468 ON t1.x = t468.x JOIN (SELECT 1 AS x) t469 ON t1.x = t469.x JOIN (SELECT 1 AS x) t470 ON t1.x = t470.x JOIN (SELECT 1 AS x) t471 ON t1.x = t471.x JOIN (SELECT 1 AS x) t472 ON t1.x = t472.x JOIN (SELECT 1 AS x) t473 ON t1.x = t473.x JOIN (SELECT 1 AS x) t474 ON t1.x = t474.x JOIN (SELECT 1 AS x) t475 ON t1.x = t475.x JOIN (SELECT 1 AS x) t476 ON t1.x = t476.x JOIN (SELECT 1 AS x) t477 ON t1.x = t477.x JOIN (SELECT 1 AS x) t478 ON t1.x = t478.x JOIN (SELECT 1 AS x) t479 ON t1.x = t479.x JOIN (SELECT 1 AS x) t480 ON t1.x = t480.x JOIN (SELECT 1 AS x) t481 ON t1.x = t481.x JOIN (SELECT 1 AS x) t482 ON t1.x = t482.x JOIN (SELECT 1 AS x) t483 ON t1.x = t483.x JOIN (SELECT 1 AS x) t484 ON t1.x = t484.x JOIN (SELECT 1 AS x) t485 ON t1.x = t485.x JOIN (SELECT 1 AS x) t486 ON t1.x = t486.x JOIN (SELECT 1 AS x) t487 ON t1.x = t487.x JOIN (SELECT 1 AS x) t488 ON t1.x = t488.x JOIN (SELECT 1 AS x) t489 ON t1.x = t489.x JOIN (SELECT 1 AS x) t490 ON t1.x = t490.x JOIN (SELECT 1 AS x) t491 ON t1.x = t491.x JOIN (SELECT 1 AS x) t492 ON t1.x = t492.x JOIN (SELECT 1 AS x) t493 ON t1.x = t493.x JOIN (SELECT 1 AS x) t494 ON t1.x = t494.x JOIN (SELECT 1 AS x) t495 ON t1.x = t495.x JOIN (SELECT 1 AS x) t496 ON t1.x = t496.x JOIN (SELECT 1 AS x) t497 ON t1.x = t497.x JOIN (SELECT 1 AS x) t498 ON t1.x = t498.x JOIN (SELECT 1 AS x) t499 ON t1.x = t499.x JOIN (SELECT 1 AS x) t500 ON t1.x = t500.x JOIN (SELECT 1 AS x) t501 ON t1.x = t501.x JOIN (SELECT 1 AS x) t502 ON t1.x = t502.x JOIN (SELECT 1 AS x) t503 ON t1.x = t503.x JOIN (SELECT 1 AS x) t504 ON t1.x = t504.x JOIN (SELECT 1 AS x) t505 ON t1.x = t505.x JOIN (SELECT 1 AS x) t506 ON t1.x = t506.x JOIN (SELECT 1 AS x) t507 ON t1.x = t507.x JOIN (SELECT 1 AS x) t508 ON t1.x = t508.x JOIN (SELECT 1 AS x) t509 ON t1.x = t509.x JOIN (SELECT 1 AS x) t510 ON t1.x = t510.x JOIN (SELECT 1 AS x) t511 ON t1.x = t511.x JOIN (SELECT 1 AS x) t512 ON t1.x = t512.x JOIN (SELECT 1 AS x) t513 ON t1.x = t513.x JOIN (SELECT 1 AS x) t514 ON t1.x = t514.x JOIN (SELECT 1 AS x) t515 ON t1.x = t515.x JOIN (SELECT 1 AS x) t516 ON t1.x = t516.x JOIN (SELECT 1 AS x) t517 ON t1.x = t517.x JOIN (SELECT 1 AS x) t518 ON t1.x = t518.x JOIN (SELECT 1 AS x) t519 ON t1.x = t519.x JOIN (SELECT 1 AS x) t520 ON t1.x = t520.x JOIN (SELECT 1 AS x) t521 ON t1.x = t521.x JOIN (SELECT 1 AS x) t522 ON t1.x = t522.x JOIN (SELECT 1 AS x) t523 ON t1.x = t523.x JOIN (SELECT 1 AS x) t524 ON t1.x = t524.x JOIN (SELECT 1 AS x) t525 ON t1.x = t525.x JOIN (SELECT 1 AS x) t526 ON t1.x = t526.x JOIN (SELECT 1 AS x) t527 ON t1.x = t527.x JOIN (SELECT 1 AS x) t528 ON t1.x = t528.x JOIN (SELECT 1 AS x) t529 ON t1.x = t529.x JOIN (SELECT 1 AS x) t530 ON t1.x = t530.x JOIN (SELECT 1 AS x) t531 ON t1.x = t531.x JOIN (SELECT 1 AS x) t532 ON t1.x = t532.x JOIN (SELECT 1 AS x) t533 ON t1.x = t533.x JOIN (SELECT 1 AS x) t534 ON t1.x = t534.x JOIN (SELECT 1 AS x) t535 ON t1.x = t535.x JOIN (SELECT 1 AS x) t536 ON t1.x = t536.x JOIN (SELECT 1 AS x) t537 ON t1.x = t537.x JOIN (SELECT 1 AS x) t538 ON t1.x = t538.x JOIN (SELECT 1 AS x) t539 ON t1.x = t539.x JOIN (SELECT 1 AS x) t540 ON t1.x = t540.x JOIN (SELECT 1 AS x) t541 ON t1.x = t541.x JOIN (SELECT 1 AS x) t542 ON t1.x = t542.x JOIN (SELECT 1 AS x) t543 ON t1.x = t543.x JOIN (SELECT 1 AS x) t544 ON t1.x = t544.x JOIN (SELECT 1 AS x) t545 ON t1.x = t545.x JOIN (SELECT 1 AS x) t546 ON t1.x = t546.x JOIN (SELECT 1 AS x) t547 ON t1.x = t547.x JOIN (SELECT 1 AS x) t548 ON t1.x = t548.x JOIN (SELECT 1 AS x) t549 ON t1.x = t549.x JOIN (SELECT 1 AS x) t550 ON t1.x = t550.x JOIN (SELECT 1 AS x) t551 ON t1.x = t551.x JOIN (SELECT 1 AS x) t552 ON t1.x = t552.x JOIN (SELECT 1 AS x) t553 ON t1.x = t553.x JOIN (SELECT 1 AS x) t554 ON t1.x = t554.x JOIN (SELECT 1 AS x) t555 ON t1.x = t555.x JOIN (SELECT 1 AS x) t556 ON t1.x = t556.x JOIN (SELECT 1 AS x) t557 ON t1.x = t557.x JOIN (SELECT 1 AS x) t558 ON t1.x = t558.x JOIN (SELECT 1 AS x) t559 ON t1.x = t559.x JOIN (SELECT 1 AS x) t560 ON t1.x = t560.x JOIN (SELECT 1 AS x) t561 ON t1.x = t561.x JOIN (SELECT 1 AS x) t562 ON t1.x = t562.x JOIN (SELECT 1 AS x) t563 ON t1.x = t563.x JOIN (SELECT 1 AS x) t564 ON t1.x = t564.x JOIN (SELECT 1 AS x) t565 ON t1.x = t565.x JOIN (SELECT 1 AS x) t566 ON t1.x = t566.x JOIN (SELECT 1 AS x) t567 ON t1.x = t567.x JOIN (SELECT 1 AS x) t568 ON t1.x = t568.x JOIN (SELECT 1 AS x) t569 ON t1.x = t569.x JOIN (SELECT 1 AS x) t570 ON t1.x = t570.x JOIN (SELECT 1 AS x) t571 ON t1.x = t571.x JOIN (SELECT 1 AS x) t572 ON t1.x = t572.x JOIN (SELECT 1 AS x) t573 ON t1.x = t573.x JOIN (SELECT 1 AS x) t574 ON t1.x = t574.x JOIN (SELECT 1 AS x) t575 ON t1.x = t575.x JOIN (SELECT 1 AS x) t576 ON t1.x = t576.x JOIN (SELECT 1 AS x) t577 ON t1.x = t577.x JOIN (SELECT 1 AS x) t578 ON t1.x = t578.x JOIN (SELECT 1 AS x) t579 ON t1.x = t579.x JOIN (SELECT 1 AS x) t580 ON t1.x = t580.x JOIN (SELECT 1 AS x) t581 ON t1.x = t581.x JOIN (SELECT 1 AS x) t582 ON t1.x = t582.x JOIN (SELECT 1 AS x) t583 ON t1.x = t583.x JOIN (SELECT 1 AS x) t584 ON t1.x = t584.x JOIN (SELECT 1 AS x) t585 ON t1.x = t585.x JOIN (SELECT 1 AS x) t586 ON t1.x = t586.x JOIN (SELECT 1 AS x) t587 ON t1.x = t587.x JOIN (SELECT 1 AS x) t588 ON t1.x = t588.x JOIN (SELECT 1 AS x) t589 ON t1.x = t589.x JOIN (SELECT 1 AS x) t590 ON t1.x = t590.x JOIN (SELECT 1 AS x) t591 ON t1.x = t591.x JOIN (SELECT 1 AS x) t592 ON t1.x = t592.x JOIN (SELECT 1 AS x) t593 ON t1.x = t593.x JOIN (SELECT 1 AS x) t594 ON t1.x = t594.x JOIN (SELECT 1 AS x) t595 ON t1.x = t595.x JOIN (SELECT 1 AS x) t596 ON t1.x = t596.x JOIN (SELECT 1 AS x) t597 ON t1.x = t597.x JOIN (SELECT 1 AS x) t598 ON t1.x = t598.x JOIN (SELECT 1 AS x) t599 ON t1.x = t599.x JOIN (SELECT 1 AS x) t600 ON t1.x = t600.x JOIN (SELECT 1 AS x) t601 ON t1.x = t601.x JOIN (SELECT 1 AS x) t602 ON t1.x = t602.x JOIN (SELECT 1 AS x) t603 ON t1.x = t603.x JOIN (SELECT 1 AS x) t604 ON t1.x = t604.x JOIN (SELECT 1 AS x) t605 ON t1.x = t605.x JOIN (SELECT 1 AS x) t606 ON t1.x = t606.x JOIN (SELECT 1 AS x) t607 ON t1.x = t607.x JOIN (SELECT 1 AS x) t608 ON t1.x = t608.x JOIN (SELECT 1 AS x) t609 ON t1.x = t609.x JOIN (SELECT 1 AS x) t610 ON t1.x = t610.x JOIN (SELECT 1 AS x) t611 ON t1.x = t611.x JOIN (SELECT 1 AS x) t612 ON t1.x = t612.x JOIN (SELECT 1 AS x) t613 ON t1.x = t613.x JOIN (SELECT 1 AS x) t614 ON t1.x = t614.x JOIN (SELECT 1 AS x) t615 ON t1.x = t615.x JOIN (SELECT 1 AS x) t616 ON t1.x = t616.x JOIN (SELECT 1 AS x) t617 ON t1.x = t617.x JOIN (SELECT 1 AS x) t618 ON t1.x = t618.x JOIN (SELECT 1 AS x) t619 ON t1.x = t619.x JOIN (SELECT 1 AS x) t620 ON t1.x = t620.x JOIN (SELECT 1 AS x) t621 ON t1.x = t621.x JOIN (SELECT 1 AS x) t622 ON t1.x = t622.x JOIN (SELECT 1 AS x) t623 ON t1.x = t623.x JOIN (SELECT 1 AS x) t624 ON t1.x = t624.x JOIN (SELECT 1 AS x) t625 ON t1.x = t625.x JOIN (SELECT 1 AS x) t626 ON t1.x = t626.x JOIN (SELECT 1 AS x) t627 ON t1.x = t627.x JOIN (SELECT 1 AS x) t628 ON t1.x = t628.x JOIN (SELECT 1 AS x) t629 ON t1.x = t629.x JOIN (SELECT 1 AS x) t630 ON t1.x = t630.x JOIN (SELECT 1 AS x) t631 ON t1.x = t631.x JOIN (SELECT 1 AS x) t632 ON t1.x = t632.x JOIN (SELECT 1 AS x) t633 ON t1.x = t633.x JOIN (SELECT 1 AS x) t634 ON t1.x = t634.x JOIN (SELECT 1 AS x) t635 ON t1.x = t635.x JOIN (SELECT 1 AS x) t636 ON t1.x = t636.x JOIN (SELECT 1 AS x) t637 ON t1.x = t637.x JOIN (SELECT 1 AS x) t638 ON t1.x = t638.x JOIN (SELECT 1 AS x) t639 ON t1.x = t639.x JOIN (SELECT 1 AS x) t640 ON t1.x = t640.x JOIN (SELECT 1 AS x) t641 ON t1.x = t641.x JOIN (SELECT 1 AS x) t642 ON t1.x = t642.x JOIN (SELECT 1 AS x) t643 ON t1.x = t643.x JOIN (SELECT 1 AS x) t644 ON t1.x = t644.x JOIN (SELECT 1 AS x) t645 ON t1.x = t645.x JOIN (SELECT 1 AS x) t646 ON t1.x = t646.x JOIN (SELECT 1 AS x) t647 ON t1.x = t647.x JOIN (SELECT 1 AS x) t648 ON t1.x = t648.x JOIN (SELECT 1 AS x) t649 ON t1.x = t649.x JOIN (SELECT 1 AS x) t650 ON t1.x = t650.x JOIN (SELECT 1 AS x) t651 ON t1.x = t651.x JOIN (SELECT 1 AS x) t652 ON t1.x = t652.x JOIN (SELECT 1 AS x) t653 ON t1.x = t653.x JOIN (SELECT 1 AS x) t654 ON t1.x = t654.x JOIN (SELECT 1 AS x) t655 ON t1.x = t655.x JOIN (SELECT 1 AS x) t656 ON t1.x = t656.x JOIN (SELECT 1 AS x) t657 ON t1.x = t657.x JOIN (SELECT 1 AS x) t658 ON t1.x = t658.x JOIN (SELECT 1 AS x) t659 ON t1.x = t659.x JOIN (SELECT 1 AS x) t660 ON t1.x = t660.x JOIN (SELECT 1 AS x) t661 ON t1.x = t661.x JOIN (SELECT 1 AS x) t662 ON t1.x = t662.x JOIN (SELECT 1 AS x) t663 ON t1.x = t663.x JOIN (SELECT 1 AS x) t664 ON t1.x = t664.x JOIN (SELECT 1 AS x) t665 ON t1.x = t665.x JOIN (SELECT 1 AS x) t666 ON t1.x = t666.x - - diff --git a/tests/queries/0_stateless/03094_one_thousand_joins.reference b/tests/queries/0_stateless/03094_one_thousand_joins.reference deleted file mode 100644 index dd729ab0220..00000000000 --- a/tests/queries/0_stateless/03094_one_thousand_joins.reference +++ /dev/null @@ -1 +0,0 @@ -1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 diff --git a/tests/queries/0_stateless/03094_one_thousand_joins.sql b/tests/queries/0_stateless/03094_one_thousand_joins.sql deleted file mode 100644 index b6345595811..00000000000 --- a/tests/queries/0_stateless/03094_one_thousand_joins.sql +++ /dev/null @@ -1,4 +0,0 @@ --- Tags: no-fasttest, long - --- Bug 33446, marked as 'long' because it still runs around 10 sec -SELECT * FROM (SELECT 1 AS x) t1 JOIN (SELECT 1 AS x) t2 ON t1.x = t2.x JOIN (SELECT 1 AS x) t3 ON t1.x = t3.x JOIN (SELECT 1 AS x) t4 ON t1.x = t4.x JOIN (SELECT 1 AS x) t5 ON t1.x = t5.x JOIN (SELECT 1 AS x) t6 ON t1.x = t6.x JOIN (SELECT 1 AS x) t7 ON t1.x = t7.x JOIN (SELECT 1 AS x) t8 ON t1.x = t8.x JOIN (SELECT 1 AS x) t9 ON t1.x = t9.x JOIN (SELECT 1 AS x) t10 ON t1.x = t10.x JOIN (SELECT 1 AS x) t11 ON t1.x = t11.x JOIN (SELECT 1 AS x) t12 ON t1.x = t12.x JOIN (SELECT 1 AS x) t13 ON t1.x = t13.x JOIN (SELECT 1 AS x) t14 ON t1.x = t14.x JOIN (SELECT 1 AS x) t15 ON t1.x = t15.x JOIN (SELECT 1 AS x) t16 ON t1.x = t16.x JOIN (SELECT 1 AS x) t17 ON t1.x = t17.x JOIN (SELECT 1 AS x) t18 ON t1.x = t18.x JOIN (SELECT 1 AS x) t19 ON t1.x = t19.x JOIN (SELECT 1 AS x) t20 ON t1.x = t20.x JOIN (SELECT 1 AS x) t21 ON t1.x = t21.x JOIN (SELECT 1 AS x) t22 ON t1.x = t22.x JOIN (SELECT 1 AS x) t23 ON t1.x = t23.x JOIN (SELECT 1 AS x) t24 ON t1.x = t24.x JOIN (SELECT 1 AS x) t25 ON t1.x = t25.x JOIN (SELECT 1 AS x) t26 ON t1.x = t26.x JOIN (SELECT 1 AS x) t27 ON t1.x = t27.x JOIN (SELECT 1 AS x) t28 ON t1.x = t28.x JOIN (SELECT 1 AS x) t29 ON t1.x = t29.x JOIN (SELECT 1 AS x) t30 ON t1.x = t30.x JOIN (SELECT 1 AS x) t31 ON t1.x = t31.x JOIN (SELECT 1 AS x) t32 ON t1.x = t32.x JOIN (SELECT 1 AS x) t33 ON t1.x = t33.x JOIN (SELECT 1 AS x) t34 ON t1.x = t34.x JOIN (SELECT 1 AS x) t35 ON t1.x = t35.x JOIN (SELECT 1 AS x) t36 ON t1.x = t36.x JOIN (SELECT 1 AS x) t37 ON t1.x = t37.x JOIN (SELECT 1 AS x) t38 ON t1.x = t38.x JOIN (SELECT 1 AS x) t39 ON t1.x = t39.x JOIN (SELECT 1 AS x) t40 ON t1.x = t40.x JOIN (SELECT 1 AS x) t41 ON t1.x = t41.x JOIN (SELECT 1 AS x) t42 ON t1.x = t42.x JOIN (SELECT 1 AS x) t43 ON t1.x = t43.x JOIN (SELECT 1 AS x) t44 ON t1.x = t44.x JOIN (SELECT 1 AS x) t45 ON t1.x = t45.x JOIN (SELECT 1 AS x) t46 ON t1.x = t46.x JOIN (SELECT 1 AS x) t47 ON t1.x = t47.x JOIN (SELECT 1 AS x) t48 ON t1.x = t48.x JOIN (SELECT 1 AS x) t49 ON t1.x = t49.x JOIN (SELECT 1 AS x) t50 ON t1.x = t50.x JOIN (SELECT 1 AS x) t51 ON t1.x = t51.x JOIN (SELECT 1 AS x) t52 ON t1.x = t52.x JOIN (SELECT 1 AS x) t53 ON t1.x = t53.x JOIN (SELECT 1 AS x) t54 ON t1.x = t54.x JOIN (SELECT 1 AS x) t55 ON t1.x = t55.x JOIN (SELECT 1 AS x) t56 ON t1.x = t56.x JOIN (SELECT 1 AS x) t57 ON t1.x = t57.x JOIN (SELECT 1 AS x) t58 ON t1.x = t58.x JOIN (SELECT 1 AS x) t59 ON t1.x = t59.x JOIN (SELECT 1 AS x) t60 ON t1.x = t60.x JOIN (SELECT 1 AS x) t61 ON t1.x = t61.x JOIN (SELECT 1 AS x) t62 ON t1.x = t62.x JOIN (SELECT 1 AS x) t63 ON t1.x = t63.x JOIN (SELECT 1 AS x) t64 ON t1.x = t64.x JOIN (SELECT 1 AS x) t65 ON t1.x = t65.x JOIN (SELECT 1 AS x) t66 ON t1.x = t66.x JOIN (SELECT 1 AS x) t67 ON t1.x = t67.x JOIN (SELECT 1 AS x) t68 ON t1.x = t68.x JOIN (SELECT 1 AS x) t69 ON t1.x = t69.x JOIN (SELECT 1 AS x) t70 ON t1.x = t70.x JOIN (SELECT 1 AS x) t71 ON t1.x = t71.x JOIN (SELECT 1 AS x) t72 ON t1.x = t72.x JOIN (SELECT 1 AS x) t73 ON t1.x = t73.x JOIN (SELECT 1 AS x) t74 ON t1.x = t74.x JOIN (SELECT 1 AS x) t75 ON t1.x = t75.x JOIN (SELECT 1 AS x) t76 ON t1.x = t76.x JOIN (SELECT 1 AS x) t77 ON t1.x = t77.x JOIN (SELECT 1 AS x) t78 ON t1.x = t78.x JOIN (SELECT 1 AS x) t79 ON t1.x = t79.x JOIN (SELECT 1 AS x) t80 ON t1.x = t80.x JOIN (SELECT 1 AS x) t81 ON t1.x = t81.x JOIN (SELECT 1 AS x) t82 ON t1.x = t82.x JOIN (SELECT 1 AS x) t83 ON t1.x = t83.x JOIN (SELECT 1 AS x) t84 ON t1.x = t84.x JOIN (SELECT 1 AS x) t85 ON t1.x = t85.x JOIN (SELECT 1 AS x) t86 ON t1.x = t86.x JOIN (SELECT 1 AS x) t87 ON t1.x = t87.x JOIN (SELECT 1 AS x) t88 ON t1.x = t88.x JOIN (SELECT 1 AS x) t89 ON t1.x = t89.x JOIN (SELECT 1 AS x) t90 ON t1.x = t90.x JOIN (SELECT 1 AS x) t91 ON t1.x = t91.x JOIN (SELECT 1 AS x) t92 ON t1.x = t92.x JOIN (SELECT 1 AS x) t93 ON t1.x = t93.x JOIN (SELECT 1 AS x) t94 ON t1.x = t94.x JOIN (SELECT 1 AS x) t95 ON t1.x = t95.x JOIN (SELECT 1 AS x) t96 ON t1.x = t96.x JOIN (SELECT 1 AS x) t97 ON t1.x = t97.x JOIN (SELECT 1 AS x) t98 ON t1.x = t98.x JOIN (SELECT 1 AS x) t99 ON t1.x = t99.x JOIN (SELECT 1 AS x) t100 ON t1.x = t100.x JOIN (SELECT 1 AS x) t101 ON t1.x = t101.x JOIN (SELECT 1 AS x) t102 ON t1.x = t102.x JOIN (SELECT 1 AS x) t103 ON t1.x = t103.x JOIN (SELECT 1 AS x) t104 ON t1.x = t104.x JOIN (SELECT 1 AS x) t105 ON t1.x = t105.x JOIN (SELECT 1 AS x) t106 ON t1.x = t106.x JOIN (SELECT 1 AS x) t107 ON t1.x = t107.x JOIN (SELECT 1 AS x) t108 ON t1.x = t108.x JOIN (SELECT 1 AS x) t109 ON t1.x = t109.x JOIN (SELECT 1 AS x) t110 ON t1.x = t110.x JOIN (SELECT 1 AS x) t111 ON t1.x = t111.x JOIN (SELECT 1 AS x) t112 ON t1.x = t112.x JOIN (SELECT 1 AS x) t113 ON t1.x = t113.x JOIN (SELECT 1 AS x) t114 ON t1.x = t114.x JOIN (SELECT 1 AS x) t115 ON t1.x = t115.x JOIN (SELECT 1 AS x) t116 ON t1.x = t116.x JOIN (SELECT 1 AS x) t117 ON t1.x = t117.x JOIN (SELECT 1 AS x) t118 ON t1.x = t118.x JOIN (SELECT 1 AS x) t119 ON t1.x = t119.x JOIN (SELECT 1 AS x) t120 ON t1.x = t120.x JOIN (SELECT 1 AS x) t121 ON t1.x = t121.x JOIN (SELECT 1 AS x) t122 ON t1.x = t122.x JOIN (SELECT 1 AS x) t123 ON t1.x = t123.x JOIN (SELECT 1 AS x) t124 ON t1.x = t124.x JOIN (SELECT 1 AS x) t125 ON t1.x = t125.x JOIN (SELECT 1 AS x) t126 ON t1.x = t126.x JOIN (SELECT 1 AS x) t127 ON t1.x = t127.x JOIN (SELECT 1 AS x) t128 ON t1.x = t128.x JOIN (SELECT 1 AS x) t129 ON t1.x = t129.x JOIN (SELECT 1 AS x) t130 ON t1.x = t130.x JOIN (SELECT 1 AS x) t131 ON t1.x = t131.x JOIN (SELECT 1 AS x) t132 ON t1.x = t132.x JOIN (SELECT 1 AS x) t133 ON t1.x = t133.x JOIN (SELECT 1 AS x) t134 ON t1.x = t134.x JOIN (SELECT 1 AS x) t135 ON t1.x = t135.x JOIN (SELECT 1 AS x) t136 ON t1.x = t136.x JOIN (SELECT 1 AS x) t137 ON t1.x = t137.x JOIN (SELECT 1 AS x) t138 ON t1.x = t138.x JOIN (SELECT 1 AS x) t139 ON t1.x = t139.x JOIN (SELECT 1 AS x) t140 ON t1.x = t140.x JOIN (SELECT 1 AS x) t141 ON t1.x = t141.x JOIN (SELECT 1 AS x) t142 ON t1.x = t142.x JOIN (SELECT 1 AS x) t143 ON t1.x = t143.x JOIN (SELECT 1 AS x) t144 ON t1.x = t144.x JOIN (SELECT 1 AS x) t145 ON t1.x = t145.x JOIN (SELECT 1 AS x) t146 ON t1.x = t146.x JOIN (SELECT 1 AS x) t147 ON t1.x = t147.x JOIN (SELECT 1 AS x) t148 ON t1.x = t148.x JOIN (SELECT 1 AS x) t149 ON t1.x = t149.x JOIN (SELECT 1 AS x) t150 ON t1.x = t150.x JOIN (SELECT 1 AS x) t151 ON t1.x = t151.x JOIN (SELECT 1 AS x) t152 ON t1.x = t152.x JOIN (SELECT 1 AS x) t153 ON t1.x = t153.x JOIN (SELECT 1 AS x) t154 ON t1.x = t154.x JOIN (SELECT 1 AS x) t155 ON t1.x = t155.x JOIN (SELECT 1 AS x) t156 ON t1.x = t156.x JOIN (SELECT 1 AS x) t157 ON t1.x = t157.x JOIN (SELECT 1 AS x) t158 ON t1.x = t158.x JOIN (SELECT 1 AS x) t159 ON t1.x = t159.x JOIN (SELECT 1 AS x) t160 ON t1.x = t160.x JOIN (SELECT 1 AS x) t161 ON t1.x = t161.x JOIN (SELECT 1 AS x) t162 ON t1.x = t162.x JOIN (SELECT 1 AS x) t163 ON t1.x = t163.x JOIN (SELECT 1 AS x) t164 ON t1.x = t164.x JOIN (SELECT 1 AS x) t165 ON t1.x = t165.x JOIN (SELECT 1 AS x) t166 ON t1.x = t166.x JOIN (SELECT 1 AS x) t167 ON t1.x = t167.x JOIN (SELECT 1 AS x) t168 ON t1.x = t168.x JOIN (SELECT 1 AS x) t169 ON t1.x = t169.x JOIN (SELECT 1 AS x) t170 ON t1.x = t170.x JOIN (SELECT 1 AS x) t171 ON t1.x = t171.x JOIN (SELECT 1 AS x) t172 ON t1.x = t172.x JOIN (SELECT 1 AS x) t173 ON t1.x = t173.x JOIN (SELECT 1 AS x) t174 ON t1.x = t174.x JOIN (SELECT 1 AS x) t175 ON t1.x = t175.x JOIN (SELECT 1 AS x) t176 ON t1.x = t176.x JOIN (SELECT 1 AS x) t177 ON t1.x = t177.x JOIN (SELECT 1 AS x) t178 ON t1.x = t178.x JOIN (SELECT 1 AS x) t179 ON t1.x = t179.x JOIN (SELECT 1 AS x) t180 ON t1.x = t180.x JOIN (SELECT 1 AS x) t181 ON t1.x = t181.x JOIN (SELECT 1 AS x) t182 ON t1.x = t182.x JOIN (SELECT 1 AS x) t183 ON t1.x = t183.x JOIN (SELECT 1 AS x) t184 ON t1.x = t184.x JOIN (SELECT 1 AS x) t185 ON t1.x = t185.x JOIN (SELECT 1 AS x) t186 ON t1.x = t186.x JOIN (SELECT 1 AS x) t187 ON t1.x = t187.x JOIN (SELECT 1 AS x) t188 ON t1.x = t188.x JOIN (SELECT 1 AS x) t189 ON t1.x = t189.x JOIN (SELECT 1 AS x) t190 ON t1.x = t190.x JOIN (SELECT 1 AS x) t191 ON t1.x = t191.x JOIN (SELECT 1 AS x) t192 ON t1.x = t192.x JOIN (SELECT 1 AS x) t193 ON t1.x = t193.x JOIN (SELECT 1 AS x) t194 ON t1.x = t194.x JOIN (SELECT 1 AS x) t195 ON t1.x = t195.x JOIN (SELECT 1 AS x) t196 ON t1.x = t196.x JOIN (SELECT 1 AS x) t197 ON t1.x = t197.x JOIN (SELECT 1 AS x) t198 ON t1.x = t198.x JOIN (SELECT 1 AS x) t199 ON t1.x = t199.x JOIN (SELECT 1 AS x) t200 ON t1.x = t200.x JOIN (SELECT 1 AS x) t201 ON t1.x = t201.x JOIN (SELECT 1 AS x) t202 ON t1.x = t202.x JOIN (SELECT 1 AS x) t203 ON t1.x = t203.x JOIN (SELECT 1 AS x) t204 ON t1.x = t204.x JOIN (SELECT 1 AS x) t205 ON t1.x = t205.x JOIN (SELECT 1 AS x) t206 ON t1.x = t206.x JOIN (SELECT 1 AS x) t207 ON t1.x = t207.x JOIN (SELECT 1 AS x) t208 ON t1.x = t208.x JOIN (SELECT 1 AS x) t209 ON t1.x = t209.x JOIN (SELECT 1 AS x) t210 ON t1.x = t210.x JOIN (SELECT 1 AS x) t211 ON t1.x = t211.x JOIN (SELECT 1 AS x) t212 ON t1.x = t212.x JOIN (SELECT 1 AS x) t213 ON t1.x = t213.x JOIN (SELECT 1 AS x) t214 ON t1.x = t214.x JOIN (SELECT 1 AS x) t215 ON t1.x = t215.x JOIN (SELECT 1 AS x) t216 ON t1.x = t216.x JOIN (SELECT 1 AS x) t217 ON t1.x = t217.x JOIN (SELECT 1 AS x) t218 ON t1.x = t218.x JOIN (SELECT 1 AS x) t219 ON t1.x = t219.x JOIN (SELECT 1 AS x) t220 ON t1.x = t220.x JOIN (SELECT 1 AS x) t221 ON t1.x = t221.x JOIN (SELECT 1 AS x) t222 ON t1.x = t222.x JOIN (SELECT 1 AS x) t223 ON t1.x = t223.x JOIN (SELECT 1 AS x) t224 ON t1.x = t224.x JOIN (SELECT 1 AS x) t225 ON t1.x = t225.x JOIN (SELECT 1 AS x) t226 ON t1.x = t226.x JOIN (SELECT 1 AS x) t227 ON t1.x = t227.x JOIN (SELECT 1 AS x) t228 ON t1.x = t228.x JOIN (SELECT 1 AS x) t229 ON t1.x = t229.x JOIN (SELECT 1 AS x) t230 ON t1.x = t230.x JOIN (SELECT 1 AS x) t231 ON t1.x = t231.x JOIN (SELECT 1 AS x) t232 ON t1.x = t232.x JOIN (SELECT 1 AS x) t233 ON t1.x = t233.x JOIN (SELECT 1 AS x) t234 ON t1.x = t234.x JOIN (SELECT 1 AS x) t235 ON t1.x = t235.x JOIN (SELECT 1 AS x) t236 ON t1.x = t236.x JOIN (SELECT 1 AS x) t237 ON t1.x = t237.x JOIN (SELECT 1 AS x) t238 ON t1.x = t238.x JOIN (SELECT 1 AS x) t239 ON t1.x = t239.x JOIN (SELECT 1 AS x) t240 ON t1.x = t240.x JOIN (SELECT 1 AS x) t241 ON t1.x = t241.x JOIN (SELECT 1 AS x) t242 ON t1.x = t242.x JOIN (SELECT 1 AS x) t243 ON t1.x = t243.x JOIN (SELECT 1 AS x) t244 ON t1.x = t244.x JOIN (SELECT 1 AS x) t245 ON t1.x = t245.x JOIN (SELECT 1 AS x) t246 ON t1.x = t246.x JOIN (SELECT 1 AS x) t247 ON t1.x = t247.x JOIN (SELECT 1 AS x) t248 ON t1.x = t248.x JOIN (SELECT 1 AS x) t249 ON t1.x = t249.x JOIN (SELECT 1 AS x) t250 ON t1.x = t250.x JOIN (SELECT 1 AS x) t251 ON t1.x = t251.x JOIN (SELECT 1 AS x) t252 ON t1.x = t252.x JOIN (SELECT 1 AS x) t253 ON t1.x = t253.x JOIN (SELECT 1 AS x) t254 ON t1.x = t254.x JOIN (SELECT 1 AS x) t255 ON t1.x = t255.x JOIN (SELECT 1 AS x) t256 ON t1.x = t256.x JOIN (SELECT 1 AS x) t257 ON t1.x = t257.x JOIN (SELECT 1 AS x) t258 ON t1.x = t258.x JOIN (SELECT 1 AS x) t259 ON t1.x = t259.x JOIN (SELECT 1 AS x) t260 ON t1.x = t260.x JOIN (SELECT 1 AS x) t261 ON t1.x = t261.x JOIN (SELECT 1 AS x) t262 ON t1.x = t262.x JOIN (SELECT 1 AS x) t263 ON t1.x = t263.x JOIN (SELECT 1 AS x) t264 ON t1.x = t264.x JOIN (SELECT 1 AS x) t265 ON t1.x = t265.x JOIN (SELECT 1 AS x) t266 ON t1.x = t266.x JOIN (SELECT 1 AS x) t267 ON t1.x = t267.x JOIN (SELECT 1 AS x) t268 ON t1.x = t268.x JOIN (SELECT 1 AS x) t269 ON t1.x = t269.x JOIN (SELECT 1 AS x) t270 ON t1.x = t270.x JOIN (SELECT 1 AS x) t271 ON t1.x = t271.x JOIN (SELECT 1 AS x) t272 ON t1.x = t272.x JOIN (SELECT 1 AS x) t273 ON t1.x = t273.x JOIN (SELECT 1 AS x) t274 ON t1.x = t274.x JOIN (SELECT 1 AS x) t275 ON t1.x = t275.x JOIN (SELECT 1 AS x) t276 ON t1.x = t276.x JOIN (SELECT 1 AS x) t277 ON t1.x = t277.x JOIN (SELECT 1 AS x) t278 ON t1.x = t278.x JOIN (SELECT 1 AS x) t279 ON t1.x = t279.x JOIN (SELECT 1 AS x) t280 ON t1.x = t280.x JOIN (SELECT 1 AS x) t281 ON t1.x = t281.x JOIN (SELECT 1 AS x) t282 ON t1.x = t282.x JOIN (SELECT 1 AS x) t283 ON t1.x = t283.x JOIN (SELECT 1 AS x) t284 ON t1.x = t284.x JOIN (SELECT 1 AS x) t285 ON t1.x = t285.x JOIN (SELECT 1 AS x) t286 ON t1.x = t286.x JOIN (SELECT 1 AS x) t287 ON t1.x = t287.x JOIN (SELECT 1 AS x) t288 ON t1.x = t288.x JOIN (SELECT 1 AS x) t289 ON t1.x = t289.x JOIN (SELECT 1 AS x) t290 ON t1.x = t290.x JOIN (SELECT 1 AS x) t291 ON t1.x = t291.x JOIN (SELECT 1 AS x) t292 ON t1.x = t292.x JOIN (SELECT 1 AS x) t293 ON t1.x = t293.x JOIN (SELECT 1 AS x) t294 ON t1.x = t294.x JOIN (SELECT 1 AS x) t295 ON t1.x = t295.x JOIN (SELECT 1 AS x) t296 ON t1.x = t296.x JOIN (SELECT 1 AS x) t297 ON t1.x = t297.x JOIN (SELECT 1 AS x) t298 ON t1.x = t298.x JOIN (SELECT 1 AS x) t299 ON t1.x = t299.x JOIN (SELECT 1 AS x) t300 ON t1.x = t300.x JOIN (SELECT 1 AS x) t301 ON t1.x = t301.x JOIN (SELECT 1 AS x) t302 ON t1.x = t302.x JOIN (SELECT 1 AS x) t303 ON t1.x = t303.x JOIN (SELECT 1 AS x) t304 ON t1.x = t304.x JOIN (SELECT 1 AS x) t305 ON t1.x = t305.x JOIN (SELECT 1 AS x) t306 ON t1.x = t306.x JOIN (SELECT 1 AS x) t307 ON t1.x = t307.x JOIN (SELECT 1 AS x) t308 ON t1.x = t308.x JOIN (SELECT 1 AS x) t309 ON t1.x = t309.x JOIN (SELECT 1 AS x) t310 ON t1.x = t310.x JOIN (SELECT 1 AS x) t311 ON t1.x = t311.x JOIN (SELECT 1 AS x) t312 ON t1.x = t312.x JOIN (SELECT 1 AS x) t313 ON t1.x = t313.x JOIN (SELECT 1 AS x) t314 ON t1.x = t314.x JOIN (SELECT 1 AS x) t315 ON t1.x = t315.x JOIN (SELECT 1 AS x) t316 ON t1.x = t316.x JOIN (SELECT 1 AS x) t317 ON t1.x = t317.x JOIN (SELECT 1 AS x) t318 ON t1.x = t318.x JOIN (SELECT 1 AS x) t319 ON t1.x = t319.x JOIN (SELECT 1 AS x) t320 ON t1.x = t320.x JOIN (SELECT 1 AS x) t321 ON t1.x = t321.x JOIN (SELECT 1 AS x) t322 ON t1.x = t322.x JOIN (SELECT 1 AS x) t323 ON t1.x = t323.x JOIN (SELECT 1 AS x) t324 ON t1.x = t324.x JOIN (SELECT 1 AS x) t325 ON t1.x = t325.x JOIN (SELECT 1 AS x) t326 ON t1.x = t326.x JOIN (SELECT 1 AS x) t327 ON t1.x = t327.x JOIN (SELECT 1 AS x) t328 ON t1.x = t328.x JOIN (SELECT 1 AS x) t329 ON t1.x = t329.x JOIN (SELECT 1 AS x) t330 ON t1.x = t330.x JOIN (SELECT 1 AS x) t331 ON t1.x = t331.x JOIN (SELECT 1 AS x) t332 ON t1.x = t332.x JOIN (SELECT 1 AS x) t333 ON t1.x = t333.x JOIN (SELECT 1 AS x) t334 ON t1.x = t334.x JOIN (SELECT 1 AS x) t335 ON t1.x = t335.x JOIN (SELECT 1 AS x) t336 ON t1.x = t336.x JOIN (SELECT 1 AS x) t337 ON t1.x = t337.x JOIN (SELECT 1 AS x) t338 ON t1.x = t338.x JOIN (SELECT 1 AS x) t339 ON t1.x = t339.x JOIN (SELECT 1 AS x) t340 ON t1.x = t340.x JOIN (SELECT 1 AS x) t341 ON t1.x = t341.x JOIN (SELECT 1 AS x) t342 ON t1.x = t342.x JOIN (SELECT 1 AS x) t343 ON t1.x = t343.x JOIN (SELECT 1 AS x) t344 ON t1.x = t344.x JOIN (SELECT 1 AS x) t345 ON t1.x = t345.x JOIN (SELECT 1 AS x) t346 ON t1.x = t346.x JOIN (SELECT 1 AS x) t347 ON t1.x = t347.x JOIN (SELECT 1 AS x) t348 ON t1.x = t348.x JOIN (SELECT 1 AS x) t349 ON t1.x = t349.x JOIN (SELECT 1 AS x) t350 ON t1.x = t350.x JOIN (SELECT 1 AS x) t351 ON t1.x = t351.x JOIN (SELECT 1 AS x) t352 ON t1.x = t352.x JOIN (SELECT 1 AS x) t353 ON t1.x = t353.x JOIN (SELECT 1 AS x) t354 ON t1.x = t354.x JOIN (SELECT 1 AS x) t355 ON t1.x = t355.x JOIN (SELECT 1 AS x) t356 ON t1.x = t356.x JOIN (SELECT 1 AS x) t357 ON t1.x = t357.x JOIN (SELECT 1 AS x) t358 ON t1.x = t358.x JOIN (SELECT 1 AS x) t359 ON t1.x = t359.x JOIN (SELECT 1 AS x) t360 ON t1.x = t360.x JOIN (SELECT 1 AS x) t361 ON t1.x = t361.x JOIN (SELECT 1 AS x) t362 ON t1.x = t362.x JOIN (SELECT 1 AS x) t363 ON t1.x = t363.x JOIN (SELECT 1 AS x) t364 ON t1.x = t364.x JOIN (SELECT 1 AS x) t365 ON t1.x = t365.x JOIN (SELECT 1 AS x) t366 ON t1.x = t366.x JOIN (SELECT 1 AS x) t367 ON t1.x = t367.x JOIN (SELECT 1 AS x) t368 ON t1.x = t368.x JOIN (SELECT 1 AS x) t369 ON t1.x = t369.x JOIN (SELECT 1 AS x) t370 ON t1.x = t370.x JOIN (SELECT 1 AS x) t371 ON t1.x = t371.x JOIN (SELECT 1 AS x) t372 ON t1.x = t372.x JOIN (SELECT 1 AS x) t373 ON t1.x = t373.x JOIN (SELECT 1 AS x) t374 ON t1.x = t374.x JOIN (SELECT 1 AS x) t375 ON t1.x = t375.x JOIN (SELECT 1 AS x) t376 ON t1.x = t376.x JOIN (SELECT 1 AS x) t377 ON t1.x = t377.x JOIN (SELECT 1 AS x) t378 ON t1.x = t378.x JOIN (SELECT 1 AS x) t379 ON t1.x = t379.x JOIN (SELECT 1 AS x) t380 ON t1.x = t380.x JOIN (SELECT 1 AS x) t381 ON t1.x = t381.x JOIN (SELECT 1 AS x) t382 ON t1.x = t382.x JOIN (SELECT 1 AS x) t383 ON t1.x = t383.x JOIN (SELECT 1 AS x) t384 ON t1.x = t384.x JOIN (SELECT 1 AS x) t385 ON t1.x = t385.x JOIN (SELECT 1 AS x) t386 ON t1.x = t386.x JOIN (SELECT 1 AS x) t387 ON t1.x = t387.x JOIN (SELECT 1 AS x) t388 ON t1.x = t388.x JOIN (SELECT 1 AS x) t389 ON t1.x = t389.x JOIN (SELECT 1 AS x) t390 ON t1.x = t390.x JOIN (SELECT 1 AS x) t391 ON t1.x = t391.x JOIN (SELECT 1 AS x) t392 ON t1.x = t392.x JOIN (SELECT 1 AS x) t393 ON t1.x = t393.x JOIN (SELECT 1 AS x) t394 ON t1.x = t394.x JOIN (SELECT 1 AS x) t395 ON t1.x = t395.x JOIN (SELECT 1 AS x) t396 ON t1.x = t396.x JOIN (SELECT 1 AS x) t397 ON t1.x = t397.x JOIN (SELECT 1 AS x) t398 ON t1.x = t398.x JOIN (SELECT 1 AS x) t399 ON t1.x = t399.x JOIN (SELECT 1 AS x) t400 ON t1.x = t400.x JOIN (SELECT 1 AS x) t401 ON t1.x = t401.x JOIN (SELECT 1 AS x) t402 ON t1.x = t402.x JOIN (SELECT 1 AS x) t403 ON t1.x = t403.x JOIN (SELECT 1 AS x) t404 ON t1.x = t404.x JOIN (SELECT 1 AS x) t405 ON t1.x = t405.x JOIN (SELECT 1 AS x) t406 ON t1.x = t406.x JOIN (SELECT 1 AS x) t407 ON t1.x = t407.x JOIN (SELECT 1 AS x) t408 ON t1.x = t408.x JOIN (SELECT 1 AS x) t409 ON t1.x = t409.x JOIN (SELECT 1 AS x) t410 ON t1.x = t410.x JOIN (SELECT 1 AS x) t411 ON t1.x = t411.x JOIN (SELECT 1 AS x) t412 ON t1.x = t412.x JOIN (SELECT 1 AS x) t413 ON t1.x = t413.x JOIN (SELECT 1 AS x) t414 ON t1.x = t414.x JOIN (SELECT 1 AS x) t415 ON t1.x = t415.x JOIN (SELECT 1 AS x) t416 ON t1.x = t416.x JOIN (SELECT 1 AS x) t417 ON t1.x = t417.x JOIN (SELECT 1 AS x) t418 ON t1.x = t418.x JOIN (SELECT 1 AS x) t419 ON t1.x = t419.x JOIN (SELECT 1 AS x) t420 ON t1.x = t420.x JOIN (SELECT 1 AS x) t421 ON t1.x = t421.x JOIN (SELECT 1 AS x) t422 ON t1.x = t422.x JOIN (SELECT 1 AS x) t423 ON t1.x = t423.x JOIN (SELECT 1 AS x) t424 ON t1.x = t424.x JOIN (SELECT 1 AS x) t425 ON t1.x = t425.x JOIN (SELECT 1 AS x) t426 ON t1.x = t426.x JOIN (SELECT 1 AS x) t427 ON t1.x = t427.x JOIN (SELECT 1 AS x) t428 ON t1.x = t428.x JOIN (SELECT 1 AS x) t429 ON t1.x = t429.x JOIN (SELECT 1 AS x) t430 ON t1.x = t430.x JOIN (SELECT 1 AS x) t431 ON t1.x = t431.x JOIN (SELECT 1 AS x) t432 ON t1.x = t432.x JOIN (SELECT 1 AS x) t433 ON t1.x = t433.x JOIN (SELECT 1 AS x) t434 ON t1.x = t434.x JOIN (SELECT 1 AS x) t435 ON t1.x = t435.x JOIN (SELECT 1 AS x) t436 ON t1.x = t436.x JOIN (SELECT 1 AS x) t437 ON t1.x = t437.x JOIN (SELECT 1 AS x) t438 ON t1.x = t438.x JOIN (SELECT 1 AS x) t439 ON t1.x = t439.x JOIN (SELECT 1 AS x) t440 ON t1.x = t440.x JOIN (SELECT 1 AS x) t441 ON t1.x = t441.x JOIN (SELECT 1 AS x) t442 ON t1.x = t442.x JOIN (SELECT 1 AS x) t443 ON t1.x = t443.x JOIN (SELECT 1 AS x) t444 ON t1.x = t444.x JOIN (SELECT 1 AS x) t445 ON t1.x = t445.x JOIN (SELECT 1 AS x) t446 ON t1.x = t446.x JOIN (SELECT 1 AS x) t447 ON t1.x = t447.x JOIN (SELECT 1 AS x) t448 ON t1.x = t448.x JOIN (SELECT 1 AS x) t449 ON t1.x = t449.x JOIN (SELECT 1 AS x) t450 ON t1.x = t450.x JOIN (SELECT 1 AS x) t451 ON t1.x = t451.x JOIN (SELECT 1 AS x) t452 ON t1.x = t452.x JOIN (SELECT 1 AS x) t453 ON t1.x = t453.x JOIN (SELECT 1 AS x) t454 ON t1.x = t454.x JOIN (SELECT 1 AS x) t455 ON t1.x = t455.x JOIN (SELECT 1 AS x) t456 ON t1.x = t456.x JOIN (SELECT 1 AS x) t457 ON t1.x = t457.x JOIN (SELECT 1 AS x) t458 ON t1.x = t458.x JOIN (SELECT 1 AS x) t459 ON t1.x = t459.x JOIN (SELECT 1 AS x) t460 ON t1.x = t460.x JOIN (SELECT 1 AS x) t461 ON t1.x = t461.x JOIN (SELECT 1 AS x) t462 ON t1.x = t462.x JOIN (SELECT 1 AS x) t463 ON t1.x = t463.x JOIN (SELECT 1 AS x) t464 ON t1.x = t464.x JOIN (SELECT 1 AS x) t465 ON t1.x = t465.x JOIN (SELECT 1 AS x) t466 ON t1.x = t466.x JOIN (SELECT 1 AS x) t467 ON t1.x = t467.x JOIN (SELECT 1 AS x) t468 ON t1.x = t468.x JOIN (SELECT 1 AS x) t469 ON t1.x = t469.x JOIN (SELECT 1 AS x) t470 ON t1.x = t470.x JOIN (SELECT 1 AS x) t471 ON t1.x = t471.x JOIN (SELECT 1 AS x) t472 ON t1.x = t472.x JOIN (SELECT 1 AS x) t473 ON t1.x = t473.x JOIN (SELECT 1 AS x) t474 ON t1.x = t474.x JOIN (SELECT 1 AS x) t475 ON t1.x = t475.x JOIN (SELECT 1 AS x) t476 ON t1.x = t476.x JOIN (SELECT 1 AS x) t477 ON t1.x = t477.x JOIN (SELECT 1 AS x) t478 ON t1.x = t478.x JOIN (SELECT 1 AS x) t479 ON t1.x = t479.x JOIN (SELECT 1 AS x) t480 ON t1.x = t480.x JOIN (SELECT 1 AS x) t481 ON t1.x = t481.x JOIN (SELECT 1 AS x) t482 ON t1.x = t482.x JOIN (SELECT 1 AS x) t483 ON t1.x = t483.x JOIN (SELECT 1 AS x) t484 ON t1.x = t484.x JOIN (SELECT 1 AS x) t485 ON t1.x = t485.x JOIN (SELECT 1 AS x) t486 ON t1.x = t486.x JOIN (SELECT 1 AS x) t487 ON t1.x = t487.x JOIN (SELECT 1 AS x) t488 ON t1.x = t488.x JOIN (SELECT 1 AS x) t489 ON t1.x = t489.x JOIN (SELECT 1 AS x) t490 ON t1.x = t490.x JOIN (SELECT 1 AS x) t491 ON t1.x = t491.x JOIN (SELECT 1 AS x) t492 ON t1.x = t492.x JOIN (SELECT 1 AS x) t493 ON t1.x = t493.x JOIN (SELECT 1 AS x) t494 ON t1.x = t494.x JOIN (SELECT 1 AS x) t495 ON t1.x = t495.x JOIN (SELECT 1 AS x) t496 ON t1.x = t496.x JOIN (SELECT 1 AS x) t497 ON t1.x = t497.x JOIN (SELECT 1 AS x) t498 ON t1.x = t498.x JOIN (SELECT 1 AS x) t499 ON t1.x = t499.x JOIN (SELECT 1 AS x) t500 ON t1.x = t500.x JOIN (SELECT 1 AS x) t501 ON t1.x = t501.x JOIN (SELECT 1 AS x) t502 ON t1.x = t502.x JOIN (SELECT 1 AS x) t503 ON t1.x = t503.x JOIN (SELECT 1 AS x) t504 ON t1.x = t504.x JOIN (SELECT 1 AS x) t505 ON t1.x = t505.x JOIN (SELECT 1 AS x) t506 ON t1.x = t506.x JOIN (SELECT 1 AS x) t507 ON t1.x = t507.x JOIN (SELECT 1 AS x) t508 ON t1.x = t508.x JOIN (SELECT 1 AS x) t509 ON t1.x = t509.x JOIN (SELECT 1 AS x) t510 ON t1.x = t510.x JOIN (SELECT 1 AS x) t511 ON t1.x = t511.x JOIN (SELECT 1 AS x) t512 ON t1.x = t512.x JOIN (SELECT 1 AS x) t513 ON t1.x = t513.x JOIN (SELECT 1 AS x) t514 ON t1.x = t514.x JOIN (SELECT 1 AS x) t515 ON t1.x = t515.x JOIN (SELECT 1 AS x) t516 ON t1.x = t516.x JOIN (SELECT 1 AS x) t517 ON t1.x = t517.x JOIN (SELECT 1 AS x) t518 ON t1.x = t518.x JOIN (SELECT 1 AS x) t519 ON t1.x = t519.x JOIN (SELECT 1 AS x) t520 ON t1.x = t520.x JOIN (SELECT 1 AS x) t521 ON t1.x = t521.x JOIN (SELECT 1 AS x) t522 ON t1.x = t522.x JOIN (SELECT 1 AS x) t523 ON t1.x = t523.x JOIN (SELECT 1 AS x) t524 ON t1.x = t524.x JOIN (SELECT 1 AS x) t525 ON t1.x = t525.x JOIN (SELECT 1 AS x) t526 ON t1.x = t526.x JOIN (SELECT 1 AS x) t527 ON t1.x = t527.x JOIN (SELECT 1 AS x) t528 ON t1.x = t528.x JOIN (SELECT 1 AS x) t529 ON t1.x = t529.x JOIN (SELECT 1 AS x) t530 ON t1.x = t530.x JOIN (SELECT 1 AS x) t531 ON t1.x = t531.x JOIN (SELECT 1 AS x) t532 ON t1.x = t532.x JOIN (SELECT 1 AS x) t533 ON t1.x = t533.x JOIN (SELECT 1 AS x) t534 ON t1.x = t534.x JOIN (SELECT 1 AS x) t535 ON t1.x = t535.x JOIN (SELECT 1 AS x) t536 ON t1.x = t536.x JOIN (SELECT 1 AS x) t537 ON t1.x = t537.x JOIN (SELECT 1 AS x) t538 ON t1.x = t538.x JOIN (SELECT 1 AS x) t539 ON t1.x = t539.x JOIN (SELECT 1 AS x) t540 ON t1.x = t540.x JOIN (SELECT 1 AS x) t541 ON t1.x = t541.x JOIN (SELECT 1 AS x) t542 ON t1.x = t542.x JOIN (SELECT 1 AS x) t543 ON t1.x = t543.x JOIN (SELECT 1 AS x) t544 ON t1.x = t544.x JOIN (SELECT 1 AS x) t545 ON t1.x = t545.x JOIN (SELECT 1 AS x) t546 ON t1.x = t546.x JOIN (SELECT 1 AS x) t547 ON t1.x = t547.x JOIN (SELECT 1 AS x) t548 ON t1.x = t548.x JOIN (SELECT 1 AS x) t549 ON t1.x = t549.x JOIN (SELECT 1 AS x) t550 ON t1.x = t550.x JOIN (SELECT 1 AS x) t551 ON t1.x = t551.x JOIN (SELECT 1 AS x) t552 ON t1.x = t552.x JOIN (SELECT 1 AS x) t553 ON t1.x = t553.x JOIN (SELECT 1 AS x) t554 ON t1.x = t554.x JOIN (SELECT 1 AS x) t555 ON t1.x = t555.x JOIN (SELECT 1 AS x) t556 ON t1.x = t556.x JOIN (SELECT 1 AS x) t557 ON t1.x = t557.x JOIN (SELECT 1 AS x) t558 ON t1.x = t558.x JOIN (SELECT 1 AS x) t559 ON t1.x = t559.x JOIN (SELECT 1 AS x) t560 ON t1.x = t560.x JOIN (SELECT 1 AS x) t561 ON t1.x = t561.x JOIN (SELECT 1 AS x) t562 ON t1.x = t562.x JOIN (SELECT 1 AS x) t563 ON t1.x = t563.x JOIN (SELECT 1 AS x) t564 ON t1.x = t564.x JOIN (SELECT 1 AS x) t565 ON t1.x = t565.x JOIN (SELECT 1 AS x) t566 ON t1.x = t566.x JOIN (SELECT 1 AS x) t567 ON t1.x = t567.x JOIN (SELECT 1 AS x) t568 ON t1.x = t568.x JOIN (SELECT 1 AS x) t569 ON t1.x = t569.x JOIN (SELECT 1 AS x) t570 ON t1.x = t570.x JOIN (SELECT 1 AS x) t571 ON t1.x = t571.x JOIN (SELECT 1 AS x) t572 ON t1.x = t572.x JOIN (SELECT 1 AS x) t573 ON t1.x = t573.x JOIN (SELECT 1 AS x) t574 ON t1.x = t574.x JOIN (SELECT 1 AS x) t575 ON t1.x = t575.x JOIN (SELECT 1 AS x) t576 ON t1.x = t576.x JOIN (SELECT 1 AS x) t577 ON t1.x = t577.x JOIN (SELECT 1 AS x) t578 ON t1.x = t578.x JOIN (SELECT 1 AS x) t579 ON t1.x = t579.x JOIN (SELECT 1 AS x) t580 ON t1.x = t580.x JOIN (SELECT 1 AS x) t581 ON t1.x = t581.x JOIN (SELECT 1 AS x) t582 ON t1.x = t582.x JOIN (SELECT 1 AS x) t583 ON t1.x = t583.x JOIN (SELECT 1 AS x) t584 ON t1.x = t584.x JOIN (SELECT 1 AS x) t585 ON t1.x = t585.x JOIN (SELECT 1 AS x) t586 ON t1.x = t586.x JOIN (SELECT 1 AS x) t587 ON t1.x = t587.x JOIN (SELECT 1 AS x) t588 ON t1.x = t588.x JOIN (SELECT 1 AS x) t589 ON t1.x = t589.x JOIN (SELECT 1 AS x) t590 ON t1.x = t590.x JOIN (SELECT 1 AS x) t591 ON t1.x = t591.x JOIN (SELECT 1 AS x) t592 ON t1.x = t592.x JOIN (SELECT 1 AS x) t593 ON t1.x = t593.x JOIN (SELECT 1 AS x) t594 ON t1.x = t594.x JOIN (SELECT 1 AS x) t595 ON t1.x = t595.x JOIN (SELECT 1 AS x) t596 ON t1.x = t596.x JOIN (SELECT 1 AS x) t597 ON t1.x = t597.x JOIN (SELECT 1 AS x) t598 ON t1.x = t598.x JOIN (SELECT 1 AS x) t599 ON t1.x = t599.x JOIN (SELECT 1 AS x) t600 ON t1.x = t600.x JOIN (SELECT 1 AS x) t601 ON t1.x = t601.x JOIN (SELECT 1 AS x) t602 ON t1.x = t602.x JOIN (SELECT 1 AS x) t603 ON t1.x = t603.x JOIN (SELECT 1 AS x) t604 ON t1.x = t604.x JOIN (SELECT 1 AS x) t605 ON t1.x = t605.x JOIN (SELECT 1 AS x) t606 ON t1.x = t606.x JOIN (SELECT 1 AS x) t607 ON t1.x = t607.x JOIN (SELECT 1 AS x) t608 ON t1.x = t608.x JOIN (SELECT 1 AS x) t609 ON t1.x = t609.x JOIN (SELECT 1 AS x) t610 ON t1.x = t610.x JOIN (SELECT 1 AS x) t611 ON t1.x = t611.x JOIN (SELECT 1 AS x) t612 ON t1.x = t612.x JOIN (SELECT 1 AS x) t613 ON t1.x = t613.x JOIN (SELECT 1 AS x) t614 ON t1.x = t614.x JOIN (SELECT 1 AS x) t615 ON t1.x = t615.x JOIN (SELECT 1 AS x) t616 ON t1.x = t616.x JOIN (SELECT 1 AS x) t617 ON t1.x = t617.x JOIN (SELECT 1 AS x) t618 ON t1.x = t618.x JOIN (SELECT 1 AS x) t619 ON t1.x = t619.x JOIN (SELECT 1 AS x) t620 ON t1.x = t620.x JOIN (SELECT 1 AS x) t621 ON t1.x = t621.x JOIN (SELECT 1 AS x) t622 ON t1.x = t622.x JOIN (SELECT 1 AS x) t623 ON t1.x = t623.x JOIN (SELECT 1 AS x) t624 ON t1.x = t624.x JOIN (SELECT 1 AS x) t625 ON t1.x = t625.x JOIN (SELECT 1 AS x) t626 ON t1.x = t626.x JOIN (SELECT 1 AS x) t627 ON t1.x = t627.x JOIN (SELECT 1 AS x) t628 ON t1.x = t628.x JOIN (SELECT 1 AS x) t629 ON t1.x = t629.x JOIN (SELECT 1 AS x) t630 ON t1.x = t630.x JOIN (SELECT 1 AS x) t631 ON t1.x = t631.x JOIN (SELECT 1 AS x) t632 ON t1.x = t632.x JOIN (SELECT 1 AS x) t633 ON t1.x = t633.x JOIN (SELECT 1 AS x) t634 ON t1.x = t634.x JOIN (SELECT 1 AS x) t635 ON t1.x = t635.x JOIN (SELECT 1 AS x) t636 ON t1.x = t636.x JOIN (SELECT 1 AS x) t637 ON t1.x = t637.x JOIN (SELECT 1 AS x) t638 ON t1.x = t638.x JOIN (SELECT 1 AS x) t639 ON t1.x = t639.x JOIN (SELECT 1 AS x) t640 ON t1.x = t640.x JOIN (SELECT 1 AS x) t641 ON t1.x = t641.x JOIN (SELECT 1 AS x) t642 ON t1.x = t642.x JOIN (SELECT 1 AS x) t643 ON t1.x = t643.x JOIN (SELECT 1 AS x) t644 ON t1.x = t644.x JOIN (SELECT 1 AS x) t645 ON t1.x = t645.x JOIN (SELECT 1 AS x) t646 ON t1.x = t646.x JOIN (SELECT 1 AS x) t647 ON t1.x = t647.x JOIN (SELECT 1 AS x) t648 ON t1.x = t648.x JOIN (SELECT 1 AS x) t649 ON t1.x = t649.x JOIN (SELECT 1 AS x) t650 ON t1.x = t650.x JOIN (SELECT 1 AS x) t651 ON t1.x = t651.x JOIN (SELECT 1 AS x) t652 ON t1.x = t652.x JOIN (SELECT 1 AS x) t653 ON t1.x = t653.x JOIN (SELECT 1 AS x) t654 ON t1.x = t654.x JOIN (SELECT 1 AS x) t655 ON t1.x = t655.x JOIN (SELECT 1 AS x) t656 ON t1.x = t656.x JOIN (SELECT 1 AS x) t657 ON t1.x = t657.x JOIN (SELECT 1 AS x) t658 ON t1.x = t658.x JOIN (SELECT 1 AS x) t659 ON t1.x = t659.x JOIN (SELECT 1 AS x) t660 ON t1.x = t660.x JOIN (SELECT 1 AS x) t661 ON t1.x = t661.x JOIN (SELECT 1 AS x) t662 ON t1.x = t662.x JOIN (SELECT 1 AS x) t663 ON t1.x = t663.x JOIN (SELECT 1 AS x) t664 ON t1.x = t664.x JOIN (SELECT 1 AS x) t665 ON t1.x = t665.x JOIN (SELECT 1 AS x) t666 ON t1.x = t666.x From 46651d21b493b8953521e530f3fa92d94fe1b895 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 22 Apr 2024 18:43:49 +0300 Subject: [PATCH 0257/1018] Fixed tests --- .../0_stateless/03035_internal_functions_direct_call.reference | 1 - .../0_stateless/03035_internal_functions_direct_call.sql | 2 -- 2 files changed, 3 deletions(-) diff --git a/tests/queries/0_stateless/03035_internal_functions_direct_call.reference b/tests/queries/0_stateless/03035_internal_functions_direct_call.reference index 4dc02ffa71d..d45b44f6052 100644 --- a/tests/queries/0_stateless/03035_internal_functions_direct_call.reference +++ b/tests/queries/0_stateless/03035_internal_functions_direct_call.reference @@ -1,5 +1,4 @@ aaa -(1,1) (1,1) 1 a1 1 1 diff --git a/tests/queries/0_stateless/03035_internal_functions_direct_call.sql b/tests/queries/0_stateless/03035_internal_functions_direct_call.sql index d90c4dceab5..951e0733dbb 100644 --- a/tests/queries/0_stateless/03035_internal_functions_direct_call.sql +++ b/tests/queries/0_stateless/03035_internal_functions_direct_call.sql @@ -16,8 +16,6 @@ SELECT __getScalar(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT __getScalar(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT __getScalar(materialize('1')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -WITH ( SELECT (1,1) ) as a SELECT materialize(a), __getScalar('17789833925953107877_7493841889429261611') SETTINGS allow_experimental_analyzer = 1; - SELECT __scalarSubqueryResult('1'); SELECT 'a' || __scalarSubqueryResult(a), materialize('1') as a; SELECT __scalarSubqueryResult(a, a), materialize('1') as a; -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } From d3a58ffbbd5e5efc5c0c9679d4b56737fe6de40f Mon Sep 17 00:00:00 2001 From: pet74alex <167422282+pet74alex@users.noreply.github.com> Date: Mon, 22 Apr 2024 19:22:40 +0300 Subject: [PATCH 0258/1018] Added generateUUIDv7* functions --- src/Functions/generateUUIDv7.cpp | 373 +++++++++++++++++++++++++++++++ 1 file changed, 373 insertions(+) create mode 100644 src/Functions/generateUUIDv7.cpp diff --git a/src/Functions/generateUUIDv7.cpp b/src/Functions/generateUUIDv7.cpp new file mode 100644 index 00000000000..4bd540d28db --- /dev/null +++ b/src/Functions/generateUUIDv7.cpp @@ -0,0 +1,373 @@ +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +namespace +{ +constexpr auto bits_in_counter = 42; +constexpr uint64_t counter_limit = (uint64_t{1} << bits_in_counter); +constexpr uint8_t random_data_offset = 6; +constexpr uint8_t random_data_count = 10; +constexpr uint8_t next_count_random_data_offset = 12; +constexpr uint8_t next_count_random_data_count = 4; + +using UUIDAsArray = std::array; + +inline uint64_t getTimestampMs() +{ + timespec tp; + clock_gettime(CLOCK_REALTIME, &tp); + const uint64_t sec = tp.tv_sec; + return sec * 1000 + tp.tv_nsec / 1000000; +} + +inline void fillTimestamp(UUIDAsArray & uuid, uint64_t timestamp) +{ + uuid[0] = (timestamp >> 40) & 0xFF; + uuid[1] = (timestamp >> 32) & 0xFF; + uuid[2] = (timestamp >> 24) & 0xFF; + uuid[3] = (timestamp >> 16) & 0xFF; + uuid[4] = (timestamp >> 8) & 0xFF; + uuid[5] = (timestamp)&0xFF; +} +} + +#define DECLARE_SEVERAL_IMPLEMENTATIONS(...) \ + DECLARE_DEFAULT_CODE(__VA_ARGS__) \ + DECLARE_AVX2_SPECIFIC_CODE(__VA_ARGS__) + +DECLARE_SEVERAL_IMPLEMENTATIONS( + + namespace UUIDv7Impl { + inline void store(UUID & new_uuid, UUIDAsArray & uuid) + { + uuid[6] = (uuid[6] & 0x0f) | 0x70; // version 7 + uuid[8] = (uuid[8] & 0x3f) | 0x80; // variant 2 + + DB::UUIDHelpers::getHighBytes(new_uuid) = unalignedLoadBigEndian(uuid.data()); + DB::UUIDHelpers::getLowBytes(new_uuid) = unalignedLoadBigEndian(uuid.data() + 8); + } + + struct UUIDv7Base + { + UUIDAsArray & uuid; + UUIDv7Base(UUIDAsArray & u) : uuid(u) { } + }; + + struct RandomData + { + static constexpr auto name = "generateUUIDv7"; + struct Data : UUIDv7Base + { + UUIDAsArray uuid_data; + + Data() : UUIDv7Base(uuid_data) { } + + void generate(UUID & new_uuid) + { + fillTimestamp(uuid, getTimestampMs()); + memcpy(uuid.data() + random_data_offset, &new_uuid, random_data_count); + store(new_uuid, uuid); + } + }; + }; + + struct CounterDataCommon : UUIDv7Base + { + CounterDataCommon(UUIDAsArray & u) : UUIDv7Base(u) { } + + uint64_t getCounter() + { + uint64_t counter = uuid[6] & 0x0f; + counter = (counter << 8) | uuid[7]; + counter = (counter << 6) | (uuid[8] & 0x3f); + counter = (counter << 8) | uuid[9]; + counter = (counter << 8) | uuid[10]; + counter = (counter << 8) | uuid[11]; + return counter; + } + + void generate(UUID & newUUID) + { + uint64_t timestamp = 0; + /// Get timestamp of the previous uuid + for (int i = 0; i != 6; ++i) + { + timestamp = (timestamp << 8) | uuid[i]; + } + + const uint64_t unix_time_ms = getTimestampMs(); + // continue incrementing counter when clock slightly goes back or when counter overflow happened during the previous UUID generation + bool need_to_increment_counter = (timestamp == unix_time_ms || timestamp < unix_time_ms + 10000); + uint64_t counter = 0; + if (need_to_increment_counter) + { + counter = getCounter(); + } + else + { + timestamp = unix_time_ms; + } + + bool counter_incremented = false; + if (need_to_increment_counter) + { + if (++counter == counter_limit) + { + ++timestamp; + // counter bytes will be filled by the random data + } + else + { + uuid[6] = counter >> 38; + uuid[7] = counter >> 30; + uuid[8] = counter >> 24; + uuid[9] = counter >> 16; + uuid[10] = counter >> 8; + uuid[11] = counter; + counter_incremented = true; + } + } + + fillTimestamp(uuid, timestamp); + + // Get the required number of random bytes: 4 in the case of incrementing existing counter, 10 in the case of renewing counter + memcpy( + uuid.data() + (counter_incremented ? next_count_random_data_offset : random_data_offset), + &newUUID, + counter_incremented ? next_count_random_data_count : random_data_count); + + store(newUUID, uuid); + } + }; + + struct ThreadLocalCounter + { + static constexpr auto name = "generateUUIDv7WithFastCounter"; + struct Data : CounterDataCommon + { + // Implement counter monotony only within one thread so function doesn't require mutexes and doesn't affect performance of the same function running simultenaously on other threads + static inline thread_local UUIDAsArray uuid_data; + Data() : CounterDataCommon(uuid_data) { } + }; + }; + + struct GlobalCounter + { + static constexpr auto name = "generateUUIDv7WithCounter"; + struct Data : std::lock_guard, CounterDataCommon + { + // Implement counter monotony whithin one timestamp accross all threads generating UUIDv7 with counter simultaneously + static inline UUIDAsArray uuid_data; + static inline std::mutex mtx; + Data() : std::lock_guard(mtx), CounterDataCommon(uuid_data) { } + }; + }; + } // namespace UUIDv7Impl + + + template + class FunctionGenerateUUIDv7Base + : public IFunction, + public FillPolicy { + public: + using FillPolicy::name; + using FillPolicyData = typename FillPolicy::Data; + + FunctionGenerateUUIDv7Base() = default; + + String getName() const final + { + return name; + } + + size_t getNumberOfArguments() const final + { + return 0; + } + + bool isDeterministicInScopeOfQuery() const final + { + return false; + } + bool useDefaultImplementationForNulls() const final + { + return false; + } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const final + { + return false; + } + bool isVariadic() const final + { + return true; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.size() > 1) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be 0 or 1.", + getName(), + arguments.size()); + + return std::make_shared(); + } + + bool isDeterministic() const override + { + return false; + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override + { + auto col_res = ColumnVector::create(); + typename ColumnVector::Container & vec_to = col_res->getData(); + + size_t size = input_rows_count; + vec_to.resize(size); + + /// RandImpl is target-dependent and is not the same in different TargetSpecific namespaces. + /// Not all random bytes produced here are required for the UUIDv7 but it's the simplest way to get the required number of them by using RandImpl + RandImpl::execute(reinterpret_cast(vec_to.data()), vec_to.size() * sizeof(UUID)); + + for (UUID & new_uuid : vec_to) + { + FillPolicyData data; + data.generate(new_uuid); + } + + return col_res; + } + }; + + using FunctionGenerateUUIDv7 = FunctionGenerateUUIDv7Base; + using FunctionGenerateUUIDv7WithCounter = FunctionGenerateUUIDv7Base; + using FunctionGenerateUUIDv7WithFastCounter = FunctionGenerateUUIDv7Base; + + ) // DECLARE_SEVERAL_IMPLEMENTATIONS +#undef DECLARE_SEVERAL_IMPLEMENTATIONS + + +class FunctionGenerateUUIDv7 : public TargetSpecific::Default::FunctionGenerateUUIDv7 +{ +public: + explicit FunctionGenerateUUIDv7(ContextPtr context) : selector(context) + { + selector.registerImplementation(); + +#if USE_MULTITARGET_CODE + selector.registerImplementation(); +#endif + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + return selector.selectAndExecute(arguments, result_type, input_rows_count); + } + + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } + +private: + ImplementationSelector selector; +}; + +class FunctionGenerateUUIDv7WithCounter : public TargetSpecific::Default::FunctionGenerateUUIDv7WithCounter +{ +public: + explicit FunctionGenerateUUIDv7WithCounter(ContextPtr context) : selector(context) + { + selector.registerImplementation(); + +#if USE_MULTITARGET_CODE + selector.registerImplementation(); +#endif + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + return selector.selectAndExecute(arguments, result_type, input_rows_count); + } + + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } + +private: + ImplementationSelector selector; +}; + + +class FunctionGenerateUUIDv7WithFastCounter : public TargetSpecific::Default::FunctionGenerateUUIDv7WithFastCounter +{ +public: + explicit FunctionGenerateUUIDv7WithFastCounter(ContextPtr context) : selector(context) + { + selector.registerImplementation(); + +#if USE_MULTITARGET_CODE + selector.registerImplementation(); +#endif + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + return selector.selectAndExecute(arguments, result_type, input_rows_count); + } + + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } + +private: + ImplementationSelector selector; +}; + + +REGISTER_FUNCTION(GenerateUUIDv7) +{ + factory.registerFunction( + FunctionDocumentation{ + .description = R"( +Generates a UUID of version 7 with current Unix time having milliseconds precision followed by random data. +This function takes an optional argument, the value of which is discarded to generate different values in case the function is called multiple times. +The function returns a value of type UUID. +)", + .examples{{"uuid", "SELECT generateUUIDv7()", ""}, {"multiple", "SELECT generateUUIDv7(1), generateUUIDv7(2)", ""}}, + .categories{"UUID"}}, + FunctionFactory::CaseSensitive); + + factory.registerFunction( + FunctionDocumentation{ + .description = R"( +Generates a UUID of version 7 with current Unix time having milliseconds precision, a monotonic counter within the same timestamp starting from the random value, and followed by 4 random bytes. +This function takes an optional argument, the value of which is discarded to generate different values in case the function is called multiple times. +The function returns a value of type UUID. +)", + .examples{ + {"uuid", "SELECT generateUUIDv7WithCounter()", ""}, + {"multiple", "SELECT generateUUIDv7WithCounter(1), generateUUIDv7WithCounter(2)", ""}}, + .categories{"UUID"}}, + FunctionFactory::CaseSensitive); + + factory.registerFunction( + FunctionDocumentation{ + .description = R"( +Generates a UUID of version 7 with current Unix time having milliseconds precision, a monotonic counter within the same timestamp and the same request starting from the random value, and followed by 4 random bytes. +This function takes an optional argument, the value of which is discarded to generate different values in case the function is called multiple times. +This function is a little bit faster version of the function GenerateUUIDv7WithCounter. It doesn't guarantee the counter monotony withing the same timestamp accross different requests. It means that two UUIDs having +The function returns a value of type UUID. +)", + .examples{ + {"uuid", "SELECT generateUUIDv7WithFastCounter()", ""}, + {"multiple", "SELECT generateUUIDv7WithFastCounter(1), generateUUIDv7WithFastCounter(2)", ""}}, + .categories{"UUID"}}, + FunctionFactory::CaseSensitive); +} + +} From 142ce60b4a9c01ca865cc2cf531238d162d0b994 Mon Sep 17 00:00:00 2001 From: pet74alex <167422282+pet74alex@users.noreply.github.com> Date: Mon, 22 Apr 2024 19:26:15 +0300 Subject: [PATCH 0259/1018] Added UUIDToNum and UUDv7ToDateTime functions --- src/Functions/FunctionsCodingUUID.cpp | 185 ++++++++++++++++++++++++++ 1 file changed, 185 insertions(+) diff --git a/src/Functions/FunctionsCodingUUID.cpp b/src/Functions/FunctionsCodingUUID.cpp index dd9170e44ad..dceff894c34 100644 --- a/src/Functions/FunctionsCodingUUID.cpp +++ b/src/Functions/FunctionsCodingUUID.cpp @@ -1,14 +1,18 @@ #include +#include #include #include +#include #include #include #include #include #include +#include #include #include #include +#include #include #include #include @@ -319,10 +323,191 @@ public: } }; + +class FunctionUUIDToNum : public IFunction +{ +public: + static constexpr auto name = "UUIDToNum"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 0; } + bool isInjective(const ColumnsWithTypeAndName &) const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + bool isVariadic() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + checkArgumentCount(arguments, name); + + if (!isUUID(arguments[0])) + { + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of first argument of function {}, expected UUID", + arguments[0]->getName(), + getName()); + } + + checkFormatArgument(arguments, name); + + return std::make_shared(uuid_bytes_length); + } + + bool useDefaultImplementationForConstants() const override { return true; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + { + const ColumnWithTypeAndName & col_type_name = arguments[0]; + const ColumnPtr & column = col_type_name.column; + + const bool defaultFormat = (parseVariant(arguments) == UUIDSerializer::Variant::Default); + if (const auto * col_in = checkAndGetColumn(column.get())) + { + const auto & vec_in = col_in->getData(); + const UUID * uuids = vec_in.data(); + const size_t size = vec_in.size(); + + auto col_res = ColumnFixedString::create(uuid_bytes_length); + + ColumnString::Chars & vec_res = col_res->getChars(); + vec_res.resize(size * uuid_bytes_length); + + size_t dst_offset = 0; + + for (size_t i = 0; i < size; ++i) + { + uint64_t hiBytes = DB::UUIDHelpers::getHighBytes(uuids[i]); + uint64_t loBytes = DB::UUIDHelpers::getLowBytes(uuids[i]); + unalignedStoreBigEndian(&vec_res[dst_offset], hiBytes); + unalignedStoreBigEndian(&vec_res[dst_offset + sizeof(hiBytes)], loBytes); + if (!defaultFormat) + { + std::swap(vec_res[dst_offset], vec_res[dst_offset + 3]); + std::swap(vec_res[dst_offset + 1], vec_res[dst_offset + 2]); + } + dst_offset += uuid_bytes_length; + } + + return col_res; + } + else + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", arguments[0].column->getName(), getName()); + } +}; + +class FunctionUUIDv7ToDateTime : public IFunction +{ +public: + static constexpr auto name = "UUIDv7ToDateTime"; + static constexpr UInt32 DATETIME_SCALE = 3; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 0; } + bool isInjective(const ColumnsWithTypeAndName &) const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + bool isVariadic() const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (arguments.empty() || arguments.size() > 2) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Wrong number of arguments for function {}: should be 1 or 2", getName()); + + if (!checkAndGetDataType(arguments[0].type.get())) + { + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of first argument of function {}, expected UUID", + arguments[0].type->getName(), + getName()); + } + + String timezone; + if (arguments.size() == 2) + { + timezone = extractTimeZoneNameFromColumn(arguments[1].column.get(), arguments[1].name); + + if (timezone.empty()) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Function {} supports a 2nd argument (optional) that must be a valid time zone", + getName()); + } + + return std::make_shared(DATETIME_SCALE, timezone); + } + + bool useDefaultImplementationForConstants() const override { return true; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + { + const ColumnWithTypeAndName & col_type_name = arguments[0]; + const ColumnPtr & column = col_type_name.column; + + if (const auto * col_in = checkAndGetColumn(column.get())) + { + const auto & vec_in = col_in->getData(); + const UUID * uuids = vec_in.data(); + const size_t size = vec_in.size(); + + auto col_res = ColumnDateTime64::create(size, DATETIME_SCALE); + auto & vec_res = col_res->getData(); + + for (size_t i = 0; i < size; ++i) + { + uint64_t hiBytes = DB::UUIDHelpers::getHighBytes(uuids[i]); + if ((hiBytes & 0xf000) == 0x7000) + { + uint64_t ms = hiBytes >> 16; + vec_res[i] = DecimalUtils::decimalFromComponents( + ms / intExp10(DATETIME_SCALE), ms % intExp10(DATETIME_SCALE), DATETIME_SCALE); + } + } + + return col_res; + } + else + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", arguments[0].column->getName(), getName()); + } +}; + REGISTER_FUNCTION(CodingUUID) { factory.registerFunction(); factory.registerFunction(); + factory.registerFunction( + FunctionDocumentation{ + .description = R"( +This function accepts a UUID and returns a FixedString(16) as its binary representation, with its format optionally specified by variant (Big-endian by default). +)", + .examples{ + {"uuid", + "select toUUID(UUIDNumToString(toFixedString('a/<@];!~p{jTj={)', 16))) as uuid, UUIDToNum(uuid) as uuidNum, " + "UUIDToNum(uuid, 2) as uuidMsNum", + R"( +┌─uuid─────────────────────────────────┬─uuidNum──────────┬─uuidMsNum────────┐ +│ 612f3c40-5d3b-217e-707b-6a546a3d7b29 │ a/<@];!~p{jTj={) │ @( + FunctionDocumentation{ + .description = R"( +This function extracts the timestamp from a UUID and returns it as a DateTime64(3) typed value. +The function expects the UUID having version 7 to be provided as the first argument. +An optional second argument can be passed to specify a timezone for the timestamp. +)", + .examples{ + {"uuid","select UUIDv7ToDateTime(generateUUIDv7())", ""}, + {"uuid","select generateUUIDv7() as uuid, UUIDv7ToDateTime(uuid), UUIDv7ToDateTime(uuid, 'America/New_York')", ""}}, + .categories{"UUID"}}, + FunctionFactory::CaseSensitive); } } From 4e4e72ead8436ec7e352be043f185412730dfdc8 Mon Sep 17 00:00:00 2001 From: pet74alex <167422282+pet74alex@users.noreply.github.com> Date: Mon, 22 Apr 2024 19:30:25 +0300 Subject: [PATCH 0260/1018] Update English version of uuid-functions.md --- .../sql-reference/functions/uuid-functions.md | 251 ++++++++++++++++++ 1 file changed, 251 insertions(+) diff --git a/docs/en/sql-reference/functions/uuid-functions.md b/docs/en/sql-reference/functions/uuid-functions.md index c338add3a57..5f1246fc3ad 100644 --- a/docs/en/sql-reference/functions/uuid-functions.md +++ b/docs/en/sql-reference/functions/uuid-functions.md @@ -51,6 +51,169 @@ SELECT generateUUIDv4(1), generateUUIDv4(2) └──────────────────────────────────────┴──────────────────────────────────────┘ ``` +## generateUUIDv7 + +Generates the [UUID](../data-types/uuid.md) of [version 7](https://datatracker.ietf.org/doc/html/draft-peabody-dispatch-new-uuid-format-04). The generated UUID contains current timestamp in milliseconds followed by version 7 and variant 2 markers and random data in the following bit layout. +``` + 0 1 2 3 + 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| unix_ts_ms | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| unix_ts_ms | ver | rand_a | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +|var| rand_b | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| rand_b | +└─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┘ +``` + +**Syntax** + +``` sql +generateUUIDv7([x]) +``` + +**Arguments** + +- `x` — [Expression](../../sql-reference/syntax.md#syntax-expressions) resulting in any of the [supported data types](../../sql-reference/data-types/index.md#data_types). The resulting value is discarded, but the expression itself if used for bypassing [common subexpression elimination](../../sql-reference/functions/index.md#common-subexpression-elimination) if the function is called multiple times in one query. Optional parameter. + +**Returned value** + +The UUID type value. + +**Usage example** + +This example demonstrates creating a table with the UUID type column and inserting a UUIDv7 value into the table. + +``` sql +CREATE TABLE t_uuid (x UUID) ENGINE=TinyLog + +INSERT INTO t_uuid SELECT generateUUIDv7() + +SELECT * FROM t_uuid +``` + +```response +┌────────────────────────────────────x─┐ +│ 018f05af-f4a8-778f-beee-1bedbc95c93b │ +└──────────────────────────────────────┘ +``` + +**Usage example if it is needed to generate multiple values in one row** + +```sql +SELECT generateUUIDv7(1), generateUUIDv7(2) +┌─generateUUIDv7(1)────────────────────┬─generateUUIDv7(2)────────────────────┐ +│ 018f05b1-8c2e-7567-a988-48d09606ae8c │ 018f05b1-8c2e-7946-895b-fcd7635da9a0 │ +└──────────────────────────────────────┴──────────────────────────────────────┘ +``` + +## generateUUIDv7WithCounter + +Generates the [UUID](../data-types/uuid.md) of [version 7](https://datatracker.ietf.org/doc/html/draft-peabody-dispatch-new-uuid-format-04). +The generated UUID contains current timestamp in milliseconds followed by version 7 and variant 2 markers, counter and random data in the following bit layout. At any given new timestamp in unix_ts_ms the counter starts from some random value and then it's being increased by 1 on each new UUID v7 with counter generation until current timestamp changes. The counter overflow causes unix_ts_ms field increment by 1 and the counter restart from a random value. Counter increment monotony at one timestamp is guaranteed across all `generateUUIDv7WithCounter` functions running simultaneously. +``` + 0 1 2 3 + 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| unix_ts_ms | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| unix_ts_ms | ver | counter_high_bits | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +|var| counter_low_bits | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| rand_b | +└─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┘ +``` + +**Syntax** + +``` sql +generateUUIDv7WithCounter([x]) +``` + +**Arguments** + +- `x` — [Expression](../../sql-reference/syntax.md#syntax-expressions) resulting in any of the [supported data types](../../sql-reference/data-types/index.md#data_types). The resulting value is discarded, but the expression itself if used for bypassing [common subexpression elimination](../../sql-reference/functions/index.md#common-subexpression-elimination) if the function is called multiple times in one query. Optional parameter. + +**Returned value** + +The UUID type value. + +**Usage example** + +This example demonstrates creating a table with the UUID type column and inserting a UUIDv7 value into the table. + +``` sql +CREATE TABLE t_uuid (x UUID) ENGINE=TinyLog + +INSERT INTO t_uuid SELECT generateUUIDv7WithCounter() + +SELECT * FROM t_uuid +``` + +```response +┌────────────────────────────────────x─┐ +│ 018f05c7-56e3-7ac3-93e9-1d93c4218e0e │ +└──────────────────────────────────────┘ +``` + +**Usage example if it is needed to generate multiple values in one row** + +```sql +SELECT generateUUIDv7WithCounter(1), generateUUIDv7WithCounter(2) +┌─generateUUIDv7WithCounter(1)─────────┬─generateUUIDv7WithCounter(2)─────────┐ +│ 018f05c9-4ab8-7b86-b64e-c9f03fbd45d1 │ 018f05c9-4ab8-7b86-b64e-c9f12efb7e16 │ +└──────────────────────────────────────┴──────────────────────────────────────┘ +``` + +## generateUUIDv7WithFastCounter + +Generates the [UUID](../data-types/uuid.md) of [version 7](https://datatracker.ietf.org/doc/html/draft-peabody-dispatch-new-uuid-format-04). +This function is a faster version of `generateUUIDv7WithCounter` function giving no guarantee on counter monotony across different requests running simultaneously. Counter increment monotony at one timestamp is guaranteed only within one thread calling this function to generate many UUIDs. + +**Syntax** + +``` sql +generateUUIDv7WithFastCounter([x]) +``` + +**Arguments** + +- `x` — [Expression](../../sql-reference/syntax.md#syntax-expressions) resulting in any of the [supported data types](../../sql-reference/data-types/index.md#data_types). The resulting value is discarded, but the expression itself if used for bypassing [common subexpression elimination](../../sql-reference/functions/index.md#common-subexpression-elimination) if the function is called multiple times in one query. Optional parameter. + +**Returned value** + +The UUID type value. + +**Usage example** + +This example demonstrates creating a table with the UUID type column and inserting a UUIDv7 value into the table. + +``` sql +CREATE TABLE t_uuid (x UUID) ENGINE=TinyLog + +INSERT INTO t_uuid SELECT generateUUIDv7WithFastCounter() + +SELECT * FROM t_uuid +``` + +```response +┌────────────────────────────────────x─┐ +│ 018f05e2-e3b2-70cb-b8be-64b09b626d32 │ +└──────────────────────────────────────┘ +``` + +**Usage example if it is needed to generate multiple values in one row** + +```sql +SELECT generateUUIDv7WithFastCounter(1), generateUUIDv7WithFastCounter(2) +┌─generateUUIDv7WithFastCounter(1)─────┬─generateUUIDv7WithFastCounter(2)─────┐ +│ 018f05e1-14ee-7bc5-9906-207153b400b1 │ 018f05e1-14ee-7bc5-9906-2072b8e96758 │ +└──────────────────────────────────────┴──────────────────────────────────────┘ +``` + ## empty Checks whether the input UUID is empty. @@ -341,6 +504,94 @@ SELECT └──────────────────┴──────────────────────────────────────┘ ``` +## UUIDToNum + +Accepts `UUID` and returns a [FixedString(16)](../../sql-reference/data-types/fixedstring.md) as its binary representation, with its format optionally specified by `variant` (`Big-endian` by default). This function replaces calls to two separate functions `UUIDStringToNum(toString(uuid))` so intermediate conversion from UUID to string is not required to extract bytes from a UUID. + +**Syntax** + +``` sql +UUIDToNum(uuid[, variant = 1]) +``` + +**Arguments** + +- `uuid` — [UUID](../data-types/uuid.md). +- `variant` — Integer, representing a variant as specified by [RFC4122](https://datatracker.ietf.org/doc/html/rfc4122#section-4.1.1). 1 = `Big-endian` (default), 2 = `Microsoft`. + +**Returned value** + +FixedString(16) + +**Usage examples** + +``` sql +SELECT + toUUID('612f3c40-5d3b-217e-707b-6a546a3d7b29') AS uuid, + UUIDToNum(uuid) AS bytes +``` + +```response +┌─uuid─────────────────────────────────┬─bytes────────────┐ +│ 612f3c40-5d3b-217e-707b-6a546a3d7b29 │ a/<@];!~p{jTj={) │ +└──────────────────────────────────────┴──────────────────┘ +``` + +``` sql +SELECT + toUUID('612f3c40-5d3b-217e-707b-6a546a3d7b29') AS uuid, + UUIDToNum(uuid, 2) AS bytes +``` + +```response +┌─uuid─────────────────────────────────┬─bytes────────────┐ +│ 612f3c40-5d3b-217e-707b-6a546a3d7b29 │ @ Date: Mon, 22 Apr 2024 18:49:46 +0200 Subject: [PATCH 0261/1018] Azure: fix building final url from endpoint object --- .../ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h index 20bf05d5ba6..c8e556d012e 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h @@ -21,6 +21,8 @@ struct AzureBlobStorageEndpoint String getEndpoint() { String url = storage_account_url; + if (url.ends_with('/')) + url = url.substr(0, url.size() - 1); if (!account_name.empty()) url += "/" + account_name; From c053d5e58ee10fd65429949c38af15c7ba57bb73 Mon Sep 17 00:00:00 2001 From: Alexey Petrunyaka <167422282+pet74alex@users.noreply.github.com> Date: Mon, 22 Apr 2024 20:01:27 +0300 Subject: [PATCH 0262/1018] Small fix in generateUUIDv7WithFastCounter documentation --- src/Functions/generateUUIDv7.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/generateUUIDv7.cpp b/src/Functions/generateUUIDv7.cpp index 4bd540d28db..1a7f358f263 100644 --- a/src/Functions/generateUUIDv7.cpp +++ b/src/Functions/generateUUIDv7.cpp @@ -360,7 +360,7 @@ The function returns a value of type UUID. .description = R"( Generates a UUID of version 7 with current Unix time having milliseconds precision, a monotonic counter within the same timestamp and the same request starting from the random value, and followed by 4 random bytes. This function takes an optional argument, the value of which is discarded to generate different values in case the function is called multiple times. -This function is a little bit faster version of the function GenerateUUIDv7WithCounter. It doesn't guarantee the counter monotony withing the same timestamp accross different requests. It means that two UUIDs having +This function is a little bit faster version of the function GenerateUUIDv7WithCounter. It doesn't guarantee the counter monotony withing the same timestamp accross different requests. The function returns a value of type UUID. )", .examples{ From 3a3aaef4c4356ba2b7e65733bdaa71e3820fb9bc Mon Sep 17 00:00:00 2001 From: Serge Klochkov <3175289+slvrtrn@users.noreply.github.com> Date: Mon, 22 Apr 2024 19:39:56 +0200 Subject: [PATCH 0263/1018] Add a note about minimal ClickHouse version to use the role query parameter --- docs/en/interfaces/http.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index 45c960c6b6f..eb1a3ba1dbc 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -327,7 +327,9 @@ Use buffering to avoid situations where a query processing error occurred after ## Setting a role with query parameters {#setting-role-with-query-parameters} -In certain scenarios, it might be required to set the granted role first, before executing the statement itself. +This is a new feature added in ClickHouse 24.4. + +In specific scenarios, setting the granted role first might be required before executing the statement itself. However, it is not possible to send `SET ROLE` and the statement together, as multi-statements are not allowed: ``` @@ -346,7 +348,7 @@ To overcome this limitation, you could use the `role` query parameter instead: curl -sS "http://localhost:8123?role=my_role" --data-binary "SELECT * FROM my_table;" ``` -This will be an equivalent of executing `SET ROLE my_role` before the statement. +This will be the equivalent of executing `SET ROLE my_role` before the statement. Additionally, it is possible to specify multiple `role` query parameters: From 252a82dc387577cf8662d2e53ebe3f9c0282869f Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 22 Apr 2024 19:16:43 +0100 Subject: [PATCH 0264/1018] impl --- src/Compression/CompressionCodecGCD.cpp | 8 ++++++-- .../0_stateless/03093_bug_gcd_codec.reference | 0 .../queries/0_stateless/03093_bug_gcd_codec.sql | 17 +++++++++++++++++ 3 files changed, 23 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03093_bug_gcd_codec.reference create mode 100644 tests/queries/0_stateless/03093_bug_gcd_codec.sql diff --git a/src/Compression/CompressionCodecGCD.cpp b/src/Compression/CompressionCodecGCD.cpp index 8b923d61650..02864f2fcee 100644 --- a/src/Compression/CompressionCodecGCD.cpp +++ b/src/Compression/CompressionCodecGCD.cpp @@ -24,6 +24,7 @@ public: void updateHash(SipHash & hash) const override; protected: + /// 1 byte (`gcd_bytes_size` value) + 1 byte (`bytes_to_skip` value) + `bytes_to_skip` bytes (trash) + `gcd_bytes_size` bytes (gcd value) + (`source_size` - `bytes_to_skip`) bytes (data) UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; @@ -54,7 +55,7 @@ UInt32 CompressionCodecGCD::getMaxCompressedDataSize(UInt32 uncompressed_size) c { return uncompressed_size + gcd_bytes_size // To store gcd - + 2; // Local header + + 2; // Values of `gcd_bytes_size` and `bytes_to_skip` } uint8_t CompressionCodecGCD::getMethodByte() const @@ -147,7 +148,7 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest, if (source_size - sizeof(T) != output_size) throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress GCD-encoded data"); - memcpy(dest, source, source_size); + memcpy(dest, source, source_size - sizeof(T)); return; } @@ -160,6 +161,7 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest, source += sizeof(T); dest += sizeof(T); } + chassert(source == source_end); } } @@ -209,6 +211,8 @@ void CompressionCodecGCD::doDecompressData(const char * source, UInt32 source_si throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress GCD-encoded data. File has wrong header"); UInt8 bytes_to_skip = uncompressed_size % bytes_size; + chassert(bytes_to_skip == source[1]); + UInt32 output_size = uncompressed_size - bytes_to_skip; if (static_cast(2 + bytes_to_skip) > source_size) diff --git a/tests/queries/0_stateless/03093_bug_gcd_codec.reference b/tests/queries/0_stateless/03093_bug_gcd_codec.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03093_bug_gcd_codec.sql b/tests/queries/0_stateless/03093_bug_gcd_codec.sql new file mode 100644 index 00000000000..d507094c1e6 --- /dev/null +++ b/tests/queries/0_stateless/03093_bug_gcd_codec.sql @@ -0,0 +1,17 @@ +-- Tags: long + +CREATE TABLE test_gcd(test_col UInt32 CODEC(GCD, LZ4)) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS index_granularity = 8192, index_granularity_bytes = 1024; + +INSERT INTO test_gcd SELECT floor(randUniform(1, 3)) FROM numbers(15000000); +OPTIMIZE TABLE test_gcd FINAL; + +CREATE TABLE test_gcd2(test_col UInt32 CODEC(GCD, LZ4)) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS index_granularity = 8192, index_granularity_bytes = 1024, min_bytes_for_wide_part = 0, max_compress_block_size = 1024, min_compress_block_size = 1024; + +INSERT INTO test_gcd2 SELECT floor(randUniform(1, 3)) FROM numbers(15000000); +OPTIMIZE TABLE test_gcd2 FINAL; From c7fcc5493cfe5b5b3e2ab884ad1273cbee25c688 Mon Sep 17 00:00:00 2001 From: HowePa <2873679104@qq.com> Date: Tue, 23 Apr 2024 11:37:55 +0800 Subject: [PATCH 0265/1018] fix typo for part log --- docs/en/operations/system-tables/part_log.md | 2 +- src/Interpreters/PartLog.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/system-tables/part_log.md b/docs/en/operations/system-tables/part_log.md index af582646653..c0caea1ce5e 100644 --- a/docs/en/operations/system-tables/part_log.md +++ b/docs/en/operations/system-tables/part_log.md @@ -14,7 +14,7 @@ The `system.part_log` table contains the following columns: - `event_type` ([Enum8](../../sql-reference/data-types/enum.md)) — Type of the event that occurred with the data part. Can have one of the following values: - `NewPart` — Inserting of a new data part. - `MergeParts` — Merging of data parts. - - `DownloadParts` — Downloading a data part. + - `DownloadPart` — Downloading a data part. - `RemovePart` — Removing or detaching a data part using [DETACH PARTITION](../../sql-reference/statements/alter/partition.md#alter_detach-partition). - `MutatePart` — Mutating of a data part. - `MovePart` — Moving the data part from the one disk to another one. diff --git a/src/Interpreters/PartLog.cpp b/src/Interpreters/PartLog.cpp index 66f933f1afa..db339375231 100644 --- a/src/Interpreters/PartLog.cpp +++ b/src/Interpreters/PartLog.cpp @@ -101,7 +101,7 @@ ColumnsDescription PartLogElement::getColumnsDescription() "Can have one of the following values: " "NewPart — Inserting of a new data part, " "MergeParts — Merging of data parts, " - "DownloadParts — Downloading a data part, " + "DownloadPart — Downloading a data part, " "RemovePart — Removing or detaching a data part using DETACH PARTITION, " "MutatePart — Mutating of a data part, " "MovePart — Moving the data part from the one disk to another one."}, From d44062edb29eb8d1512725edfb0c1c1c6799dc73 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 23 Apr 2024 15:11:22 +0800 Subject: [PATCH 0266/1018] Fix LowCardinality(Nullable) key in hyperrectangle --- src/Storages/MergeTree/KeyCondition.cpp | 16 ++++++++++------ .../MergeTree/MergeTreeDataSelectExecutor.cpp | 6 ++++-- ...lity_nullable_non_first_primary_key.reference | 1 + ...ardinality_nullable_non_first_primary_key.sql | 10 ++++++++++ 4 files changed, 25 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/03129_low_cardinality_nullable_non_first_primary_key.reference create mode 100644 tests/queries/0_stateless/03129_low_cardinality_nullable_non_first_primary_key.sql diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 2d57ea40c9c..80f425c3e06 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -2250,9 +2250,11 @@ static BoolMask forAnyHyperrectangle( if (left_bounded && right_bounded) hyperrectangle[prefix_size] = Range(left_keys[prefix_size], true, right_keys[prefix_size], true); else if (left_bounded) - hyperrectangle[prefix_size] = Range::createLeftBounded(left_keys[prefix_size], true, data_types[prefix_size]->isNullable()); + hyperrectangle[prefix_size] + = Range::createLeftBounded(left_keys[prefix_size], true, isNullableOrLowCardinalityNullable(data_types[prefix_size])); else if (right_bounded) - hyperrectangle[prefix_size] = Range::createRightBounded(right_keys[prefix_size], true, data_types[prefix_size]->isNullable()); + hyperrectangle[prefix_size] + = Range::createRightBounded(right_keys[prefix_size], true, isNullableOrLowCardinalityNullable(data_types[prefix_size])); return callback(hyperrectangle); } @@ -2262,13 +2264,15 @@ static BoolMask forAnyHyperrectangle( if (left_bounded && right_bounded) hyperrectangle[prefix_size] = Range(left_keys[prefix_size], false, right_keys[prefix_size], false); else if (left_bounded) - hyperrectangle[prefix_size] = Range::createLeftBounded(left_keys[prefix_size], false, data_types[prefix_size]->isNullable()); + hyperrectangle[prefix_size] + = Range::createLeftBounded(left_keys[prefix_size], false, isNullableOrLowCardinalityNullable(data_types[prefix_size])); else if (right_bounded) - hyperrectangle[prefix_size] = Range::createRightBounded(right_keys[prefix_size], false, data_types[prefix_size]->isNullable()); + hyperrectangle[prefix_size] + = Range::createRightBounded(right_keys[prefix_size], false, isNullableOrLowCardinalityNullable(data_types[prefix_size])); for (size_t i = prefix_size + 1; i < key_size; ++i) { - if (data_types[i]->isNullable()) + if (isNullableOrLowCardinalityNullable(data_types[i])) hyperrectangle[i] = Range::createWholeUniverse(); else hyperrectangle[i] = Range::createWholeUniverseWithoutNull(); @@ -2324,7 +2328,7 @@ BoolMask KeyCondition::checkInRange( key_ranges.reserve(used_key_size); for (size_t i = 0; i < used_key_size; ++i) { - if (data_types[i]->isNullable()) + if (isNullableOrLowCardinalityNullable(data_types[i])) key_ranges.push_back(Range::createWholeUniverse()); else key_ranges.push_back(Range::createWholeUniverseWithoutNull()); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 345872efddf..cd0a7615762 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -383,7 +383,8 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( if (has_lower_limit) { if (!key_condition.addCondition( - sampling_key.column_names[0], Range::createLeftBounded(lower, true, sampling_key.data_types[0]->isNullable()))) + sampling_key.column_names[0], + Range::createLeftBounded(lower, true, isNullableOrLowCardinalityNullable(sampling_key.data_types[0])))) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Sampling column not in primary key"); ASTPtr args = std::make_shared(); @@ -401,7 +402,8 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( if (has_upper_limit) { if (!key_condition.addCondition( - sampling_key.column_names[0], Range::createRightBounded(upper, false, sampling_key.data_types[0]->isNullable()))) + sampling_key.column_names[0], + Range::createRightBounded(upper, false, isNullableOrLowCardinalityNullable(sampling_key.data_types[0])))) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Sampling column not in primary key"); ASTPtr args = std::make_shared(); diff --git a/tests/queries/0_stateless/03129_low_cardinality_nullable_non_first_primary_key.reference b/tests/queries/0_stateless/03129_low_cardinality_nullable_non_first_primary_key.reference new file mode 100644 index 00000000000..712cd79a3f2 --- /dev/null +++ b/tests/queries/0_stateless/03129_low_cardinality_nullable_non_first_primary_key.reference @@ -0,0 +1 @@ +1650 diff --git a/tests/queries/0_stateless/03129_low_cardinality_nullable_non_first_primary_key.sql b/tests/queries/0_stateless/03129_low_cardinality_nullable_non_first_primary_key.sql new file mode 100644 index 00000000000..457c288da69 --- /dev/null +++ b/tests/queries/0_stateless/03129_low_cardinality_nullable_non_first_primary_key.sql @@ -0,0 +1,10 @@ +DROP TABLE IF EXISTS small; + +CREATE TABLE small (`dt` DateTime, `user_email` LowCardinality(Nullable(String))) +ENGINE = MergeTree order by (dt, user_email) settings allow_nullable_key = 1, min_bytes_for_wide_part=0, min_rows_for_wide_part=0; + +INSERT INTO small (dt, user_email) SELECT number, if(number % 3 = 2, NULL, number) FROM numbers(1e2); + +SELECT SUM(dt::int) FROM small WHERE user_email IS NULL; + +DROP TABLE small; From 447aa5bf6926a8bf6442727d9bb2989340cda342 Mon Sep 17 00:00:00 2001 From: Alexey Petrunyaka <167422282+pet74alex@users.noreply.github.com> Date: Tue, 23 Apr 2024 12:32:08 +0300 Subject: [PATCH 0267/1018] Mistypes fixes in generateUUIDv7.cpp --- src/Functions/generateUUIDv7.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/generateUUIDv7.cpp b/src/Functions/generateUUIDv7.cpp index 1a7f358f263..9354e75d4f3 100644 --- a/src/Functions/generateUUIDv7.cpp +++ b/src/Functions/generateUUIDv7.cpp @@ -165,7 +165,7 @@ DECLARE_SEVERAL_IMPLEMENTATIONS( static constexpr auto name = "generateUUIDv7WithCounter"; struct Data : std::lock_guard, CounterDataCommon { - // Implement counter monotony whithin one timestamp accross all threads generating UUIDv7 with counter simultaneously + // Implement counter monotony within one timestamp across all threads generating UUIDv7 with counter simultaneously static inline UUIDAsArray uuid_data; static inline std::mutex mtx; Data() : std::lock_guard(mtx), CounterDataCommon(uuid_data) { } @@ -360,7 +360,7 @@ The function returns a value of type UUID. .description = R"( Generates a UUID of version 7 with current Unix time having milliseconds precision, a monotonic counter within the same timestamp and the same request starting from the random value, and followed by 4 random bytes. This function takes an optional argument, the value of which is discarded to generate different values in case the function is called multiple times. -This function is a little bit faster version of the function GenerateUUIDv7WithCounter. It doesn't guarantee the counter monotony withing the same timestamp accross different requests. +This function is a little bit faster version of the function GenerateUUIDv7WithCounter. It doesn't guarantee the counter monotony within the same timestamp across different requests. The function returns a value of type UUID. )", .examples{ From e9f80b8631341b57a8f3f72e7b2ff5de2fa810b7 Mon Sep 17 00:00:00 2001 From: Alexey Petrunyaka <167422282+pet74alex@users.noreply.github.com> Date: Tue, 23 Apr 2024 12:36:07 +0300 Subject: [PATCH 0268/1018] Update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index e63a7608210..811bf3f8e9c 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1008,7 +1008,9 @@ URLPathHierarchy USearch UUIDNumToString UUIDStringToNum +UUIDToNum UUIDs +UUIDv UUid Uber Uint @@ -1043,6 +1045,8 @@ Wether WikiStat WindowView Winkler +WithCounter +WithFastCounter WithNames WithNamesAndTypes WordNet From 2ba6be6d8b88a1f6d876e9ea86a88a49fa5f9a3d Mon Sep 17 00:00:00 2001 From: Alexey Petrunyaka <167422282+pet74alex@users.noreply.github.com> Date: Tue, 23 Apr 2024 12:39:29 +0300 Subject: [PATCH 0269/1018] Small style fix in generateUUIDv7.cpp --- src/Functions/generateUUIDv7.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Functions/generateUUIDv7.cpp b/src/Functions/generateUUIDv7.cpp index 9354e75d4f3..e6f78543c0d 100644 --- a/src/Functions/generateUUIDv7.cpp +++ b/src/Functions/generateUUIDv7.cpp @@ -46,7 +46,8 @@ inline void fillTimestamp(UUIDAsArray & uuid, uint64_t timestamp) DECLARE_SEVERAL_IMPLEMENTATIONS( - namespace UUIDv7Impl { + namespace UUIDv7Impl + { inline void store(UUID & new_uuid, UUIDAsArray & uuid) { uuid[6] = (uuid[6] & 0x0f) | 0x70; // version 7 From 9c744e50ec6494a14f24004a58ba0d67686388e4 Mon Sep 17 00:00:00 2001 From: Alexey Petrunyaka <167422282+pet74alex@users.noreply.github.com> Date: Tue, 23 Apr 2024 12:55:17 +0300 Subject: [PATCH 0270/1018] Update generateUUIDv7.cpp for style check test --- src/Functions/generateUUIDv7.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Functions/generateUUIDv7.cpp b/src/Functions/generateUUIDv7.cpp index e6f78543c0d..17619caffec 100644 --- a/src/Functions/generateUUIDv7.cpp +++ b/src/Functions/generateUUIDv7.cpp @@ -172,8 +172,7 @@ DECLARE_SEVERAL_IMPLEMENTATIONS( Data() : std::lock_guard(mtx), CounterDataCommon(uuid_data) { } }; }; - } // namespace UUIDv7Impl - + } template class FunctionGenerateUUIDv7Base From 771219a118f0b348bb27a3b6be84a4dade640e11 Mon Sep 17 00:00:00 2001 From: Dani Pozo Date: Tue, 23 Apr 2024 12:34:26 +0200 Subject: [PATCH 0271/1018] Apply suggestion to use pop_back --- .../ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h index c8e556d012e..8470b843e10 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h @@ -22,7 +22,7 @@ struct AzureBlobStorageEndpoint { String url = storage_account_url; if (url.ends_with('/')) - url = url.substr(0, url.size() - 1); + url.pop_back(); if (!account_name.empty()) url += "/" + account_name; From 7c13ea020c7894dbabe8e73cf64d2a80a27a3006 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 23 Apr 2024 14:17:30 +0300 Subject: [PATCH 0272/1018] PlannerJoins remove unused comments --- src/Planner/PlannerJoins.cpp | 112 +++-------------------------------- 1 file changed, 9 insertions(+), 103 deletions(-) diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 94ee249106a..78b0e966ca4 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -125,9 +125,8 @@ TableExpressionSet extractTableExpressionsSet(const QueryTreeNodePtr & node) return res; } -std::optional extractJoinTableSideFromExpression(//const ActionsDAG::Node * expression_root_node, +std::optional extractJoinTableSideFromExpression( const IQueryTreeNode * expression_root_node, - //const std::unordered_set & join_expression_dag_input_nodes, const TableExpressionSet & left_table_expressions, const TableExpressionSet & right_table_expressions, const JoinNode & join_node) @@ -136,30 +135,11 @@ std::optional extractJoinTableSideFromExpression(//const ActionsD std::vector nodes_to_process; nodes_to_process.push_back(expression_root_node); - // std::cerr << "==== extractJoinTableSideFromExpression\n"; - // std::cerr << "inp nodes" << std::endl; - // for (const auto * node : join_expression_dag_input_nodes) - // std::cerr << reinterpret_cast(node) << ' ' << node->result_name << std::endl; - - - // std::cerr << "l names" << std::endl; - // for (const auto & l : left_table_expression_columns_names) - // std::cerr << l << std::endl; - - // std::cerr << "r names" << std::endl; - // for (const auto & r : right_table_expression_columns_names) - // std::cerr << r << std::endl; - - // const auto * left_table_expr = join_node.getLeftTableExpression().get(); - // const auto * right_table_expr = join_node.getRightTableExpression().get(); - while (!nodes_to_process.empty()) { const auto * node_to_process = nodes_to_process.back(); nodes_to_process.pop_back(); - //std::cerr << "... " << reinterpret_cast(node_to_process) << ' ' << node_to_process->result_name << std::endl; - if (const auto * function_node = node_to_process->as()) { for (const auto & child : function_node->getArguments()) @@ -172,22 +152,7 @@ std::optional extractJoinTableSideFromExpression(//const ActionsD if (!column_node) continue; - // if (!join_expression_dag_input_nodes.contains(node_to_process)) - // continue; - const auto & input_name = column_node->getColumnName(); - - // bool left_table_expression_contains_input = left_table_expression_columns_names.contains(input_name); - // bool right_table_expression_contains_input = right_table_expression_columns_names.contains(input_name); - - // if (!left_table_expression_contains_input && !right_table_expression_contains_input) - // throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, - // "JOIN {} actions has column {} that do not exist in left {} or right {} table expression columns", - // join_node.formatASTForErrorMessage(), - // input_name, - // boost::join(left_table_expression_columns_names, ", "), - // boost::join(right_table_expression_columns_names, ", ")); - const auto * column_source = column_node->getColumnSource().get(); if (!column_source) throw Exception(ErrorCodes::LOGICAL_ERROR, "No source for column {} in JOIN {}", input_name, join_node.formatASTForErrorMessage()); @@ -235,9 +200,6 @@ void buildJoinClause( ActionsDAGPtr & left_dag, ActionsDAGPtr & right_dag, const PlannerContextPtr & planner_context, - //ActionsDAGPtr join_expression_dag, - //const std::unordered_set & join_expression_dag_input_nodes, - //const ActionsDAG::Node * join_expressions_actions_node, const QueryTreeNodePtr & join_expression, const TableExpressionSet & left_table_expressions, const TableExpressionSet & right_table_expressions, @@ -245,22 +207,16 @@ void buildJoinClause( JoinClause & join_clause) { std::string function_name; - - //std::cerr << join_expression_dag->dumpDAG() << std::endl; auto * function_node = join_expression->as(); if (function_node) function_name = function_node->getFunction()->getName(); - // if (join_expressions_actions_node->function) - // function_name = join_expressions_actions_node->function->getName(); - /// For 'and' function go into children if (function_name == "and") { for (const auto & child : function_node->getArguments()) { - buildJoinClause(//join_expression_dag, - //join_expression_dag_input_nodes, + buildJoinClause( left_dag, right_dag, planner_context, @@ -279,17 +235,15 @@ void buildJoinClause( if (function_name == "equals" || function_name == "isNotDistinctFrom" || is_asof_join_inequality) { - const auto left_child = function_node->getArguments().getNodes().at(0);//join_expressions_actions_node->children.at(0); - const auto right_child = function_node->getArguments().getNodes().at(1); //join_expressions_actions_node->children.at(1); + const auto left_child = function_node->getArguments().getNodes().at(0); + const auto right_child = function_node->getArguments().getNodes().at(1); auto left_expression_side_optional = extractJoinTableSideFromExpression(left_child.get(), - //join_expression_dag_input_nodes, left_table_expressions, right_table_expressions, join_node); auto right_expression_side_optional = extractJoinTableSideFromExpression(right_child.get(), - //join_expression_dag_input_nodes, left_table_expressions, right_table_expressions, join_node); @@ -314,7 +268,6 @@ void buildJoinClause( } else { - // std::cerr << "===============\n"; auto left_expression_side = *left_expression_side_optional; auto right_expression_side = *right_expression_side_optional; @@ -361,8 +314,7 @@ void buildJoinClause( return; } - auto expression_side_optional = extractJoinTableSideFromExpression(//join_expressions_actions_node, - //join_expression_dag_input_nodes, + auto expression_side_optional = extractJoinTableSideFromExpression( join_expression.get(), left_table_expressions, right_table_expressions, @@ -377,32 +329,15 @@ void buildJoinClause( join_clause.addCondition(expression_side, node); } -JoinClausesAndActions buildJoinClausesAndActions(//const ColumnsWithTypeAndName & join_expression_input_columns, +JoinClausesAndActions buildJoinClausesAndActions( const ColumnsWithTypeAndName & left_table_expression_columns, const ColumnsWithTypeAndName & right_table_expression_columns, const JoinNode & join_node, const PlannerContextPtr & planner_context) { - //ActionsDAGPtr join_expression_actions = std::make_shared(join_expression_input_columns); - ActionsDAGPtr left_join_actions = std::make_shared(left_table_expression_columns); ActionsDAGPtr right_join_actions = std::make_shared(right_table_expression_columns); - // LOG_TRACE(getLogger("Planner"), "buildJoinClausesAndActions cols {} ", left_join_actions->dumpDAG()); - // LOG_TRACE(getLogger("Planner"), "buildJoinClausesAndActions cols {} ", right_join_actions->dumpDAG()); - - /** In ActionsDAG if input node has constant representation additional constant column is added. - * That way we cannot simply check that node has INPUT type during resolution of expression join table side. - * Put all nodes after actions dag initialization in set. - * To check if actions dag node is input column, we check if set contains it. - */ - // const auto & join_expression_actions_nodes = join_expression_actions->getNodes(); - - // std::unordered_set join_expression_dag_input_nodes; - // join_expression_dag_input_nodes.reserve(join_expression_actions_nodes.size()); - // for (const auto & node : join_expression_actions_nodes) - // join_expression_dag_input_nodes.insert(&node); - /** It is possible to have constant value in JOIN ON section, that we need to ignore during DAG construction. * If we do not ignore it, this function will be replaced by underlying constant. * For example ASOF JOIN does not support JOIN with constants, and we should process it like ordinary JOIN. @@ -411,9 +346,6 @@ JoinClausesAndActions buildJoinClausesAndActions(//const ColumnsWithTypeAndName * ON (t1.id = t2.id) AND 1 != 1 AND (t1.value >= t1.value); */ auto join_expression = join_node.getJoinExpression(); - // LOG_TRACE(getLogger("Planner"), "buildJoinClausesAndActions expr {} ", join_expression->formatConvertedASTForErrorMessage()); - // LOG_TRACE(getLogger("Planner"), "buildJoinClausesAndActions expr {} ", join_expression->dumpTree()); - auto * constant_join_expression = join_expression->as(); if (constant_join_expression && constant_join_expression->hasSourceExpression()) @@ -425,19 +357,6 @@ JoinClausesAndActions buildJoinClausesAndActions(//const ColumnsWithTypeAndName "JOIN {} join expression expected function", join_node.formatASTForErrorMessage()); - // PlannerActionsVisitor join_expression_visitor(planner_context); - // auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(join_expression_actions, join_expression); - // if (join_expression_dag_node_raw_pointers.size() != 1) - // throw Exception(ErrorCodes::LOGICAL_ERROR, - // "JOIN {} ON clause contains multiple expressions", - // join_node.formatASTForErrorMessage()); - - // const auto * join_expressions_actions_root_node = join_expression_dag_node_raw_pointers[0]; - // if (!join_expressions_actions_root_node->function) - // throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, - // "JOIN {} join expression expected function", - // join_node.formatASTForErrorMessage()); - size_t left_table_expression_columns_size = left_table_expression_columns.size(); Names join_left_actions_names; @@ -470,7 +389,6 @@ JoinClausesAndActions buildJoinClausesAndActions(//const ColumnsWithTypeAndName auto join_right_table_expressions = extractTableExpressionsSet(join_node.getRightTableExpression()); JoinClausesAndActions result; - //result.join_expression_actions = join_expression_actions; const auto & function_name = function_node->getFunction()->getName(); if (function_name == "or") @@ -479,8 +397,7 @@ JoinClausesAndActions buildJoinClausesAndActions(//const ColumnsWithTypeAndName { result.join_clauses.emplace_back(); - buildJoinClause(//join_expression_actions, - //join_expression_dag_input_nodes, + buildJoinClause( left_join_actions, right_join_actions, planner_context, @@ -499,9 +416,7 @@ JoinClausesAndActions buildJoinClausesAndActions(//const ColumnsWithTypeAndName left_join_actions, right_join_actions, planner_context, - //join_expression_actions, - //join_expression_dag_input_nodes, - join_expression, //join_expressions_actions_root_node, + join_expression, join_left_table_expressions, join_right_table_expressions, join_node, @@ -621,12 +536,6 @@ JoinClausesAndActions buildJoinClausesAndActions(//const ColumnsWithTypeAndName result.left_join_expressions_actions = left_join_actions->clone(); result.left_join_tmp_expression_actions = std::move(left_join_actions); result.left_join_expressions_actions->removeUnusedActions(join_left_actions_names); - - // for (const auto & name : join_right_actions_names) - // std::cerr << ".. " << name << std::endl; - - // std::cerr << right_join_actions->dumpDAG() << std::endl; - result.right_join_expressions_actions = right_join_actions->clone(); result.right_join_tmp_expression_actions = std::move(right_join_actions); result.right_join_expressions_actions->removeUnusedActions(join_right_actions_names); @@ -648,10 +557,7 @@ JoinClausesAndActions buildJoinClausesAndActions( "JOIN {} join does not have ON section", join_node_typed.formatASTForErrorMessage()); - // auto join_expression_input_columns = left_table_expression_columns; - // join_expression_input_columns.insert(join_expression_input_columns.end(), right_table_expression_columns.begin(), right_table_expression_columns.end()); - - return buildJoinClausesAndActions(/*join_expression_input_columns,*/ left_table_expression_columns, right_table_expression_columns, join_node_typed, planner_context); + return buildJoinClausesAndActions(left_table_expression_columns, right_table_expression_columns, join_node_typed, planner_context); } std::optional tryExtractConstantFromJoinNode(const QueryTreeNodePtr & join_node) From 35d700a5af99a8046faa4f75993973291d92d0c9 Mon Sep 17 00:00:00 2001 From: Alexey Petrunyaka <167422282+pet74alex@users.noreply.github.com> Date: Tue, 23 Apr 2024 14:22:58 +0300 Subject: [PATCH 0273/1018] Update generateUUIDv7.cpp small fixes for clang-tidy checks --- src/Functions/generateUUIDv7.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/generateUUIDv7.cpp b/src/Functions/generateUUIDv7.cpp index 17619caffec..101fe4a279d 100644 --- a/src/Functions/generateUUIDv7.cpp +++ b/src/Functions/generateUUIDv7.cpp @@ -60,7 +60,7 @@ DECLARE_SEVERAL_IMPLEMENTATIONS( struct UUIDv7Base { UUIDAsArray & uuid; - UUIDv7Base(UUIDAsArray & u) : uuid(u) { } + explicit UUIDv7Base(UUIDAsArray & u) : uuid(u) { } }; struct RandomData @@ -83,7 +83,7 @@ DECLARE_SEVERAL_IMPLEMENTATIONS( struct CounterDataCommon : UUIDv7Base { - CounterDataCommon(UUIDAsArray & u) : UUIDv7Base(u) { } + explicit CounterDataCommon(UUIDAsArray & u) : UUIDv7Base(u) { } uint64_t getCounter() { From 8d7221fe353696ae51d3249c26d5acfec1850b87 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 10 Apr 2024 18:56:30 +0000 Subject: [PATCH 0274/1018] Add test for bug 33446 --- tests/performance/one_thousand_joins.xml | 6 ++++++ .../queries/0_stateless/03094_one_thousand_joins.reference | 1 + tests/queries/0_stateless/03094_one_thousand_joins.sql | 4 ++++ 3 files changed, 11 insertions(+) create mode 100644 tests/performance/one_thousand_joins.xml create mode 100644 tests/queries/0_stateless/03094_one_thousand_joins.reference create mode 100644 tests/queries/0_stateless/03094_one_thousand_joins.sql diff --git a/tests/performance/one_thousand_joins.xml b/tests/performance/one_thousand_joins.xml new file mode 100644 index 00000000000..d7e66bb1c67 --- /dev/null +++ b/tests/performance/one_thousand_joins.xml @@ -0,0 +1,6 @@ + + + +SELECT * FROM (SELECT 1 AS x) t1 JOIN (SELECT 1 AS x) t2 ON t1.x = t2.x JOIN (SELECT 1 AS x) t3 ON t1.x = t3.x JOIN (SELECT 1 AS x) t4 ON t1.x = t4.x JOIN (SELECT 1 AS x) t5 ON t1.x = t5.x JOIN (SELECT 1 AS x) t6 ON t1.x = t6.x JOIN (SELECT 1 AS x) t7 ON t1.x = t7.x JOIN (SELECT 1 AS x) t8 ON t1.x = t8.x JOIN (SELECT 1 AS x) t9 ON t1.x = t9.x JOIN (SELECT 1 AS x) t10 ON t1.x = t10.x JOIN (SELECT 1 AS x) t11 ON t1.x = t11.x JOIN (SELECT 1 AS x) t12 ON t1.x = t12.x JOIN (SELECT 1 AS x) t13 ON t1.x = t13.x JOIN (SELECT 1 AS x) t14 ON t1.x = t14.x JOIN (SELECT 1 AS x) t15 ON t1.x = t15.x JOIN (SELECT 1 AS x) t16 ON t1.x = t16.x JOIN (SELECT 1 AS x) t17 ON t1.x = t17.x JOIN (SELECT 1 AS x) t18 ON t1.x = t18.x JOIN (SELECT 1 AS x) t19 ON t1.x = t19.x JOIN (SELECT 1 AS x) t20 ON t1.x = t20.x JOIN (SELECT 1 AS x) t21 ON t1.x = t21.x JOIN (SELECT 1 AS x) t22 ON t1.x = t22.x JOIN (SELECT 1 AS x) t23 ON t1.x = t23.x JOIN (SELECT 1 AS x) t24 ON t1.x = t24.x JOIN (SELECT 1 AS x) t25 ON t1.x = t25.x JOIN (SELECT 1 AS x) t26 ON t1.x = t26.x JOIN (SELECT 1 AS x) t27 ON t1.x = t27.x JOIN (SELECT 1 AS x) t28 ON t1.x = t28.x JOIN (SELECT 1 AS x) t29 ON t1.x = t29.x JOIN (SELECT 1 AS x) t30 ON t1.x = t30.x JOIN (SELECT 1 AS x) t31 ON t1.x = t31.x JOIN (SELECT 1 AS x) t32 ON t1.x = t32.x JOIN (SELECT 1 AS x) t33 ON t1.x = t33.x JOIN (SELECT 1 AS x) t34 ON t1.x = t34.x JOIN (SELECT 1 AS x) t35 ON t1.x = t35.x JOIN (SELECT 1 AS x) t36 ON t1.x = t36.x JOIN (SELECT 1 AS x) t37 ON t1.x = t37.x JOIN (SELECT 1 AS x) t38 ON t1.x = t38.x JOIN (SELECT 1 AS x) t39 ON t1.x = t39.x JOIN (SELECT 1 AS x) t40 ON t1.x = t40.x JOIN (SELECT 1 AS x) t41 ON t1.x = t41.x JOIN (SELECT 1 AS x) t42 ON t1.x = t42.x JOIN (SELECT 1 AS x) t43 ON t1.x = t43.x JOIN (SELECT 1 AS x) t44 ON t1.x = t44.x JOIN (SELECT 1 AS x) t45 ON t1.x = t45.x JOIN (SELECT 1 AS x) t46 ON t1.x = t46.x JOIN (SELECT 1 AS x) t47 ON t1.x = t47.x JOIN (SELECT 1 AS x) t48 ON t1.x = t48.x JOIN (SELECT 1 AS x) t49 ON t1.x = t49.x JOIN (SELECT 1 AS x) t50 ON t1.x = t50.x JOIN (SELECT 1 AS x) t51 ON t1.x = t51.x JOIN (SELECT 1 AS x) t52 ON t1.x = t52.x JOIN (SELECT 1 AS x) t53 ON t1.x = t53.x JOIN (SELECT 1 AS x) t54 ON t1.x = t54.x JOIN (SELECT 1 AS x) t55 ON t1.x = t55.x JOIN (SELECT 1 AS x) t56 ON t1.x = t56.x JOIN (SELECT 1 AS x) t57 ON t1.x = t57.x JOIN (SELECT 1 AS x) t58 ON t1.x = t58.x JOIN (SELECT 1 AS x) t59 ON t1.x = t59.x JOIN (SELECT 1 AS x) t60 ON t1.x = t60.x JOIN (SELECT 1 AS x) t61 ON t1.x = t61.x JOIN (SELECT 1 AS x) t62 ON t1.x = t62.x JOIN (SELECT 1 AS x) t63 ON t1.x = t63.x JOIN (SELECT 1 AS x) t64 ON t1.x = t64.x JOIN (SELECT 1 AS x) t65 ON t1.x = t65.x JOIN (SELECT 1 AS x) t66 ON t1.x = t66.x JOIN (SELECT 1 AS x) t67 ON t1.x = t67.x JOIN (SELECT 1 AS x) t68 ON t1.x = t68.x JOIN (SELECT 1 AS x) t69 ON t1.x = t69.x JOIN (SELECT 1 AS x) t70 ON t1.x = t70.x JOIN (SELECT 1 AS x) t71 ON t1.x = t71.x JOIN (SELECT 1 AS x) t72 ON t1.x = t72.x JOIN (SELECT 1 AS x) t73 ON t1.x = t73.x JOIN (SELECT 1 AS x) t74 ON t1.x = t74.x JOIN (SELECT 1 AS x) t75 ON t1.x = t75.x JOIN (SELECT 1 AS x) t76 ON t1.x = t76.x JOIN (SELECT 1 AS x) t77 ON t1.x = t77.x JOIN (SELECT 1 AS x) t78 ON t1.x = t78.x JOIN (SELECT 1 AS x) t79 ON t1.x = t79.x JOIN (SELECT 1 AS x) t80 ON t1.x = t80.x JOIN (SELECT 1 AS x) t81 ON t1.x = t81.x JOIN (SELECT 1 AS x) t82 ON t1.x = t82.x JOIN (SELECT 1 AS x) t83 ON t1.x = t83.x JOIN (SELECT 1 AS x) t84 ON t1.x = t84.x JOIN (SELECT 1 AS x) t85 ON t1.x = t85.x JOIN (SELECT 1 AS x) t86 ON t1.x = t86.x JOIN (SELECT 1 AS x) t87 ON t1.x = t87.x JOIN (SELECT 1 AS x) t88 ON t1.x = t88.x JOIN (SELECT 1 AS x) t89 ON t1.x = t89.x JOIN (SELECT 1 AS x) t90 ON t1.x = t90.x JOIN (SELECT 1 AS x) t91 ON t1.x = t91.x JOIN (SELECT 1 AS x) t92 ON t1.x = t92.x JOIN (SELECT 1 AS x) t93 ON t1.x = t93.x JOIN (SELECT 1 AS x) t94 ON t1.x = t94.x JOIN (SELECT 1 AS x) t95 ON t1.x = t95.x JOIN (SELECT 1 AS x) t96 ON t1.x = t96.x JOIN (SELECT 1 AS x) t97 ON t1.x = t97.x JOIN (SELECT 1 AS x) t98 ON t1.x = t98.x JOIN (SELECT 1 AS x) t99 ON t1.x = t99.x JOIN (SELECT 1 AS x) t100 ON t1.x = t100.x JOIN (SELECT 1 AS x) t101 ON t1.x = t101.x JOIN (SELECT 1 AS x) t102 ON t1.x = t102.x JOIN (SELECT 1 AS x) t103 ON t1.x = t103.x JOIN (SELECT 1 AS x) t104 ON t1.x = t104.x JOIN (SELECT 1 AS x) t105 ON t1.x = t105.x JOIN (SELECT 1 AS x) t106 ON t1.x = t106.x JOIN (SELECT 1 AS x) t107 ON t1.x = t107.x JOIN (SELECT 1 AS x) t108 ON t1.x = t108.x JOIN (SELECT 1 AS x) t109 ON t1.x = t109.x JOIN (SELECT 1 AS x) t110 ON t1.x = t110.x JOIN (SELECT 1 AS x) t111 ON t1.x = t111.x JOIN (SELECT 1 AS x) t112 ON t1.x = t112.x JOIN (SELECT 1 AS x) t113 ON t1.x = t113.x JOIN (SELECT 1 AS x) t114 ON t1.x = t114.x JOIN (SELECT 1 AS x) t115 ON t1.x = t115.x JOIN (SELECT 1 AS x) t116 ON t1.x = t116.x JOIN (SELECT 1 AS x) t117 ON t1.x = t117.x JOIN (SELECT 1 AS x) t118 ON t1.x = t118.x JOIN (SELECT 1 AS x) t119 ON t1.x = t119.x JOIN (SELECT 1 AS x) t120 ON t1.x = t120.x JOIN (SELECT 1 AS x) t121 ON t1.x = t121.x JOIN (SELECT 1 AS x) t122 ON t1.x = t122.x JOIN (SELECT 1 AS x) t123 ON t1.x = t123.x JOIN (SELECT 1 AS x) t124 ON t1.x = t124.x JOIN (SELECT 1 AS x) t125 ON t1.x = t125.x JOIN (SELECT 1 AS x) t126 ON t1.x = t126.x JOIN (SELECT 1 AS x) t127 ON t1.x = t127.x JOIN (SELECT 1 AS x) t128 ON t1.x = t128.x JOIN (SELECT 1 AS x) t129 ON t1.x = t129.x JOIN (SELECT 1 AS x) t130 ON t1.x = t130.x JOIN (SELECT 1 AS x) t131 ON t1.x = t131.x JOIN (SELECT 1 AS x) t132 ON t1.x = t132.x JOIN (SELECT 1 AS x) t133 ON t1.x = t133.x JOIN (SELECT 1 AS x) t134 ON t1.x = t134.x JOIN (SELECT 1 AS x) t135 ON t1.x = t135.x JOIN (SELECT 1 AS x) t136 ON t1.x = t136.x JOIN (SELECT 1 AS x) t137 ON t1.x = t137.x JOIN (SELECT 1 AS x) t138 ON t1.x = t138.x JOIN (SELECT 1 AS x) t139 ON t1.x = t139.x JOIN (SELECT 1 AS x) t140 ON t1.x = t140.x JOIN (SELECT 1 AS x) t141 ON t1.x = t141.x JOIN (SELECT 1 AS x) t142 ON t1.x = t142.x JOIN (SELECT 1 AS x) t143 ON t1.x = t143.x JOIN (SELECT 1 AS x) t144 ON t1.x = t144.x JOIN (SELECT 1 AS x) t145 ON t1.x = t145.x JOIN (SELECT 1 AS x) t146 ON t1.x = t146.x JOIN (SELECT 1 AS x) t147 ON t1.x = t147.x JOIN (SELECT 1 AS x) t148 ON t1.x = t148.x JOIN (SELECT 1 AS x) t149 ON t1.x = t149.x JOIN (SELECT 1 AS x) t150 ON t1.x = t150.x JOIN (SELECT 1 AS x) t151 ON t1.x = t151.x JOIN (SELECT 1 AS x) t152 ON t1.x = t152.x JOIN (SELECT 1 AS x) t153 ON t1.x = t153.x JOIN (SELECT 1 AS x) t154 ON t1.x = t154.x JOIN (SELECT 1 AS x) t155 ON t1.x = t155.x JOIN (SELECT 1 AS x) t156 ON t1.x = t156.x JOIN (SELECT 1 AS x) t157 ON t1.x = t157.x JOIN (SELECT 1 AS x) t158 ON t1.x = t158.x JOIN (SELECT 1 AS x) t159 ON t1.x = t159.x JOIN (SELECT 1 AS x) t160 ON t1.x = t160.x JOIN (SELECT 1 AS x) t161 ON t1.x = t161.x JOIN (SELECT 1 AS x) t162 ON t1.x = t162.x JOIN (SELECT 1 AS x) t163 ON t1.x = t163.x JOIN (SELECT 1 AS x) t164 ON t1.x = t164.x JOIN (SELECT 1 AS x) t165 ON t1.x = t165.x JOIN (SELECT 1 AS x) t166 ON t1.x = t166.x JOIN (SELECT 1 AS x) t167 ON t1.x = t167.x JOIN (SELECT 1 AS x) t168 ON t1.x = t168.x JOIN (SELECT 1 AS x) t169 ON t1.x = t169.x JOIN (SELECT 1 AS x) t170 ON t1.x = t170.x JOIN (SELECT 1 AS x) t171 ON t1.x = t171.x JOIN (SELECT 1 AS x) t172 ON t1.x = t172.x JOIN (SELECT 1 AS x) t173 ON t1.x = t173.x JOIN (SELECT 1 AS x) t174 ON t1.x = t174.x JOIN (SELECT 1 AS x) t175 ON t1.x = t175.x JOIN (SELECT 1 AS x) t176 ON t1.x = t176.x JOIN (SELECT 1 AS x) t177 ON t1.x = t177.x JOIN (SELECT 1 AS x) t178 ON t1.x = t178.x JOIN (SELECT 1 AS x) t179 ON t1.x = t179.x JOIN (SELECT 1 AS x) t180 ON t1.x = t180.x JOIN (SELECT 1 AS x) t181 ON t1.x = t181.x JOIN (SELECT 1 AS x) t182 ON t1.x = t182.x JOIN (SELECT 1 AS x) t183 ON t1.x = t183.x JOIN (SELECT 1 AS x) t184 ON t1.x = t184.x JOIN (SELECT 1 AS x) t185 ON t1.x = t185.x JOIN (SELECT 1 AS x) t186 ON t1.x = t186.x JOIN (SELECT 1 AS x) t187 ON t1.x = t187.x JOIN (SELECT 1 AS x) t188 ON t1.x = t188.x JOIN (SELECT 1 AS x) t189 ON t1.x = t189.x JOIN (SELECT 1 AS x) t190 ON t1.x = t190.x JOIN (SELECT 1 AS x) t191 ON t1.x = t191.x JOIN (SELECT 1 AS x) t192 ON t1.x = t192.x JOIN (SELECT 1 AS x) t193 ON t1.x = t193.x JOIN (SELECT 1 AS x) t194 ON t1.x = t194.x JOIN (SELECT 1 AS x) t195 ON t1.x = t195.x JOIN (SELECT 1 AS x) t196 ON t1.x = t196.x JOIN (SELECT 1 AS x) t197 ON t1.x = t197.x JOIN (SELECT 1 AS x) t198 ON t1.x = t198.x JOIN (SELECT 1 AS x) t199 ON t1.x = t199.x JOIN (SELECT 1 AS x) t200 ON t1.x = t200.x JOIN (SELECT 1 AS x) t201 ON t1.x = t201.x JOIN (SELECT 1 AS x) t202 ON t1.x = t202.x JOIN (SELECT 1 AS x) t203 ON t1.x = t203.x JOIN (SELECT 1 AS x) t204 ON t1.x = t204.x JOIN (SELECT 1 AS x) t205 ON t1.x = t205.x JOIN (SELECT 1 AS x) t206 ON t1.x = t206.x JOIN (SELECT 1 AS x) t207 ON t1.x = t207.x JOIN (SELECT 1 AS x) t208 ON t1.x = t208.x JOIN (SELECT 1 AS x) t209 ON t1.x = t209.x JOIN (SELECT 1 AS x) t210 ON t1.x = t210.x JOIN (SELECT 1 AS x) t211 ON t1.x = t211.x JOIN (SELECT 1 AS x) t212 ON t1.x = t212.x JOIN (SELECT 1 AS x) t213 ON t1.x = t213.x JOIN (SELECT 1 AS x) t214 ON t1.x = t214.x JOIN (SELECT 1 AS x) t215 ON t1.x = t215.x JOIN (SELECT 1 AS x) t216 ON t1.x = t216.x JOIN (SELECT 1 AS x) t217 ON t1.x = t217.x JOIN (SELECT 1 AS x) t218 ON t1.x = t218.x JOIN (SELECT 1 AS x) t219 ON t1.x = t219.x JOIN (SELECT 1 AS x) t220 ON t1.x = t220.x JOIN (SELECT 1 AS x) t221 ON t1.x = t221.x JOIN (SELECT 1 AS x) t222 ON t1.x = t222.x JOIN (SELECT 1 AS x) t223 ON t1.x = t223.x JOIN (SELECT 1 AS x) t224 ON t1.x = t224.x JOIN (SELECT 1 AS x) t225 ON t1.x = t225.x JOIN (SELECT 1 AS x) t226 ON t1.x = t226.x JOIN (SELECT 1 AS x) t227 ON t1.x = t227.x JOIN (SELECT 1 AS x) t228 ON t1.x = t228.x JOIN (SELECT 1 AS x) t229 ON t1.x = t229.x JOIN (SELECT 1 AS x) t230 ON t1.x = t230.x JOIN (SELECT 1 AS x) t231 ON t1.x = t231.x JOIN (SELECT 1 AS x) t232 ON t1.x = t232.x JOIN (SELECT 1 AS x) t233 ON t1.x = t233.x JOIN (SELECT 1 AS x) t234 ON t1.x = t234.x JOIN (SELECT 1 AS x) t235 ON t1.x = t235.x JOIN (SELECT 1 AS x) t236 ON t1.x = t236.x JOIN (SELECT 1 AS x) t237 ON t1.x = t237.x JOIN (SELECT 1 AS x) t238 ON t1.x = t238.x JOIN (SELECT 1 AS x) t239 ON t1.x = t239.x JOIN (SELECT 1 AS x) t240 ON t1.x = t240.x JOIN (SELECT 1 AS x) t241 ON t1.x = t241.x JOIN (SELECT 1 AS x) t242 ON t1.x = t242.x JOIN (SELECT 1 AS x) t243 ON t1.x = t243.x JOIN (SELECT 1 AS x) t244 ON t1.x = t244.x JOIN (SELECT 1 AS x) t245 ON t1.x = t245.x JOIN (SELECT 1 AS x) t246 ON t1.x = t246.x JOIN (SELECT 1 AS x) t247 ON t1.x = t247.x JOIN (SELECT 1 AS x) t248 ON t1.x = t248.x JOIN (SELECT 1 AS x) t249 ON t1.x = t249.x JOIN (SELECT 1 AS x) t250 ON t1.x = t250.x JOIN (SELECT 1 AS x) t251 ON t1.x = t251.x JOIN (SELECT 1 AS x) t252 ON t1.x = t252.x JOIN (SELECT 1 AS x) t253 ON t1.x = t253.x JOIN (SELECT 1 AS x) t254 ON t1.x = t254.x JOIN (SELECT 1 AS x) t255 ON t1.x = t255.x JOIN (SELECT 1 AS x) t256 ON t1.x = t256.x JOIN (SELECT 1 AS x) t257 ON t1.x = t257.x JOIN (SELECT 1 AS x) t258 ON t1.x = t258.x JOIN (SELECT 1 AS x) t259 ON t1.x = t259.x JOIN (SELECT 1 AS x) t260 ON t1.x = t260.x JOIN (SELECT 1 AS x) t261 ON t1.x = t261.x JOIN (SELECT 1 AS x) t262 ON t1.x = t262.x JOIN (SELECT 1 AS x) t263 ON t1.x = t263.x JOIN (SELECT 1 AS x) t264 ON t1.x = t264.x JOIN (SELECT 1 AS x) t265 ON t1.x = t265.x JOIN (SELECT 1 AS x) t266 ON t1.x = t266.x JOIN (SELECT 1 AS x) t267 ON t1.x = t267.x JOIN (SELECT 1 AS x) t268 ON t1.x = t268.x JOIN (SELECT 1 AS x) t269 ON t1.x = t269.x JOIN (SELECT 1 AS x) t270 ON t1.x = t270.x JOIN (SELECT 1 AS x) t271 ON t1.x = t271.x JOIN (SELECT 1 AS x) t272 ON t1.x = t272.x JOIN (SELECT 1 AS x) t273 ON t1.x = t273.x JOIN (SELECT 1 AS x) t274 ON t1.x = t274.x JOIN (SELECT 1 AS x) t275 ON t1.x = t275.x JOIN (SELECT 1 AS x) t276 ON t1.x = t276.x JOIN (SELECT 1 AS x) t277 ON t1.x = t277.x JOIN (SELECT 1 AS x) t278 ON t1.x = t278.x JOIN (SELECT 1 AS x) t279 ON t1.x = t279.x JOIN (SELECT 1 AS x) t280 ON t1.x = t280.x JOIN (SELECT 1 AS x) t281 ON t1.x = t281.x JOIN (SELECT 1 AS x) t282 ON t1.x = t282.x JOIN (SELECT 1 AS x) t283 ON t1.x = t283.x JOIN (SELECT 1 AS x) t284 ON t1.x = t284.x JOIN (SELECT 1 AS x) t285 ON t1.x = t285.x JOIN (SELECT 1 AS x) t286 ON t1.x = t286.x JOIN (SELECT 1 AS x) t287 ON t1.x = t287.x JOIN (SELECT 1 AS x) t288 ON t1.x = t288.x JOIN (SELECT 1 AS x) t289 ON t1.x = t289.x JOIN (SELECT 1 AS x) t290 ON t1.x = t290.x JOIN (SELECT 1 AS x) t291 ON t1.x = t291.x JOIN (SELECT 1 AS x) t292 ON t1.x = t292.x JOIN (SELECT 1 AS x) t293 ON t1.x = t293.x JOIN (SELECT 1 AS x) t294 ON t1.x = t294.x JOIN (SELECT 1 AS x) t295 ON t1.x = t295.x JOIN (SELECT 1 AS x) t296 ON t1.x = t296.x JOIN (SELECT 1 AS x) t297 ON t1.x = t297.x JOIN (SELECT 1 AS x) t298 ON t1.x = t298.x JOIN (SELECT 1 AS x) t299 ON t1.x = t299.x JOIN (SELECT 1 AS x) t300 ON t1.x = t300.x JOIN (SELECT 1 AS x) t301 ON t1.x = t301.x JOIN (SELECT 1 AS x) t302 ON t1.x = t302.x JOIN (SELECT 1 AS x) t303 ON t1.x = t303.x JOIN (SELECT 1 AS x) t304 ON t1.x = t304.x JOIN (SELECT 1 AS x) t305 ON t1.x = t305.x JOIN (SELECT 1 AS x) t306 ON t1.x = t306.x JOIN (SELECT 1 AS x) t307 ON t1.x = t307.x JOIN (SELECT 1 AS x) t308 ON t1.x = t308.x JOIN (SELECT 1 AS x) t309 ON t1.x = t309.x JOIN (SELECT 1 AS x) t310 ON t1.x = t310.x JOIN (SELECT 1 AS x) t311 ON t1.x = t311.x JOIN (SELECT 1 AS x) t312 ON t1.x = t312.x JOIN (SELECT 1 AS x) t313 ON t1.x = t313.x JOIN (SELECT 1 AS x) t314 ON t1.x = t314.x JOIN (SELECT 1 AS x) t315 ON t1.x = t315.x JOIN (SELECT 1 AS x) t316 ON t1.x = t316.x JOIN (SELECT 1 AS x) t317 ON t1.x = t317.x JOIN (SELECT 1 AS x) t318 ON t1.x = t318.x JOIN (SELECT 1 AS x) t319 ON t1.x = t319.x JOIN (SELECT 1 AS x) t320 ON t1.x = t320.x JOIN (SELECT 1 AS x) t321 ON t1.x = t321.x JOIN (SELECT 1 AS x) t322 ON t1.x = t322.x JOIN (SELECT 1 AS x) t323 ON t1.x = t323.x JOIN (SELECT 1 AS x) t324 ON t1.x = t324.x JOIN (SELECT 1 AS x) t325 ON t1.x = t325.x JOIN (SELECT 1 AS x) t326 ON t1.x = t326.x JOIN (SELECT 1 AS x) t327 ON t1.x = t327.x JOIN (SELECT 1 AS x) t328 ON t1.x = t328.x JOIN (SELECT 1 AS x) t329 ON t1.x = t329.x JOIN (SELECT 1 AS x) t330 ON t1.x = t330.x JOIN (SELECT 1 AS x) t331 ON t1.x = t331.x JOIN (SELECT 1 AS x) t332 ON t1.x = t332.x JOIN (SELECT 1 AS x) t333 ON t1.x = t333.x JOIN (SELECT 1 AS x) t334 ON t1.x = t334.x JOIN (SELECT 1 AS x) t335 ON t1.x = t335.x JOIN (SELECT 1 AS x) t336 ON t1.x = t336.x JOIN (SELECT 1 AS x) t337 ON t1.x = t337.x JOIN (SELECT 1 AS x) t338 ON t1.x = t338.x JOIN (SELECT 1 AS x) t339 ON t1.x = t339.x JOIN (SELECT 1 AS x) t340 ON t1.x = t340.x JOIN (SELECT 1 AS x) t341 ON t1.x = t341.x JOIN (SELECT 1 AS x) t342 ON t1.x = t342.x JOIN (SELECT 1 AS x) t343 ON t1.x = t343.x JOIN (SELECT 1 AS x) t344 ON t1.x = t344.x JOIN (SELECT 1 AS x) t345 ON t1.x = t345.x JOIN (SELECT 1 AS x) t346 ON t1.x = t346.x JOIN (SELECT 1 AS x) t347 ON t1.x = t347.x JOIN (SELECT 1 AS x) t348 ON t1.x = t348.x JOIN (SELECT 1 AS x) t349 ON t1.x = t349.x JOIN (SELECT 1 AS x) t350 ON t1.x = t350.x JOIN (SELECT 1 AS x) t351 ON t1.x = t351.x JOIN (SELECT 1 AS x) t352 ON t1.x = t352.x JOIN (SELECT 1 AS x) t353 ON t1.x = t353.x JOIN (SELECT 1 AS x) t354 ON t1.x = t354.x JOIN (SELECT 1 AS x) t355 ON t1.x = t355.x JOIN (SELECT 1 AS x) t356 ON t1.x = t356.x JOIN (SELECT 1 AS x) t357 ON t1.x = t357.x JOIN (SELECT 1 AS x) t358 ON t1.x = t358.x JOIN (SELECT 1 AS x) t359 ON t1.x = t359.x JOIN (SELECT 1 AS x) t360 ON t1.x = t360.x JOIN (SELECT 1 AS x) t361 ON t1.x = t361.x JOIN (SELECT 1 AS x) t362 ON t1.x = t362.x JOIN (SELECT 1 AS x) t363 ON t1.x = t363.x JOIN (SELECT 1 AS x) t364 ON t1.x = t364.x JOIN (SELECT 1 AS x) t365 ON t1.x = t365.x JOIN (SELECT 1 AS x) t366 ON t1.x = t366.x JOIN (SELECT 1 AS x) t367 ON t1.x = t367.x JOIN (SELECT 1 AS x) t368 ON t1.x = t368.x JOIN (SELECT 1 AS x) t369 ON t1.x = t369.x JOIN (SELECT 1 AS x) t370 ON t1.x = t370.x JOIN (SELECT 1 AS x) t371 ON t1.x = t371.x JOIN (SELECT 1 AS x) t372 ON t1.x = t372.x JOIN (SELECT 1 AS x) t373 ON t1.x = t373.x JOIN (SELECT 1 AS x) t374 ON t1.x = t374.x JOIN (SELECT 1 AS x) t375 ON t1.x = t375.x JOIN (SELECT 1 AS x) t376 ON t1.x = t376.x JOIN (SELECT 1 AS x) t377 ON t1.x = t377.x JOIN (SELECT 1 AS x) t378 ON t1.x = t378.x JOIN (SELECT 1 AS x) t379 ON t1.x = t379.x JOIN (SELECT 1 AS x) t380 ON t1.x = t380.x JOIN (SELECT 1 AS x) t381 ON t1.x = t381.x JOIN (SELECT 1 AS x) t382 ON t1.x = t382.x JOIN (SELECT 1 AS x) t383 ON t1.x = t383.x JOIN (SELECT 1 AS x) t384 ON t1.x = t384.x JOIN (SELECT 1 AS x) t385 ON t1.x = t385.x JOIN (SELECT 1 AS x) t386 ON t1.x = t386.x JOIN (SELECT 1 AS x) t387 ON t1.x = t387.x JOIN (SELECT 1 AS x) t388 ON t1.x = t388.x JOIN (SELECT 1 AS x) t389 ON t1.x = t389.x JOIN (SELECT 1 AS x) t390 ON t1.x = t390.x JOIN (SELECT 1 AS x) t391 ON t1.x = t391.x JOIN (SELECT 1 AS x) t392 ON t1.x = t392.x JOIN (SELECT 1 AS x) t393 ON t1.x = t393.x JOIN (SELECT 1 AS x) t394 ON t1.x = t394.x JOIN (SELECT 1 AS x) t395 ON t1.x = t395.x JOIN (SELECT 1 AS x) t396 ON t1.x = t396.x JOIN (SELECT 1 AS x) t397 ON t1.x = t397.x JOIN (SELECT 1 AS x) t398 ON t1.x = t398.x JOIN (SELECT 1 AS x) t399 ON t1.x = t399.x JOIN (SELECT 1 AS x) t400 ON t1.x = t400.x JOIN (SELECT 1 AS x) t401 ON t1.x = t401.x JOIN (SELECT 1 AS x) t402 ON t1.x = t402.x JOIN (SELECT 1 AS x) t403 ON t1.x = t403.x JOIN (SELECT 1 AS x) t404 ON t1.x = t404.x JOIN (SELECT 1 AS x) t405 ON t1.x = t405.x JOIN (SELECT 1 AS x) t406 ON t1.x = t406.x JOIN (SELECT 1 AS x) t407 ON t1.x = t407.x JOIN (SELECT 1 AS x) t408 ON t1.x = t408.x JOIN (SELECT 1 AS x) t409 ON t1.x = t409.x JOIN (SELECT 1 AS x) t410 ON t1.x = t410.x JOIN (SELECT 1 AS x) t411 ON t1.x = t411.x JOIN (SELECT 1 AS x) t412 ON t1.x = t412.x JOIN (SELECT 1 AS x) t413 ON t1.x = t413.x JOIN (SELECT 1 AS x) t414 ON t1.x = t414.x JOIN (SELECT 1 AS x) t415 ON t1.x = t415.x JOIN (SELECT 1 AS x) t416 ON t1.x = t416.x JOIN (SELECT 1 AS x) t417 ON t1.x = t417.x JOIN (SELECT 1 AS x) t418 ON t1.x = t418.x JOIN (SELECT 1 AS x) t419 ON t1.x = t419.x JOIN (SELECT 1 AS x) t420 ON t1.x = t420.x JOIN (SELECT 1 AS x) t421 ON t1.x = t421.x JOIN (SELECT 1 AS x) t422 ON t1.x = t422.x JOIN (SELECT 1 AS x) t423 ON t1.x = t423.x JOIN (SELECT 1 AS x) t424 ON t1.x = t424.x JOIN (SELECT 1 AS x) t425 ON t1.x = t425.x JOIN (SELECT 1 AS x) t426 ON t1.x = t426.x JOIN (SELECT 1 AS x) t427 ON t1.x = t427.x JOIN (SELECT 1 AS x) t428 ON t1.x = t428.x JOIN (SELECT 1 AS x) t429 ON t1.x = t429.x JOIN (SELECT 1 AS x) t430 ON t1.x = t430.x JOIN (SELECT 1 AS x) t431 ON t1.x = t431.x JOIN (SELECT 1 AS x) t432 ON t1.x = t432.x JOIN (SELECT 1 AS x) t433 ON t1.x = t433.x JOIN (SELECT 1 AS x) t434 ON t1.x = t434.x JOIN (SELECT 1 AS x) t435 ON t1.x = t435.x JOIN (SELECT 1 AS x) t436 ON t1.x = t436.x JOIN (SELECT 1 AS x) t437 ON t1.x = t437.x JOIN (SELECT 1 AS x) t438 ON t1.x = t438.x JOIN (SELECT 1 AS x) t439 ON t1.x = t439.x JOIN (SELECT 1 AS x) t440 ON t1.x = t440.x JOIN (SELECT 1 AS x) t441 ON t1.x = t441.x JOIN (SELECT 1 AS x) t442 ON t1.x = t442.x JOIN (SELECT 1 AS x) t443 ON t1.x = t443.x JOIN (SELECT 1 AS x) t444 ON t1.x = t444.x JOIN (SELECT 1 AS x) t445 ON t1.x = t445.x JOIN (SELECT 1 AS x) t446 ON t1.x = t446.x JOIN (SELECT 1 AS x) t447 ON t1.x = t447.x JOIN (SELECT 1 AS x) t448 ON t1.x = t448.x JOIN (SELECT 1 AS x) t449 ON t1.x = t449.x JOIN (SELECT 1 AS x) t450 ON t1.x = t450.x JOIN (SELECT 1 AS x) t451 ON t1.x = t451.x JOIN (SELECT 1 AS x) t452 ON t1.x = t452.x JOIN (SELECT 1 AS x) t453 ON t1.x = t453.x JOIN (SELECT 1 AS x) t454 ON t1.x = t454.x JOIN (SELECT 1 AS x) t455 ON t1.x = t455.x JOIN (SELECT 1 AS x) t456 ON t1.x = t456.x JOIN (SELECT 1 AS x) t457 ON t1.x = t457.x JOIN (SELECT 1 AS x) t458 ON t1.x = t458.x JOIN (SELECT 1 AS x) t459 ON t1.x = t459.x JOIN (SELECT 1 AS x) t460 ON t1.x = t460.x JOIN (SELECT 1 AS x) t461 ON t1.x = t461.x JOIN (SELECT 1 AS x) t462 ON t1.x = t462.x JOIN (SELECT 1 AS x) t463 ON t1.x = t463.x JOIN (SELECT 1 AS x) t464 ON t1.x = t464.x JOIN (SELECT 1 AS x) t465 ON t1.x = t465.x JOIN (SELECT 1 AS x) t466 ON t1.x = t466.x JOIN (SELECT 1 AS x) t467 ON t1.x = t467.x JOIN (SELECT 1 AS x) t468 ON t1.x = t468.x JOIN (SELECT 1 AS x) t469 ON t1.x = t469.x JOIN (SELECT 1 AS x) t470 ON t1.x = t470.x JOIN (SELECT 1 AS x) t471 ON t1.x = t471.x JOIN (SELECT 1 AS x) t472 ON t1.x = t472.x JOIN (SELECT 1 AS x) t473 ON t1.x = t473.x JOIN (SELECT 1 AS x) t474 ON t1.x = t474.x JOIN (SELECT 1 AS x) t475 ON t1.x = t475.x JOIN (SELECT 1 AS x) t476 ON t1.x = t476.x JOIN (SELECT 1 AS x) t477 ON t1.x = t477.x JOIN (SELECT 1 AS x) t478 ON t1.x = t478.x JOIN (SELECT 1 AS x) t479 ON t1.x = t479.x JOIN (SELECT 1 AS x) t480 ON t1.x = t480.x JOIN (SELECT 1 AS x) t481 ON t1.x = t481.x JOIN (SELECT 1 AS x) t482 ON t1.x = t482.x JOIN (SELECT 1 AS x) t483 ON t1.x = t483.x JOIN (SELECT 1 AS x) t484 ON t1.x = t484.x JOIN (SELECT 1 AS x) t485 ON t1.x = t485.x JOIN (SELECT 1 AS x) t486 ON t1.x = t486.x JOIN (SELECT 1 AS x) t487 ON t1.x = t487.x JOIN (SELECT 1 AS x) t488 ON t1.x = t488.x JOIN (SELECT 1 AS x) t489 ON t1.x = t489.x JOIN (SELECT 1 AS x) t490 ON t1.x = t490.x JOIN (SELECT 1 AS x) t491 ON t1.x = t491.x JOIN (SELECT 1 AS x) t492 ON t1.x = t492.x JOIN (SELECT 1 AS x) t493 ON t1.x = t493.x JOIN (SELECT 1 AS x) t494 ON t1.x = t494.x JOIN (SELECT 1 AS x) t495 ON t1.x = t495.x JOIN (SELECT 1 AS x) t496 ON t1.x = t496.x JOIN (SELECT 1 AS x) t497 ON t1.x = t497.x JOIN (SELECT 1 AS x) t498 ON t1.x = t498.x JOIN (SELECT 1 AS x) t499 ON t1.x = t499.x JOIN (SELECT 1 AS x) t500 ON t1.x = t500.x JOIN (SELECT 1 AS x) t501 ON t1.x = t501.x JOIN (SELECT 1 AS x) t502 ON t1.x = t502.x JOIN (SELECT 1 AS x) t503 ON t1.x = t503.x JOIN (SELECT 1 AS x) t504 ON t1.x = t504.x JOIN (SELECT 1 AS x) t505 ON t1.x = t505.x JOIN (SELECT 1 AS x) t506 ON t1.x = t506.x JOIN (SELECT 1 AS x) t507 ON t1.x = t507.x JOIN (SELECT 1 AS x) t508 ON t1.x = t508.x JOIN (SELECT 1 AS x) t509 ON t1.x = t509.x JOIN (SELECT 1 AS x) t510 ON t1.x = t510.x JOIN (SELECT 1 AS x) t511 ON t1.x = t511.x JOIN (SELECT 1 AS x) t512 ON t1.x = t512.x JOIN (SELECT 1 AS x) t513 ON t1.x = t513.x JOIN (SELECT 1 AS x) t514 ON t1.x = t514.x JOIN (SELECT 1 AS x) t515 ON t1.x = t515.x JOIN (SELECT 1 AS x) t516 ON t1.x = t516.x JOIN (SELECT 1 AS x) t517 ON t1.x = t517.x JOIN (SELECT 1 AS x) t518 ON t1.x = t518.x JOIN (SELECT 1 AS x) t519 ON t1.x = t519.x JOIN (SELECT 1 AS x) t520 ON t1.x = t520.x JOIN (SELECT 1 AS x) t521 ON t1.x = t521.x JOIN (SELECT 1 AS x) t522 ON t1.x = t522.x JOIN (SELECT 1 AS x) t523 ON t1.x = t523.x JOIN (SELECT 1 AS x) t524 ON t1.x = t524.x JOIN (SELECT 1 AS x) t525 ON t1.x = t525.x JOIN (SELECT 1 AS x) t526 ON t1.x = t526.x JOIN (SELECT 1 AS x) t527 ON t1.x = t527.x JOIN (SELECT 1 AS x) t528 ON t1.x = t528.x JOIN (SELECT 1 AS x) t529 ON t1.x = t529.x JOIN (SELECT 1 AS x) t530 ON t1.x = t530.x JOIN (SELECT 1 AS x) t531 ON t1.x = t531.x JOIN (SELECT 1 AS x) t532 ON t1.x = t532.x JOIN (SELECT 1 AS x) t533 ON t1.x = t533.x JOIN (SELECT 1 AS x) t534 ON t1.x = t534.x JOIN (SELECT 1 AS x) t535 ON t1.x = t535.x JOIN (SELECT 1 AS x) t536 ON t1.x = t536.x JOIN (SELECT 1 AS x) t537 ON t1.x = t537.x JOIN (SELECT 1 AS x) t538 ON t1.x = t538.x JOIN (SELECT 1 AS x) t539 ON t1.x = t539.x JOIN (SELECT 1 AS x) t540 ON t1.x = t540.x JOIN (SELECT 1 AS x) t541 ON t1.x = t541.x JOIN (SELECT 1 AS x) t542 ON t1.x = t542.x JOIN (SELECT 1 AS x) t543 ON t1.x = t543.x JOIN (SELECT 1 AS x) t544 ON t1.x = t544.x JOIN (SELECT 1 AS x) t545 ON t1.x = t545.x JOIN (SELECT 1 AS x) t546 ON t1.x = t546.x JOIN (SELECT 1 AS x) t547 ON t1.x = t547.x JOIN (SELECT 1 AS x) t548 ON t1.x = t548.x JOIN (SELECT 1 AS x) t549 ON t1.x = t549.x JOIN (SELECT 1 AS x) t550 ON t1.x = t550.x JOIN (SELECT 1 AS x) t551 ON t1.x = t551.x JOIN (SELECT 1 AS x) t552 ON t1.x = t552.x JOIN (SELECT 1 AS x) t553 ON t1.x = t553.x JOIN (SELECT 1 AS x) t554 ON t1.x = t554.x JOIN (SELECT 1 AS x) t555 ON t1.x = t555.x JOIN (SELECT 1 AS x) t556 ON t1.x = t556.x JOIN (SELECT 1 AS x) t557 ON t1.x = t557.x JOIN (SELECT 1 AS x) t558 ON t1.x = t558.x JOIN (SELECT 1 AS x) t559 ON t1.x = t559.x JOIN (SELECT 1 AS x) t560 ON t1.x = t560.x JOIN (SELECT 1 AS x) t561 ON t1.x = t561.x JOIN (SELECT 1 AS x) t562 ON t1.x = t562.x JOIN (SELECT 1 AS x) t563 ON t1.x = t563.x JOIN (SELECT 1 AS x) t564 ON t1.x = t564.x JOIN (SELECT 1 AS x) t565 ON t1.x = t565.x JOIN (SELECT 1 AS x) t566 ON t1.x = t566.x JOIN (SELECT 1 AS x) t567 ON t1.x = t567.x JOIN (SELECT 1 AS x) t568 ON t1.x = t568.x JOIN (SELECT 1 AS x) t569 ON t1.x = t569.x JOIN (SELECT 1 AS x) t570 ON t1.x = t570.x JOIN (SELECT 1 AS x) t571 ON t1.x = t571.x JOIN (SELECT 1 AS x) t572 ON t1.x = t572.x JOIN (SELECT 1 AS x) t573 ON t1.x = t573.x JOIN (SELECT 1 AS x) t574 ON t1.x = t574.x JOIN (SELECT 1 AS x) t575 ON t1.x = t575.x JOIN (SELECT 1 AS x) t576 ON t1.x = t576.x JOIN (SELECT 1 AS x) t577 ON t1.x = t577.x JOIN (SELECT 1 AS x) t578 ON t1.x = t578.x JOIN (SELECT 1 AS x) t579 ON t1.x = t579.x JOIN (SELECT 1 AS x) t580 ON t1.x = t580.x JOIN (SELECT 1 AS x) t581 ON t1.x = t581.x JOIN (SELECT 1 AS x) t582 ON t1.x = t582.x JOIN (SELECT 1 AS x) t583 ON t1.x = t583.x JOIN (SELECT 1 AS x) t584 ON t1.x = t584.x JOIN (SELECT 1 AS x) t585 ON t1.x = t585.x JOIN (SELECT 1 AS x) t586 ON t1.x = t586.x JOIN (SELECT 1 AS x) t587 ON t1.x = t587.x JOIN (SELECT 1 AS x) t588 ON t1.x = t588.x JOIN (SELECT 1 AS x) t589 ON t1.x = t589.x JOIN (SELECT 1 AS x) t590 ON t1.x = t590.x JOIN (SELECT 1 AS x) t591 ON t1.x = t591.x JOIN (SELECT 1 AS x) t592 ON t1.x = t592.x JOIN (SELECT 1 AS x) t593 ON t1.x = t593.x JOIN (SELECT 1 AS x) t594 ON t1.x = t594.x JOIN (SELECT 1 AS x) t595 ON t1.x = t595.x JOIN (SELECT 1 AS x) t596 ON t1.x = t596.x JOIN (SELECT 1 AS x) t597 ON t1.x = t597.x JOIN (SELECT 1 AS x) t598 ON t1.x = t598.x JOIN (SELECT 1 AS x) t599 ON t1.x = t599.x JOIN (SELECT 1 AS x) t600 ON t1.x = t600.x JOIN (SELECT 1 AS x) t601 ON t1.x = t601.x JOIN (SELECT 1 AS x) t602 ON t1.x = t602.x JOIN (SELECT 1 AS x) t603 ON t1.x = t603.x JOIN (SELECT 1 AS x) t604 ON t1.x = t604.x JOIN (SELECT 1 AS x) t605 ON t1.x = t605.x JOIN (SELECT 1 AS x) t606 ON t1.x = t606.x JOIN (SELECT 1 AS x) t607 ON t1.x = t607.x JOIN (SELECT 1 AS x) t608 ON t1.x = t608.x JOIN (SELECT 1 AS x) t609 ON t1.x = t609.x JOIN (SELECT 1 AS x) t610 ON t1.x = t610.x JOIN (SELECT 1 AS x) t611 ON t1.x = t611.x JOIN (SELECT 1 AS x) t612 ON t1.x = t612.x JOIN (SELECT 1 AS x) t613 ON t1.x = t613.x JOIN (SELECT 1 AS x) t614 ON t1.x = t614.x JOIN (SELECT 1 AS x) t615 ON t1.x = t615.x JOIN (SELECT 1 AS x) t616 ON t1.x = t616.x JOIN (SELECT 1 AS x) t617 ON t1.x = t617.x JOIN (SELECT 1 AS x) t618 ON t1.x = t618.x JOIN (SELECT 1 AS x) t619 ON t1.x = t619.x JOIN (SELECT 1 AS x) t620 ON t1.x = t620.x JOIN (SELECT 1 AS x) t621 ON t1.x = t621.x JOIN (SELECT 1 AS x) t622 ON t1.x = t622.x JOIN (SELECT 1 AS x) t623 ON t1.x = t623.x JOIN (SELECT 1 AS x) t624 ON t1.x = t624.x JOIN (SELECT 1 AS x) t625 ON t1.x = t625.x JOIN (SELECT 1 AS x) t626 ON t1.x = t626.x JOIN (SELECT 1 AS x) t627 ON t1.x = t627.x JOIN (SELECT 1 AS x) t628 ON t1.x = t628.x JOIN (SELECT 1 AS x) t629 ON t1.x = t629.x JOIN (SELECT 1 AS x) t630 ON t1.x = t630.x JOIN (SELECT 1 AS x) t631 ON t1.x = t631.x JOIN (SELECT 1 AS x) t632 ON t1.x = t632.x JOIN (SELECT 1 AS x) t633 ON t1.x = t633.x JOIN (SELECT 1 AS x) t634 ON t1.x = t634.x JOIN (SELECT 1 AS x) t635 ON t1.x = t635.x JOIN (SELECT 1 AS x) t636 ON t1.x = t636.x JOIN (SELECT 1 AS x) t637 ON t1.x = t637.x JOIN (SELECT 1 AS x) t638 ON t1.x = t638.x JOIN (SELECT 1 AS x) t639 ON t1.x = t639.x JOIN (SELECT 1 AS x) t640 ON t1.x = t640.x JOIN (SELECT 1 AS x) t641 ON t1.x = t641.x JOIN (SELECT 1 AS x) t642 ON t1.x = t642.x JOIN (SELECT 1 AS x) t643 ON t1.x = t643.x JOIN (SELECT 1 AS x) t644 ON t1.x = t644.x JOIN (SELECT 1 AS x) t645 ON t1.x = t645.x JOIN (SELECT 1 AS x) t646 ON t1.x = t646.x JOIN (SELECT 1 AS x) t647 ON t1.x = t647.x JOIN (SELECT 1 AS x) t648 ON t1.x = t648.x JOIN (SELECT 1 AS x) t649 ON t1.x = t649.x JOIN (SELECT 1 AS x) t650 ON t1.x = t650.x JOIN (SELECT 1 AS x) t651 ON t1.x = t651.x JOIN (SELECT 1 AS x) t652 ON t1.x = t652.x JOIN (SELECT 1 AS x) t653 ON t1.x = t653.x JOIN (SELECT 1 AS x) t654 ON t1.x = t654.x JOIN (SELECT 1 AS x) t655 ON t1.x = t655.x JOIN (SELECT 1 AS x) t656 ON t1.x = t656.x JOIN (SELECT 1 AS x) t657 ON t1.x = t657.x JOIN (SELECT 1 AS x) t658 ON t1.x = t658.x JOIN (SELECT 1 AS x) t659 ON t1.x = t659.x JOIN (SELECT 1 AS x) t660 ON t1.x = t660.x JOIN (SELECT 1 AS x) t661 ON t1.x = t661.x JOIN (SELECT 1 AS x) t662 ON t1.x = t662.x JOIN (SELECT 1 AS x) t663 ON t1.x = t663.x JOIN (SELECT 1 AS x) t664 ON t1.x = t664.x JOIN (SELECT 1 AS x) t665 ON t1.x = t665.x JOIN (SELECT 1 AS x) t666 ON t1.x = t666.x + + diff --git a/tests/queries/0_stateless/03094_one_thousand_joins.reference b/tests/queries/0_stateless/03094_one_thousand_joins.reference new file mode 100644 index 00000000000..52176202f05 --- /dev/null +++ b/tests/queries/0_stateless/03094_one_thousand_joins.reference @@ -0,0 +1 @@ +1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 diff --git a/tests/queries/0_stateless/03094_one_thousand_joins.sql b/tests/queries/0_stateless/03094_one_thousand_joins.sql new file mode 100644 index 00000000000..f7ca2134d2e --- /dev/null +++ b/tests/queries/0_stateless/03094_one_thousand_joins.sql @@ -0,0 +1,4 @@ +-- Tags: no-fasttest, long + +-- Bug 33446, marked as 'long' because it still runs around 10 sec +SELECT * FROM (SELECT 1 AS x) t1 JOIN (SELECT 1 AS x) t2 ON t1.x = t2.x JOIN (SELECT 1 AS x) t3 ON t1.x = t3.x JOIN (SELECT 1 AS x) t4 ON t1.x = t4.x JOIN (SELECT 1 AS x) t5 ON t1.x = t5.x JOIN (SELECT 1 AS x) t6 ON t1.x = t6.x JOIN (SELECT 1 AS x) t7 ON t1.x = t7.x JOIN (SELECT 1 AS x) t8 ON t1.x = t8.x JOIN (SELECT 1 AS x) t9 ON t1.x = t9.x JOIN (SELECT 1 AS x) t10 ON t1.x = t10.x JOIN (SELECT 1 AS x) t11 ON t1.x = t11.x JOIN (SELECT 1 AS x) t12 ON t1.x = t12.x JOIN (SELECT 1 AS x) t13 ON t1.x = t13.x JOIN (SELECT 1 AS x) t14 ON t1.x = t14.x JOIN (SELECT 1 AS x) t15 ON t1.x = t15.x JOIN (SELECT 1 AS x) t16 ON t1.x = t16.x JOIN (SELECT 1 AS x) t17 ON t1.x = t17.x JOIN (SELECT 1 AS x) t18 ON t1.x = t18.x JOIN (SELECT 1 AS x) t19 ON t1.x = t19.x JOIN (SELECT 1 AS x) t20 ON t1.x = t20.x JOIN (SELECT 1 AS x) t21 ON t1.x = t21.x JOIN (SELECT 1 AS x) t22 ON t1.x = t22.x JOIN (SELECT 1 AS x) t23 ON t1.x = t23.x JOIN (SELECT 1 AS x) t24 ON t1.x = t24.x JOIN (SELECT 1 AS x) t25 ON t1.x = t25.x JOIN (SELECT 1 AS x) t26 ON t1.x = t26.x JOIN (SELECT 1 AS x) t27 ON t1.x = t27.x JOIN (SELECT 1 AS x) t28 ON t1.x = t28.x JOIN (SELECT 1 AS x) t29 ON t1.x = t29.x JOIN (SELECT 1 AS x) t30 ON t1.x = t30.x JOIN (SELECT 1 AS x) t31 ON t1.x = t31.x JOIN (SELECT 1 AS x) t32 ON t1.x = t32.x JOIN (SELECT 1 AS x) t33 ON t1.x = t33.x JOIN (SELECT 1 AS x) t34 ON t1.x = t34.x JOIN (SELECT 1 AS x) t35 ON t1.x = t35.x JOIN (SELECT 1 AS x) t36 ON t1.x = t36.x JOIN (SELECT 1 AS x) t37 ON t1.x = t37.x JOIN (SELECT 1 AS x) t38 ON t1.x = t38.x JOIN (SELECT 1 AS x) t39 ON t1.x = t39.x JOIN (SELECT 1 AS x) t40 ON t1.x = t40.x JOIN (SELECT 1 AS x) t41 ON t1.x = t41.x JOIN (SELECT 1 AS x) t42 ON t1.x = t42.x JOIN (SELECT 1 AS x) t43 ON t1.x = t43.x JOIN (SELECT 1 AS x) t44 ON t1.x = t44.x JOIN (SELECT 1 AS x) t45 ON t1.x = t45.x JOIN (SELECT 1 AS x) t46 ON t1.x = t46.x JOIN (SELECT 1 AS x) t47 ON t1.x = t47.x JOIN (SELECT 1 AS x) t48 ON t1.x = t48.x JOIN (SELECT 1 AS x) t49 ON t1.x = t49.x JOIN (SELECT 1 AS x) t50 ON t1.x = t50.x JOIN (SELECT 1 AS x) t51 ON t1.x = t51.x JOIN (SELECT 1 AS x) t52 ON t1.x = t52.x JOIN (SELECT 1 AS x) t53 ON t1.x = t53.x JOIN (SELECT 1 AS x) t54 ON t1.x = t54.x JOIN (SELECT 1 AS x) t55 ON t1.x = t55.x JOIN (SELECT 1 AS x) t56 ON t1.x = t56.x JOIN (SELECT 1 AS x) t57 ON t1.x = t57.x JOIN (SELECT 1 AS x) t58 ON t1.x = t58.x JOIN (SELECT 1 AS x) t59 ON t1.x = t59.x JOIN (SELECT 1 AS x) t60 ON t1.x = t60.x JOIN (SELECT 1 AS x) t61 ON t1.x = t61.x JOIN (SELECT 1 AS x) t62 ON t1.x = t62.x JOIN (SELECT 1 AS x) t63 ON t1.x = t63.x JOIN (SELECT 1 AS x) t64 ON t1.x = t64.x JOIN (SELECT 1 AS x) t65 ON t1.x = t65.x JOIN (SELECT 1 AS x) t66 ON t1.x = t66.x JOIN (SELECT 1 AS x) t67 ON t1.x = t67.x JOIN (SELECT 1 AS x) t68 ON t1.x = t68.x JOIN (SELECT 1 AS x) t69 ON t1.x = t69.x JOIN (SELECT 1 AS x) t70 ON t1.x = t70.x JOIN (SELECT 1 AS x) t71 ON t1.x = t71.x JOIN (SELECT 1 AS x) t72 ON t1.x = t72.x JOIN (SELECT 1 AS x) t73 ON t1.x = t73.x JOIN (SELECT 1 AS x) t74 ON t1.x = t74.x JOIN (SELECT 1 AS x) t75 ON t1.x = t75.x JOIN (SELECT 1 AS x) t76 ON t1.x = t76.x JOIN (SELECT 1 AS x) t77 ON t1.x = t77.x JOIN (SELECT 1 AS x) t78 ON t1.x = t78.x JOIN (SELECT 1 AS x) t79 ON t1.x = t79.x JOIN (SELECT 1 AS x) t80 ON t1.x = t80.x JOIN (SELECT 1 AS x) t81 ON t1.x = t81.x JOIN (SELECT 1 AS x) t82 ON t1.x = t82.x JOIN (SELECT 1 AS x) t83 ON t1.x = t83.x JOIN (SELECT 1 AS x) t84 ON t1.x = t84.x JOIN (SELECT 1 AS x) t85 ON t1.x = t85.x JOIN (SELECT 1 AS x) t86 ON t1.x = t86.x JOIN (SELECT 1 AS x) t87 ON t1.x = t87.x JOIN (SELECT 1 AS x) t88 ON t1.x = t88.x JOIN (SELECT 1 AS x) t89 ON t1.x = t89.x JOIN (SELECT 1 AS x) t90 ON t1.x = t90.x JOIN (SELECT 1 AS x) t91 ON t1.x = t91.x JOIN (SELECT 1 AS x) t92 ON t1.x = t92.x JOIN (SELECT 1 AS x) t93 ON t1.x = t93.x JOIN (SELECT 1 AS x) t94 ON t1.x = t94.x JOIN (SELECT 1 AS x) t95 ON t1.x = t95.x JOIN (SELECT 1 AS x) t96 ON t1.x = t96.x JOIN (SELECT 1 AS x) t97 ON t1.x = t97.x JOIN (SELECT 1 AS x) t98 ON t1.x = t98.x JOIN (SELECT 1 AS x) t99 ON t1.x = t99.x JOIN (SELECT 1 AS x) t100 ON t1.x = t100.x JOIN (SELECT 1 AS x) t101 ON t1.x = t101.x JOIN (SELECT 1 AS x) t102 ON t1.x = t102.x JOIN (SELECT 1 AS x) t103 ON t1.x = t103.x JOIN (SELECT 1 AS x) t104 ON t1.x = t104.x JOIN (SELECT 1 AS x) t105 ON t1.x = t105.x JOIN (SELECT 1 AS x) t106 ON t1.x = t106.x JOIN (SELECT 1 AS x) t107 ON t1.x = t107.x JOIN (SELECT 1 AS x) t108 ON t1.x = t108.x JOIN (SELECT 1 AS x) t109 ON t1.x = t109.x JOIN (SELECT 1 AS x) t110 ON t1.x = t110.x JOIN (SELECT 1 AS x) t111 ON t1.x = t111.x JOIN (SELECT 1 AS x) t112 ON t1.x = t112.x JOIN (SELECT 1 AS x) t113 ON t1.x = t113.x JOIN (SELECT 1 AS x) t114 ON t1.x = t114.x JOIN (SELECT 1 AS x) t115 ON t1.x = t115.x JOIN (SELECT 1 AS x) t116 ON t1.x = t116.x JOIN (SELECT 1 AS x) t117 ON t1.x = t117.x JOIN (SELECT 1 AS x) t118 ON t1.x = t118.x JOIN (SELECT 1 AS x) t119 ON t1.x = t119.x JOIN (SELECT 1 AS x) t120 ON t1.x = t120.x JOIN (SELECT 1 AS x) t121 ON t1.x = t121.x JOIN (SELECT 1 AS x) t122 ON t1.x = t122.x JOIN (SELECT 1 AS x) t123 ON t1.x = t123.x JOIN (SELECT 1 AS x) t124 ON t1.x = t124.x JOIN (SELECT 1 AS x) t125 ON t1.x = t125.x JOIN (SELECT 1 AS x) t126 ON t1.x = t126.x JOIN (SELECT 1 AS x) t127 ON t1.x = t127.x JOIN (SELECT 1 AS x) t128 ON t1.x = t128.x JOIN (SELECT 1 AS x) t129 ON t1.x = t129.x JOIN (SELECT 1 AS x) t130 ON t1.x = t130.x JOIN (SELECT 1 AS x) t131 ON t1.x = t131.x JOIN (SELECT 1 AS x) t132 ON t1.x = t132.x JOIN (SELECT 1 AS x) t133 ON t1.x = t133.x JOIN (SELECT 1 AS x) t134 ON t1.x = t134.x JOIN (SELECT 1 AS x) t135 ON t1.x = t135.x JOIN (SELECT 1 AS x) t136 ON t1.x = t136.x JOIN (SELECT 1 AS x) t137 ON t1.x = t137.x JOIN (SELECT 1 AS x) t138 ON t1.x = t138.x JOIN (SELECT 1 AS x) t139 ON t1.x = t139.x JOIN (SELECT 1 AS x) t140 ON t1.x = t140.x JOIN (SELECT 1 AS x) t141 ON t1.x = t141.x JOIN (SELECT 1 AS x) t142 ON t1.x = t142.x JOIN (SELECT 1 AS x) t143 ON t1.x = t143.x JOIN (SELECT 1 AS x) t144 ON t1.x = t144.x JOIN (SELECT 1 AS x) t145 ON t1.x = t145.x JOIN (SELECT 1 AS x) t146 ON t1.x = t146.x JOIN (SELECT 1 AS x) t147 ON t1.x = t147.x JOIN (SELECT 1 AS x) t148 ON t1.x = t148.x JOIN (SELECT 1 AS x) t149 ON t1.x = t149.x JOIN (SELECT 1 AS x) t150 ON t1.x = t150.x JOIN (SELECT 1 AS x) t151 ON t1.x = t151.x JOIN (SELECT 1 AS x) t152 ON t1.x = t152.x JOIN (SELECT 1 AS x) t153 ON t1.x = t153.x JOIN (SELECT 1 AS x) t154 ON t1.x = t154.x JOIN (SELECT 1 AS x) t155 ON t1.x = t155.x JOIN (SELECT 1 AS x) t156 ON t1.x = t156.x JOIN (SELECT 1 AS x) t157 ON t1.x = t157.x JOIN (SELECT 1 AS x) t158 ON t1.x = t158.x JOIN (SELECT 1 AS x) t159 ON t1.x = t159.x JOIN (SELECT 1 AS x) t160 ON t1.x = t160.x JOIN (SELECT 1 AS x) t161 ON t1.x = t161.x JOIN (SELECT 1 AS x) t162 ON t1.x = t162.x JOIN (SELECT 1 AS x) t163 ON t1.x = t163.x JOIN (SELECT 1 AS x) t164 ON t1.x = t164.x JOIN (SELECT 1 AS x) t165 ON t1.x = t165.x JOIN (SELECT 1 AS x) t166 ON t1.x = t166.x JOIN (SELECT 1 AS x) t167 ON t1.x = t167.x JOIN (SELECT 1 AS x) t168 ON t1.x = t168.x JOIN (SELECT 1 AS x) t169 ON t1.x = t169.x JOIN (SELECT 1 AS x) t170 ON t1.x = t170.x JOIN (SELECT 1 AS x) t171 ON t1.x = t171.x JOIN (SELECT 1 AS x) t172 ON t1.x = t172.x JOIN (SELECT 1 AS x) t173 ON t1.x = t173.x JOIN (SELECT 1 AS x) t174 ON t1.x = t174.x JOIN (SELECT 1 AS x) t175 ON t1.x = t175.x JOIN (SELECT 1 AS x) t176 ON t1.x = t176.x JOIN (SELECT 1 AS x) t177 ON t1.x = t177.x JOIN (SELECT 1 AS x) t178 ON t1.x = t178.x JOIN (SELECT 1 AS x) t179 ON t1.x = t179.x JOIN (SELECT 1 AS x) t180 ON t1.x = t180.x JOIN (SELECT 1 AS x) t181 ON t1.x = t181.x JOIN (SELECT 1 AS x) t182 ON t1.x = t182.x JOIN (SELECT 1 AS x) t183 ON t1.x = t183.x JOIN (SELECT 1 AS x) t184 ON t1.x = t184.x JOIN (SELECT 1 AS x) t185 ON t1.x = t185.x JOIN (SELECT 1 AS x) t186 ON t1.x = t186.x JOIN (SELECT 1 AS x) t187 ON t1.x = t187.x JOIN (SELECT 1 AS x) t188 ON t1.x = t188.x JOIN (SELECT 1 AS x) t189 ON t1.x = t189.x JOIN (SELECT 1 AS x) t190 ON t1.x = t190.x JOIN (SELECT 1 AS x) t191 ON t1.x = t191.x JOIN (SELECT 1 AS x) t192 ON t1.x = t192.x JOIN (SELECT 1 AS x) t193 ON t1.x = t193.x JOIN (SELECT 1 AS x) t194 ON t1.x = t194.x JOIN (SELECT 1 AS x) t195 ON t1.x = t195.x JOIN (SELECT 1 AS x) t196 ON t1.x = t196.x JOIN (SELECT 1 AS x) t197 ON t1.x = t197.x JOIN (SELECT 1 AS x) t198 ON t1.x = t198.x JOIN (SELECT 1 AS x) t199 ON t1.x = t199.x JOIN (SELECT 1 AS x) t200 ON t1.x = t200.x JOIN (SELECT 1 AS x) t201 ON t1.x = t201.x JOIN (SELECT 1 AS x) t202 ON t1.x = t202.x JOIN (SELECT 1 AS x) t203 ON t1.x = t203.x JOIN (SELECT 1 AS x) t204 ON t1.x = t204.x JOIN (SELECT 1 AS x) t205 ON t1.x = t205.x JOIN (SELECT 1 AS x) t206 ON t1.x = t206.x JOIN (SELECT 1 AS x) t207 ON t1.x = t207.x JOIN (SELECT 1 AS x) t208 ON t1.x = t208.x JOIN (SELECT 1 AS x) t209 ON t1.x = t209.x JOIN (SELECT 1 AS x) t210 ON t1.x = t210.x JOIN (SELECT 1 AS x) t211 ON t1.x = t211.x JOIN (SELECT 1 AS x) t212 ON t1.x = t212.x JOIN (SELECT 1 AS x) t213 ON t1.x = t213.x JOIN (SELECT 1 AS x) t214 ON t1.x = t214.x JOIN (SELECT 1 AS x) t215 ON t1.x = t215.x JOIN (SELECT 1 AS x) t216 ON t1.x = t216.x JOIN (SELECT 1 AS x) t217 ON t1.x = t217.x JOIN (SELECT 1 AS x) t218 ON t1.x = t218.x JOIN (SELECT 1 AS x) t219 ON t1.x = t219.x JOIN (SELECT 1 AS x) t220 ON t1.x = t220.x JOIN (SELECT 1 AS x) t221 ON t1.x = t221.x JOIN (SELECT 1 AS x) t222 ON t1.x = t222.x JOIN (SELECT 1 AS x) t223 ON t1.x = t223.x JOIN (SELECT 1 AS x) t224 ON t1.x = t224.x JOIN (SELECT 1 AS x) t225 ON t1.x = t225.x JOIN (SELECT 1 AS x) t226 ON t1.x = t226.x JOIN (SELECT 1 AS x) t227 ON t1.x = t227.x JOIN (SELECT 1 AS x) t228 ON t1.x = t228.x JOIN (SELECT 1 AS x) t229 ON t1.x = t229.x JOIN (SELECT 1 AS x) t230 ON t1.x = t230.x JOIN (SELECT 1 AS x) t231 ON t1.x = t231.x JOIN (SELECT 1 AS x) t232 ON t1.x = t232.x JOIN (SELECT 1 AS x) t233 ON t1.x = t233.x JOIN (SELECT 1 AS x) t234 ON t1.x = t234.x JOIN (SELECT 1 AS x) t235 ON t1.x = t235.x JOIN (SELECT 1 AS x) t236 ON t1.x = t236.x JOIN (SELECT 1 AS x) t237 ON t1.x = t237.x JOIN (SELECT 1 AS x) t238 ON t1.x = t238.x JOIN (SELECT 1 AS x) t239 ON t1.x = t239.x JOIN (SELECT 1 AS x) t240 ON t1.x = t240.x JOIN (SELECT 1 AS x) t241 ON t1.x = t241.x JOIN (SELECT 1 AS x) t242 ON t1.x = t242.x JOIN (SELECT 1 AS x) t243 ON t1.x = t243.x JOIN (SELECT 1 AS x) t244 ON t1.x = t244.x JOIN (SELECT 1 AS x) t245 ON t1.x = t245.x JOIN (SELECT 1 AS x) t246 ON t1.x = t246.x JOIN (SELECT 1 AS x) t247 ON t1.x = t247.x JOIN (SELECT 1 AS x) t248 ON t1.x = t248.x JOIN (SELECT 1 AS x) t249 ON t1.x = t249.x JOIN (SELECT 1 AS x) t250 ON t1.x = t250.x JOIN (SELECT 1 AS x) t251 ON t1.x = t251.x JOIN (SELECT 1 AS x) t252 ON t1.x = t252.x JOIN (SELECT 1 AS x) t253 ON t1.x = t253.x JOIN (SELECT 1 AS x) t254 ON t1.x = t254.x JOIN (SELECT 1 AS x) t255 ON t1.x = t255.x JOIN (SELECT 1 AS x) t256 ON t1.x = t256.x JOIN (SELECT 1 AS x) t257 ON t1.x = t257.x JOIN (SELECT 1 AS x) t258 ON t1.x = t258.x JOIN (SELECT 1 AS x) t259 ON t1.x = t259.x JOIN (SELECT 1 AS x) t260 ON t1.x = t260.x JOIN (SELECT 1 AS x) t261 ON t1.x = t261.x JOIN (SELECT 1 AS x) t262 ON t1.x = t262.x JOIN (SELECT 1 AS x) t263 ON t1.x = t263.x JOIN (SELECT 1 AS x) t264 ON t1.x = t264.x JOIN (SELECT 1 AS x) t265 ON t1.x = t265.x JOIN (SELECT 1 AS x) t266 ON t1.x = t266.x JOIN (SELECT 1 AS x) t267 ON t1.x = t267.x JOIN (SELECT 1 AS x) t268 ON t1.x = t268.x JOIN (SELECT 1 AS x) t269 ON t1.x = t269.x JOIN (SELECT 1 AS x) t270 ON t1.x = t270.x JOIN (SELECT 1 AS x) t271 ON t1.x = t271.x JOIN (SELECT 1 AS x) t272 ON t1.x = t272.x JOIN (SELECT 1 AS x) t273 ON t1.x = t273.x JOIN (SELECT 1 AS x) t274 ON t1.x = t274.x JOIN (SELECT 1 AS x) t275 ON t1.x = t275.x JOIN (SELECT 1 AS x) t276 ON t1.x = t276.x JOIN (SELECT 1 AS x) t277 ON t1.x = t277.x JOIN (SELECT 1 AS x) t278 ON t1.x = t278.x JOIN (SELECT 1 AS x) t279 ON t1.x = t279.x JOIN (SELECT 1 AS x) t280 ON t1.x = t280.x JOIN (SELECT 1 AS x) t281 ON t1.x = t281.x JOIN (SELECT 1 AS x) t282 ON t1.x = t282.x JOIN (SELECT 1 AS x) t283 ON t1.x = t283.x JOIN (SELECT 1 AS x) t284 ON t1.x = t284.x JOIN (SELECT 1 AS x) t285 ON t1.x = t285.x JOIN (SELECT 1 AS x) t286 ON t1.x = t286.x JOIN (SELECT 1 AS x) t287 ON t1.x = t287.x JOIN (SELECT 1 AS x) t288 ON t1.x = t288.x JOIN (SELECT 1 AS x) t289 ON t1.x = t289.x JOIN (SELECT 1 AS x) t290 ON t1.x = t290.x JOIN (SELECT 1 AS x) t291 ON t1.x = t291.x JOIN (SELECT 1 AS x) t292 ON t1.x = t292.x JOIN (SELECT 1 AS x) t293 ON t1.x = t293.x JOIN (SELECT 1 AS x) t294 ON t1.x = t294.x JOIN (SELECT 1 AS x) t295 ON t1.x = t295.x JOIN (SELECT 1 AS x) t296 ON t1.x = t296.x JOIN (SELECT 1 AS x) t297 ON t1.x = t297.x JOIN (SELECT 1 AS x) t298 ON t1.x = t298.x JOIN (SELECT 1 AS x) t299 ON t1.x = t299.x JOIN (SELECT 1 AS x) t300 ON t1.x = t300.x JOIN (SELECT 1 AS x) t301 ON t1.x = t301.x JOIN (SELECT 1 AS x) t302 ON t1.x = t302.x JOIN (SELECT 1 AS x) t303 ON t1.x = t303.x JOIN (SELECT 1 AS x) t304 ON t1.x = t304.x JOIN (SELECT 1 AS x) t305 ON t1.x = t305.x JOIN (SELECT 1 AS x) t306 ON t1.x = t306.x JOIN (SELECT 1 AS x) t307 ON t1.x = t307.x JOIN (SELECT 1 AS x) t308 ON t1.x = t308.x JOIN (SELECT 1 AS x) t309 ON t1.x = t309.x JOIN (SELECT 1 AS x) t310 ON t1.x = t310.x JOIN (SELECT 1 AS x) t311 ON t1.x = t311.x JOIN (SELECT 1 AS x) t312 ON t1.x = t312.x JOIN (SELECT 1 AS x) t313 ON t1.x = t313.x JOIN (SELECT 1 AS x) t314 ON t1.x = t314.x JOIN (SELECT 1 AS x) t315 ON t1.x = t315.x JOIN (SELECT 1 AS x) t316 ON t1.x = t316.x JOIN (SELECT 1 AS x) t317 ON t1.x = t317.x JOIN (SELECT 1 AS x) t318 ON t1.x = t318.x JOIN (SELECT 1 AS x) t319 ON t1.x = t319.x JOIN (SELECT 1 AS x) t320 ON t1.x = t320.x JOIN (SELECT 1 AS x) t321 ON t1.x = t321.x JOIN (SELECT 1 AS x) t322 ON t1.x = t322.x JOIN (SELECT 1 AS x) t323 ON t1.x = t323.x JOIN (SELECT 1 AS x) t324 ON t1.x = t324.x JOIN (SELECT 1 AS x) t325 ON t1.x = t325.x JOIN (SELECT 1 AS x) t326 ON t1.x = t326.x JOIN (SELECT 1 AS x) t327 ON t1.x = t327.x JOIN (SELECT 1 AS x) t328 ON t1.x = t328.x JOIN (SELECT 1 AS x) t329 ON t1.x = t329.x JOIN (SELECT 1 AS x) t330 ON t1.x = t330.x JOIN (SELECT 1 AS x) t331 ON t1.x = t331.x JOIN (SELECT 1 AS x) t332 ON t1.x = t332.x JOIN (SELECT 1 AS x) t333 ON t1.x = t333.x JOIN (SELECT 1 AS x) t334 ON t1.x = t334.x JOIN (SELECT 1 AS x) t335 ON t1.x = t335.x JOIN (SELECT 1 AS x) t336 ON t1.x = t336.x JOIN (SELECT 1 AS x) t337 ON t1.x = t337.x JOIN (SELECT 1 AS x) t338 ON t1.x = t338.x JOIN (SELECT 1 AS x) t339 ON t1.x = t339.x JOIN (SELECT 1 AS x) t340 ON t1.x = t340.x JOIN (SELECT 1 AS x) t341 ON t1.x = t341.x JOIN (SELECT 1 AS x) t342 ON t1.x = t342.x JOIN (SELECT 1 AS x) t343 ON t1.x = t343.x JOIN (SELECT 1 AS x) t344 ON t1.x = t344.x JOIN (SELECT 1 AS x) t345 ON t1.x = t345.x JOIN (SELECT 1 AS x) t346 ON t1.x = t346.x JOIN (SELECT 1 AS x) t347 ON t1.x = t347.x JOIN (SELECT 1 AS x) t348 ON t1.x = t348.x JOIN (SELECT 1 AS x) t349 ON t1.x = t349.x JOIN (SELECT 1 AS x) t350 ON t1.x = t350.x JOIN (SELECT 1 AS x) t351 ON t1.x = t351.x JOIN (SELECT 1 AS x) t352 ON t1.x = t352.x JOIN (SELECT 1 AS x) t353 ON t1.x = t353.x JOIN (SELECT 1 AS x) t354 ON t1.x = t354.x JOIN (SELECT 1 AS x) t355 ON t1.x = t355.x JOIN (SELECT 1 AS x) t356 ON t1.x = t356.x JOIN (SELECT 1 AS x) t357 ON t1.x = t357.x JOIN (SELECT 1 AS x) t358 ON t1.x = t358.x JOIN (SELECT 1 AS x) t359 ON t1.x = t359.x JOIN (SELECT 1 AS x) t360 ON t1.x = t360.x JOIN (SELECT 1 AS x) t361 ON t1.x = t361.x JOIN (SELECT 1 AS x) t362 ON t1.x = t362.x JOIN (SELECT 1 AS x) t363 ON t1.x = t363.x JOIN (SELECT 1 AS x) t364 ON t1.x = t364.x JOIN (SELECT 1 AS x) t365 ON t1.x = t365.x JOIN (SELECT 1 AS x) t366 ON t1.x = t366.x JOIN (SELECT 1 AS x) t367 ON t1.x = t367.x JOIN (SELECT 1 AS x) t368 ON t1.x = t368.x JOIN (SELECT 1 AS x) t369 ON t1.x = t369.x JOIN (SELECT 1 AS x) t370 ON t1.x = t370.x JOIN (SELECT 1 AS x) t371 ON t1.x = t371.x JOIN (SELECT 1 AS x) t372 ON t1.x = t372.x JOIN (SELECT 1 AS x) t373 ON t1.x = t373.x JOIN (SELECT 1 AS x) t374 ON t1.x = t374.x JOIN (SELECT 1 AS x) t375 ON t1.x = t375.x JOIN (SELECT 1 AS x) t376 ON t1.x = t376.x JOIN (SELECT 1 AS x) t377 ON t1.x = t377.x JOIN (SELECT 1 AS x) t378 ON t1.x = t378.x JOIN (SELECT 1 AS x) t379 ON t1.x = t379.x JOIN (SELECT 1 AS x) t380 ON t1.x = t380.x JOIN (SELECT 1 AS x) t381 ON t1.x = t381.x JOIN (SELECT 1 AS x) t382 ON t1.x = t382.x JOIN (SELECT 1 AS x) t383 ON t1.x = t383.x JOIN (SELECT 1 AS x) t384 ON t1.x = t384.x JOIN (SELECT 1 AS x) t385 ON t1.x = t385.x JOIN (SELECT 1 AS x) t386 ON t1.x = t386.x JOIN (SELECT 1 AS x) t387 ON t1.x = t387.x JOIN (SELECT 1 AS x) t388 ON t1.x = t388.x JOIN (SELECT 1 AS x) t389 ON t1.x = t389.x JOIN (SELECT 1 AS x) t390 ON t1.x = t390.x JOIN (SELECT 1 AS x) t391 ON t1.x = t391.x JOIN (SELECT 1 AS x) t392 ON t1.x = t392.x JOIN (SELECT 1 AS x) t393 ON t1.x = t393.x JOIN (SELECT 1 AS x) t394 ON t1.x = t394.x JOIN (SELECT 1 AS x) t395 ON t1.x = t395.x JOIN (SELECT 1 AS x) t396 ON t1.x = t396.x JOIN (SELECT 1 AS x) t397 ON t1.x = t397.x JOIN (SELECT 1 AS x) t398 ON t1.x = t398.x JOIN (SELECT 1 AS x) t399 ON t1.x = t399.x JOIN (SELECT 1 AS x) t400 ON t1.x = t400.x JOIN (SELECT 1 AS x) t401 ON t1.x = t401.x JOIN (SELECT 1 AS x) t402 ON t1.x = t402.x JOIN (SELECT 1 AS x) t403 ON t1.x = t403.x JOIN (SELECT 1 AS x) t404 ON t1.x = t404.x JOIN (SELECT 1 AS x) t405 ON t1.x = t405.x JOIN (SELECT 1 AS x) t406 ON t1.x = t406.x JOIN (SELECT 1 AS x) t407 ON t1.x = t407.x JOIN (SELECT 1 AS x) t408 ON t1.x = t408.x JOIN (SELECT 1 AS x) t409 ON t1.x = t409.x JOIN (SELECT 1 AS x) t410 ON t1.x = t410.x JOIN (SELECT 1 AS x) t411 ON t1.x = t411.x JOIN (SELECT 1 AS x) t412 ON t1.x = t412.x JOIN (SELECT 1 AS x) t413 ON t1.x = t413.x JOIN (SELECT 1 AS x) t414 ON t1.x = t414.x JOIN (SELECT 1 AS x) t415 ON t1.x = t415.x JOIN (SELECT 1 AS x) t416 ON t1.x = t416.x JOIN (SELECT 1 AS x) t417 ON t1.x = t417.x JOIN (SELECT 1 AS x) t418 ON t1.x = t418.x JOIN (SELECT 1 AS x) t419 ON t1.x = t419.x JOIN (SELECT 1 AS x) t420 ON t1.x = t420.x JOIN (SELECT 1 AS x) t421 ON t1.x = t421.x JOIN (SELECT 1 AS x) t422 ON t1.x = t422.x JOIN (SELECT 1 AS x) t423 ON t1.x = t423.x JOIN (SELECT 1 AS x) t424 ON t1.x = t424.x JOIN (SELECT 1 AS x) t425 ON t1.x = t425.x JOIN (SELECT 1 AS x) t426 ON t1.x = t426.x JOIN (SELECT 1 AS x) t427 ON t1.x = t427.x JOIN (SELECT 1 AS x) t428 ON t1.x = t428.x JOIN (SELECT 1 AS x) t429 ON t1.x = t429.x JOIN (SELECT 1 AS x) t430 ON t1.x = t430.x JOIN (SELECT 1 AS x) t431 ON t1.x = t431.x JOIN (SELECT 1 AS x) t432 ON t1.x = t432.x JOIN (SELECT 1 AS x) t433 ON t1.x = t433.x JOIN (SELECT 1 AS x) t434 ON t1.x = t434.x JOIN (SELECT 1 AS x) t435 ON t1.x = t435.x JOIN (SELECT 1 AS x) t436 ON t1.x = t436.x JOIN (SELECT 1 AS x) t437 ON t1.x = t437.x JOIN (SELECT 1 AS x) t438 ON t1.x = t438.x JOIN (SELECT 1 AS x) t439 ON t1.x = t439.x JOIN (SELECT 1 AS x) t440 ON t1.x = t440.x JOIN (SELECT 1 AS x) t441 ON t1.x = t441.x JOIN (SELECT 1 AS x) t442 ON t1.x = t442.x JOIN (SELECT 1 AS x) t443 ON t1.x = t443.x JOIN (SELECT 1 AS x) t444 ON t1.x = t444.x JOIN (SELECT 1 AS x) t445 ON t1.x = t445.x JOIN (SELECT 1 AS x) t446 ON t1.x = t446.x JOIN (SELECT 1 AS x) t447 ON t1.x = t447.x JOIN (SELECT 1 AS x) t448 ON t1.x = t448.x JOIN (SELECT 1 AS x) t449 ON t1.x = t449.x JOIN (SELECT 1 AS x) t450 ON t1.x = t450.x JOIN (SELECT 1 AS x) t451 ON t1.x = t451.x JOIN (SELECT 1 AS x) t452 ON t1.x = t452.x JOIN (SELECT 1 AS x) t453 ON t1.x = t453.x JOIN (SELECT 1 AS x) t454 ON t1.x = t454.x JOIN (SELECT 1 AS x) t455 ON t1.x = t455.x JOIN (SELECT 1 AS x) t456 ON t1.x = t456.x JOIN (SELECT 1 AS x) t457 ON t1.x = t457.x JOIN (SELECT 1 AS x) t458 ON t1.x = t458.x JOIN (SELECT 1 AS x) t459 ON t1.x = t459.x JOIN (SELECT 1 AS x) t460 ON t1.x = t460.x JOIN (SELECT 1 AS x) t461 ON t1.x = t461.x JOIN (SELECT 1 AS x) t462 ON t1.x = t462.x JOIN (SELECT 1 AS x) t463 ON t1.x = t463.x JOIN (SELECT 1 AS x) t464 ON t1.x = t464.x JOIN (SELECT 1 AS x) t465 ON t1.x = t465.x JOIN (SELECT 1 AS x) t466 ON t1.x = t466.x JOIN (SELECT 1 AS x) t467 ON t1.x = t467.x JOIN (SELECT 1 AS x) t468 ON t1.x = t468.x JOIN (SELECT 1 AS x) t469 ON t1.x = t469.x JOIN (SELECT 1 AS x) t470 ON t1.x = t470.x JOIN (SELECT 1 AS x) t471 ON t1.x = t471.x JOIN (SELECT 1 AS x) t472 ON t1.x = t472.x JOIN (SELECT 1 AS x) t473 ON t1.x = t473.x JOIN (SELECT 1 AS x) t474 ON t1.x = t474.x JOIN (SELECT 1 AS x) t475 ON t1.x = t475.x JOIN (SELECT 1 AS x) t476 ON t1.x = t476.x JOIN (SELECT 1 AS x) t477 ON t1.x = t477.x JOIN (SELECT 1 AS x) t478 ON t1.x = t478.x JOIN (SELECT 1 AS x) t479 ON t1.x = t479.x JOIN (SELECT 1 AS x) t480 ON t1.x = t480.x JOIN (SELECT 1 AS x) t481 ON t1.x = t481.x JOIN (SELECT 1 AS x) t482 ON t1.x = t482.x JOIN (SELECT 1 AS x) t483 ON t1.x = t483.x JOIN (SELECT 1 AS x) t484 ON t1.x = t484.x JOIN (SELECT 1 AS x) t485 ON t1.x = t485.x JOIN (SELECT 1 AS x) t486 ON t1.x = t486.x JOIN (SELECT 1 AS x) t487 ON t1.x = t487.x JOIN (SELECT 1 AS x) t488 ON t1.x = t488.x JOIN (SELECT 1 AS x) t489 ON t1.x = t489.x JOIN (SELECT 1 AS x) t490 ON t1.x = t490.x JOIN (SELECT 1 AS x) t491 ON t1.x = t491.x JOIN (SELECT 1 AS x) t492 ON t1.x = t492.x JOIN (SELECT 1 AS x) t493 ON t1.x = t493.x JOIN (SELECT 1 AS x) t494 ON t1.x = t494.x JOIN (SELECT 1 AS x) t495 ON t1.x = t495.x JOIN (SELECT 1 AS x) t496 ON t1.x = t496.x JOIN (SELECT 1 AS x) t497 ON t1.x = t497.x JOIN (SELECT 1 AS x) t498 ON t1.x = t498.x JOIN (SELECT 1 AS x) t499 ON t1.x = t499.x JOIN (SELECT 1 AS x) t500 ON t1.x = t500.x JOIN (SELECT 1 AS x) t501 ON t1.x = t501.x JOIN (SELECT 1 AS x) t502 ON t1.x = t502.x JOIN (SELECT 1 AS x) t503 ON t1.x = t503.x JOIN (SELECT 1 AS x) t504 ON t1.x = t504.x JOIN (SELECT 1 AS x) t505 ON t1.x = t505.x JOIN (SELECT 1 AS x) t506 ON t1.x = t506.x JOIN (SELECT 1 AS x) t507 ON t1.x = t507.x JOIN (SELECT 1 AS x) t508 ON t1.x = t508.x JOIN (SELECT 1 AS x) t509 ON t1.x = t509.x JOIN (SELECT 1 AS x) t510 ON t1.x = t510.x JOIN (SELECT 1 AS x) t511 ON t1.x = t511.x JOIN (SELECT 1 AS x) t512 ON t1.x = t512.x JOIN (SELECT 1 AS x) t513 ON t1.x = t513.x JOIN (SELECT 1 AS x) t514 ON t1.x = t514.x JOIN (SELECT 1 AS x) t515 ON t1.x = t515.x JOIN (SELECT 1 AS x) t516 ON t1.x = t516.x JOIN (SELECT 1 AS x) t517 ON t1.x = t517.x JOIN (SELECT 1 AS x) t518 ON t1.x = t518.x JOIN (SELECT 1 AS x) t519 ON t1.x = t519.x JOIN (SELECT 1 AS x) t520 ON t1.x = t520.x JOIN (SELECT 1 AS x) t521 ON t1.x = t521.x JOIN (SELECT 1 AS x) t522 ON t1.x = t522.x JOIN (SELECT 1 AS x) t523 ON t1.x = t523.x JOIN (SELECT 1 AS x) t524 ON t1.x = t524.x JOIN (SELECT 1 AS x) t525 ON t1.x = t525.x JOIN (SELECT 1 AS x) t526 ON t1.x = t526.x JOIN (SELECT 1 AS x) t527 ON t1.x = t527.x JOIN (SELECT 1 AS x) t528 ON t1.x = t528.x JOIN (SELECT 1 AS x) t529 ON t1.x = t529.x JOIN (SELECT 1 AS x) t530 ON t1.x = t530.x JOIN (SELECT 1 AS x) t531 ON t1.x = t531.x JOIN (SELECT 1 AS x) t532 ON t1.x = t532.x JOIN (SELECT 1 AS x) t533 ON t1.x = t533.x JOIN (SELECT 1 AS x) t534 ON t1.x = t534.x JOIN (SELECT 1 AS x) t535 ON t1.x = t535.x JOIN (SELECT 1 AS x) t536 ON t1.x = t536.x JOIN (SELECT 1 AS x) t537 ON t1.x = t537.x JOIN (SELECT 1 AS x) t538 ON t1.x = t538.x JOIN (SELECT 1 AS x) t539 ON t1.x = t539.x JOIN (SELECT 1 AS x) t540 ON t1.x = t540.x JOIN (SELECT 1 AS x) t541 ON t1.x = t541.x JOIN (SELECT 1 AS x) t542 ON t1.x = t542.x JOIN (SELECT 1 AS x) t543 ON t1.x = t543.x JOIN (SELECT 1 AS x) t544 ON t1.x = t544.x JOIN (SELECT 1 AS x) t545 ON t1.x = t545.x JOIN (SELECT 1 AS x) t546 ON t1.x = t546.x JOIN (SELECT 1 AS x) t547 ON t1.x = t547.x JOIN (SELECT 1 AS x) t548 ON t1.x = t548.x JOIN (SELECT 1 AS x) t549 ON t1.x = t549.x JOIN (SELECT 1 AS x) t550 ON t1.x = t550.x JOIN (SELECT 1 AS x) t551 ON t1.x = t551.x JOIN (SELECT 1 AS x) t552 ON t1.x = t552.x JOIN (SELECT 1 AS x) t553 ON t1.x = t553.x JOIN (SELECT 1 AS x) t554 ON t1.x = t554.x JOIN (SELECT 1 AS x) t555 ON t1.x = t555.x JOIN (SELECT 1 AS x) t556 ON t1.x = t556.x JOIN (SELECT 1 AS x) t557 ON t1.x = t557.x JOIN (SELECT 1 AS x) t558 ON t1.x = t558.x JOIN (SELECT 1 AS x) t559 ON t1.x = t559.x JOIN (SELECT 1 AS x) t560 ON t1.x = t560.x JOIN (SELECT 1 AS x) t561 ON t1.x = t561.x JOIN (SELECT 1 AS x) t562 ON t1.x = t562.x JOIN (SELECT 1 AS x) t563 ON t1.x = t563.x JOIN (SELECT 1 AS x) t564 ON t1.x = t564.x JOIN (SELECT 1 AS x) t565 ON t1.x = t565.x JOIN (SELECT 1 AS x) t566 ON t1.x = t566.x JOIN (SELECT 1 AS x) t567 ON t1.x = t567.x JOIN (SELECT 1 AS x) t568 ON t1.x = t568.x JOIN (SELECT 1 AS x) t569 ON t1.x = t569.x JOIN (SELECT 1 AS x) t570 ON t1.x = t570.x JOIN (SELECT 1 AS x) t571 ON t1.x = t571.x JOIN (SELECT 1 AS x) t572 ON t1.x = t572.x JOIN (SELECT 1 AS x) t573 ON t1.x = t573.x JOIN (SELECT 1 AS x) t574 ON t1.x = t574.x JOIN (SELECT 1 AS x) t575 ON t1.x = t575.x JOIN (SELECT 1 AS x) t576 ON t1.x = t576.x JOIN (SELECT 1 AS x) t577 ON t1.x = t577.x JOIN (SELECT 1 AS x) t578 ON t1.x = t578.x JOIN (SELECT 1 AS x) t579 ON t1.x = t579.x JOIN (SELECT 1 AS x) t580 ON t1.x = t580.x JOIN (SELECT 1 AS x) t581 ON t1.x = t581.x JOIN (SELECT 1 AS x) t582 ON t1.x = t582.x JOIN (SELECT 1 AS x) t583 ON t1.x = t583.x JOIN (SELECT 1 AS x) t584 ON t1.x = t584.x JOIN (SELECT 1 AS x) t585 ON t1.x = t585.x JOIN (SELECT 1 AS x) t586 ON t1.x = t586.x JOIN (SELECT 1 AS x) t587 ON t1.x = t587.x JOIN (SELECT 1 AS x) t588 ON t1.x = t588.x JOIN (SELECT 1 AS x) t589 ON t1.x = t589.x JOIN (SELECT 1 AS x) t590 ON t1.x = t590.x JOIN (SELECT 1 AS x) t591 ON t1.x = t591.x JOIN (SELECT 1 AS x) t592 ON t1.x = t592.x JOIN (SELECT 1 AS x) t593 ON t1.x = t593.x JOIN (SELECT 1 AS x) t594 ON t1.x = t594.x JOIN (SELECT 1 AS x) t595 ON t1.x = t595.x JOIN (SELECT 1 AS x) t596 ON t1.x = t596.x JOIN (SELECT 1 AS x) t597 ON t1.x = t597.x JOIN (SELECT 1 AS x) t598 ON t1.x = t598.x JOIN (SELECT 1 AS x) t599 ON t1.x = t599.x JOIN (SELECT 1 AS x) t600 ON t1.x = t600.x JOIN (SELECT 1 AS x) t601 ON t1.x = t601.x JOIN (SELECT 1 AS x) t602 ON t1.x = t602.x JOIN (SELECT 1 AS x) t603 ON t1.x = t603.x JOIN (SELECT 1 AS x) t604 ON t1.x = t604.x JOIN (SELECT 1 AS x) t605 ON t1.x = t605.x JOIN (SELECT 1 AS x) t606 ON t1.x = t606.x JOIN (SELECT 1 AS x) t607 ON t1.x = t607.x JOIN (SELECT 1 AS x) t608 ON t1.x = t608.x JOIN (SELECT 1 AS x) t609 ON t1.x = t609.x JOIN (SELECT 1 AS x) t610 ON t1.x = t610.x JOIN (SELECT 1 AS x) t611 ON t1.x = t611.x JOIN (SELECT 1 AS x) t612 ON t1.x = t612.x JOIN (SELECT 1 AS x) t613 ON t1.x = t613.x JOIN (SELECT 1 AS x) t614 ON t1.x = t614.x JOIN (SELECT 1 AS x) t615 ON t1.x = t615.x JOIN (SELECT 1 AS x) t616 ON t1.x = t616.x JOIN (SELECT 1 AS x) t617 ON t1.x = t617.x JOIN (SELECT 1 AS x) t618 ON t1.x = t618.x JOIN (SELECT 1 AS x) t619 ON t1.x = t619.x JOIN (SELECT 1 AS x) t620 ON t1.x = t620.x JOIN (SELECT 1 AS x) t621 ON t1.x = t621.x JOIN (SELECT 1 AS x) t622 ON t1.x = t622.x JOIN (SELECT 1 AS x) t623 ON t1.x = t623.x JOIN (SELECT 1 AS x) t624 ON t1.x = t624.x JOIN (SELECT 1 AS x) t625 ON t1.x = t625.x JOIN (SELECT 1 AS x) t626 ON t1.x = t626.x JOIN (SELECT 1 AS x) t627 ON t1.x = t627.x JOIN (SELECT 1 AS x) t628 ON t1.x = t628.x JOIN (SELECT 1 AS x) t629 ON t1.x = t629.x JOIN (SELECT 1 AS x) t630 ON t1.x = t630.x JOIN (SELECT 1 AS x) t631 ON t1.x = t631.x JOIN (SELECT 1 AS x) t632 ON t1.x = t632.x JOIN (SELECT 1 AS x) t633 ON t1.x = t633.x JOIN (SELECT 1 AS x) t634 ON t1.x = t634.x JOIN (SELECT 1 AS x) t635 ON t1.x = t635.x JOIN (SELECT 1 AS x) t636 ON t1.x = t636.x JOIN (SELECT 1 AS x) t637 ON t1.x = t637.x JOIN (SELECT 1 AS x) t638 ON t1.x = t638.x JOIN (SELECT 1 AS x) t639 ON t1.x = t639.x JOIN (SELECT 1 AS x) t640 ON t1.x = t640.x JOIN (SELECT 1 AS x) t641 ON t1.x = t641.x JOIN (SELECT 1 AS x) t642 ON t1.x = t642.x JOIN (SELECT 1 AS x) t643 ON t1.x = t643.x JOIN (SELECT 1 AS x) t644 ON t1.x = t644.x JOIN (SELECT 1 AS x) t645 ON t1.x = t645.x JOIN (SELECT 1 AS x) t646 ON t1.x = t646.x JOIN (SELECT 1 AS x) t647 ON t1.x = t647.x JOIN (SELECT 1 AS x) t648 ON t1.x = t648.x JOIN (SELECT 1 AS x) t649 ON t1.x = t649.x JOIN (SELECT 1 AS x) t650 ON t1.x = t650.x JOIN (SELECT 1 AS x) t651 ON t1.x = t651.x JOIN (SELECT 1 AS x) t652 ON t1.x = t652.x JOIN (SELECT 1 AS x) t653 ON t1.x = t653.x JOIN (SELECT 1 AS x) t654 ON t1.x = t654.x JOIN (SELECT 1 AS x) t655 ON t1.x = t655.x JOIN (SELECT 1 AS x) t656 ON t1.x = t656.x JOIN (SELECT 1 AS x) t657 ON t1.x = t657.x JOIN (SELECT 1 AS x) t658 ON t1.x = t658.x JOIN (SELECT 1 AS x) t659 ON t1.x = t659.x JOIN (SELECT 1 AS x) t660 ON t1.x = t660.x JOIN (SELECT 1 AS x) t661 ON t1.x = t661.x JOIN (SELECT 1 AS x) t662 ON t1.x = t662.x JOIN (SELECT 1 AS x) t663 ON t1.x = t663.x JOIN (SELECT 1 AS x) t664 ON t1.x = t664.x JOIN (SELECT 1 AS x) t665 ON t1.x = t665.x JOIN (SELECT 1 AS x) t666 ON t1.x = t666.x JOIN (SELECT 1 AS x) t667 ON t1.x = t667.x JOIN (SELECT 1 AS x) t668 ON t1.x = t668.x JOIN (SELECT 1 AS x) t669 ON t1.x = t669.x JOIN (SELECT 1 AS x) t670 ON t1.x = t670.x JOIN (SELECT 1 AS x) t671 ON t1.x = t671.x JOIN (SELECT 1 AS x) t672 ON t1.x = t672.x JOIN (SELECT 1 AS x) t673 ON t1.x = t673.x JOIN (SELECT 1 AS x) t674 ON t1.x = t674.x JOIN (SELECT 1 AS x) t675 ON t1.x = t675.x JOIN (SELECT 1 AS x) t676 ON t1.x = t676.x JOIN (SELECT 1 AS x) t677 ON t1.x = t677.x JOIN (SELECT 1 AS x) t678 ON t1.x = t678.x JOIN (SELECT 1 AS x) t679 ON t1.x = t679.x JOIN (SELECT 1 AS x) t680 ON t1.x = t680.x JOIN (SELECT 1 AS x) t681 ON t1.x = t681.x JOIN (SELECT 1 AS x) t682 ON t1.x = t682.x JOIN (SELECT 1 AS x) t683 ON t1.x = t683.x JOIN (SELECT 1 AS x) t684 ON t1.x = t684.x JOIN (SELECT 1 AS x) t685 ON t1.x = t685.x JOIN (SELECT 1 AS x) t686 ON t1.x = t686.x JOIN (SELECT 1 AS x) t687 ON t1.x = t687.x JOIN (SELECT 1 AS x) t688 ON t1.x = t688.x JOIN (SELECT 1 AS x) t689 ON t1.x = t689.x JOIN (SELECT 1 AS x) t690 ON t1.x = t690.x JOIN (SELECT 1 AS x) t691 ON t1.x = t691.x JOIN (SELECT 1 AS x) t692 ON t1.x = t692.x JOIN (SELECT 1 AS x) t693 ON t1.x = t693.x JOIN (SELECT 1 AS x) t694 ON t1.x = t694.x JOIN (SELECT 1 AS x) t695 ON t1.x = t695.x JOIN (SELECT 1 AS x) t696 ON t1.x = t696.x JOIN (SELECT 1 AS x) t697 ON t1.x = t697.x JOIN (SELECT 1 AS x) t698 ON t1.x = t698.x JOIN (SELECT 1 AS x) t699 ON t1.x = t699.x JOIN (SELECT 1 AS x) t700 ON t1.x = t700.x JOIN (SELECT 1 AS x) t701 ON t1.x = t701.x JOIN (SELECT 1 AS x) t702 ON t1.x = t702.x JOIN (SELECT 1 AS x) t703 ON t1.x = t703.x JOIN (SELECT 1 AS x) t704 ON t1.x = t704.x JOIN (SELECT 1 AS x) t705 ON t1.x = t705.x JOIN (SELECT 1 AS x) t706 ON t1.x = t706.x JOIN (SELECT 1 AS x) t707 ON t1.x = t707.x JOIN (SELECT 1 AS x) t708 ON t1.x = t708.x JOIN (SELECT 1 AS x) t709 ON t1.x = t709.x JOIN (SELECT 1 AS x) t710 ON t1.x = t710.x JOIN (SELECT 1 AS x) t711 ON t1.x = t711.x JOIN (SELECT 1 AS x) t712 ON t1.x = t712.x JOIN (SELECT 1 AS x) t713 ON t1.x = t713.x JOIN (SELECT 1 AS x) t714 ON t1.x = t714.x JOIN (SELECT 1 AS x) t715 ON t1.x = t715.x JOIN (SELECT 1 AS x) t716 ON t1.x = t716.x JOIN (SELECT 1 AS x) t717 ON t1.x = t717.x JOIN (SELECT 1 AS x) t718 ON t1.x = t718.x JOIN (SELECT 1 AS x) t719 ON t1.x = t719.x JOIN (SELECT 1 AS x) t720 ON t1.x = t720.x JOIN (SELECT 1 AS x) t721 ON t1.x = t721.x JOIN (SELECT 1 AS x) t722 ON t1.x = t722.x JOIN (SELECT 1 AS x) t723 ON t1.x = t723.x JOIN (SELECT 1 AS x) t724 ON t1.x = t724.x JOIN (SELECT 1 AS x) t725 ON t1.x = t725.x JOIN (SELECT 1 AS x) t726 ON t1.x = t726.x JOIN (SELECT 1 AS x) t727 ON t1.x = t727.x JOIN (SELECT 1 AS x) t728 ON t1.x = t728.x JOIN (SELECT 1 AS x) t729 ON t1.x = t729.x JOIN (SELECT 1 AS x) t730 ON t1.x = t730.x JOIN (SELECT 1 AS x) t731 ON t1.x = t731.x JOIN (SELECT 1 AS x) t732 ON t1.x = t732.x JOIN (SELECT 1 AS x) t733 ON t1.x = t733.x JOIN (SELECT 1 AS x) t734 ON t1.x = t734.x JOIN (SELECT 1 AS x) t735 ON t1.x = t735.x JOIN (SELECT 1 AS x) t736 ON t1.x = t736.x JOIN (SELECT 1 AS x) t737 ON t1.x = t737.x JOIN (SELECT 1 AS x) t738 ON t1.x = t738.x JOIN (SELECT 1 AS x) t739 ON t1.x = t739.x JOIN (SELECT 1 AS x) t740 ON t1.x = t740.x JOIN (SELECT 1 AS x) t741 ON t1.x = t741.x JOIN (SELECT 1 AS x) t742 ON t1.x = t742.x JOIN (SELECT 1 AS x) t743 ON t1.x = t743.x JOIN (SELECT 1 AS x) t744 ON t1.x = t744.x JOIN (SELECT 1 AS x) t745 ON t1.x = t745.x JOIN (SELECT 1 AS x) t746 ON t1.x = t746.x JOIN (SELECT 1 AS x) t747 ON t1.x = t747.x JOIN (SELECT 1 AS x) t748 ON t1.x = t748.x JOIN (SELECT 1 AS x) t749 ON t1.x = t749.x JOIN (SELECT 1 AS x) t750 ON t1.x = t750.x JOIN (SELECT 1 AS x) t751 ON t1.x = t751.x JOIN (SELECT 1 AS x) t752 ON t1.x = t752.x JOIN (SELECT 1 AS x) t753 ON t1.x = t753.x JOIN (SELECT 1 AS x) t754 ON t1.x = t754.x JOIN (SELECT 1 AS x) t755 ON t1.x = t755.x JOIN (SELECT 1 AS x) t756 ON t1.x = t756.x JOIN (SELECT 1 AS x) t757 ON t1.x = t757.x JOIN (SELECT 1 AS x) t758 ON t1.x = t758.x JOIN (SELECT 1 AS x) t759 ON t1.x = t759.x JOIN (SELECT 1 AS x) t760 ON t1.x = t760.x JOIN (SELECT 1 AS x) t761 ON t1.x = t761.x JOIN (SELECT 1 AS x) t762 ON t1.x = t762.x JOIN (SELECT 1 AS x) t763 ON t1.x = t763.x JOIN (SELECT 1 AS x) t764 ON t1.x = t764.x JOIN (SELECT 1 AS x) t765 ON t1.x = t765.x JOIN (SELECT 1 AS x) t766 ON t1.x = t766.x JOIN (SELECT 1 AS x) t767 ON t1.x = t767.x JOIN (SELECT 1 AS x) t768 ON t1.x = t768.x JOIN (SELECT 1 AS x) t769 ON t1.x = t769.x JOIN (SELECT 1 AS x) t770 ON t1.x = t770.x JOIN (SELECT 1 AS x) t771 ON t1.x = t771.x JOIN (SELECT 1 AS x) t772 ON t1.x = t772.x JOIN (SELECT 1 AS x) t773 ON t1.x = t773.x JOIN (SELECT 1 AS x) t774 ON t1.x = t774.x JOIN (SELECT 1 AS x) t775 ON t1.x = t775.x JOIN (SELECT 1 AS x) t776 ON t1.x = t776.x JOIN (SELECT 1 AS x) t777 ON t1.x = t777.x JOIN (SELECT 1 AS x) t778 ON t1.x = t778.x JOIN (SELECT 1 AS x) t779 ON t1.x = t779.x JOIN (SELECT 1 AS x) t780 ON t1.x = t780.x JOIN (SELECT 1 AS x) t781 ON t1.x = t781.x JOIN (SELECT 1 AS x) t782 ON t1.x = t782.x JOIN (SELECT 1 AS x) t783 ON t1.x = t783.x JOIN (SELECT 1 AS x) t784 ON t1.x = t784.x JOIN (SELECT 1 AS x) t785 ON t1.x = t785.x JOIN (SELECT 1 AS x) t786 ON t1.x = t786.x JOIN (SELECT 1 AS x) t787 ON t1.x = t787.x JOIN (SELECT 1 AS x) t788 ON t1.x = t788.x JOIN (SELECT 1 AS x) t789 ON t1.x = t789.x JOIN (SELECT 1 AS x) t790 ON t1.x = t790.x JOIN (SELECT 1 AS x) t791 ON t1.x = t791.x JOIN (SELECT 1 AS x) t792 ON t1.x = t792.x JOIN (SELECT 1 AS x) t793 ON t1.x = t793.x JOIN (SELECT 1 AS x) t794 ON t1.x = t794.x JOIN (SELECT 1 AS x) t795 ON t1.x = t795.x JOIN (SELECT 1 AS x) t796 ON t1.x = t796.x JOIN (SELECT 1 AS x) t797 ON t1.x = t797.x JOIN (SELECT 1 AS x) t798 ON t1.x = t798.x JOIN (SELECT 1 AS x) t799 ON t1.x = t799.x JOIN (SELECT 1 AS x) t800 ON t1.x = t800.x JOIN (SELECT 1 AS x) t801 ON t1.x = t801.x JOIN (SELECT 1 AS x) t802 ON t1.x = t802.x JOIN (SELECT 1 AS x) t803 ON t1.x = t803.x JOIN (SELECT 1 AS x) t804 ON t1.x = t804.x JOIN (SELECT 1 AS x) t805 ON t1.x = t805.x JOIN (SELECT 1 AS x) t806 ON t1.x = t806.x JOIN (SELECT 1 AS x) t807 ON t1.x = t807.x JOIN (SELECT 1 AS x) t808 ON t1.x = t808.x JOIN (SELECT 1 AS x) t809 ON t1.x = t809.x JOIN (SELECT 1 AS x) t810 ON t1.x = t810.x JOIN (SELECT 1 AS x) t811 ON t1.x = t811.x JOIN (SELECT 1 AS x) t812 ON t1.x = t812.x JOIN (SELECT 1 AS x) t813 ON t1.x = t813.x JOIN (SELECT 1 AS x) t814 ON t1.x = t814.x JOIN (SELECT 1 AS x) t815 ON t1.x = t815.x JOIN (SELECT 1 AS x) t816 ON t1.x = t816.x JOIN (SELECT 1 AS x) t817 ON t1.x = t817.x JOIN (SELECT 1 AS x) t818 ON t1.x = t818.x JOIN (SELECT 1 AS x) t819 ON t1.x = t819.x JOIN (SELECT 1 AS x) t820 ON t1.x = t820.x JOIN (SELECT 1 AS x) t821 ON t1.x = t821.x JOIN (SELECT 1 AS x) t822 ON t1.x = t822.x JOIN (SELECT 1 AS x) t823 ON t1.x = t823.x JOIN (SELECT 1 AS x) t824 ON t1.x = t824.x JOIN (SELECT 1 AS x) t825 ON t1.x = t825.x JOIN (SELECT 1 AS x) t826 ON t1.x = t826.x JOIN (SELECT 1 AS x) t827 ON t1.x = t827.x JOIN (SELECT 1 AS x) t828 ON t1.x = t828.x JOIN (SELECT 1 AS x) t829 ON t1.x = t829.x JOIN (SELECT 1 AS x) t830 ON t1.x = t830.x JOIN (SELECT 1 AS x) t831 ON t1.x = t831.x JOIN (SELECT 1 AS x) t832 ON t1.x = t832.x JOIN (SELECT 1 AS x) t833 ON t1.x = t833.x JOIN (SELECT 1 AS x) t834 ON t1.x = t834.x JOIN (SELECT 1 AS x) t835 ON t1.x = t835.x JOIN (SELECT 1 AS x) t836 ON t1.x = t836.x JOIN (SELECT 1 AS x) t837 ON t1.x = t837.x JOIN (SELECT 1 AS x) t838 ON t1.x = t838.x JOIN (SELECT 1 AS x) t839 ON t1.x = t839.x JOIN (SELECT 1 AS x) t840 ON t1.x = t840.x JOIN (SELECT 1 AS x) t841 ON t1.x = t841.x JOIN (SELECT 1 AS x) t842 ON t1.x = t842.x JOIN (SELECT 1 AS x) t843 ON t1.x = t843.x JOIN (SELECT 1 AS x) t844 ON t1.x = t844.x JOIN (SELECT 1 AS x) t845 ON t1.x = t845.x JOIN (SELECT 1 AS x) t846 ON t1.x = t846.x JOIN (SELECT 1 AS x) t847 ON t1.x = t847.x JOIN (SELECT 1 AS x) t848 ON t1.x = t848.x JOIN (SELECT 1 AS x) t849 ON t1.x = t849.x JOIN (SELECT 1 AS x) t850 ON t1.x = t850.x JOIN (SELECT 1 AS x) t851 ON t1.x = t851.x JOIN (SELECT 1 AS x) t852 ON t1.x = t852.x JOIN (SELECT 1 AS x) t853 ON t1.x = t853.x JOIN (SELECT 1 AS x) t854 ON t1.x = t854.x JOIN (SELECT 1 AS x) t855 ON t1.x = t855.x JOIN (SELECT 1 AS x) t856 ON t1.x = t856.x JOIN (SELECT 1 AS x) t857 ON t1.x = t857.x JOIN (SELECT 1 AS x) t858 ON t1.x = t858.x JOIN (SELECT 1 AS x) t859 ON t1.x = t859.x JOIN (SELECT 1 AS x) t860 ON t1.x = t860.x JOIN (SELECT 1 AS x) t861 ON t1.x = t861.x JOIN (SELECT 1 AS x) t862 ON t1.x = t862.x JOIN (SELECT 1 AS x) t863 ON t1.x = t863.x JOIN (SELECT 1 AS x) t864 ON t1.x = t864.x JOIN (SELECT 1 AS x) t865 ON t1.x = t865.x JOIN (SELECT 1 AS x) t866 ON t1.x = t866.x JOIN (SELECT 1 AS x) t867 ON t1.x = t867.x JOIN (SELECT 1 AS x) t868 ON t1.x = t868.x JOIN (SELECT 1 AS x) t869 ON t1.x = t869.x JOIN (SELECT 1 AS x) t870 ON t1.x = t870.x JOIN (SELECT 1 AS x) t871 ON t1.x = t871.x JOIN (SELECT 1 AS x) t872 ON t1.x = t872.x JOIN (SELECT 1 AS x) t873 ON t1.x = t873.x JOIN (SELECT 1 AS x) t874 ON t1.x = t874.x JOIN (SELECT 1 AS x) t875 ON t1.x = t875.x JOIN (SELECT 1 AS x) t876 ON t1.x = t876.x JOIN (SELECT 1 AS x) t877 ON t1.x = t877.x JOIN (SELECT 1 AS x) t878 ON t1.x = t878.x JOIN (SELECT 1 AS x) t879 ON t1.x = t879.x JOIN (SELECT 1 AS x) t880 ON t1.x = t880.x JOIN (SELECT 1 AS x) t881 ON t1.x = t881.x JOIN (SELECT 1 AS x) t882 ON t1.x = t882.x JOIN (SELECT 1 AS x) t883 ON t1.x = t883.x JOIN (SELECT 1 AS x) t884 ON t1.x = t884.x JOIN (SELECT 1 AS x) t885 ON t1.x = t885.x JOIN (SELECT 1 AS x) t886 ON t1.x = t886.x JOIN (SELECT 1 AS x) t887 ON t1.x = t887.x JOIN (SELECT 1 AS x) t888 ON t1.x = t888.x JOIN (SELECT 1 AS x) t889 ON t1.x = t889.x JOIN (SELECT 1 AS x) t890 ON t1.x = t890.x JOIN (SELECT 1 AS x) t891 ON t1.x = t891.x JOIN (SELECT 1 AS x) t892 ON t1.x = t892.x JOIN (SELECT 1 AS x) t893 ON t1.x = t893.x JOIN (SELECT 1 AS x) t894 ON t1.x = t894.x JOIN (SELECT 1 AS x) t895 ON t1.x = t895.x JOIN (SELECT 1 AS x) t896 ON t1.x = t896.x JOIN (SELECT 1 AS x) t897 ON t1.x = t897.x JOIN (SELECT 1 AS x) t898 ON t1.x = t898.x JOIN (SELECT 1 AS x) t899 ON t1.x = t899.x JOIN (SELECT 1 AS x) t900 ON t1.x = t900.x JOIN (SELECT 1 AS x) t901 ON t1.x = t901.x JOIN (SELECT 1 AS x) t902 ON t1.x = t902.x JOIN (SELECT 1 AS x) t903 ON t1.x = t903.x JOIN (SELECT 1 AS x) t904 ON t1.x = t904.x JOIN (SELECT 1 AS x) t905 ON t1.x = t905.x JOIN (SELECT 1 AS x) t906 ON t1.x = t906.x JOIN (SELECT 1 AS x) t907 ON t1.x = t907.x JOIN (SELECT 1 AS x) t908 ON t1.x = t908.x JOIN (SELECT 1 AS x) t909 ON t1.x = t909.x JOIN (SELECT 1 AS x) t910 ON t1.x = t910.x JOIN (SELECT 1 AS x) t911 ON t1.x = t911.x JOIN (SELECT 1 AS x) t912 ON t1.x = t912.x JOIN (SELECT 1 AS x) t913 ON t1.x = t913.x JOIN (SELECT 1 AS x) t914 ON t1.x = t914.x JOIN (SELECT 1 AS x) t915 ON t1.x = t915.x JOIN (SELECT 1 AS x) t916 ON t1.x = t916.x JOIN (SELECT 1 AS x) t917 ON t1.x = t917.x JOIN (SELECT 1 AS x) t918 ON t1.x = t918.x JOIN (SELECT 1 AS x) t919 ON t1.x = t919.x JOIN (SELECT 1 AS x) t920 ON t1.x = t920.x JOIN (SELECT 1 AS x) t921 ON t1.x = t921.x JOIN (SELECT 1 AS x) t922 ON t1.x = t922.x JOIN (SELECT 1 AS x) t923 ON t1.x = t923.x JOIN (SELECT 1 AS x) t924 ON t1.x = t924.x JOIN (SELECT 1 AS x) t925 ON t1.x = t925.x JOIN (SELECT 1 AS x) t926 ON t1.x = t926.x JOIN (SELECT 1 AS x) t927 ON t1.x = t927.x JOIN (SELECT 1 AS x) t928 ON t1.x = t928.x JOIN (SELECT 1 AS x) t929 ON t1.x = t929.x JOIN (SELECT 1 AS x) t930 ON t1.x = t930.x JOIN (SELECT 1 AS x) t931 ON t1.x = t931.x JOIN (SELECT 1 AS x) t932 ON t1.x = t932.x JOIN (SELECT 1 AS x) t933 ON t1.x = t933.x JOIN (SELECT 1 AS x) t934 ON t1.x = t934.x JOIN (SELECT 1 AS x) t935 ON t1.x = t935.x JOIN (SELECT 1 AS x) t936 ON t1.x = t936.x JOIN (SELECT 1 AS x) t937 ON t1.x = t937.x JOIN (SELECT 1 AS x) t938 ON t1.x = t938.x JOIN (SELECT 1 AS x) t939 ON t1.x = t939.x JOIN (SELECT 1 AS x) t940 ON t1.x = t940.x JOIN (SELECT 1 AS x) t941 ON t1.x = t941.x JOIN (SELECT 1 AS x) t942 ON t1.x = t942.x JOIN (SELECT 1 AS x) t943 ON t1.x = t943.x JOIN (SELECT 1 AS x) t944 ON t1.x = t944.x JOIN (SELECT 1 AS x) t945 ON t1.x = t945.x JOIN (SELECT 1 AS x) t946 ON t1.x = t946.x JOIN (SELECT 1 AS x) t947 ON t1.x = t947.x JOIN (SELECT 1 AS x) t948 ON t1.x = t948.x JOIN (SELECT 1 AS x) t949 ON t1.x = t949.x JOIN (SELECT 1 AS x) t950 ON t1.x = t950.x JOIN (SELECT 1 AS x) t951 ON t1.x = t951.x JOIN (SELECT 1 AS x) t952 ON t1.x = t952.x JOIN (SELECT 1 AS x) t953 ON t1.x = t953.x JOIN (SELECT 1 AS x) t954 ON t1.x = t954.x JOIN (SELECT 1 AS x) t955 ON t1.x = t955.x JOIN (SELECT 1 AS x) t956 ON t1.x = t956.x JOIN (SELECT 1 AS x) t957 ON t1.x = t957.x JOIN (SELECT 1 AS x) t958 ON t1.x = t958.x JOIN (SELECT 1 AS x) t959 ON t1.x = t959.x JOIN (SELECT 1 AS x) t960 ON t1.x = t960.x JOIN (SELECT 1 AS x) t961 ON t1.x = t961.x JOIN (SELECT 1 AS x) t962 ON t1.x = t962.x JOIN (SELECT 1 AS x) t963 ON t1.x = t963.x JOIN (SELECT 1 AS x) t964 ON t1.x = t964.x JOIN (SELECT 1 AS x) t965 ON t1.x = t965.x JOIN (SELECT 1 AS x) t966 ON t1.x = t966.x JOIN (SELECT 1 AS x) t967 ON t1.x = t967.x JOIN (SELECT 1 AS x) t968 ON t1.x = t968.x JOIN (SELECT 1 AS x) t969 ON t1.x = t969.x JOIN (SELECT 1 AS x) t970 ON t1.x = t970.x JOIN (SELECT 1 AS x) t971 ON t1.x = t971.x JOIN (SELECT 1 AS x) t972 ON t1.x = t972.x JOIN (SELECT 1 AS x) t973 ON t1.x = t973.x JOIN (SELECT 1 AS x) t974 ON t1.x = t974.x JOIN (SELECT 1 AS x) t975 ON t1.x = t975.x JOIN (SELECT 1 AS x) t976 ON t1.x = t976.x JOIN (SELECT 1 AS x) t977 ON t1.x = t977.x JOIN (SELECT 1 AS x) t978 ON t1.x = t978.x JOIN (SELECT 1 AS x) t979 ON t1.x = t979.x JOIN (SELECT 1 AS x) t980 ON t1.x = t980.x JOIN (SELECT 1 AS x) t981 ON t1.x = t981.x JOIN (SELECT 1 AS x) t982 ON t1.x = t982.x JOIN (SELECT 1 AS x) t983 ON t1.x = t983.x JOIN (SELECT 1 AS x) t984 ON t1.x = t984.x JOIN (SELECT 1 AS x) t985 ON t1.x = t985.x JOIN (SELECT 1 AS x) t986 ON t1.x = t986.x JOIN (SELECT 1 AS x) t987 ON t1.x = t987.x JOIN (SELECT 1 AS x) t988 ON t1.x = t988.x JOIN (SELECT 1 AS x) t989 ON t1.x = t989.x JOIN (SELECT 1 AS x) t990 ON t1.x = t990.x JOIN (SELECT 1 AS x) t991 ON t1.x = t991.x JOIN (SELECT 1 AS x) t992 ON t1.x = t992.x JOIN (SELECT 1 AS x) t993 ON t1.x = t993.x JOIN (SELECT 1 AS x) t994 ON t1.x = t994.x JOIN (SELECT 1 AS x) t995 ON t1.x = t995.x JOIN (SELECT 1 AS x) t996 ON t1.x = t996.x JOIN (SELECT 1 AS x) t997 ON t1.x = t997.x JOIN (SELECT 1 AS x) t998 ON t1.x = t998.x JOIN (SELECT 1 AS x) t999 ON t1.x = t999.x JOIN (SELECT 1 AS x) t1000 ON t1.x = t1000.x From f4ba22e0d2e11d8dc74e86e43bfffdc546ce4ea2 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 18 Apr 2024 12:26:40 +0000 Subject: [PATCH 0275/1018] Reduce test size --- tests/queries/0_stateless/03094_one_thousand_joins.reference | 2 +- tests/queries/0_stateless/03094_one_thousand_joins.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03094_one_thousand_joins.reference b/tests/queries/0_stateless/03094_one_thousand_joins.reference index 52176202f05..dd729ab0220 100644 --- a/tests/queries/0_stateless/03094_one_thousand_joins.reference +++ b/tests/queries/0_stateless/03094_one_thousand_joins.reference @@ -1 +1 @@ -1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 +1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 diff --git a/tests/queries/0_stateless/03094_one_thousand_joins.sql b/tests/queries/0_stateless/03094_one_thousand_joins.sql index f7ca2134d2e..b6345595811 100644 --- a/tests/queries/0_stateless/03094_one_thousand_joins.sql +++ b/tests/queries/0_stateless/03094_one_thousand_joins.sql @@ -1,4 +1,4 @@ -- Tags: no-fasttest, long -- Bug 33446, marked as 'long' because it still runs around 10 sec -SELECT * FROM (SELECT 1 AS x) t1 JOIN (SELECT 1 AS x) t2 ON t1.x = t2.x JOIN (SELECT 1 AS x) t3 ON t1.x = t3.x JOIN (SELECT 1 AS x) t4 ON t1.x = t4.x JOIN (SELECT 1 AS x) t5 ON t1.x = t5.x JOIN (SELECT 1 AS x) t6 ON t1.x = t6.x JOIN (SELECT 1 AS x) t7 ON t1.x = t7.x JOIN (SELECT 1 AS x) t8 ON t1.x = t8.x JOIN (SELECT 1 AS x) t9 ON t1.x = t9.x JOIN (SELECT 1 AS x) t10 ON t1.x = t10.x JOIN (SELECT 1 AS x) t11 ON t1.x = t11.x JOIN (SELECT 1 AS x) t12 ON t1.x = t12.x JOIN (SELECT 1 AS x) t13 ON t1.x = t13.x JOIN (SELECT 1 AS x) t14 ON t1.x = t14.x JOIN (SELECT 1 AS x) t15 ON t1.x = t15.x JOIN (SELECT 1 AS x) t16 ON t1.x = t16.x JOIN (SELECT 1 AS x) t17 ON t1.x = t17.x JOIN (SELECT 1 AS x) t18 ON t1.x = t18.x JOIN (SELECT 1 AS x) t19 ON t1.x = t19.x JOIN (SELECT 1 AS x) t20 ON t1.x = t20.x JOIN (SELECT 1 AS x) t21 ON t1.x = t21.x JOIN (SELECT 1 AS x) t22 ON t1.x = t22.x JOIN (SELECT 1 AS x) t23 ON t1.x = t23.x JOIN (SELECT 1 AS x) t24 ON t1.x = t24.x JOIN (SELECT 1 AS x) t25 ON t1.x = t25.x JOIN (SELECT 1 AS x) t26 ON t1.x = t26.x JOIN (SELECT 1 AS x) t27 ON t1.x = t27.x JOIN (SELECT 1 AS x) t28 ON t1.x = t28.x JOIN (SELECT 1 AS x) t29 ON t1.x = t29.x JOIN (SELECT 1 AS x) t30 ON t1.x = t30.x JOIN (SELECT 1 AS x) t31 ON t1.x = t31.x JOIN (SELECT 1 AS x) t32 ON t1.x = t32.x JOIN (SELECT 1 AS x) t33 ON t1.x = t33.x JOIN (SELECT 1 AS x) t34 ON t1.x = t34.x JOIN (SELECT 1 AS x) t35 ON t1.x = t35.x JOIN (SELECT 1 AS x) t36 ON t1.x = t36.x JOIN (SELECT 1 AS x) t37 ON t1.x = t37.x JOIN (SELECT 1 AS x) t38 ON t1.x = t38.x JOIN (SELECT 1 AS x) t39 ON t1.x = t39.x JOIN (SELECT 1 AS x) t40 ON t1.x = t40.x JOIN (SELECT 1 AS x) t41 ON t1.x = t41.x JOIN (SELECT 1 AS x) t42 ON t1.x = t42.x JOIN (SELECT 1 AS x) t43 ON t1.x = t43.x JOIN (SELECT 1 AS x) t44 ON t1.x = t44.x JOIN (SELECT 1 AS x) t45 ON t1.x = t45.x JOIN (SELECT 1 AS x) t46 ON t1.x = t46.x JOIN (SELECT 1 AS x) t47 ON t1.x = t47.x JOIN (SELECT 1 AS x) t48 ON t1.x = t48.x JOIN (SELECT 1 AS x) t49 ON t1.x = t49.x JOIN (SELECT 1 AS x) t50 ON t1.x = t50.x JOIN (SELECT 1 AS x) t51 ON t1.x = t51.x JOIN (SELECT 1 AS x) t52 ON t1.x = t52.x JOIN (SELECT 1 AS x) t53 ON t1.x = t53.x JOIN (SELECT 1 AS x) t54 ON t1.x = t54.x JOIN (SELECT 1 AS x) t55 ON t1.x = t55.x JOIN (SELECT 1 AS x) t56 ON t1.x = t56.x JOIN (SELECT 1 AS x) t57 ON t1.x = t57.x JOIN (SELECT 1 AS x) t58 ON t1.x = t58.x JOIN (SELECT 1 AS x) t59 ON t1.x = t59.x JOIN (SELECT 1 AS x) t60 ON t1.x = t60.x JOIN (SELECT 1 AS x) t61 ON t1.x = t61.x JOIN (SELECT 1 AS x) t62 ON t1.x = t62.x JOIN (SELECT 1 AS x) t63 ON t1.x = t63.x JOIN (SELECT 1 AS x) t64 ON t1.x = t64.x JOIN (SELECT 1 AS x) t65 ON t1.x = t65.x JOIN (SELECT 1 AS x) t66 ON t1.x = t66.x JOIN (SELECT 1 AS x) t67 ON t1.x = t67.x JOIN (SELECT 1 AS x) t68 ON t1.x = t68.x JOIN (SELECT 1 AS x) t69 ON t1.x = t69.x JOIN (SELECT 1 AS x) t70 ON t1.x = t70.x JOIN (SELECT 1 AS x) t71 ON t1.x = t71.x JOIN (SELECT 1 AS x) t72 ON t1.x = t72.x JOIN (SELECT 1 AS x) t73 ON t1.x = t73.x JOIN (SELECT 1 AS x) t74 ON t1.x = t74.x JOIN (SELECT 1 AS x) t75 ON t1.x = t75.x JOIN (SELECT 1 AS x) t76 ON t1.x = t76.x JOIN (SELECT 1 AS x) t77 ON t1.x = t77.x JOIN (SELECT 1 AS x) t78 ON t1.x = t78.x JOIN (SELECT 1 AS x) t79 ON t1.x = t79.x JOIN (SELECT 1 AS x) t80 ON t1.x = t80.x JOIN (SELECT 1 AS x) t81 ON t1.x = t81.x JOIN (SELECT 1 AS x) t82 ON t1.x = t82.x JOIN (SELECT 1 AS x) t83 ON t1.x = t83.x JOIN (SELECT 1 AS x) t84 ON t1.x = t84.x JOIN (SELECT 1 AS x) t85 ON t1.x = t85.x JOIN (SELECT 1 AS x) t86 ON t1.x = t86.x JOIN (SELECT 1 AS x) t87 ON t1.x = t87.x JOIN (SELECT 1 AS x) t88 ON t1.x = t88.x JOIN (SELECT 1 AS x) t89 ON t1.x = t89.x JOIN (SELECT 1 AS x) t90 ON t1.x = t90.x JOIN (SELECT 1 AS x) t91 ON t1.x = t91.x JOIN (SELECT 1 AS x) t92 ON t1.x = t92.x JOIN (SELECT 1 AS x) t93 ON t1.x = t93.x JOIN (SELECT 1 AS x) t94 ON t1.x = t94.x JOIN (SELECT 1 AS x) t95 ON t1.x = t95.x JOIN (SELECT 1 AS x) t96 ON t1.x = t96.x JOIN (SELECT 1 AS x) t97 ON t1.x = t97.x JOIN (SELECT 1 AS x) t98 ON t1.x = t98.x JOIN (SELECT 1 AS x) t99 ON t1.x = t99.x JOIN (SELECT 1 AS x) t100 ON t1.x = t100.x JOIN (SELECT 1 AS x) t101 ON t1.x = t101.x JOIN (SELECT 1 AS x) t102 ON t1.x = t102.x JOIN (SELECT 1 AS x) t103 ON t1.x = t103.x JOIN (SELECT 1 AS x) t104 ON t1.x = t104.x JOIN (SELECT 1 AS x) t105 ON t1.x = t105.x JOIN (SELECT 1 AS x) t106 ON t1.x = t106.x JOIN (SELECT 1 AS x) t107 ON t1.x = t107.x JOIN (SELECT 1 AS x) t108 ON t1.x = t108.x JOIN (SELECT 1 AS x) t109 ON t1.x = t109.x JOIN (SELECT 1 AS x) t110 ON t1.x = t110.x JOIN (SELECT 1 AS x) t111 ON t1.x = t111.x JOIN (SELECT 1 AS x) t112 ON t1.x = t112.x JOIN (SELECT 1 AS x) t113 ON t1.x = t113.x JOIN (SELECT 1 AS x) t114 ON t1.x = t114.x JOIN (SELECT 1 AS x) t115 ON t1.x = t115.x JOIN (SELECT 1 AS x) t116 ON t1.x = t116.x JOIN (SELECT 1 AS x) t117 ON t1.x = t117.x JOIN (SELECT 1 AS x) t118 ON t1.x = t118.x JOIN (SELECT 1 AS x) t119 ON t1.x = t119.x JOIN (SELECT 1 AS x) t120 ON t1.x = t120.x JOIN (SELECT 1 AS x) t121 ON t1.x = t121.x JOIN (SELECT 1 AS x) t122 ON t1.x = t122.x JOIN (SELECT 1 AS x) t123 ON t1.x = t123.x JOIN (SELECT 1 AS x) t124 ON t1.x = t124.x JOIN (SELECT 1 AS x) t125 ON t1.x = t125.x JOIN (SELECT 1 AS x) t126 ON t1.x = t126.x JOIN (SELECT 1 AS x) t127 ON t1.x = t127.x JOIN (SELECT 1 AS x) t128 ON t1.x = t128.x JOIN (SELECT 1 AS x) t129 ON t1.x = t129.x JOIN (SELECT 1 AS x) t130 ON t1.x = t130.x JOIN (SELECT 1 AS x) t131 ON t1.x = t131.x JOIN (SELECT 1 AS x) t132 ON t1.x = t132.x JOIN (SELECT 1 AS x) t133 ON t1.x = t133.x JOIN (SELECT 1 AS x) t134 ON t1.x = t134.x JOIN (SELECT 1 AS x) t135 ON t1.x = t135.x JOIN (SELECT 1 AS x) t136 ON t1.x = t136.x JOIN (SELECT 1 AS x) t137 ON t1.x = t137.x JOIN (SELECT 1 AS x) t138 ON t1.x = t138.x JOIN (SELECT 1 AS x) t139 ON t1.x = t139.x JOIN (SELECT 1 AS x) t140 ON t1.x = t140.x JOIN (SELECT 1 AS x) t141 ON t1.x = t141.x JOIN (SELECT 1 AS x) t142 ON t1.x = t142.x JOIN (SELECT 1 AS x) t143 ON t1.x = t143.x JOIN (SELECT 1 AS x) t144 ON t1.x = t144.x JOIN (SELECT 1 AS x) t145 ON t1.x = t145.x JOIN (SELECT 1 AS x) t146 ON t1.x = t146.x JOIN (SELECT 1 AS x) t147 ON t1.x = t147.x JOIN (SELECT 1 AS x) t148 ON t1.x = t148.x JOIN (SELECT 1 AS x) t149 ON t1.x = t149.x JOIN (SELECT 1 AS x) t150 ON t1.x = t150.x JOIN (SELECT 1 AS x) t151 ON t1.x = t151.x JOIN (SELECT 1 AS x) t152 ON t1.x = t152.x JOIN (SELECT 1 AS x) t153 ON t1.x = t153.x JOIN (SELECT 1 AS x) t154 ON t1.x = t154.x JOIN (SELECT 1 AS x) t155 ON t1.x = t155.x JOIN (SELECT 1 AS x) t156 ON t1.x = t156.x JOIN (SELECT 1 AS x) t157 ON t1.x = t157.x JOIN (SELECT 1 AS x) t158 ON t1.x = t158.x JOIN (SELECT 1 AS x) t159 ON t1.x = t159.x JOIN (SELECT 1 AS x) t160 ON t1.x = t160.x JOIN (SELECT 1 AS x) t161 ON t1.x = t161.x JOIN (SELECT 1 AS x) t162 ON t1.x = t162.x JOIN (SELECT 1 AS x) t163 ON t1.x = t163.x JOIN (SELECT 1 AS x) t164 ON t1.x = t164.x JOIN (SELECT 1 AS x) t165 ON t1.x = t165.x JOIN (SELECT 1 AS x) t166 ON t1.x = t166.x JOIN (SELECT 1 AS x) t167 ON t1.x = t167.x JOIN (SELECT 1 AS x) t168 ON t1.x = t168.x JOIN (SELECT 1 AS x) t169 ON t1.x = t169.x JOIN (SELECT 1 AS x) t170 ON t1.x = t170.x JOIN (SELECT 1 AS x) t171 ON t1.x = t171.x JOIN (SELECT 1 AS x) t172 ON t1.x = t172.x JOIN (SELECT 1 AS x) t173 ON t1.x = t173.x JOIN (SELECT 1 AS x) t174 ON t1.x = t174.x JOIN (SELECT 1 AS x) t175 ON t1.x = t175.x JOIN (SELECT 1 AS x) t176 ON t1.x = t176.x JOIN (SELECT 1 AS x) t177 ON t1.x = t177.x JOIN (SELECT 1 AS x) t178 ON t1.x = t178.x JOIN (SELECT 1 AS x) t179 ON t1.x = t179.x JOIN (SELECT 1 AS x) t180 ON t1.x = t180.x JOIN (SELECT 1 AS x) t181 ON t1.x = t181.x JOIN (SELECT 1 AS x) t182 ON t1.x = t182.x JOIN (SELECT 1 AS x) t183 ON t1.x = t183.x JOIN (SELECT 1 AS x) t184 ON t1.x = t184.x JOIN (SELECT 1 AS x) t185 ON t1.x = t185.x JOIN (SELECT 1 AS x) t186 ON t1.x = t186.x JOIN (SELECT 1 AS x) t187 ON t1.x = t187.x JOIN (SELECT 1 AS x) t188 ON t1.x = t188.x JOIN (SELECT 1 AS x) t189 ON t1.x = t189.x JOIN (SELECT 1 AS x) t190 ON t1.x = t190.x JOIN (SELECT 1 AS x) t191 ON t1.x = t191.x JOIN (SELECT 1 AS x) t192 ON t1.x = t192.x JOIN (SELECT 1 AS x) t193 ON t1.x = t193.x JOIN (SELECT 1 AS x) t194 ON t1.x = t194.x JOIN (SELECT 1 AS x) t195 ON t1.x = t195.x JOIN (SELECT 1 AS x) t196 ON t1.x = t196.x JOIN (SELECT 1 AS x) t197 ON t1.x = t197.x JOIN (SELECT 1 AS x) t198 ON t1.x = t198.x JOIN (SELECT 1 AS x) t199 ON t1.x = t199.x JOIN (SELECT 1 AS x) t200 ON t1.x = t200.x JOIN (SELECT 1 AS x) t201 ON t1.x = t201.x JOIN (SELECT 1 AS x) t202 ON t1.x = t202.x JOIN (SELECT 1 AS x) t203 ON t1.x = t203.x JOIN (SELECT 1 AS x) t204 ON t1.x = t204.x JOIN (SELECT 1 AS x) t205 ON t1.x = t205.x JOIN (SELECT 1 AS x) t206 ON t1.x = t206.x JOIN (SELECT 1 AS x) t207 ON t1.x = t207.x JOIN (SELECT 1 AS x) t208 ON t1.x = t208.x JOIN (SELECT 1 AS x) t209 ON t1.x = t209.x JOIN (SELECT 1 AS x) t210 ON t1.x = t210.x JOIN (SELECT 1 AS x) t211 ON t1.x = t211.x JOIN (SELECT 1 AS x) t212 ON t1.x = t212.x JOIN (SELECT 1 AS x) t213 ON t1.x = t213.x JOIN (SELECT 1 AS x) t214 ON t1.x = t214.x JOIN (SELECT 1 AS x) t215 ON t1.x = t215.x JOIN (SELECT 1 AS x) t216 ON t1.x = t216.x JOIN (SELECT 1 AS x) t217 ON t1.x = t217.x JOIN (SELECT 1 AS x) t218 ON t1.x = t218.x JOIN (SELECT 1 AS x) t219 ON t1.x = t219.x JOIN (SELECT 1 AS x) t220 ON t1.x = t220.x JOIN (SELECT 1 AS x) t221 ON t1.x = t221.x JOIN (SELECT 1 AS x) t222 ON t1.x = t222.x JOIN (SELECT 1 AS x) t223 ON t1.x = t223.x JOIN (SELECT 1 AS x) t224 ON t1.x = t224.x JOIN (SELECT 1 AS x) t225 ON t1.x = t225.x JOIN (SELECT 1 AS x) t226 ON t1.x = t226.x JOIN (SELECT 1 AS x) t227 ON t1.x = t227.x JOIN (SELECT 1 AS x) t228 ON t1.x = t228.x JOIN (SELECT 1 AS x) t229 ON t1.x = t229.x JOIN (SELECT 1 AS x) t230 ON t1.x = t230.x JOIN (SELECT 1 AS x) t231 ON t1.x = t231.x JOIN (SELECT 1 AS x) t232 ON t1.x = t232.x JOIN (SELECT 1 AS x) t233 ON t1.x = t233.x JOIN (SELECT 1 AS x) t234 ON t1.x = t234.x JOIN (SELECT 1 AS x) t235 ON t1.x = t235.x JOIN (SELECT 1 AS x) t236 ON t1.x = t236.x JOIN (SELECT 1 AS x) t237 ON t1.x = t237.x JOIN (SELECT 1 AS x) t238 ON t1.x = t238.x JOIN (SELECT 1 AS x) t239 ON t1.x = t239.x JOIN (SELECT 1 AS x) t240 ON t1.x = t240.x JOIN (SELECT 1 AS x) t241 ON t1.x = t241.x JOIN (SELECT 1 AS x) t242 ON t1.x = t242.x JOIN (SELECT 1 AS x) t243 ON t1.x = t243.x JOIN (SELECT 1 AS x) t244 ON t1.x = t244.x JOIN (SELECT 1 AS x) t245 ON t1.x = t245.x JOIN (SELECT 1 AS x) t246 ON t1.x = t246.x JOIN (SELECT 1 AS x) t247 ON t1.x = t247.x JOIN (SELECT 1 AS x) t248 ON t1.x = t248.x JOIN (SELECT 1 AS x) t249 ON t1.x = t249.x JOIN (SELECT 1 AS x) t250 ON t1.x = t250.x JOIN (SELECT 1 AS x) t251 ON t1.x = t251.x JOIN (SELECT 1 AS x) t252 ON t1.x = t252.x JOIN (SELECT 1 AS x) t253 ON t1.x = t253.x JOIN (SELECT 1 AS x) t254 ON t1.x = t254.x JOIN (SELECT 1 AS x) t255 ON t1.x = t255.x JOIN (SELECT 1 AS x) t256 ON t1.x = t256.x JOIN (SELECT 1 AS x) t257 ON t1.x = t257.x JOIN (SELECT 1 AS x) t258 ON t1.x = t258.x JOIN (SELECT 1 AS x) t259 ON t1.x = t259.x JOIN (SELECT 1 AS x) t260 ON t1.x = t260.x JOIN (SELECT 1 AS x) t261 ON t1.x = t261.x JOIN (SELECT 1 AS x) t262 ON t1.x = t262.x JOIN (SELECT 1 AS x) t263 ON t1.x = t263.x JOIN (SELECT 1 AS x) t264 ON t1.x = t264.x JOIN (SELECT 1 AS x) t265 ON t1.x = t265.x JOIN (SELECT 1 AS x) t266 ON t1.x = t266.x JOIN (SELECT 1 AS x) t267 ON t1.x = t267.x JOIN (SELECT 1 AS x) t268 ON t1.x = t268.x JOIN (SELECT 1 AS x) t269 ON t1.x = t269.x JOIN (SELECT 1 AS x) t270 ON t1.x = t270.x JOIN (SELECT 1 AS x) t271 ON t1.x = t271.x JOIN (SELECT 1 AS x) t272 ON t1.x = t272.x JOIN (SELECT 1 AS x) t273 ON t1.x = t273.x JOIN (SELECT 1 AS x) t274 ON t1.x = t274.x JOIN (SELECT 1 AS x) t275 ON t1.x = t275.x JOIN (SELECT 1 AS x) t276 ON t1.x = t276.x JOIN (SELECT 1 AS x) t277 ON t1.x = t277.x JOIN (SELECT 1 AS x) t278 ON t1.x = t278.x JOIN (SELECT 1 AS x) t279 ON t1.x = t279.x JOIN (SELECT 1 AS x) t280 ON t1.x = t280.x JOIN (SELECT 1 AS x) t281 ON t1.x = t281.x JOIN (SELECT 1 AS x) t282 ON t1.x = t282.x JOIN (SELECT 1 AS x) t283 ON t1.x = t283.x JOIN (SELECT 1 AS x) t284 ON t1.x = t284.x JOIN (SELECT 1 AS x) t285 ON t1.x = t285.x JOIN (SELECT 1 AS x) t286 ON t1.x = t286.x JOIN (SELECT 1 AS x) t287 ON t1.x = t287.x JOIN (SELECT 1 AS x) t288 ON t1.x = t288.x JOIN (SELECT 1 AS x) t289 ON t1.x = t289.x JOIN (SELECT 1 AS x) t290 ON t1.x = t290.x JOIN (SELECT 1 AS x) t291 ON t1.x = t291.x JOIN (SELECT 1 AS x) t292 ON t1.x = t292.x JOIN (SELECT 1 AS x) t293 ON t1.x = t293.x JOIN (SELECT 1 AS x) t294 ON t1.x = t294.x JOIN (SELECT 1 AS x) t295 ON t1.x = t295.x JOIN (SELECT 1 AS x) t296 ON t1.x = t296.x JOIN (SELECT 1 AS x) t297 ON t1.x = t297.x JOIN (SELECT 1 AS x) t298 ON t1.x = t298.x JOIN (SELECT 1 AS x) t299 ON t1.x = t299.x JOIN (SELECT 1 AS x) t300 ON t1.x = t300.x JOIN (SELECT 1 AS x) t301 ON t1.x = t301.x JOIN (SELECT 1 AS x) t302 ON t1.x = t302.x JOIN (SELECT 1 AS x) t303 ON t1.x = t303.x JOIN (SELECT 1 AS x) t304 ON t1.x = t304.x JOIN (SELECT 1 AS x) t305 ON t1.x = t305.x JOIN (SELECT 1 AS x) t306 ON t1.x = t306.x JOIN (SELECT 1 AS x) t307 ON t1.x = t307.x JOIN (SELECT 1 AS x) t308 ON t1.x = t308.x JOIN (SELECT 1 AS x) t309 ON t1.x = t309.x JOIN (SELECT 1 AS x) t310 ON t1.x = t310.x JOIN (SELECT 1 AS x) t311 ON t1.x = t311.x JOIN (SELECT 1 AS x) t312 ON t1.x = t312.x JOIN (SELECT 1 AS x) t313 ON t1.x = t313.x JOIN (SELECT 1 AS x) t314 ON t1.x = t314.x JOIN (SELECT 1 AS x) t315 ON t1.x = t315.x JOIN (SELECT 1 AS x) t316 ON t1.x = t316.x JOIN (SELECT 1 AS x) t317 ON t1.x = t317.x JOIN (SELECT 1 AS x) t318 ON t1.x = t318.x JOIN (SELECT 1 AS x) t319 ON t1.x = t319.x JOIN (SELECT 1 AS x) t320 ON t1.x = t320.x JOIN (SELECT 1 AS x) t321 ON t1.x = t321.x JOIN (SELECT 1 AS x) t322 ON t1.x = t322.x JOIN (SELECT 1 AS x) t323 ON t1.x = t323.x JOIN (SELECT 1 AS x) t324 ON t1.x = t324.x JOIN (SELECT 1 AS x) t325 ON t1.x = t325.x JOIN (SELECT 1 AS x) t326 ON t1.x = t326.x JOIN (SELECT 1 AS x) t327 ON t1.x = t327.x JOIN (SELECT 1 AS x) t328 ON t1.x = t328.x JOIN (SELECT 1 AS x) t329 ON t1.x = t329.x JOIN (SELECT 1 AS x) t330 ON t1.x = t330.x JOIN (SELECT 1 AS x) t331 ON t1.x = t331.x JOIN (SELECT 1 AS x) t332 ON t1.x = t332.x JOIN (SELECT 1 AS x) t333 ON t1.x = t333.x JOIN (SELECT 1 AS x) t334 ON t1.x = t334.x JOIN (SELECT 1 AS x) t335 ON t1.x = t335.x JOIN (SELECT 1 AS x) t336 ON t1.x = t336.x JOIN (SELECT 1 AS x) t337 ON t1.x = t337.x JOIN (SELECT 1 AS x) t338 ON t1.x = t338.x JOIN (SELECT 1 AS x) t339 ON t1.x = t339.x JOIN (SELECT 1 AS x) t340 ON t1.x = t340.x JOIN (SELECT 1 AS x) t341 ON t1.x = t341.x JOIN (SELECT 1 AS x) t342 ON t1.x = t342.x JOIN (SELECT 1 AS x) t343 ON t1.x = t343.x JOIN (SELECT 1 AS x) t344 ON t1.x = t344.x JOIN (SELECT 1 AS x) t345 ON t1.x = t345.x JOIN (SELECT 1 AS x) t346 ON t1.x = t346.x JOIN (SELECT 1 AS x) t347 ON t1.x = t347.x JOIN (SELECT 1 AS x) t348 ON t1.x = t348.x JOIN (SELECT 1 AS x) t349 ON t1.x = t349.x JOIN (SELECT 1 AS x) t350 ON t1.x = t350.x JOIN (SELECT 1 AS x) t351 ON t1.x = t351.x JOIN (SELECT 1 AS x) t352 ON t1.x = t352.x JOIN (SELECT 1 AS x) t353 ON t1.x = t353.x JOIN (SELECT 1 AS x) t354 ON t1.x = t354.x JOIN (SELECT 1 AS x) t355 ON t1.x = t355.x JOIN (SELECT 1 AS x) t356 ON t1.x = t356.x JOIN (SELECT 1 AS x) t357 ON t1.x = t357.x JOIN (SELECT 1 AS x) t358 ON t1.x = t358.x JOIN (SELECT 1 AS x) t359 ON t1.x = t359.x JOIN (SELECT 1 AS x) t360 ON t1.x = t360.x JOIN (SELECT 1 AS x) t361 ON t1.x = t361.x JOIN (SELECT 1 AS x) t362 ON t1.x = t362.x JOIN (SELECT 1 AS x) t363 ON t1.x = t363.x JOIN (SELECT 1 AS x) t364 ON t1.x = t364.x JOIN (SELECT 1 AS x) t365 ON t1.x = t365.x JOIN (SELECT 1 AS x) t366 ON t1.x = t366.x JOIN (SELECT 1 AS x) t367 ON t1.x = t367.x JOIN (SELECT 1 AS x) t368 ON t1.x = t368.x JOIN (SELECT 1 AS x) t369 ON t1.x = t369.x JOIN (SELECT 1 AS x) t370 ON t1.x = t370.x JOIN (SELECT 1 AS x) t371 ON t1.x = t371.x JOIN (SELECT 1 AS x) t372 ON t1.x = t372.x JOIN (SELECT 1 AS x) t373 ON t1.x = t373.x JOIN (SELECT 1 AS x) t374 ON t1.x = t374.x JOIN (SELECT 1 AS x) t375 ON t1.x = t375.x JOIN (SELECT 1 AS x) t376 ON t1.x = t376.x JOIN (SELECT 1 AS x) t377 ON t1.x = t377.x JOIN (SELECT 1 AS x) t378 ON t1.x = t378.x JOIN (SELECT 1 AS x) t379 ON t1.x = t379.x JOIN (SELECT 1 AS x) t380 ON t1.x = t380.x JOIN (SELECT 1 AS x) t381 ON t1.x = t381.x JOIN (SELECT 1 AS x) t382 ON t1.x = t382.x JOIN (SELECT 1 AS x) t383 ON t1.x = t383.x JOIN (SELECT 1 AS x) t384 ON t1.x = t384.x JOIN (SELECT 1 AS x) t385 ON t1.x = t385.x JOIN (SELECT 1 AS x) t386 ON t1.x = t386.x JOIN (SELECT 1 AS x) t387 ON t1.x = t387.x JOIN (SELECT 1 AS x) t388 ON t1.x = t388.x JOIN (SELECT 1 AS x) t389 ON t1.x = t389.x JOIN (SELECT 1 AS x) t390 ON t1.x = t390.x JOIN (SELECT 1 AS x) t391 ON t1.x = t391.x JOIN (SELECT 1 AS x) t392 ON t1.x = t392.x JOIN (SELECT 1 AS x) t393 ON t1.x = t393.x JOIN (SELECT 1 AS x) t394 ON t1.x = t394.x JOIN (SELECT 1 AS x) t395 ON t1.x = t395.x JOIN (SELECT 1 AS x) t396 ON t1.x = t396.x JOIN (SELECT 1 AS x) t397 ON t1.x = t397.x JOIN (SELECT 1 AS x) t398 ON t1.x = t398.x JOIN (SELECT 1 AS x) t399 ON t1.x = t399.x JOIN (SELECT 1 AS x) t400 ON t1.x = t400.x JOIN (SELECT 1 AS x) t401 ON t1.x = t401.x JOIN (SELECT 1 AS x) t402 ON t1.x = t402.x JOIN (SELECT 1 AS x) t403 ON t1.x = t403.x JOIN (SELECT 1 AS x) t404 ON t1.x = t404.x JOIN (SELECT 1 AS x) t405 ON t1.x = t405.x JOIN (SELECT 1 AS x) t406 ON t1.x = t406.x JOIN (SELECT 1 AS x) t407 ON t1.x = t407.x JOIN (SELECT 1 AS x) t408 ON t1.x = t408.x JOIN (SELECT 1 AS x) t409 ON t1.x = t409.x JOIN (SELECT 1 AS x) t410 ON t1.x = t410.x JOIN (SELECT 1 AS x) t411 ON t1.x = t411.x JOIN (SELECT 1 AS x) t412 ON t1.x = t412.x JOIN (SELECT 1 AS x) t413 ON t1.x = t413.x JOIN (SELECT 1 AS x) t414 ON t1.x = t414.x JOIN (SELECT 1 AS x) t415 ON t1.x = t415.x JOIN (SELECT 1 AS x) t416 ON t1.x = t416.x JOIN (SELECT 1 AS x) t417 ON t1.x = t417.x JOIN (SELECT 1 AS x) t418 ON t1.x = t418.x JOIN (SELECT 1 AS x) t419 ON t1.x = t419.x JOIN (SELECT 1 AS x) t420 ON t1.x = t420.x JOIN (SELECT 1 AS x) t421 ON t1.x = t421.x JOIN (SELECT 1 AS x) t422 ON t1.x = t422.x JOIN (SELECT 1 AS x) t423 ON t1.x = t423.x JOIN (SELECT 1 AS x) t424 ON t1.x = t424.x JOIN (SELECT 1 AS x) t425 ON t1.x = t425.x JOIN (SELECT 1 AS x) t426 ON t1.x = t426.x JOIN (SELECT 1 AS x) t427 ON t1.x = t427.x JOIN (SELECT 1 AS x) t428 ON t1.x = t428.x JOIN (SELECT 1 AS x) t429 ON t1.x = t429.x JOIN (SELECT 1 AS x) t430 ON t1.x = t430.x JOIN (SELECT 1 AS x) t431 ON t1.x = t431.x JOIN (SELECT 1 AS x) t432 ON t1.x = t432.x JOIN (SELECT 1 AS x) t433 ON t1.x = t433.x JOIN (SELECT 1 AS x) t434 ON t1.x = t434.x JOIN (SELECT 1 AS x) t435 ON t1.x = t435.x JOIN (SELECT 1 AS x) t436 ON t1.x = t436.x JOIN (SELECT 1 AS x) t437 ON t1.x = t437.x JOIN (SELECT 1 AS x) t438 ON t1.x = t438.x JOIN (SELECT 1 AS x) t439 ON t1.x = t439.x JOIN (SELECT 1 AS x) t440 ON t1.x = t440.x JOIN (SELECT 1 AS x) t441 ON t1.x = t441.x JOIN (SELECT 1 AS x) t442 ON t1.x = t442.x JOIN (SELECT 1 AS x) t443 ON t1.x = t443.x JOIN (SELECT 1 AS x) t444 ON t1.x = t444.x JOIN (SELECT 1 AS x) t445 ON t1.x = t445.x JOIN (SELECT 1 AS x) t446 ON t1.x = t446.x JOIN (SELECT 1 AS x) t447 ON t1.x = t447.x JOIN (SELECT 1 AS x) t448 ON t1.x = t448.x JOIN (SELECT 1 AS x) t449 ON t1.x = t449.x JOIN (SELECT 1 AS x) t450 ON t1.x = t450.x JOIN (SELECT 1 AS x) t451 ON t1.x = t451.x JOIN (SELECT 1 AS x) t452 ON t1.x = t452.x JOIN (SELECT 1 AS x) t453 ON t1.x = t453.x JOIN (SELECT 1 AS x) t454 ON t1.x = t454.x JOIN (SELECT 1 AS x) t455 ON t1.x = t455.x JOIN (SELECT 1 AS x) t456 ON t1.x = t456.x JOIN (SELECT 1 AS x) t457 ON t1.x = t457.x JOIN (SELECT 1 AS x) t458 ON t1.x = t458.x JOIN (SELECT 1 AS x) t459 ON t1.x = t459.x JOIN (SELECT 1 AS x) t460 ON t1.x = t460.x JOIN (SELECT 1 AS x) t461 ON t1.x = t461.x JOIN (SELECT 1 AS x) t462 ON t1.x = t462.x JOIN (SELECT 1 AS x) t463 ON t1.x = t463.x JOIN (SELECT 1 AS x) t464 ON t1.x = t464.x JOIN (SELECT 1 AS x) t465 ON t1.x = t465.x JOIN (SELECT 1 AS x) t466 ON t1.x = t466.x JOIN (SELECT 1 AS x) t467 ON t1.x = t467.x JOIN (SELECT 1 AS x) t468 ON t1.x = t468.x JOIN (SELECT 1 AS x) t469 ON t1.x = t469.x JOIN (SELECT 1 AS x) t470 ON t1.x = t470.x JOIN (SELECT 1 AS x) t471 ON t1.x = t471.x JOIN (SELECT 1 AS x) t472 ON t1.x = t472.x JOIN (SELECT 1 AS x) t473 ON t1.x = t473.x JOIN (SELECT 1 AS x) t474 ON t1.x = t474.x JOIN (SELECT 1 AS x) t475 ON t1.x = t475.x JOIN (SELECT 1 AS x) t476 ON t1.x = t476.x JOIN (SELECT 1 AS x) t477 ON t1.x = t477.x JOIN (SELECT 1 AS x) t478 ON t1.x = t478.x JOIN (SELECT 1 AS x) t479 ON t1.x = t479.x JOIN (SELECT 1 AS x) t480 ON t1.x = t480.x JOIN (SELECT 1 AS x) t481 ON t1.x = t481.x JOIN (SELECT 1 AS x) t482 ON t1.x = t482.x JOIN (SELECT 1 AS x) t483 ON t1.x = t483.x JOIN (SELECT 1 AS x) t484 ON t1.x = t484.x JOIN (SELECT 1 AS x) t485 ON t1.x = t485.x JOIN (SELECT 1 AS x) t486 ON t1.x = t486.x JOIN (SELECT 1 AS x) t487 ON t1.x = t487.x JOIN (SELECT 1 AS x) t488 ON t1.x = t488.x JOIN (SELECT 1 AS x) t489 ON t1.x = t489.x JOIN (SELECT 1 AS x) t490 ON t1.x = t490.x JOIN (SELECT 1 AS x) t491 ON t1.x = t491.x JOIN (SELECT 1 AS x) t492 ON t1.x = t492.x JOIN (SELECT 1 AS x) t493 ON t1.x = t493.x JOIN (SELECT 1 AS x) t494 ON t1.x = t494.x JOIN (SELECT 1 AS x) t495 ON t1.x = t495.x JOIN (SELECT 1 AS x) t496 ON t1.x = t496.x JOIN (SELECT 1 AS x) t497 ON t1.x = t497.x JOIN (SELECT 1 AS x) t498 ON t1.x = t498.x JOIN (SELECT 1 AS x) t499 ON t1.x = t499.x JOIN (SELECT 1 AS x) t500 ON t1.x = t500.x JOIN (SELECT 1 AS x) t501 ON t1.x = t501.x JOIN (SELECT 1 AS x) t502 ON t1.x = t502.x JOIN (SELECT 1 AS x) t503 ON t1.x = t503.x JOIN (SELECT 1 AS x) t504 ON t1.x = t504.x JOIN (SELECT 1 AS x) t505 ON t1.x = t505.x JOIN (SELECT 1 AS x) t506 ON t1.x = t506.x JOIN (SELECT 1 AS x) t507 ON t1.x = t507.x JOIN (SELECT 1 AS x) t508 ON t1.x = t508.x JOIN (SELECT 1 AS x) t509 ON t1.x = t509.x JOIN (SELECT 1 AS x) t510 ON t1.x = t510.x JOIN (SELECT 1 AS x) t511 ON t1.x = t511.x JOIN (SELECT 1 AS x) t512 ON t1.x = t512.x JOIN (SELECT 1 AS x) t513 ON t1.x = t513.x JOIN (SELECT 1 AS x) t514 ON t1.x = t514.x JOIN (SELECT 1 AS x) t515 ON t1.x = t515.x JOIN (SELECT 1 AS x) t516 ON t1.x = t516.x JOIN (SELECT 1 AS x) t517 ON t1.x = t517.x JOIN (SELECT 1 AS x) t518 ON t1.x = t518.x JOIN (SELECT 1 AS x) t519 ON t1.x = t519.x JOIN (SELECT 1 AS x) t520 ON t1.x = t520.x JOIN (SELECT 1 AS x) t521 ON t1.x = t521.x JOIN (SELECT 1 AS x) t522 ON t1.x = t522.x JOIN (SELECT 1 AS x) t523 ON t1.x = t523.x JOIN (SELECT 1 AS x) t524 ON t1.x = t524.x JOIN (SELECT 1 AS x) t525 ON t1.x = t525.x JOIN (SELECT 1 AS x) t526 ON t1.x = t526.x JOIN (SELECT 1 AS x) t527 ON t1.x = t527.x JOIN (SELECT 1 AS x) t528 ON t1.x = t528.x JOIN (SELECT 1 AS x) t529 ON t1.x = t529.x JOIN (SELECT 1 AS x) t530 ON t1.x = t530.x JOIN (SELECT 1 AS x) t531 ON t1.x = t531.x JOIN (SELECT 1 AS x) t532 ON t1.x = t532.x JOIN (SELECT 1 AS x) t533 ON t1.x = t533.x JOIN (SELECT 1 AS x) t534 ON t1.x = t534.x JOIN (SELECT 1 AS x) t535 ON t1.x = t535.x JOIN (SELECT 1 AS x) t536 ON t1.x = t536.x JOIN (SELECT 1 AS x) t537 ON t1.x = t537.x JOIN (SELECT 1 AS x) t538 ON t1.x = t538.x JOIN (SELECT 1 AS x) t539 ON t1.x = t539.x JOIN (SELECT 1 AS x) t540 ON t1.x = t540.x JOIN (SELECT 1 AS x) t541 ON t1.x = t541.x JOIN (SELECT 1 AS x) t542 ON t1.x = t542.x JOIN (SELECT 1 AS x) t543 ON t1.x = t543.x JOIN (SELECT 1 AS x) t544 ON t1.x = t544.x JOIN (SELECT 1 AS x) t545 ON t1.x = t545.x JOIN (SELECT 1 AS x) t546 ON t1.x = t546.x JOIN (SELECT 1 AS x) t547 ON t1.x = t547.x JOIN (SELECT 1 AS x) t548 ON t1.x = t548.x JOIN (SELECT 1 AS x) t549 ON t1.x = t549.x JOIN (SELECT 1 AS x) t550 ON t1.x = t550.x JOIN (SELECT 1 AS x) t551 ON t1.x = t551.x JOIN (SELECT 1 AS x) t552 ON t1.x = t552.x JOIN (SELECT 1 AS x) t553 ON t1.x = t553.x JOIN (SELECT 1 AS x) t554 ON t1.x = t554.x JOIN (SELECT 1 AS x) t555 ON t1.x = t555.x JOIN (SELECT 1 AS x) t556 ON t1.x = t556.x JOIN (SELECT 1 AS x) t557 ON t1.x = t557.x JOIN (SELECT 1 AS x) t558 ON t1.x = t558.x JOIN (SELECT 1 AS x) t559 ON t1.x = t559.x JOIN (SELECT 1 AS x) t560 ON t1.x = t560.x JOIN (SELECT 1 AS x) t561 ON t1.x = t561.x JOIN (SELECT 1 AS x) t562 ON t1.x = t562.x JOIN (SELECT 1 AS x) t563 ON t1.x = t563.x JOIN (SELECT 1 AS x) t564 ON t1.x = t564.x JOIN (SELECT 1 AS x) t565 ON t1.x = t565.x JOIN (SELECT 1 AS x) t566 ON t1.x = t566.x JOIN (SELECT 1 AS x) t567 ON t1.x = t567.x JOIN (SELECT 1 AS x) t568 ON t1.x = t568.x JOIN (SELECT 1 AS x) t569 ON t1.x = t569.x JOIN (SELECT 1 AS x) t570 ON t1.x = t570.x JOIN (SELECT 1 AS x) t571 ON t1.x = t571.x JOIN (SELECT 1 AS x) t572 ON t1.x = t572.x JOIN (SELECT 1 AS x) t573 ON t1.x = t573.x JOIN (SELECT 1 AS x) t574 ON t1.x = t574.x JOIN (SELECT 1 AS x) t575 ON t1.x = t575.x JOIN (SELECT 1 AS x) t576 ON t1.x = t576.x JOIN (SELECT 1 AS x) t577 ON t1.x = t577.x JOIN (SELECT 1 AS x) t578 ON t1.x = t578.x JOIN (SELECT 1 AS x) t579 ON t1.x = t579.x JOIN (SELECT 1 AS x) t580 ON t1.x = t580.x JOIN (SELECT 1 AS x) t581 ON t1.x = t581.x JOIN (SELECT 1 AS x) t582 ON t1.x = t582.x JOIN (SELECT 1 AS x) t583 ON t1.x = t583.x JOIN (SELECT 1 AS x) t584 ON t1.x = t584.x JOIN (SELECT 1 AS x) t585 ON t1.x = t585.x JOIN (SELECT 1 AS x) t586 ON t1.x = t586.x JOIN (SELECT 1 AS x) t587 ON t1.x = t587.x JOIN (SELECT 1 AS x) t588 ON t1.x = t588.x JOIN (SELECT 1 AS x) t589 ON t1.x = t589.x JOIN (SELECT 1 AS x) t590 ON t1.x = t590.x JOIN (SELECT 1 AS x) t591 ON t1.x = t591.x JOIN (SELECT 1 AS x) t592 ON t1.x = t592.x JOIN (SELECT 1 AS x) t593 ON t1.x = t593.x JOIN (SELECT 1 AS x) t594 ON t1.x = t594.x JOIN (SELECT 1 AS x) t595 ON t1.x = t595.x JOIN (SELECT 1 AS x) t596 ON t1.x = t596.x JOIN (SELECT 1 AS x) t597 ON t1.x = t597.x JOIN (SELECT 1 AS x) t598 ON t1.x = t598.x JOIN (SELECT 1 AS x) t599 ON t1.x = t599.x JOIN (SELECT 1 AS x) t600 ON t1.x = t600.x JOIN (SELECT 1 AS x) t601 ON t1.x = t601.x JOIN (SELECT 1 AS x) t602 ON t1.x = t602.x JOIN (SELECT 1 AS x) t603 ON t1.x = t603.x JOIN (SELECT 1 AS x) t604 ON t1.x = t604.x JOIN (SELECT 1 AS x) t605 ON t1.x = t605.x JOIN (SELECT 1 AS x) t606 ON t1.x = t606.x JOIN (SELECT 1 AS x) t607 ON t1.x = t607.x JOIN (SELECT 1 AS x) t608 ON t1.x = t608.x JOIN (SELECT 1 AS x) t609 ON t1.x = t609.x JOIN (SELECT 1 AS x) t610 ON t1.x = t610.x JOIN (SELECT 1 AS x) t611 ON t1.x = t611.x JOIN (SELECT 1 AS x) t612 ON t1.x = t612.x JOIN (SELECT 1 AS x) t613 ON t1.x = t613.x JOIN (SELECT 1 AS x) t614 ON t1.x = t614.x JOIN (SELECT 1 AS x) t615 ON t1.x = t615.x JOIN (SELECT 1 AS x) t616 ON t1.x = t616.x JOIN (SELECT 1 AS x) t617 ON t1.x = t617.x JOIN (SELECT 1 AS x) t618 ON t1.x = t618.x JOIN (SELECT 1 AS x) t619 ON t1.x = t619.x JOIN (SELECT 1 AS x) t620 ON t1.x = t620.x JOIN (SELECT 1 AS x) t621 ON t1.x = t621.x JOIN (SELECT 1 AS x) t622 ON t1.x = t622.x JOIN (SELECT 1 AS x) t623 ON t1.x = t623.x JOIN (SELECT 1 AS x) t624 ON t1.x = t624.x JOIN (SELECT 1 AS x) t625 ON t1.x = t625.x JOIN (SELECT 1 AS x) t626 ON t1.x = t626.x JOIN (SELECT 1 AS x) t627 ON t1.x = t627.x JOIN (SELECT 1 AS x) t628 ON t1.x = t628.x JOIN (SELECT 1 AS x) t629 ON t1.x = t629.x JOIN (SELECT 1 AS x) t630 ON t1.x = t630.x JOIN (SELECT 1 AS x) t631 ON t1.x = t631.x JOIN (SELECT 1 AS x) t632 ON t1.x = t632.x JOIN (SELECT 1 AS x) t633 ON t1.x = t633.x JOIN (SELECT 1 AS x) t634 ON t1.x = t634.x JOIN (SELECT 1 AS x) t635 ON t1.x = t635.x JOIN (SELECT 1 AS x) t636 ON t1.x = t636.x JOIN (SELECT 1 AS x) t637 ON t1.x = t637.x JOIN (SELECT 1 AS x) t638 ON t1.x = t638.x JOIN (SELECT 1 AS x) t639 ON t1.x = t639.x JOIN (SELECT 1 AS x) t640 ON t1.x = t640.x JOIN (SELECT 1 AS x) t641 ON t1.x = t641.x JOIN (SELECT 1 AS x) t642 ON t1.x = t642.x JOIN (SELECT 1 AS x) t643 ON t1.x = t643.x JOIN (SELECT 1 AS x) t644 ON t1.x = t644.x JOIN (SELECT 1 AS x) t645 ON t1.x = t645.x JOIN (SELECT 1 AS x) t646 ON t1.x = t646.x JOIN (SELECT 1 AS x) t647 ON t1.x = t647.x JOIN (SELECT 1 AS x) t648 ON t1.x = t648.x JOIN (SELECT 1 AS x) t649 ON t1.x = t649.x JOIN (SELECT 1 AS x) t650 ON t1.x = t650.x JOIN (SELECT 1 AS x) t651 ON t1.x = t651.x JOIN (SELECT 1 AS x) t652 ON t1.x = t652.x JOIN (SELECT 1 AS x) t653 ON t1.x = t653.x JOIN (SELECT 1 AS x) t654 ON t1.x = t654.x JOIN (SELECT 1 AS x) t655 ON t1.x = t655.x JOIN (SELECT 1 AS x) t656 ON t1.x = t656.x JOIN (SELECT 1 AS x) t657 ON t1.x = t657.x JOIN (SELECT 1 AS x) t658 ON t1.x = t658.x JOIN (SELECT 1 AS x) t659 ON t1.x = t659.x JOIN (SELECT 1 AS x) t660 ON t1.x = t660.x JOIN (SELECT 1 AS x) t661 ON t1.x = t661.x JOIN (SELECT 1 AS x) t662 ON t1.x = t662.x JOIN (SELECT 1 AS x) t663 ON t1.x = t663.x JOIN (SELECT 1 AS x) t664 ON t1.x = t664.x JOIN (SELECT 1 AS x) t665 ON t1.x = t665.x JOIN (SELECT 1 AS x) t666 ON t1.x = t666.x JOIN (SELECT 1 AS x) t667 ON t1.x = t667.x JOIN (SELECT 1 AS x) t668 ON t1.x = t668.x JOIN (SELECT 1 AS x) t669 ON t1.x = t669.x JOIN (SELECT 1 AS x) t670 ON t1.x = t670.x JOIN (SELECT 1 AS x) t671 ON t1.x = t671.x JOIN (SELECT 1 AS x) t672 ON t1.x = t672.x JOIN (SELECT 1 AS x) t673 ON t1.x = t673.x JOIN (SELECT 1 AS x) t674 ON t1.x = t674.x JOIN (SELECT 1 AS x) t675 ON t1.x = t675.x JOIN (SELECT 1 AS x) t676 ON t1.x = t676.x JOIN (SELECT 1 AS x) t677 ON t1.x = t677.x JOIN (SELECT 1 AS x) t678 ON t1.x = t678.x JOIN (SELECT 1 AS x) t679 ON t1.x = t679.x JOIN (SELECT 1 AS x) t680 ON t1.x = t680.x JOIN (SELECT 1 AS x) t681 ON t1.x = t681.x JOIN (SELECT 1 AS x) t682 ON t1.x = t682.x JOIN (SELECT 1 AS x) t683 ON t1.x = t683.x JOIN (SELECT 1 AS x) t684 ON t1.x = t684.x JOIN (SELECT 1 AS x) t685 ON t1.x = t685.x JOIN (SELECT 1 AS x) t686 ON t1.x = t686.x JOIN (SELECT 1 AS x) t687 ON t1.x = t687.x JOIN (SELECT 1 AS x) t688 ON t1.x = t688.x JOIN (SELECT 1 AS x) t689 ON t1.x = t689.x JOIN (SELECT 1 AS x) t690 ON t1.x = t690.x JOIN (SELECT 1 AS x) t691 ON t1.x = t691.x JOIN (SELECT 1 AS x) t692 ON t1.x = t692.x JOIN (SELECT 1 AS x) t693 ON t1.x = t693.x JOIN (SELECT 1 AS x) t694 ON t1.x = t694.x JOIN (SELECT 1 AS x) t695 ON t1.x = t695.x JOIN (SELECT 1 AS x) t696 ON t1.x = t696.x JOIN (SELECT 1 AS x) t697 ON t1.x = t697.x JOIN (SELECT 1 AS x) t698 ON t1.x = t698.x JOIN (SELECT 1 AS x) t699 ON t1.x = t699.x JOIN (SELECT 1 AS x) t700 ON t1.x = t700.x JOIN (SELECT 1 AS x) t701 ON t1.x = t701.x JOIN (SELECT 1 AS x) t702 ON t1.x = t702.x JOIN (SELECT 1 AS x) t703 ON t1.x = t703.x JOIN (SELECT 1 AS x) t704 ON t1.x = t704.x JOIN (SELECT 1 AS x) t705 ON t1.x = t705.x JOIN (SELECT 1 AS x) t706 ON t1.x = t706.x JOIN (SELECT 1 AS x) t707 ON t1.x = t707.x JOIN (SELECT 1 AS x) t708 ON t1.x = t708.x JOIN (SELECT 1 AS x) t709 ON t1.x = t709.x JOIN (SELECT 1 AS x) t710 ON t1.x = t710.x JOIN (SELECT 1 AS x) t711 ON t1.x = t711.x JOIN (SELECT 1 AS x) t712 ON t1.x = t712.x JOIN (SELECT 1 AS x) t713 ON t1.x = t713.x JOIN (SELECT 1 AS x) t714 ON t1.x = t714.x JOIN (SELECT 1 AS x) t715 ON t1.x = t715.x JOIN (SELECT 1 AS x) t716 ON t1.x = t716.x JOIN (SELECT 1 AS x) t717 ON t1.x = t717.x JOIN (SELECT 1 AS x) t718 ON t1.x = t718.x JOIN (SELECT 1 AS x) t719 ON t1.x = t719.x JOIN (SELECT 1 AS x) t720 ON t1.x = t720.x JOIN (SELECT 1 AS x) t721 ON t1.x = t721.x JOIN (SELECT 1 AS x) t722 ON t1.x = t722.x JOIN (SELECT 1 AS x) t723 ON t1.x = t723.x JOIN (SELECT 1 AS x) t724 ON t1.x = t724.x JOIN (SELECT 1 AS x) t725 ON t1.x = t725.x JOIN (SELECT 1 AS x) t726 ON t1.x = t726.x JOIN (SELECT 1 AS x) t727 ON t1.x = t727.x JOIN (SELECT 1 AS x) t728 ON t1.x = t728.x JOIN (SELECT 1 AS x) t729 ON t1.x = t729.x JOIN (SELECT 1 AS x) t730 ON t1.x = t730.x JOIN (SELECT 1 AS x) t731 ON t1.x = t731.x JOIN (SELECT 1 AS x) t732 ON t1.x = t732.x JOIN (SELECT 1 AS x) t733 ON t1.x = t733.x JOIN (SELECT 1 AS x) t734 ON t1.x = t734.x JOIN (SELECT 1 AS x) t735 ON t1.x = t735.x JOIN (SELECT 1 AS x) t736 ON t1.x = t736.x JOIN (SELECT 1 AS x) t737 ON t1.x = t737.x JOIN (SELECT 1 AS x) t738 ON t1.x = t738.x JOIN (SELECT 1 AS x) t739 ON t1.x = t739.x JOIN (SELECT 1 AS x) t740 ON t1.x = t740.x JOIN (SELECT 1 AS x) t741 ON t1.x = t741.x JOIN (SELECT 1 AS x) t742 ON t1.x = t742.x JOIN (SELECT 1 AS x) t743 ON t1.x = t743.x JOIN (SELECT 1 AS x) t744 ON t1.x = t744.x JOIN (SELECT 1 AS x) t745 ON t1.x = t745.x JOIN (SELECT 1 AS x) t746 ON t1.x = t746.x JOIN (SELECT 1 AS x) t747 ON t1.x = t747.x JOIN (SELECT 1 AS x) t748 ON t1.x = t748.x JOIN (SELECT 1 AS x) t749 ON t1.x = t749.x JOIN (SELECT 1 AS x) t750 ON t1.x = t750.x JOIN (SELECT 1 AS x) t751 ON t1.x = t751.x JOIN (SELECT 1 AS x) t752 ON t1.x = t752.x JOIN (SELECT 1 AS x) t753 ON t1.x = t753.x JOIN (SELECT 1 AS x) t754 ON t1.x = t754.x JOIN (SELECT 1 AS x) t755 ON t1.x = t755.x JOIN (SELECT 1 AS x) t756 ON t1.x = t756.x JOIN (SELECT 1 AS x) t757 ON t1.x = t757.x JOIN (SELECT 1 AS x) t758 ON t1.x = t758.x JOIN (SELECT 1 AS x) t759 ON t1.x = t759.x JOIN (SELECT 1 AS x) t760 ON t1.x = t760.x JOIN (SELECT 1 AS x) t761 ON t1.x = t761.x JOIN (SELECT 1 AS x) t762 ON t1.x = t762.x JOIN (SELECT 1 AS x) t763 ON t1.x = t763.x JOIN (SELECT 1 AS x) t764 ON t1.x = t764.x JOIN (SELECT 1 AS x) t765 ON t1.x = t765.x JOIN (SELECT 1 AS x) t766 ON t1.x = t766.x JOIN (SELECT 1 AS x) t767 ON t1.x = t767.x JOIN (SELECT 1 AS x) t768 ON t1.x = t768.x JOIN (SELECT 1 AS x) t769 ON t1.x = t769.x JOIN (SELECT 1 AS x) t770 ON t1.x = t770.x JOIN (SELECT 1 AS x) t771 ON t1.x = t771.x JOIN (SELECT 1 AS x) t772 ON t1.x = t772.x JOIN (SELECT 1 AS x) t773 ON t1.x = t773.x JOIN (SELECT 1 AS x) t774 ON t1.x = t774.x JOIN (SELECT 1 AS x) t775 ON t1.x = t775.x JOIN (SELECT 1 AS x) t776 ON t1.x = t776.x JOIN (SELECT 1 AS x) t777 ON t1.x = t777.x JOIN (SELECT 1 AS x) t778 ON t1.x = t778.x JOIN (SELECT 1 AS x) t779 ON t1.x = t779.x JOIN (SELECT 1 AS x) t780 ON t1.x = t780.x JOIN (SELECT 1 AS x) t781 ON t1.x = t781.x JOIN (SELECT 1 AS x) t782 ON t1.x = t782.x JOIN (SELECT 1 AS x) t783 ON t1.x = t783.x JOIN (SELECT 1 AS x) t784 ON t1.x = t784.x JOIN (SELECT 1 AS x) t785 ON t1.x = t785.x JOIN (SELECT 1 AS x) t786 ON t1.x = t786.x JOIN (SELECT 1 AS x) t787 ON t1.x = t787.x JOIN (SELECT 1 AS x) t788 ON t1.x = t788.x JOIN (SELECT 1 AS x) t789 ON t1.x = t789.x JOIN (SELECT 1 AS x) t790 ON t1.x = t790.x JOIN (SELECT 1 AS x) t791 ON t1.x = t791.x JOIN (SELECT 1 AS x) t792 ON t1.x = t792.x JOIN (SELECT 1 AS x) t793 ON t1.x = t793.x JOIN (SELECT 1 AS x) t794 ON t1.x = t794.x JOIN (SELECT 1 AS x) t795 ON t1.x = t795.x JOIN (SELECT 1 AS x) t796 ON t1.x = t796.x JOIN (SELECT 1 AS x) t797 ON t1.x = t797.x JOIN (SELECT 1 AS x) t798 ON t1.x = t798.x JOIN (SELECT 1 AS x) t799 ON t1.x = t799.x JOIN (SELECT 1 AS x) t800 ON t1.x = t800.x JOIN (SELECT 1 AS x) t801 ON t1.x = t801.x JOIN (SELECT 1 AS x) t802 ON t1.x = t802.x JOIN (SELECT 1 AS x) t803 ON t1.x = t803.x JOIN (SELECT 1 AS x) t804 ON t1.x = t804.x JOIN (SELECT 1 AS x) t805 ON t1.x = t805.x JOIN (SELECT 1 AS x) t806 ON t1.x = t806.x JOIN (SELECT 1 AS x) t807 ON t1.x = t807.x JOIN (SELECT 1 AS x) t808 ON t1.x = t808.x JOIN (SELECT 1 AS x) t809 ON t1.x = t809.x JOIN (SELECT 1 AS x) t810 ON t1.x = t810.x JOIN (SELECT 1 AS x) t811 ON t1.x = t811.x JOIN (SELECT 1 AS x) t812 ON t1.x = t812.x JOIN (SELECT 1 AS x) t813 ON t1.x = t813.x JOIN (SELECT 1 AS x) t814 ON t1.x = t814.x JOIN (SELECT 1 AS x) t815 ON t1.x = t815.x JOIN (SELECT 1 AS x) t816 ON t1.x = t816.x JOIN (SELECT 1 AS x) t817 ON t1.x = t817.x JOIN (SELECT 1 AS x) t818 ON t1.x = t818.x JOIN (SELECT 1 AS x) t819 ON t1.x = t819.x JOIN (SELECT 1 AS x) t820 ON t1.x = t820.x JOIN (SELECT 1 AS x) t821 ON t1.x = t821.x JOIN (SELECT 1 AS x) t822 ON t1.x = t822.x JOIN (SELECT 1 AS x) t823 ON t1.x = t823.x JOIN (SELECT 1 AS x) t824 ON t1.x = t824.x JOIN (SELECT 1 AS x) t825 ON t1.x = t825.x JOIN (SELECT 1 AS x) t826 ON t1.x = t826.x JOIN (SELECT 1 AS x) t827 ON t1.x = t827.x JOIN (SELECT 1 AS x) t828 ON t1.x = t828.x JOIN (SELECT 1 AS x) t829 ON t1.x = t829.x JOIN (SELECT 1 AS x) t830 ON t1.x = t830.x JOIN (SELECT 1 AS x) t831 ON t1.x = t831.x JOIN (SELECT 1 AS x) t832 ON t1.x = t832.x JOIN (SELECT 1 AS x) t833 ON t1.x = t833.x JOIN (SELECT 1 AS x) t834 ON t1.x = t834.x JOIN (SELECT 1 AS x) t835 ON t1.x = t835.x JOIN (SELECT 1 AS x) t836 ON t1.x = t836.x JOIN (SELECT 1 AS x) t837 ON t1.x = t837.x JOIN (SELECT 1 AS x) t838 ON t1.x = t838.x JOIN (SELECT 1 AS x) t839 ON t1.x = t839.x JOIN (SELECT 1 AS x) t840 ON t1.x = t840.x JOIN (SELECT 1 AS x) t841 ON t1.x = t841.x JOIN (SELECT 1 AS x) t842 ON t1.x = t842.x JOIN (SELECT 1 AS x) t843 ON t1.x = t843.x JOIN (SELECT 1 AS x) t844 ON t1.x = t844.x JOIN (SELECT 1 AS x) t845 ON t1.x = t845.x JOIN (SELECT 1 AS x) t846 ON t1.x = t846.x JOIN (SELECT 1 AS x) t847 ON t1.x = t847.x JOIN (SELECT 1 AS x) t848 ON t1.x = t848.x JOIN (SELECT 1 AS x) t849 ON t1.x = t849.x JOIN (SELECT 1 AS x) t850 ON t1.x = t850.x JOIN (SELECT 1 AS x) t851 ON t1.x = t851.x JOIN (SELECT 1 AS x) t852 ON t1.x = t852.x JOIN (SELECT 1 AS x) t853 ON t1.x = t853.x JOIN (SELECT 1 AS x) t854 ON t1.x = t854.x JOIN (SELECT 1 AS x) t855 ON t1.x = t855.x JOIN (SELECT 1 AS x) t856 ON t1.x = t856.x JOIN (SELECT 1 AS x) t857 ON t1.x = t857.x JOIN (SELECT 1 AS x) t858 ON t1.x = t858.x JOIN (SELECT 1 AS x) t859 ON t1.x = t859.x JOIN (SELECT 1 AS x) t860 ON t1.x = t860.x JOIN (SELECT 1 AS x) t861 ON t1.x = t861.x JOIN (SELECT 1 AS x) t862 ON t1.x = t862.x JOIN (SELECT 1 AS x) t863 ON t1.x = t863.x JOIN (SELECT 1 AS x) t864 ON t1.x = t864.x JOIN (SELECT 1 AS x) t865 ON t1.x = t865.x JOIN (SELECT 1 AS x) t866 ON t1.x = t866.x JOIN (SELECT 1 AS x) t867 ON t1.x = t867.x JOIN (SELECT 1 AS x) t868 ON t1.x = t868.x JOIN (SELECT 1 AS x) t869 ON t1.x = t869.x JOIN (SELECT 1 AS x) t870 ON t1.x = t870.x JOIN (SELECT 1 AS x) t871 ON t1.x = t871.x JOIN (SELECT 1 AS x) t872 ON t1.x = t872.x JOIN (SELECT 1 AS x) t873 ON t1.x = t873.x JOIN (SELECT 1 AS x) t874 ON t1.x = t874.x JOIN (SELECT 1 AS x) t875 ON t1.x = t875.x JOIN (SELECT 1 AS x) t876 ON t1.x = t876.x JOIN (SELECT 1 AS x) t877 ON t1.x = t877.x JOIN (SELECT 1 AS x) t878 ON t1.x = t878.x JOIN (SELECT 1 AS x) t879 ON t1.x = t879.x JOIN (SELECT 1 AS x) t880 ON t1.x = t880.x JOIN (SELECT 1 AS x) t881 ON t1.x = t881.x JOIN (SELECT 1 AS x) t882 ON t1.x = t882.x JOIN (SELECT 1 AS x) t883 ON t1.x = t883.x JOIN (SELECT 1 AS x) t884 ON t1.x = t884.x JOIN (SELECT 1 AS x) t885 ON t1.x = t885.x JOIN (SELECT 1 AS x) t886 ON t1.x = t886.x JOIN (SELECT 1 AS x) t887 ON t1.x = t887.x JOIN (SELECT 1 AS x) t888 ON t1.x = t888.x JOIN (SELECT 1 AS x) t889 ON t1.x = t889.x JOIN (SELECT 1 AS x) t890 ON t1.x = t890.x JOIN (SELECT 1 AS x) t891 ON t1.x = t891.x JOIN (SELECT 1 AS x) t892 ON t1.x = t892.x JOIN (SELECT 1 AS x) t893 ON t1.x = t893.x JOIN (SELECT 1 AS x) t894 ON t1.x = t894.x JOIN (SELECT 1 AS x) t895 ON t1.x = t895.x JOIN (SELECT 1 AS x) t896 ON t1.x = t896.x JOIN (SELECT 1 AS x) t897 ON t1.x = t897.x JOIN (SELECT 1 AS x) t898 ON t1.x = t898.x JOIN (SELECT 1 AS x) t899 ON t1.x = t899.x JOIN (SELECT 1 AS x) t900 ON t1.x = t900.x JOIN (SELECT 1 AS x) t901 ON t1.x = t901.x JOIN (SELECT 1 AS x) t902 ON t1.x = t902.x JOIN (SELECT 1 AS x) t903 ON t1.x = t903.x JOIN (SELECT 1 AS x) t904 ON t1.x = t904.x JOIN (SELECT 1 AS x) t905 ON t1.x = t905.x JOIN (SELECT 1 AS x) t906 ON t1.x = t906.x JOIN (SELECT 1 AS x) t907 ON t1.x = t907.x JOIN (SELECT 1 AS x) t908 ON t1.x = t908.x JOIN (SELECT 1 AS x) t909 ON t1.x = t909.x JOIN (SELECT 1 AS x) t910 ON t1.x = t910.x JOIN (SELECT 1 AS x) t911 ON t1.x = t911.x JOIN (SELECT 1 AS x) t912 ON t1.x = t912.x JOIN (SELECT 1 AS x) t913 ON t1.x = t913.x JOIN (SELECT 1 AS x) t914 ON t1.x = t914.x JOIN (SELECT 1 AS x) t915 ON t1.x = t915.x JOIN (SELECT 1 AS x) t916 ON t1.x = t916.x JOIN (SELECT 1 AS x) t917 ON t1.x = t917.x JOIN (SELECT 1 AS x) t918 ON t1.x = t918.x JOIN (SELECT 1 AS x) t919 ON t1.x = t919.x JOIN (SELECT 1 AS x) t920 ON t1.x = t920.x JOIN (SELECT 1 AS x) t921 ON t1.x = t921.x JOIN (SELECT 1 AS x) t922 ON t1.x = t922.x JOIN (SELECT 1 AS x) t923 ON t1.x = t923.x JOIN (SELECT 1 AS x) t924 ON t1.x = t924.x JOIN (SELECT 1 AS x) t925 ON t1.x = t925.x JOIN (SELECT 1 AS x) t926 ON t1.x = t926.x JOIN (SELECT 1 AS x) t927 ON t1.x = t927.x JOIN (SELECT 1 AS x) t928 ON t1.x = t928.x JOIN (SELECT 1 AS x) t929 ON t1.x = t929.x JOIN (SELECT 1 AS x) t930 ON t1.x = t930.x JOIN (SELECT 1 AS x) t931 ON t1.x = t931.x JOIN (SELECT 1 AS x) t932 ON t1.x = t932.x JOIN (SELECT 1 AS x) t933 ON t1.x = t933.x JOIN (SELECT 1 AS x) t934 ON t1.x = t934.x JOIN (SELECT 1 AS x) t935 ON t1.x = t935.x JOIN (SELECT 1 AS x) t936 ON t1.x = t936.x JOIN (SELECT 1 AS x) t937 ON t1.x = t937.x JOIN (SELECT 1 AS x) t938 ON t1.x = t938.x JOIN (SELECT 1 AS x) t939 ON t1.x = t939.x JOIN (SELECT 1 AS x) t940 ON t1.x = t940.x JOIN (SELECT 1 AS x) t941 ON t1.x = t941.x JOIN (SELECT 1 AS x) t942 ON t1.x = t942.x JOIN (SELECT 1 AS x) t943 ON t1.x = t943.x JOIN (SELECT 1 AS x) t944 ON t1.x = t944.x JOIN (SELECT 1 AS x) t945 ON t1.x = t945.x JOIN (SELECT 1 AS x) t946 ON t1.x = t946.x JOIN (SELECT 1 AS x) t947 ON t1.x = t947.x JOIN (SELECT 1 AS x) t948 ON t1.x = t948.x JOIN (SELECT 1 AS x) t949 ON t1.x = t949.x JOIN (SELECT 1 AS x) t950 ON t1.x = t950.x JOIN (SELECT 1 AS x) t951 ON t1.x = t951.x JOIN (SELECT 1 AS x) t952 ON t1.x = t952.x JOIN (SELECT 1 AS x) t953 ON t1.x = t953.x JOIN (SELECT 1 AS x) t954 ON t1.x = t954.x JOIN (SELECT 1 AS x) t955 ON t1.x = t955.x JOIN (SELECT 1 AS x) t956 ON t1.x = t956.x JOIN (SELECT 1 AS x) t957 ON t1.x = t957.x JOIN (SELECT 1 AS x) t958 ON t1.x = t958.x JOIN (SELECT 1 AS x) t959 ON t1.x = t959.x JOIN (SELECT 1 AS x) t960 ON t1.x = t960.x JOIN (SELECT 1 AS x) t961 ON t1.x = t961.x JOIN (SELECT 1 AS x) t962 ON t1.x = t962.x JOIN (SELECT 1 AS x) t963 ON t1.x = t963.x JOIN (SELECT 1 AS x) t964 ON t1.x = t964.x JOIN (SELECT 1 AS x) t965 ON t1.x = t965.x JOIN (SELECT 1 AS x) t966 ON t1.x = t966.x JOIN (SELECT 1 AS x) t967 ON t1.x = t967.x JOIN (SELECT 1 AS x) t968 ON t1.x = t968.x JOIN (SELECT 1 AS x) t969 ON t1.x = t969.x JOIN (SELECT 1 AS x) t970 ON t1.x = t970.x JOIN (SELECT 1 AS x) t971 ON t1.x = t971.x JOIN (SELECT 1 AS x) t972 ON t1.x = t972.x JOIN (SELECT 1 AS x) t973 ON t1.x = t973.x JOIN (SELECT 1 AS x) t974 ON t1.x = t974.x JOIN (SELECT 1 AS x) t975 ON t1.x = t975.x JOIN (SELECT 1 AS x) t976 ON t1.x = t976.x JOIN (SELECT 1 AS x) t977 ON t1.x = t977.x JOIN (SELECT 1 AS x) t978 ON t1.x = t978.x JOIN (SELECT 1 AS x) t979 ON t1.x = t979.x JOIN (SELECT 1 AS x) t980 ON t1.x = t980.x JOIN (SELECT 1 AS x) t981 ON t1.x = t981.x JOIN (SELECT 1 AS x) t982 ON t1.x = t982.x JOIN (SELECT 1 AS x) t983 ON t1.x = t983.x JOIN (SELECT 1 AS x) t984 ON t1.x = t984.x JOIN (SELECT 1 AS x) t985 ON t1.x = t985.x JOIN (SELECT 1 AS x) t986 ON t1.x = t986.x JOIN (SELECT 1 AS x) t987 ON t1.x = t987.x JOIN (SELECT 1 AS x) t988 ON t1.x = t988.x JOIN (SELECT 1 AS x) t989 ON t1.x = t989.x JOIN (SELECT 1 AS x) t990 ON t1.x = t990.x JOIN (SELECT 1 AS x) t991 ON t1.x = t991.x JOIN (SELECT 1 AS x) t992 ON t1.x = t992.x JOIN (SELECT 1 AS x) t993 ON t1.x = t993.x JOIN (SELECT 1 AS x) t994 ON t1.x = t994.x JOIN (SELECT 1 AS x) t995 ON t1.x = t995.x JOIN (SELECT 1 AS x) t996 ON t1.x = t996.x JOIN (SELECT 1 AS x) t997 ON t1.x = t997.x JOIN (SELECT 1 AS x) t998 ON t1.x = t998.x JOIN (SELECT 1 AS x) t999 ON t1.x = t999.x JOIN (SELECT 1 AS x) t1000 ON t1.x = t1000.x +SELECT * FROM (SELECT 1 AS x) t1 JOIN (SELECT 1 AS x) t2 ON t1.x = t2.x JOIN (SELECT 1 AS x) t3 ON t1.x = t3.x JOIN (SELECT 1 AS x) t4 ON t1.x = t4.x JOIN (SELECT 1 AS x) t5 ON t1.x = t5.x JOIN (SELECT 1 AS x) t6 ON t1.x = t6.x JOIN (SELECT 1 AS x) t7 ON t1.x = t7.x JOIN (SELECT 1 AS x) t8 ON t1.x = t8.x JOIN (SELECT 1 AS x) t9 ON t1.x = t9.x JOIN (SELECT 1 AS x) t10 ON t1.x = t10.x JOIN (SELECT 1 AS x) t11 ON t1.x = t11.x JOIN (SELECT 1 AS x) t12 ON t1.x = t12.x JOIN (SELECT 1 AS x) t13 ON t1.x = t13.x JOIN (SELECT 1 AS x) t14 ON t1.x = t14.x JOIN (SELECT 1 AS x) t15 ON t1.x = t15.x JOIN (SELECT 1 AS x) t16 ON t1.x = t16.x JOIN (SELECT 1 AS x) t17 ON t1.x = t17.x JOIN (SELECT 1 AS x) t18 ON t1.x = t18.x JOIN (SELECT 1 AS x) t19 ON t1.x = t19.x JOIN (SELECT 1 AS x) t20 ON t1.x = t20.x JOIN (SELECT 1 AS x) t21 ON t1.x = t21.x JOIN (SELECT 1 AS x) t22 ON t1.x = t22.x JOIN (SELECT 1 AS x) t23 ON t1.x = t23.x JOIN (SELECT 1 AS x) t24 ON t1.x = t24.x JOIN (SELECT 1 AS x) t25 ON t1.x = t25.x JOIN (SELECT 1 AS x) t26 ON t1.x = t26.x JOIN (SELECT 1 AS x) t27 ON t1.x = t27.x JOIN (SELECT 1 AS x) t28 ON t1.x = t28.x JOIN (SELECT 1 AS x) t29 ON t1.x = t29.x JOIN (SELECT 1 AS x) t30 ON t1.x = t30.x JOIN (SELECT 1 AS x) t31 ON t1.x = t31.x JOIN (SELECT 1 AS x) t32 ON t1.x = t32.x JOIN (SELECT 1 AS x) t33 ON t1.x = t33.x JOIN (SELECT 1 AS x) t34 ON t1.x = t34.x JOIN (SELECT 1 AS x) t35 ON t1.x = t35.x JOIN (SELECT 1 AS x) t36 ON t1.x = t36.x JOIN (SELECT 1 AS x) t37 ON t1.x = t37.x JOIN (SELECT 1 AS x) t38 ON t1.x = t38.x JOIN (SELECT 1 AS x) t39 ON t1.x = t39.x JOIN (SELECT 1 AS x) t40 ON t1.x = t40.x JOIN (SELECT 1 AS x) t41 ON t1.x = t41.x JOIN (SELECT 1 AS x) t42 ON t1.x = t42.x JOIN (SELECT 1 AS x) t43 ON t1.x = t43.x JOIN (SELECT 1 AS x) t44 ON t1.x = t44.x JOIN (SELECT 1 AS x) t45 ON t1.x = t45.x JOIN (SELECT 1 AS x) t46 ON t1.x = t46.x JOIN (SELECT 1 AS x) t47 ON t1.x = t47.x JOIN (SELECT 1 AS x) t48 ON t1.x = t48.x JOIN (SELECT 1 AS x) t49 ON t1.x = t49.x JOIN (SELECT 1 AS x) t50 ON t1.x = t50.x JOIN (SELECT 1 AS x) t51 ON t1.x = t51.x JOIN (SELECT 1 AS x) t52 ON t1.x = t52.x JOIN (SELECT 1 AS x) t53 ON t1.x = t53.x JOIN (SELECT 1 AS x) t54 ON t1.x = t54.x JOIN (SELECT 1 AS x) t55 ON t1.x = t55.x JOIN (SELECT 1 AS x) t56 ON t1.x = t56.x JOIN (SELECT 1 AS x) t57 ON t1.x = t57.x JOIN (SELECT 1 AS x) t58 ON t1.x = t58.x JOIN (SELECT 1 AS x) t59 ON t1.x = t59.x JOIN (SELECT 1 AS x) t60 ON t1.x = t60.x JOIN (SELECT 1 AS x) t61 ON t1.x = t61.x JOIN (SELECT 1 AS x) t62 ON t1.x = t62.x JOIN (SELECT 1 AS x) t63 ON t1.x = t63.x JOIN (SELECT 1 AS x) t64 ON t1.x = t64.x JOIN (SELECT 1 AS x) t65 ON t1.x = t65.x JOIN (SELECT 1 AS x) t66 ON t1.x = t66.x JOIN (SELECT 1 AS x) t67 ON t1.x = t67.x JOIN (SELECT 1 AS x) t68 ON t1.x = t68.x JOIN (SELECT 1 AS x) t69 ON t1.x = t69.x JOIN (SELECT 1 AS x) t70 ON t1.x = t70.x JOIN (SELECT 1 AS x) t71 ON t1.x = t71.x JOIN (SELECT 1 AS x) t72 ON t1.x = t72.x JOIN (SELECT 1 AS x) t73 ON t1.x = t73.x JOIN (SELECT 1 AS x) t74 ON t1.x = t74.x JOIN (SELECT 1 AS x) t75 ON t1.x = t75.x JOIN (SELECT 1 AS x) t76 ON t1.x = t76.x JOIN (SELECT 1 AS x) t77 ON t1.x = t77.x JOIN (SELECT 1 AS x) t78 ON t1.x = t78.x JOIN (SELECT 1 AS x) t79 ON t1.x = t79.x JOIN (SELECT 1 AS x) t80 ON t1.x = t80.x JOIN (SELECT 1 AS x) t81 ON t1.x = t81.x JOIN (SELECT 1 AS x) t82 ON t1.x = t82.x JOIN (SELECT 1 AS x) t83 ON t1.x = t83.x JOIN (SELECT 1 AS x) t84 ON t1.x = t84.x JOIN (SELECT 1 AS x) t85 ON t1.x = t85.x JOIN (SELECT 1 AS x) t86 ON t1.x = t86.x JOIN (SELECT 1 AS x) t87 ON t1.x = t87.x JOIN (SELECT 1 AS x) t88 ON t1.x = t88.x JOIN (SELECT 1 AS x) t89 ON t1.x = t89.x JOIN (SELECT 1 AS x) t90 ON t1.x = t90.x JOIN (SELECT 1 AS x) t91 ON t1.x = t91.x JOIN (SELECT 1 AS x) t92 ON t1.x = t92.x JOIN (SELECT 1 AS x) t93 ON t1.x = t93.x JOIN (SELECT 1 AS x) t94 ON t1.x = t94.x JOIN (SELECT 1 AS x) t95 ON t1.x = t95.x JOIN (SELECT 1 AS x) t96 ON t1.x = t96.x JOIN (SELECT 1 AS x) t97 ON t1.x = t97.x JOIN (SELECT 1 AS x) t98 ON t1.x = t98.x JOIN (SELECT 1 AS x) t99 ON t1.x = t99.x JOIN (SELECT 1 AS x) t100 ON t1.x = t100.x JOIN (SELECT 1 AS x) t101 ON t1.x = t101.x JOIN (SELECT 1 AS x) t102 ON t1.x = t102.x JOIN (SELECT 1 AS x) t103 ON t1.x = t103.x JOIN (SELECT 1 AS x) t104 ON t1.x = t104.x JOIN (SELECT 1 AS x) t105 ON t1.x = t105.x JOIN (SELECT 1 AS x) t106 ON t1.x = t106.x JOIN (SELECT 1 AS x) t107 ON t1.x = t107.x JOIN (SELECT 1 AS x) t108 ON t1.x = t108.x JOIN (SELECT 1 AS x) t109 ON t1.x = t109.x JOIN (SELECT 1 AS x) t110 ON t1.x = t110.x JOIN (SELECT 1 AS x) t111 ON t1.x = t111.x JOIN (SELECT 1 AS x) t112 ON t1.x = t112.x JOIN (SELECT 1 AS x) t113 ON t1.x = t113.x JOIN (SELECT 1 AS x) t114 ON t1.x = t114.x JOIN (SELECT 1 AS x) t115 ON t1.x = t115.x JOIN (SELECT 1 AS x) t116 ON t1.x = t116.x JOIN (SELECT 1 AS x) t117 ON t1.x = t117.x JOIN (SELECT 1 AS x) t118 ON t1.x = t118.x JOIN (SELECT 1 AS x) t119 ON t1.x = t119.x JOIN (SELECT 1 AS x) t120 ON t1.x = t120.x JOIN (SELECT 1 AS x) t121 ON t1.x = t121.x JOIN (SELECT 1 AS x) t122 ON t1.x = t122.x JOIN (SELECT 1 AS x) t123 ON t1.x = t123.x JOIN (SELECT 1 AS x) t124 ON t1.x = t124.x JOIN (SELECT 1 AS x) t125 ON t1.x = t125.x JOIN (SELECT 1 AS x) t126 ON t1.x = t126.x JOIN (SELECT 1 AS x) t127 ON t1.x = t127.x JOIN (SELECT 1 AS x) t128 ON t1.x = t128.x JOIN (SELECT 1 AS x) t129 ON t1.x = t129.x JOIN (SELECT 1 AS x) t130 ON t1.x = t130.x JOIN (SELECT 1 AS x) t131 ON t1.x = t131.x JOIN (SELECT 1 AS x) t132 ON t1.x = t132.x JOIN (SELECT 1 AS x) t133 ON t1.x = t133.x JOIN (SELECT 1 AS x) t134 ON t1.x = t134.x JOIN (SELECT 1 AS x) t135 ON t1.x = t135.x JOIN (SELECT 1 AS x) t136 ON t1.x = t136.x JOIN (SELECT 1 AS x) t137 ON t1.x = t137.x JOIN (SELECT 1 AS x) t138 ON t1.x = t138.x JOIN (SELECT 1 AS x) t139 ON t1.x = t139.x JOIN (SELECT 1 AS x) t140 ON t1.x = t140.x JOIN (SELECT 1 AS x) t141 ON t1.x = t141.x JOIN (SELECT 1 AS x) t142 ON t1.x = t142.x JOIN (SELECT 1 AS x) t143 ON t1.x = t143.x JOIN (SELECT 1 AS x) t144 ON t1.x = t144.x JOIN (SELECT 1 AS x) t145 ON t1.x = t145.x JOIN (SELECT 1 AS x) t146 ON t1.x = t146.x JOIN (SELECT 1 AS x) t147 ON t1.x = t147.x JOIN (SELECT 1 AS x) t148 ON t1.x = t148.x JOIN (SELECT 1 AS x) t149 ON t1.x = t149.x JOIN (SELECT 1 AS x) t150 ON t1.x = t150.x JOIN (SELECT 1 AS x) t151 ON t1.x = t151.x JOIN (SELECT 1 AS x) t152 ON t1.x = t152.x JOIN (SELECT 1 AS x) t153 ON t1.x = t153.x JOIN (SELECT 1 AS x) t154 ON t1.x = t154.x JOIN (SELECT 1 AS x) t155 ON t1.x = t155.x JOIN (SELECT 1 AS x) t156 ON t1.x = t156.x JOIN (SELECT 1 AS x) t157 ON t1.x = t157.x JOIN (SELECT 1 AS x) t158 ON t1.x = t158.x JOIN (SELECT 1 AS x) t159 ON t1.x = t159.x JOIN (SELECT 1 AS x) t160 ON t1.x = t160.x JOIN (SELECT 1 AS x) t161 ON t1.x = t161.x JOIN (SELECT 1 AS x) t162 ON t1.x = t162.x JOIN (SELECT 1 AS x) t163 ON t1.x = t163.x JOIN (SELECT 1 AS x) t164 ON t1.x = t164.x JOIN (SELECT 1 AS x) t165 ON t1.x = t165.x JOIN (SELECT 1 AS x) t166 ON t1.x = t166.x JOIN (SELECT 1 AS x) t167 ON t1.x = t167.x JOIN (SELECT 1 AS x) t168 ON t1.x = t168.x JOIN (SELECT 1 AS x) t169 ON t1.x = t169.x JOIN (SELECT 1 AS x) t170 ON t1.x = t170.x JOIN (SELECT 1 AS x) t171 ON t1.x = t171.x JOIN (SELECT 1 AS x) t172 ON t1.x = t172.x JOIN (SELECT 1 AS x) t173 ON t1.x = t173.x JOIN (SELECT 1 AS x) t174 ON t1.x = t174.x JOIN (SELECT 1 AS x) t175 ON t1.x = t175.x JOIN (SELECT 1 AS x) t176 ON t1.x = t176.x JOIN (SELECT 1 AS x) t177 ON t1.x = t177.x JOIN (SELECT 1 AS x) t178 ON t1.x = t178.x JOIN (SELECT 1 AS x) t179 ON t1.x = t179.x JOIN (SELECT 1 AS x) t180 ON t1.x = t180.x JOIN (SELECT 1 AS x) t181 ON t1.x = t181.x JOIN (SELECT 1 AS x) t182 ON t1.x = t182.x JOIN (SELECT 1 AS x) t183 ON t1.x = t183.x JOIN (SELECT 1 AS x) t184 ON t1.x = t184.x JOIN (SELECT 1 AS x) t185 ON t1.x = t185.x JOIN (SELECT 1 AS x) t186 ON t1.x = t186.x JOIN (SELECT 1 AS x) t187 ON t1.x = t187.x JOIN (SELECT 1 AS x) t188 ON t1.x = t188.x JOIN (SELECT 1 AS x) t189 ON t1.x = t189.x JOIN (SELECT 1 AS x) t190 ON t1.x = t190.x JOIN (SELECT 1 AS x) t191 ON t1.x = t191.x JOIN (SELECT 1 AS x) t192 ON t1.x = t192.x JOIN (SELECT 1 AS x) t193 ON t1.x = t193.x JOIN (SELECT 1 AS x) t194 ON t1.x = t194.x JOIN (SELECT 1 AS x) t195 ON t1.x = t195.x JOIN (SELECT 1 AS x) t196 ON t1.x = t196.x JOIN (SELECT 1 AS x) t197 ON t1.x = t197.x JOIN (SELECT 1 AS x) t198 ON t1.x = t198.x JOIN (SELECT 1 AS x) t199 ON t1.x = t199.x JOIN (SELECT 1 AS x) t200 ON t1.x = t200.x JOIN (SELECT 1 AS x) t201 ON t1.x = t201.x JOIN (SELECT 1 AS x) t202 ON t1.x = t202.x JOIN (SELECT 1 AS x) t203 ON t1.x = t203.x JOIN (SELECT 1 AS x) t204 ON t1.x = t204.x JOIN (SELECT 1 AS x) t205 ON t1.x = t205.x JOIN (SELECT 1 AS x) t206 ON t1.x = t206.x JOIN (SELECT 1 AS x) t207 ON t1.x = t207.x JOIN (SELECT 1 AS x) t208 ON t1.x = t208.x JOIN (SELECT 1 AS x) t209 ON t1.x = t209.x JOIN (SELECT 1 AS x) t210 ON t1.x = t210.x JOIN (SELECT 1 AS x) t211 ON t1.x = t211.x JOIN (SELECT 1 AS x) t212 ON t1.x = t212.x JOIN (SELECT 1 AS x) t213 ON t1.x = t213.x JOIN (SELECT 1 AS x) t214 ON t1.x = t214.x JOIN (SELECT 1 AS x) t215 ON t1.x = t215.x JOIN (SELECT 1 AS x) t216 ON t1.x = t216.x JOIN (SELECT 1 AS x) t217 ON t1.x = t217.x JOIN (SELECT 1 AS x) t218 ON t1.x = t218.x JOIN (SELECT 1 AS x) t219 ON t1.x = t219.x JOIN (SELECT 1 AS x) t220 ON t1.x = t220.x JOIN (SELECT 1 AS x) t221 ON t1.x = t221.x JOIN (SELECT 1 AS x) t222 ON t1.x = t222.x JOIN (SELECT 1 AS x) t223 ON t1.x = t223.x JOIN (SELECT 1 AS x) t224 ON t1.x = t224.x JOIN (SELECT 1 AS x) t225 ON t1.x = t225.x JOIN (SELECT 1 AS x) t226 ON t1.x = t226.x JOIN (SELECT 1 AS x) t227 ON t1.x = t227.x JOIN (SELECT 1 AS x) t228 ON t1.x = t228.x JOIN (SELECT 1 AS x) t229 ON t1.x = t229.x JOIN (SELECT 1 AS x) t230 ON t1.x = t230.x JOIN (SELECT 1 AS x) t231 ON t1.x = t231.x JOIN (SELECT 1 AS x) t232 ON t1.x = t232.x JOIN (SELECT 1 AS x) t233 ON t1.x = t233.x JOIN (SELECT 1 AS x) t234 ON t1.x = t234.x JOIN (SELECT 1 AS x) t235 ON t1.x = t235.x JOIN (SELECT 1 AS x) t236 ON t1.x = t236.x JOIN (SELECT 1 AS x) t237 ON t1.x = t237.x JOIN (SELECT 1 AS x) t238 ON t1.x = t238.x JOIN (SELECT 1 AS x) t239 ON t1.x = t239.x JOIN (SELECT 1 AS x) t240 ON t1.x = t240.x JOIN (SELECT 1 AS x) t241 ON t1.x = t241.x JOIN (SELECT 1 AS x) t242 ON t1.x = t242.x JOIN (SELECT 1 AS x) t243 ON t1.x = t243.x JOIN (SELECT 1 AS x) t244 ON t1.x = t244.x JOIN (SELECT 1 AS x) t245 ON t1.x = t245.x JOIN (SELECT 1 AS x) t246 ON t1.x = t246.x JOIN (SELECT 1 AS x) t247 ON t1.x = t247.x JOIN (SELECT 1 AS x) t248 ON t1.x = t248.x JOIN (SELECT 1 AS x) t249 ON t1.x = t249.x JOIN (SELECT 1 AS x) t250 ON t1.x = t250.x JOIN (SELECT 1 AS x) t251 ON t1.x = t251.x JOIN (SELECT 1 AS x) t252 ON t1.x = t252.x JOIN (SELECT 1 AS x) t253 ON t1.x = t253.x JOIN (SELECT 1 AS x) t254 ON t1.x = t254.x JOIN (SELECT 1 AS x) t255 ON t1.x = t255.x JOIN (SELECT 1 AS x) t256 ON t1.x = t256.x JOIN (SELECT 1 AS x) t257 ON t1.x = t257.x JOIN (SELECT 1 AS x) t258 ON t1.x = t258.x JOIN (SELECT 1 AS x) t259 ON t1.x = t259.x JOIN (SELECT 1 AS x) t260 ON t1.x = t260.x JOIN (SELECT 1 AS x) t261 ON t1.x = t261.x JOIN (SELECT 1 AS x) t262 ON t1.x = t262.x JOIN (SELECT 1 AS x) t263 ON t1.x = t263.x JOIN (SELECT 1 AS x) t264 ON t1.x = t264.x JOIN (SELECT 1 AS x) t265 ON t1.x = t265.x JOIN (SELECT 1 AS x) t266 ON t1.x = t266.x JOIN (SELECT 1 AS x) t267 ON t1.x = t267.x JOIN (SELECT 1 AS x) t268 ON t1.x = t268.x JOIN (SELECT 1 AS x) t269 ON t1.x = t269.x JOIN (SELECT 1 AS x) t270 ON t1.x = t270.x JOIN (SELECT 1 AS x) t271 ON t1.x = t271.x JOIN (SELECT 1 AS x) t272 ON t1.x = t272.x JOIN (SELECT 1 AS x) t273 ON t1.x = t273.x JOIN (SELECT 1 AS x) t274 ON t1.x = t274.x JOIN (SELECT 1 AS x) t275 ON t1.x = t275.x JOIN (SELECT 1 AS x) t276 ON t1.x = t276.x JOIN (SELECT 1 AS x) t277 ON t1.x = t277.x JOIN (SELECT 1 AS x) t278 ON t1.x = t278.x JOIN (SELECT 1 AS x) t279 ON t1.x = t279.x JOIN (SELECT 1 AS x) t280 ON t1.x = t280.x JOIN (SELECT 1 AS x) t281 ON t1.x = t281.x JOIN (SELECT 1 AS x) t282 ON t1.x = t282.x JOIN (SELECT 1 AS x) t283 ON t1.x = t283.x JOIN (SELECT 1 AS x) t284 ON t1.x = t284.x JOIN (SELECT 1 AS x) t285 ON t1.x = t285.x JOIN (SELECT 1 AS x) t286 ON t1.x = t286.x JOIN (SELECT 1 AS x) t287 ON t1.x = t287.x JOIN (SELECT 1 AS x) t288 ON t1.x = t288.x JOIN (SELECT 1 AS x) t289 ON t1.x = t289.x JOIN (SELECT 1 AS x) t290 ON t1.x = t290.x JOIN (SELECT 1 AS x) t291 ON t1.x = t291.x JOIN (SELECT 1 AS x) t292 ON t1.x = t292.x JOIN (SELECT 1 AS x) t293 ON t1.x = t293.x JOIN (SELECT 1 AS x) t294 ON t1.x = t294.x JOIN (SELECT 1 AS x) t295 ON t1.x = t295.x JOIN (SELECT 1 AS x) t296 ON t1.x = t296.x JOIN (SELECT 1 AS x) t297 ON t1.x = t297.x JOIN (SELECT 1 AS x) t298 ON t1.x = t298.x JOIN (SELECT 1 AS x) t299 ON t1.x = t299.x JOIN (SELECT 1 AS x) t300 ON t1.x = t300.x JOIN (SELECT 1 AS x) t301 ON t1.x = t301.x JOIN (SELECT 1 AS x) t302 ON t1.x = t302.x JOIN (SELECT 1 AS x) t303 ON t1.x = t303.x JOIN (SELECT 1 AS x) t304 ON t1.x = t304.x JOIN (SELECT 1 AS x) t305 ON t1.x = t305.x JOIN (SELECT 1 AS x) t306 ON t1.x = t306.x JOIN (SELECT 1 AS x) t307 ON t1.x = t307.x JOIN (SELECT 1 AS x) t308 ON t1.x = t308.x JOIN (SELECT 1 AS x) t309 ON t1.x = t309.x JOIN (SELECT 1 AS x) t310 ON t1.x = t310.x JOIN (SELECT 1 AS x) t311 ON t1.x = t311.x JOIN (SELECT 1 AS x) t312 ON t1.x = t312.x JOIN (SELECT 1 AS x) t313 ON t1.x = t313.x JOIN (SELECT 1 AS x) t314 ON t1.x = t314.x JOIN (SELECT 1 AS x) t315 ON t1.x = t315.x JOIN (SELECT 1 AS x) t316 ON t1.x = t316.x JOIN (SELECT 1 AS x) t317 ON t1.x = t317.x JOIN (SELECT 1 AS x) t318 ON t1.x = t318.x JOIN (SELECT 1 AS x) t319 ON t1.x = t319.x JOIN (SELECT 1 AS x) t320 ON t1.x = t320.x JOIN (SELECT 1 AS x) t321 ON t1.x = t321.x JOIN (SELECT 1 AS x) t322 ON t1.x = t322.x JOIN (SELECT 1 AS x) t323 ON t1.x = t323.x JOIN (SELECT 1 AS x) t324 ON t1.x = t324.x JOIN (SELECT 1 AS x) t325 ON t1.x = t325.x JOIN (SELECT 1 AS x) t326 ON t1.x = t326.x JOIN (SELECT 1 AS x) t327 ON t1.x = t327.x JOIN (SELECT 1 AS x) t328 ON t1.x = t328.x JOIN (SELECT 1 AS x) t329 ON t1.x = t329.x JOIN (SELECT 1 AS x) t330 ON t1.x = t330.x JOIN (SELECT 1 AS x) t331 ON t1.x = t331.x JOIN (SELECT 1 AS x) t332 ON t1.x = t332.x JOIN (SELECT 1 AS x) t333 ON t1.x = t333.x JOIN (SELECT 1 AS x) t334 ON t1.x = t334.x JOIN (SELECT 1 AS x) t335 ON t1.x = t335.x JOIN (SELECT 1 AS x) t336 ON t1.x = t336.x JOIN (SELECT 1 AS x) t337 ON t1.x = t337.x JOIN (SELECT 1 AS x) t338 ON t1.x = t338.x JOIN (SELECT 1 AS x) t339 ON t1.x = t339.x JOIN (SELECT 1 AS x) t340 ON t1.x = t340.x JOIN (SELECT 1 AS x) t341 ON t1.x = t341.x JOIN (SELECT 1 AS x) t342 ON t1.x = t342.x JOIN (SELECT 1 AS x) t343 ON t1.x = t343.x JOIN (SELECT 1 AS x) t344 ON t1.x = t344.x JOIN (SELECT 1 AS x) t345 ON t1.x = t345.x JOIN (SELECT 1 AS x) t346 ON t1.x = t346.x JOIN (SELECT 1 AS x) t347 ON t1.x = t347.x JOIN (SELECT 1 AS x) t348 ON t1.x = t348.x JOIN (SELECT 1 AS x) t349 ON t1.x = t349.x JOIN (SELECT 1 AS x) t350 ON t1.x = t350.x JOIN (SELECT 1 AS x) t351 ON t1.x = t351.x JOIN (SELECT 1 AS x) t352 ON t1.x = t352.x JOIN (SELECT 1 AS x) t353 ON t1.x = t353.x JOIN (SELECT 1 AS x) t354 ON t1.x = t354.x JOIN (SELECT 1 AS x) t355 ON t1.x = t355.x JOIN (SELECT 1 AS x) t356 ON t1.x = t356.x JOIN (SELECT 1 AS x) t357 ON t1.x = t357.x JOIN (SELECT 1 AS x) t358 ON t1.x = t358.x JOIN (SELECT 1 AS x) t359 ON t1.x = t359.x JOIN (SELECT 1 AS x) t360 ON t1.x = t360.x JOIN (SELECT 1 AS x) t361 ON t1.x = t361.x JOIN (SELECT 1 AS x) t362 ON t1.x = t362.x JOIN (SELECT 1 AS x) t363 ON t1.x = t363.x JOIN (SELECT 1 AS x) t364 ON t1.x = t364.x JOIN (SELECT 1 AS x) t365 ON t1.x = t365.x JOIN (SELECT 1 AS x) t366 ON t1.x = t366.x JOIN (SELECT 1 AS x) t367 ON t1.x = t367.x JOIN (SELECT 1 AS x) t368 ON t1.x = t368.x JOIN (SELECT 1 AS x) t369 ON t1.x = t369.x JOIN (SELECT 1 AS x) t370 ON t1.x = t370.x JOIN (SELECT 1 AS x) t371 ON t1.x = t371.x JOIN (SELECT 1 AS x) t372 ON t1.x = t372.x JOIN (SELECT 1 AS x) t373 ON t1.x = t373.x JOIN (SELECT 1 AS x) t374 ON t1.x = t374.x JOIN (SELECT 1 AS x) t375 ON t1.x = t375.x JOIN (SELECT 1 AS x) t376 ON t1.x = t376.x JOIN (SELECT 1 AS x) t377 ON t1.x = t377.x JOIN (SELECT 1 AS x) t378 ON t1.x = t378.x JOIN (SELECT 1 AS x) t379 ON t1.x = t379.x JOIN (SELECT 1 AS x) t380 ON t1.x = t380.x JOIN (SELECT 1 AS x) t381 ON t1.x = t381.x JOIN (SELECT 1 AS x) t382 ON t1.x = t382.x JOIN (SELECT 1 AS x) t383 ON t1.x = t383.x JOIN (SELECT 1 AS x) t384 ON t1.x = t384.x JOIN (SELECT 1 AS x) t385 ON t1.x = t385.x JOIN (SELECT 1 AS x) t386 ON t1.x = t386.x JOIN (SELECT 1 AS x) t387 ON t1.x = t387.x JOIN (SELECT 1 AS x) t388 ON t1.x = t388.x JOIN (SELECT 1 AS x) t389 ON t1.x = t389.x JOIN (SELECT 1 AS x) t390 ON t1.x = t390.x JOIN (SELECT 1 AS x) t391 ON t1.x = t391.x JOIN (SELECT 1 AS x) t392 ON t1.x = t392.x JOIN (SELECT 1 AS x) t393 ON t1.x = t393.x JOIN (SELECT 1 AS x) t394 ON t1.x = t394.x JOIN (SELECT 1 AS x) t395 ON t1.x = t395.x JOIN (SELECT 1 AS x) t396 ON t1.x = t396.x JOIN (SELECT 1 AS x) t397 ON t1.x = t397.x JOIN (SELECT 1 AS x) t398 ON t1.x = t398.x JOIN (SELECT 1 AS x) t399 ON t1.x = t399.x JOIN (SELECT 1 AS x) t400 ON t1.x = t400.x JOIN (SELECT 1 AS x) t401 ON t1.x = t401.x JOIN (SELECT 1 AS x) t402 ON t1.x = t402.x JOIN (SELECT 1 AS x) t403 ON t1.x = t403.x JOIN (SELECT 1 AS x) t404 ON t1.x = t404.x JOIN (SELECT 1 AS x) t405 ON t1.x = t405.x JOIN (SELECT 1 AS x) t406 ON t1.x = t406.x JOIN (SELECT 1 AS x) t407 ON t1.x = t407.x JOIN (SELECT 1 AS x) t408 ON t1.x = t408.x JOIN (SELECT 1 AS x) t409 ON t1.x = t409.x JOIN (SELECT 1 AS x) t410 ON t1.x = t410.x JOIN (SELECT 1 AS x) t411 ON t1.x = t411.x JOIN (SELECT 1 AS x) t412 ON t1.x = t412.x JOIN (SELECT 1 AS x) t413 ON t1.x = t413.x JOIN (SELECT 1 AS x) t414 ON t1.x = t414.x JOIN (SELECT 1 AS x) t415 ON t1.x = t415.x JOIN (SELECT 1 AS x) t416 ON t1.x = t416.x JOIN (SELECT 1 AS x) t417 ON t1.x = t417.x JOIN (SELECT 1 AS x) t418 ON t1.x = t418.x JOIN (SELECT 1 AS x) t419 ON t1.x = t419.x JOIN (SELECT 1 AS x) t420 ON t1.x = t420.x JOIN (SELECT 1 AS x) t421 ON t1.x = t421.x JOIN (SELECT 1 AS x) t422 ON t1.x = t422.x JOIN (SELECT 1 AS x) t423 ON t1.x = t423.x JOIN (SELECT 1 AS x) t424 ON t1.x = t424.x JOIN (SELECT 1 AS x) t425 ON t1.x = t425.x JOIN (SELECT 1 AS x) t426 ON t1.x = t426.x JOIN (SELECT 1 AS x) t427 ON t1.x = t427.x JOIN (SELECT 1 AS x) t428 ON t1.x = t428.x JOIN (SELECT 1 AS x) t429 ON t1.x = t429.x JOIN (SELECT 1 AS x) t430 ON t1.x = t430.x JOIN (SELECT 1 AS x) t431 ON t1.x = t431.x JOIN (SELECT 1 AS x) t432 ON t1.x = t432.x JOIN (SELECT 1 AS x) t433 ON t1.x = t433.x JOIN (SELECT 1 AS x) t434 ON t1.x = t434.x JOIN (SELECT 1 AS x) t435 ON t1.x = t435.x JOIN (SELECT 1 AS x) t436 ON t1.x = t436.x JOIN (SELECT 1 AS x) t437 ON t1.x = t437.x JOIN (SELECT 1 AS x) t438 ON t1.x = t438.x JOIN (SELECT 1 AS x) t439 ON t1.x = t439.x JOIN (SELECT 1 AS x) t440 ON t1.x = t440.x JOIN (SELECT 1 AS x) t441 ON t1.x = t441.x JOIN (SELECT 1 AS x) t442 ON t1.x = t442.x JOIN (SELECT 1 AS x) t443 ON t1.x = t443.x JOIN (SELECT 1 AS x) t444 ON t1.x = t444.x JOIN (SELECT 1 AS x) t445 ON t1.x = t445.x JOIN (SELECT 1 AS x) t446 ON t1.x = t446.x JOIN (SELECT 1 AS x) t447 ON t1.x = t447.x JOIN (SELECT 1 AS x) t448 ON t1.x = t448.x JOIN (SELECT 1 AS x) t449 ON t1.x = t449.x JOIN (SELECT 1 AS x) t450 ON t1.x = t450.x JOIN (SELECT 1 AS x) t451 ON t1.x = t451.x JOIN (SELECT 1 AS x) t452 ON t1.x = t452.x JOIN (SELECT 1 AS x) t453 ON t1.x = t453.x JOIN (SELECT 1 AS x) t454 ON t1.x = t454.x JOIN (SELECT 1 AS x) t455 ON t1.x = t455.x JOIN (SELECT 1 AS x) t456 ON t1.x = t456.x JOIN (SELECT 1 AS x) t457 ON t1.x = t457.x JOIN (SELECT 1 AS x) t458 ON t1.x = t458.x JOIN (SELECT 1 AS x) t459 ON t1.x = t459.x JOIN (SELECT 1 AS x) t460 ON t1.x = t460.x JOIN (SELECT 1 AS x) t461 ON t1.x = t461.x JOIN (SELECT 1 AS x) t462 ON t1.x = t462.x JOIN (SELECT 1 AS x) t463 ON t1.x = t463.x JOIN (SELECT 1 AS x) t464 ON t1.x = t464.x JOIN (SELECT 1 AS x) t465 ON t1.x = t465.x JOIN (SELECT 1 AS x) t466 ON t1.x = t466.x JOIN (SELECT 1 AS x) t467 ON t1.x = t467.x JOIN (SELECT 1 AS x) t468 ON t1.x = t468.x JOIN (SELECT 1 AS x) t469 ON t1.x = t469.x JOIN (SELECT 1 AS x) t470 ON t1.x = t470.x JOIN (SELECT 1 AS x) t471 ON t1.x = t471.x JOIN (SELECT 1 AS x) t472 ON t1.x = t472.x JOIN (SELECT 1 AS x) t473 ON t1.x = t473.x JOIN (SELECT 1 AS x) t474 ON t1.x = t474.x JOIN (SELECT 1 AS x) t475 ON t1.x = t475.x JOIN (SELECT 1 AS x) t476 ON t1.x = t476.x JOIN (SELECT 1 AS x) t477 ON t1.x = t477.x JOIN (SELECT 1 AS x) t478 ON t1.x = t478.x JOIN (SELECT 1 AS x) t479 ON t1.x = t479.x JOIN (SELECT 1 AS x) t480 ON t1.x = t480.x JOIN (SELECT 1 AS x) t481 ON t1.x = t481.x JOIN (SELECT 1 AS x) t482 ON t1.x = t482.x JOIN (SELECT 1 AS x) t483 ON t1.x = t483.x JOIN (SELECT 1 AS x) t484 ON t1.x = t484.x JOIN (SELECT 1 AS x) t485 ON t1.x = t485.x JOIN (SELECT 1 AS x) t486 ON t1.x = t486.x JOIN (SELECT 1 AS x) t487 ON t1.x = t487.x JOIN (SELECT 1 AS x) t488 ON t1.x = t488.x JOIN (SELECT 1 AS x) t489 ON t1.x = t489.x JOIN (SELECT 1 AS x) t490 ON t1.x = t490.x JOIN (SELECT 1 AS x) t491 ON t1.x = t491.x JOIN (SELECT 1 AS x) t492 ON t1.x = t492.x JOIN (SELECT 1 AS x) t493 ON t1.x = t493.x JOIN (SELECT 1 AS x) t494 ON t1.x = t494.x JOIN (SELECT 1 AS x) t495 ON t1.x = t495.x JOIN (SELECT 1 AS x) t496 ON t1.x = t496.x JOIN (SELECT 1 AS x) t497 ON t1.x = t497.x JOIN (SELECT 1 AS x) t498 ON t1.x = t498.x JOIN (SELECT 1 AS x) t499 ON t1.x = t499.x JOIN (SELECT 1 AS x) t500 ON t1.x = t500.x JOIN (SELECT 1 AS x) t501 ON t1.x = t501.x JOIN (SELECT 1 AS x) t502 ON t1.x = t502.x JOIN (SELECT 1 AS x) t503 ON t1.x = t503.x JOIN (SELECT 1 AS x) t504 ON t1.x = t504.x JOIN (SELECT 1 AS x) t505 ON t1.x = t505.x JOIN (SELECT 1 AS x) t506 ON t1.x = t506.x JOIN (SELECT 1 AS x) t507 ON t1.x = t507.x JOIN (SELECT 1 AS x) t508 ON t1.x = t508.x JOIN (SELECT 1 AS x) t509 ON t1.x = t509.x JOIN (SELECT 1 AS x) t510 ON t1.x = t510.x JOIN (SELECT 1 AS x) t511 ON t1.x = t511.x JOIN (SELECT 1 AS x) t512 ON t1.x = t512.x JOIN (SELECT 1 AS x) t513 ON t1.x = t513.x JOIN (SELECT 1 AS x) t514 ON t1.x = t514.x JOIN (SELECT 1 AS x) t515 ON t1.x = t515.x JOIN (SELECT 1 AS x) t516 ON t1.x = t516.x JOIN (SELECT 1 AS x) t517 ON t1.x = t517.x JOIN (SELECT 1 AS x) t518 ON t1.x = t518.x JOIN (SELECT 1 AS x) t519 ON t1.x = t519.x JOIN (SELECT 1 AS x) t520 ON t1.x = t520.x JOIN (SELECT 1 AS x) t521 ON t1.x = t521.x JOIN (SELECT 1 AS x) t522 ON t1.x = t522.x JOIN (SELECT 1 AS x) t523 ON t1.x = t523.x JOIN (SELECT 1 AS x) t524 ON t1.x = t524.x JOIN (SELECT 1 AS x) t525 ON t1.x = t525.x JOIN (SELECT 1 AS x) t526 ON t1.x = t526.x JOIN (SELECT 1 AS x) t527 ON t1.x = t527.x JOIN (SELECT 1 AS x) t528 ON t1.x = t528.x JOIN (SELECT 1 AS x) t529 ON t1.x = t529.x JOIN (SELECT 1 AS x) t530 ON t1.x = t530.x JOIN (SELECT 1 AS x) t531 ON t1.x = t531.x JOIN (SELECT 1 AS x) t532 ON t1.x = t532.x JOIN (SELECT 1 AS x) t533 ON t1.x = t533.x JOIN (SELECT 1 AS x) t534 ON t1.x = t534.x JOIN (SELECT 1 AS x) t535 ON t1.x = t535.x JOIN (SELECT 1 AS x) t536 ON t1.x = t536.x JOIN (SELECT 1 AS x) t537 ON t1.x = t537.x JOIN (SELECT 1 AS x) t538 ON t1.x = t538.x JOIN (SELECT 1 AS x) t539 ON t1.x = t539.x JOIN (SELECT 1 AS x) t540 ON t1.x = t540.x JOIN (SELECT 1 AS x) t541 ON t1.x = t541.x JOIN (SELECT 1 AS x) t542 ON t1.x = t542.x JOIN (SELECT 1 AS x) t543 ON t1.x = t543.x JOIN (SELECT 1 AS x) t544 ON t1.x = t544.x JOIN (SELECT 1 AS x) t545 ON t1.x = t545.x JOIN (SELECT 1 AS x) t546 ON t1.x = t546.x JOIN (SELECT 1 AS x) t547 ON t1.x = t547.x JOIN (SELECT 1 AS x) t548 ON t1.x = t548.x JOIN (SELECT 1 AS x) t549 ON t1.x = t549.x JOIN (SELECT 1 AS x) t550 ON t1.x = t550.x JOIN (SELECT 1 AS x) t551 ON t1.x = t551.x JOIN (SELECT 1 AS x) t552 ON t1.x = t552.x JOIN (SELECT 1 AS x) t553 ON t1.x = t553.x JOIN (SELECT 1 AS x) t554 ON t1.x = t554.x JOIN (SELECT 1 AS x) t555 ON t1.x = t555.x JOIN (SELECT 1 AS x) t556 ON t1.x = t556.x JOIN (SELECT 1 AS x) t557 ON t1.x = t557.x JOIN (SELECT 1 AS x) t558 ON t1.x = t558.x JOIN (SELECT 1 AS x) t559 ON t1.x = t559.x JOIN (SELECT 1 AS x) t560 ON t1.x = t560.x JOIN (SELECT 1 AS x) t561 ON t1.x = t561.x JOIN (SELECT 1 AS x) t562 ON t1.x = t562.x JOIN (SELECT 1 AS x) t563 ON t1.x = t563.x JOIN (SELECT 1 AS x) t564 ON t1.x = t564.x JOIN (SELECT 1 AS x) t565 ON t1.x = t565.x JOIN (SELECT 1 AS x) t566 ON t1.x = t566.x JOIN (SELECT 1 AS x) t567 ON t1.x = t567.x JOIN (SELECT 1 AS x) t568 ON t1.x = t568.x JOIN (SELECT 1 AS x) t569 ON t1.x = t569.x JOIN (SELECT 1 AS x) t570 ON t1.x = t570.x JOIN (SELECT 1 AS x) t571 ON t1.x = t571.x JOIN (SELECT 1 AS x) t572 ON t1.x = t572.x JOIN (SELECT 1 AS x) t573 ON t1.x = t573.x JOIN (SELECT 1 AS x) t574 ON t1.x = t574.x JOIN (SELECT 1 AS x) t575 ON t1.x = t575.x JOIN (SELECT 1 AS x) t576 ON t1.x = t576.x JOIN (SELECT 1 AS x) t577 ON t1.x = t577.x JOIN (SELECT 1 AS x) t578 ON t1.x = t578.x JOIN (SELECT 1 AS x) t579 ON t1.x = t579.x JOIN (SELECT 1 AS x) t580 ON t1.x = t580.x JOIN (SELECT 1 AS x) t581 ON t1.x = t581.x JOIN (SELECT 1 AS x) t582 ON t1.x = t582.x JOIN (SELECT 1 AS x) t583 ON t1.x = t583.x JOIN (SELECT 1 AS x) t584 ON t1.x = t584.x JOIN (SELECT 1 AS x) t585 ON t1.x = t585.x JOIN (SELECT 1 AS x) t586 ON t1.x = t586.x JOIN (SELECT 1 AS x) t587 ON t1.x = t587.x JOIN (SELECT 1 AS x) t588 ON t1.x = t588.x JOIN (SELECT 1 AS x) t589 ON t1.x = t589.x JOIN (SELECT 1 AS x) t590 ON t1.x = t590.x JOIN (SELECT 1 AS x) t591 ON t1.x = t591.x JOIN (SELECT 1 AS x) t592 ON t1.x = t592.x JOIN (SELECT 1 AS x) t593 ON t1.x = t593.x JOIN (SELECT 1 AS x) t594 ON t1.x = t594.x JOIN (SELECT 1 AS x) t595 ON t1.x = t595.x JOIN (SELECT 1 AS x) t596 ON t1.x = t596.x JOIN (SELECT 1 AS x) t597 ON t1.x = t597.x JOIN (SELECT 1 AS x) t598 ON t1.x = t598.x JOIN (SELECT 1 AS x) t599 ON t1.x = t599.x JOIN (SELECT 1 AS x) t600 ON t1.x = t600.x JOIN (SELECT 1 AS x) t601 ON t1.x = t601.x JOIN (SELECT 1 AS x) t602 ON t1.x = t602.x JOIN (SELECT 1 AS x) t603 ON t1.x = t603.x JOIN (SELECT 1 AS x) t604 ON t1.x = t604.x JOIN (SELECT 1 AS x) t605 ON t1.x = t605.x JOIN (SELECT 1 AS x) t606 ON t1.x = t606.x JOIN (SELECT 1 AS x) t607 ON t1.x = t607.x JOIN (SELECT 1 AS x) t608 ON t1.x = t608.x JOIN (SELECT 1 AS x) t609 ON t1.x = t609.x JOIN (SELECT 1 AS x) t610 ON t1.x = t610.x JOIN (SELECT 1 AS x) t611 ON t1.x = t611.x JOIN (SELECT 1 AS x) t612 ON t1.x = t612.x JOIN (SELECT 1 AS x) t613 ON t1.x = t613.x JOIN (SELECT 1 AS x) t614 ON t1.x = t614.x JOIN (SELECT 1 AS x) t615 ON t1.x = t615.x JOIN (SELECT 1 AS x) t616 ON t1.x = t616.x JOIN (SELECT 1 AS x) t617 ON t1.x = t617.x JOIN (SELECT 1 AS x) t618 ON t1.x = t618.x JOIN (SELECT 1 AS x) t619 ON t1.x = t619.x JOIN (SELECT 1 AS x) t620 ON t1.x = t620.x JOIN (SELECT 1 AS x) t621 ON t1.x = t621.x JOIN (SELECT 1 AS x) t622 ON t1.x = t622.x JOIN (SELECT 1 AS x) t623 ON t1.x = t623.x JOIN (SELECT 1 AS x) t624 ON t1.x = t624.x JOIN (SELECT 1 AS x) t625 ON t1.x = t625.x JOIN (SELECT 1 AS x) t626 ON t1.x = t626.x JOIN (SELECT 1 AS x) t627 ON t1.x = t627.x JOIN (SELECT 1 AS x) t628 ON t1.x = t628.x JOIN (SELECT 1 AS x) t629 ON t1.x = t629.x JOIN (SELECT 1 AS x) t630 ON t1.x = t630.x JOIN (SELECT 1 AS x) t631 ON t1.x = t631.x JOIN (SELECT 1 AS x) t632 ON t1.x = t632.x JOIN (SELECT 1 AS x) t633 ON t1.x = t633.x JOIN (SELECT 1 AS x) t634 ON t1.x = t634.x JOIN (SELECT 1 AS x) t635 ON t1.x = t635.x JOIN (SELECT 1 AS x) t636 ON t1.x = t636.x JOIN (SELECT 1 AS x) t637 ON t1.x = t637.x JOIN (SELECT 1 AS x) t638 ON t1.x = t638.x JOIN (SELECT 1 AS x) t639 ON t1.x = t639.x JOIN (SELECT 1 AS x) t640 ON t1.x = t640.x JOIN (SELECT 1 AS x) t641 ON t1.x = t641.x JOIN (SELECT 1 AS x) t642 ON t1.x = t642.x JOIN (SELECT 1 AS x) t643 ON t1.x = t643.x JOIN (SELECT 1 AS x) t644 ON t1.x = t644.x JOIN (SELECT 1 AS x) t645 ON t1.x = t645.x JOIN (SELECT 1 AS x) t646 ON t1.x = t646.x JOIN (SELECT 1 AS x) t647 ON t1.x = t647.x JOIN (SELECT 1 AS x) t648 ON t1.x = t648.x JOIN (SELECT 1 AS x) t649 ON t1.x = t649.x JOIN (SELECT 1 AS x) t650 ON t1.x = t650.x JOIN (SELECT 1 AS x) t651 ON t1.x = t651.x JOIN (SELECT 1 AS x) t652 ON t1.x = t652.x JOIN (SELECT 1 AS x) t653 ON t1.x = t653.x JOIN (SELECT 1 AS x) t654 ON t1.x = t654.x JOIN (SELECT 1 AS x) t655 ON t1.x = t655.x JOIN (SELECT 1 AS x) t656 ON t1.x = t656.x JOIN (SELECT 1 AS x) t657 ON t1.x = t657.x JOIN (SELECT 1 AS x) t658 ON t1.x = t658.x JOIN (SELECT 1 AS x) t659 ON t1.x = t659.x JOIN (SELECT 1 AS x) t660 ON t1.x = t660.x JOIN (SELECT 1 AS x) t661 ON t1.x = t661.x JOIN (SELECT 1 AS x) t662 ON t1.x = t662.x JOIN (SELECT 1 AS x) t663 ON t1.x = t663.x JOIN (SELECT 1 AS x) t664 ON t1.x = t664.x JOIN (SELECT 1 AS x) t665 ON t1.x = t665.x JOIN (SELECT 1 AS x) t666 ON t1.x = t666.x From c46fb6661240fb1e0a45b748bd1be120536d5940 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 23 Apr 2024 13:27:38 +0000 Subject: [PATCH 0276/1018] Disable for tsan --- tests/queries/0_stateless/03094_one_thousand_joins.sql | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03094_one_thousand_joins.sql b/tests/queries/0_stateless/03094_one_thousand_joins.sql index b6345595811..801cf0be2bc 100644 --- a/tests/queries/0_stateless/03094_one_thousand_joins.sql +++ b/tests/queries/0_stateless/03094_one_thousand_joins.sql @@ -1,4 +1,6 @@ --- Tags: no-fasttest, long +-- Tags: no-fasttest, no-tsan, long + +-- (no-tsan because it has a small maximum stack size and the test would fail with TOO_DEEP_RECURSION) -- Bug 33446, marked as 'long' because it still runs around 10 sec SELECT * FROM (SELECT 1 AS x) t1 JOIN (SELECT 1 AS x) t2 ON t1.x = t2.x JOIN (SELECT 1 AS x) t3 ON t1.x = t3.x JOIN (SELECT 1 AS x) t4 ON t1.x = t4.x JOIN (SELECT 1 AS x) t5 ON t1.x = t5.x JOIN (SELECT 1 AS x) t6 ON t1.x = t6.x JOIN (SELECT 1 AS x) t7 ON t1.x = t7.x JOIN (SELECT 1 AS x) t8 ON t1.x = t8.x JOIN (SELECT 1 AS x) t9 ON t1.x = t9.x JOIN (SELECT 1 AS x) t10 ON t1.x = t10.x JOIN (SELECT 1 AS x) t11 ON t1.x = t11.x JOIN (SELECT 1 AS x) t12 ON t1.x = t12.x JOIN (SELECT 1 AS x) t13 ON t1.x = t13.x JOIN (SELECT 1 AS x) t14 ON t1.x = t14.x JOIN (SELECT 1 AS x) t15 ON t1.x = t15.x JOIN (SELECT 1 AS x) t16 ON t1.x = t16.x JOIN (SELECT 1 AS x) t17 ON t1.x = t17.x JOIN (SELECT 1 AS x) t18 ON t1.x = t18.x JOIN (SELECT 1 AS x) t19 ON t1.x = t19.x JOIN (SELECT 1 AS x) t20 ON t1.x = t20.x JOIN (SELECT 1 AS x) t21 ON t1.x = t21.x JOIN (SELECT 1 AS x) t22 ON t1.x = t22.x JOIN (SELECT 1 AS x) t23 ON t1.x = t23.x JOIN (SELECT 1 AS x) t24 ON t1.x = t24.x JOIN (SELECT 1 AS x) t25 ON t1.x = t25.x JOIN (SELECT 1 AS x) t26 ON t1.x = t26.x JOIN (SELECT 1 AS x) t27 ON t1.x = t27.x JOIN (SELECT 1 AS x) t28 ON t1.x = t28.x JOIN (SELECT 1 AS x) t29 ON t1.x = t29.x JOIN (SELECT 1 AS x) t30 ON t1.x = t30.x JOIN (SELECT 1 AS x) t31 ON t1.x = t31.x JOIN (SELECT 1 AS x) t32 ON t1.x = t32.x JOIN (SELECT 1 AS x) t33 ON t1.x = t33.x JOIN (SELECT 1 AS x) t34 ON t1.x = t34.x JOIN (SELECT 1 AS x) t35 ON t1.x = t35.x JOIN (SELECT 1 AS x) t36 ON t1.x = t36.x JOIN (SELECT 1 AS x) t37 ON t1.x = t37.x JOIN (SELECT 1 AS x) t38 ON t1.x = t38.x JOIN (SELECT 1 AS x) t39 ON t1.x = t39.x JOIN (SELECT 1 AS x) t40 ON t1.x = t40.x JOIN (SELECT 1 AS x) t41 ON t1.x = t41.x JOIN (SELECT 1 AS x) t42 ON t1.x = t42.x JOIN (SELECT 1 AS x) t43 ON t1.x = t43.x JOIN (SELECT 1 AS x) t44 ON t1.x = t44.x JOIN (SELECT 1 AS x) t45 ON t1.x = t45.x JOIN (SELECT 1 AS x) t46 ON t1.x = t46.x JOIN (SELECT 1 AS x) t47 ON t1.x = t47.x JOIN (SELECT 1 AS x) t48 ON t1.x = t48.x JOIN (SELECT 1 AS x) t49 ON t1.x = t49.x JOIN (SELECT 1 AS x) t50 ON t1.x = t50.x JOIN (SELECT 1 AS x) t51 ON t1.x = t51.x JOIN (SELECT 1 AS x) t52 ON t1.x = t52.x JOIN (SELECT 1 AS x) t53 ON t1.x = t53.x JOIN (SELECT 1 AS x) t54 ON t1.x = t54.x JOIN (SELECT 1 AS x) t55 ON t1.x = t55.x JOIN (SELECT 1 AS x) t56 ON t1.x = t56.x JOIN (SELECT 1 AS x) t57 ON t1.x = t57.x JOIN (SELECT 1 AS x) t58 ON t1.x = t58.x JOIN (SELECT 1 AS x) t59 ON t1.x = t59.x JOIN (SELECT 1 AS x) t60 ON t1.x = t60.x JOIN (SELECT 1 AS x) t61 ON t1.x = t61.x JOIN (SELECT 1 AS x) t62 ON t1.x = t62.x JOIN (SELECT 1 AS x) t63 ON t1.x = t63.x JOIN (SELECT 1 AS x) t64 ON t1.x = t64.x JOIN (SELECT 1 AS x) t65 ON t1.x = t65.x JOIN (SELECT 1 AS x) t66 ON t1.x = t66.x JOIN (SELECT 1 AS x) t67 ON t1.x = t67.x JOIN (SELECT 1 AS x) t68 ON t1.x = t68.x JOIN (SELECT 1 AS x) t69 ON t1.x = t69.x JOIN (SELECT 1 AS x) t70 ON t1.x = t70.x JOIN (SELECT 1 AS x) t71 ON t1.x = t71.x JOIN (SELECT 1 AS x) t72 ON t1.x = t72.x JOIN (SELECT 1 AS x) t73 ON t1.x = t73.x JOIN (SELECT 1 AS x) t74 ON t1.x = t74.x JOIN (SELECT 1 AS x) t75 ON t1.x = t75.x JOIN (SELECT 1 AS x) t76 ON t1.x = t76.x JOIN (SELECT 1 AS x) t77 ON t1.x = t77.x JOIN (SELECT 1 AS x) t78 ON t1.x = t78.x JOIN (SELECT 1 AS x) t79 ON t1.x = t79.x JOIN (SELECT 1 AS x) t80 ON t1.x = t80.x JOIN (SELECT 1 AS x) t81 ON t1.x = t81.x JOIN (SELECT 1 AS x) t82 ON t1.x = t82.x JOIN (SELECT 1 AS x) t83 ON t1.x = t83.x JOIN (SELECT 1 AS x) t84 ON t1.x = t84.x JOIN (SELECT 1 AS x) t85 ON t1.x = t85.x JOIN (SELECT 1 AS x) t86 ON t1.x = t86.x JOIN (SELECT 1 AS x) t87 ON t1.x = t87.x JOIN (SELECT 1 AS x) t88 ON t1.x = t88.x JOIN (SELECT 1 AS x) t89 ON t1.x = t89.x JOIN (SELECT 1 AS x) t90 ON t1.x = t90.x JOIN (SELECT 1 AS x) t91 ON t1.x = t91.x JOIN (SELECT 1 AS x) t92 ON t1.x = t92.x JOIN (SELECT 1 AS x) t93 ON t1.x = t93.x JOIN (SELECT 1 AS x) t94 ON t1.x = t94.x JOIN (SELECT 1 AS x) t95 ON t1.x = t95.x JOIN (SELECT 1 AS x) t96 ON t1.x = t96.x JOIN (SELECT 1 AS x) t97 ON t1.x = t97.x JOIN (SELECT 1 AS x) t98 ON t1.x = t98.x JOIN (SELECT 1 AS x) t99 ON t1.x = t99.x JOIN (SELECT 1 AS x) t100 ON t1.x = t100.x JOIN (SELECT 1 AS x) t101 ON t1.x = t101.x JOIN (SELECT 1 AS x) t102 ON t1.x = t102.x JOIN (SELECT 1 AS x) t103 ON t1.x = t103.x JOIN (SELECT 1 AS x) t104 ON t1.x = t104.x JOIN (SELECT 1 AS x) t105 ON t1.x = t105.x JOIN (SELECT 1 AS x) t106 ON t1.x = t106.x JOIN (SELECT 1 AS x) t107 ON t1.x = t107.x JOIN (SELECT 1 AS x) t108 ON t1.x = t108.x JOIN (SELECT 1 AS x) t109 ON t1.x = t109.x JOIN (SELECT 1 AS x) t110 ON t1.x = t110.x JOIN (SELECT 1 AS x) t111 ON t1.x = t111.x JOIN (SELECT 1 AS x) t112 ON t1.x = t112.x JOIN (SELECT 1 AS x) t113 ON t1.x = t113.x JOIN (SELECT 1 AS x) t114 ON t1.x = t114.x JOIN (SELECT 1 AS x) t115 ON t1.x = t115.x JOIN (SELECT 1 AS x) t116 ON t1.x = t116.x JOIN (SELECT 1 AS x) t117 ON t1.x = t117.x JOIN (SELECT 1 AS x) t118 ON t1.x = t118.x JOIN (SELECT 1 AS x) t119 ON t1.x = t119.x JOIN (SELECT 1 AS x) t120 ON t1.x = t120.x JOIN (SELECT 1 AS x) t121 ON t1.x = t121.x JOIN (SELECT 1 AS x) t122 ON t1.x = t122.x JOIN (SELECT 1 AS x) t123 ON t1.x = t123.x JOIN (SELECT 1 AS x) t124 ON t1.x = t124.x JOIN (SELECT 1 AS x) t125 ON t1.x = t125.x JOIN (SELECT 1 AS x) t126 ON t1.x = t126.x JOIN (SELECT 1 AS x) t127 ON t1.x = t127.x JOIN (SELECT 1 AS x) t128 ON t1.x = t128.x JOIN (SELECT 1 AS x) t129 ON t1.x = t129.x JOIN (SELECT 1 AS x) t130 ON t1.x = t130.x JOIN (SELECT 1 AS x) t131 ON t1.x = t131.x JOIN (SELECT 1 AS x) t132 ON t1.x = t132.x JOIN (SELECT 1 AS x) t133 ON t1.x = t133.x JOIN (SELECT 1 AS x) t134 ON t1.x = t134.x JOIN (SELECT 1 AS x) t135 ON t1.x = t135.x JOIN (SELECT 1 AS x) t136 ON t1.x = t136.x JOIN (SELECT 1 AS x) t137 ON t1.x = t137.x JOIN (SELECT 1 AS x) t138 ON t1.x = t138.x JOIN (SELECT 1 AS x) t139 ON t1.x = t139.x JOIN (SELECT 1 AS x) t140 ON t1.x = t140.x JOIN (SELECT 1 AS x) t141 ON t1.x = t141.x JOIN (SELECT 1 AS x) t142 ON t1.x = t142.x JOIN (SELECT 1 AS x) t143 ON t1.x = t143.x JOIN (SELECT 1 AS x) t144 ON t1.x = t144.x JOIN (SELECT 1 AS x) t145 ON t1.x = t145.x JOIN (SELECT 1 AS x) t146 ON t1.x = t146.x JOIN (SELECT 1 AS x) t147 ON t1.x = t147.x JOIN (SELECT 1 AS x) t148 ON t1.x = t148.x JOIN (SELECT 1 AS x) t149 ON t1.x = t149.x JOIN (SELECT 1 AS x) t150 ON t1.x = t150.x JOIN (SELECT 1 AS x) t151 ON t1.x = t151.x JOIN (SELECT 1 AS x) t152 ON t1.x = t152.x JOIN (SELECT 1 AS x) t153 ON t1.x = t153.x JOIN (SELECT 1 AS x) t154 ON t1.x = t154.x JOIN (SELECT 1 AS x) t155 ON t1.x = t155.x JOIN (SELECT 1 AS x) t156 ON t1.x = t156.x JOIN (SELECT 1 AS x) t157 ON t1.x = t157.x JOIN (SELECT 1 AS x) t158 ON t1.x = t158.x JOIN (SELECT 1 AS x) t159 ON t1.x = t159.x JOIN (SELECT 1 AS x) t160 ON t1.x = t160.x JOIN (SELECT 1 AS x) t161 ON t1.x = t161.x JOIN (SELECT 1 AS x) t162 ON t1.x = t162.x JOIN (SELECT 1 AS x) t163 ON t1.x = t163.x JOIN (SELECT 1 AS x) t164 ON t1.x = t164.x JOIN (SELECT 1 AS x) t165 ON t1.x = t165.x JOIN (SELECT 1 AS x) t166 ON t1.x = t166.x JOIN (SELECT 1 AS x) t167 ON t1.x = t167.x JOIN (SELECT 1 AS x) t168 ON t1.x = t168.x JOIN (SELECT 1 AS x) t169 ON t1.x = t169.x JOIN (SELECT 1 AS x) t170 ON t1.x = t170.x JOIN (SELECT 1 AS x) t171 ON t1.x = t171.x JOIN (SELECT 1 AS x) t172 ON t1.x = t172.x JOIN (SELECT 1 AS x) t173 ON t1.x = t173.x JOIN (SELECT 1 AS x) t174 ON t1.x = t174.x JOIN (SELECT 1 AS x) t175 ON t1.x = t175.x JOIN (SELECT 1 AS x) t176 ON t1.x = t176.x JOIN (SELECT 1 AS x) t177 ON t1.x = t177.x JOIN (SELECT 1 AS x) t178 ON t1.x = t178.x JOIN (SELECT 1 AS x) t179 ON t1.x = t179.x JOIN (SELECT 1 AS x) t180 ON t1.x = t180.x JOIN (SELECT 1 AS x) t181 ON t1.x = t181.x JOIN (SELECT 1 AS x) t182 ON t1.x = t182.x JOIN (SELECT 1 AS x) t183 ON t1.x = t183.x JOIN (SELECT 1 AS x) t184 ON t1.x = t184.x JOIN (SELECT 1 AS x) t185 ON t1.x = t185.x JOIN (SELECT 1 AS x) t186 ON t1.x = t186.x JOIN (SELECT 1 AS x) t187 ON t1.x = t187.x JOIN (SELECT 1 AS x) t188 ON t1.x = t188.x JOIN (SELECT 1 AS x) t189 ON t1.x = t189.x JOIN (SELECT 1 AS x) t190 ON t1.x = t190.x JOIN (SELECT 1 AS x) t191 ON t1.x = t191.x JOIN (SELECT 1 AS x) t192 ON t1.x = t192.x JOIN (SELECT 1 AS x) t193 ON t1.x = t193.x JOIN (SELECT 1 AS x) t194 ON t1.x = t194.x JOIN (SELECT 1 AS x) t195 ON t1.x = t195.x JOIN (SELECT 1 AS x) t196 ON t1.x = t196.x JOIN (SELECT 1 AS x) t197 ON t1.x = t197.x JOIN (SELECT 1 AS x) t198 ON t1.x = t198.x JOIN (SELECT 1 AS x) t199 ON t1.x = t199.x JOIN (SELECT 1 AS x) t200 ON t1.x = t200.x JOIN (SELECT 1 AS x) t201 ON t1.x = t201.x JOIN (SELECT 1 AS x) t202 ON t1.x = t202.x JOIN (SELECT 1 AS x) t203 ON t1.x = t203.x JOIN (SELECT 1 AS x) t204 ON t1.x = t204.x JOIN (SELECT 1 AS x) t205 ON t1.x = t205.x JOIN (SELECT 1 AS x) t206 ON t1.x = t206.x JOIN (SELECT 1 AS x) t207 ON t1.x = t207.x JOIN (SELECT 1 AS x) t208 ON t1.x = t208.x JOIN (SELECT 1 AS x) t209 ON t1.x = t209.x JOIN (SELECT 1 AS x) t210 ON t1.x = t210.x JOIN (SELECT 1 AS x) t211 ON t1.x = t211.x JOIN (SELECT 1 AS x) t212 ON t1.x = t212.x JOIN (SELECT 1 AS x) t213 ON t1.x = t213.x JOIN (SELECT 1 AS x) t214 ON t1.x = t214.x JOIN (SELECT 1 AS x) t215 ON t1.x = t215.x JOIN (SELECT 1 AS x) t216 ON t1.x = t216.x JOIN (SELECT 1 AS x) t217 ON t1.x = t217.x JOIN (SELECT 1 AS x) t218 ON t1.x = t218.x JOIN (SELECT 1 AS x) t219 ON t1.x = t219.x JOIN (SELECT 1 AS x) t220 ON t1.x = t220.x JOIN (SELECT 1 AS x) t221 ON t1.x = t221.x JOIN (SELECT 1 AS x) t222 ON t1.x = t222.x JOIN (SELECT 1 AS x) t223 ON t1.x = t223.x JOIN (SELECT 1 AS x) t224 ON t1.x = t224.x JOIN (SELECT 1 AS x) t225 ON t1.x = t225.x JOIN (SELECT 1 AS x) t226 ON t1.x = t226.x JOIN (SELECT 1 AS x) t227 ON t1.x = t227.x JOIN (SELECT 1 AS x) t228 ON t1.x = t228.x JOIN (SELECT 1 AS x) t229 ON t1.x = t229.x JOIN (SELECT 1 AS x) t230 ON t1.x = t230.x JOIN (SELECT 1 AS x) t231 ON t1.x = t231.x JOIN (SELECT 1 AS x) t232 ON t1.x = t232.x JOIN (SELECT 1 AS x) t233 ON t1.x = t233.x JOIN (SELECT 1 AS x) t234 ON t1.x = t234.x JOIN (SELECT 1 AS x) t235 ON t1.x = t235.x JOIN (SELECT 1 AS x) t236 ON t1.x = t236.x JOIN (SELECT 1 AS x) t237 ON t1.x = t237.x JOIN (SELECT 1 AS x) t238 ON t1.x = t238.x JOIN (SELECT 1 AS x) t239 ON t1.x = t239.x JOIN (SELECT 1 AS x) t240 ON t1.x = t240.x JOIN (SELECT 1 AS x) t241 ON t1.x = t241.x JOIN (SELECT 1 AS x) t242 ON t1.x = t242.x JOIN (SELECT 1 AS x) t243 ON t1.x = t243.x JOIN (SELECT 1 AS x) t244 ON t1.x = t244.x JOIN (SELECT 1 AS x) t245 ON t1.x = t245.x JOIN (SELECT 1 AS x) t246 ON t1.x = t246.x JOIN (SELECT 1 AS x) t247 ON t1.x = t247.x JOIN (SELECT 1 AS x) t248 ON t1.x = t248.x JOIN (SELECT 1 AS x) t249 ON t1.x = t249.x JOIN (SELECT 1 AS x) t250 ON t1.x = t250.x JOIN (SELECT 1 AS x) t251 ON t1.x = t251.x JOIN (SELECT 1 AS x) t252 ON t1.x = t252.x JOIN (SELECT 1 AS x) t253 ON t1.x = t253.x JOIN (SELECT 1 AS x) t254 ON t1.x = t254.x JOIN (SELECT 1 AS x) t255 ON t1.x = t255.x JOIN (SELECT 1 AS x) t256 ON t1.x = t256.x JOIN (SELECT 1 AS x) t257 ON t1.x = t257.x JOIN (SELECT 1 AS x) t258 ON t1.x = t258.x JOIN (SELECT 1 AS x) t259 ON t1.x = t259.x JOIN (SELECT 1 AS x) t260 ON t1.x = t260.x JOIN (SELECT 1 AS x) t261 ON t1.x = t261.x JOIN (SELECT 1 AS x) t262 ON t1.x = t262.x JOIN (SELECT 1 AS x) t263 ON t1.x = t263.x JOIN (SELECT 1 AS x) t264 ON t1.x = t264.x JOIN (SELECT 1 AS x) t265 ON t1.x = t265.x JOIN (SELECT 1 AS x) t266 ON t1.x = t266.x JOIN (SELECT 1 AS x) t267 ON t1.x = t267.x JOIN (SELECT 1 AS x) t268 ON t1.x = t268.x JOIN (SELECT 1 AS x) t269 ON t1.x = t269.x JOIN (SELECT 1 AS x) t270 ON t1.x = t270.x JOIN (SELECT 1 AS x) t271 ON t1.x = t271.x JOIN (SELECT 1 AS x) t272 ON t1.x = t272.x JOIN (SELECT 1 AS x) t273 ON t1.x = t273.x JOIN (SELECT 1 AS x) t274 ON t1.x = t274.x JOIN (SELECT 1 AS x) t275 ON t1.x = t275.x JOIN (SELECT 1 AS x) t276 ON t1.x = t276.x JOIN (SELECT 1 AS x) t277 ON t1.x = t277.x JOIN (SELECT 1 AS x) t278 ON t1.x = t278.x JOIN (SELECT 1 AS x) t279 ON t1.x = t279.x JOIN (SELECT 1 AS x) t280 ON t1.x = t280.x JOIN (SELECT 1 AS x) t281 ON t1.x = t281.x JOIN (SELECT 1 AS x) t282 ON t1.x = t282.x JOIN (SELECT 1 AS x) t283 ON t1.x = t283.x JOIN (SELECT 1 AS x) t284 ON t1.x = t284.x JOIN (SELECT 1 AS x) t285 ON t1.x = t285.x JOIN (SELECT 1 AS x) t286 ON t1.x = t286.x JOIN (SELECT 1 AS x) t287 ON t1.x = t287.x JOIN (SELECT 1 AS x) t288 ON t1.x = t288.x JOIN (SELECT 1 AS x) t289 ON t1.x = t289.x JOIN (SELECT 1 AS x) t290 ON t1.x = t290.x JOIN (SELECT 1 AS x) t291 ON t1.x = t291.x JOIN (SELECT 1 AS x) t292 ON t1.x = t292.x JOIN (SELECT 1 AS x) t293 ON t1.x = t293.x JOIN (SELECT 1 AS x) t294 ON t1.x = t294.x JOIN (SELECT 1 AS x) t295 ON t1.x = t295.x JOIN (SELECT 1 AS x) t296 ON t1.x = t296.x JOIN (SELECT 1 AS x) t297 ON t1.x = t297.x JOIN (SELECT 1 AS x) t298 ON t1.x = t298.x JOIN (SELECT 1 AS x) t299 ON t1.x = t299.x JOIN (SELECT 1 AS x) t300 ON t1.x = t300.x JOIN (SELECT 1 AS x) t301 ON t1.x = t301.x JOIN (SELECT 1 AS x) t302 ON t1.x = t302.x JOIN (SELECT 1 AS x) t303 ON t1.x = t303.x JOIN (SELECT 1 AS x) t304 ON t1.x = t304.x JOIN (SELECT 1 AS x) t305 ON t1.x = t305.x JOIN (SELECT 1 AS x) t306 ON t1.x = t306.x JOIN (SELECT 1 AS x) t307 ON t1.x = t307.x JOIN (SELECT 1 AS x) t308 ON t1.x = t308.x JOIN (SELECT 1 AS x) t309 ON t1.x = t309.x JOIN (SELECT 1 AS x) t310 ON t1.x = t310.x JOIN (SELECT 1 AS x) t311 ON t1.x = t311.x JOIN (SELECT 1 AS x) t312 ON t1.x = t312.x JOIN (SELECT 1 AS x) t313 ON t1.x = t313.x JOIN (SELECT 1 AS x) t314 ON t1.x = t314.x JOIN (SELECT 1 AS x) t315 ON t1.x = t315.x JOIN (SELECT 1 AS x) t316 ON t1.x = t316.x JOIN (SELECT 1 AS x) t317 ON t1.x = t317.x JOIN (SELECT 1 AS x) t318 ON t1.x = t318.x JOIN (SELECT 1 AS x) t319 ON t1.x = t319.x JOIN (SELECT 1 AS x) t320 ON t1.x = t320.x JOIN (SELECT 1 AS x) t321 ON t1.x = t321.x JOIN (SELECT 1 AS x) t322 ON t1.x = t322.x JOIN (SELECT 1 AS x) t323 ON t1.x = t323.x JOIN (SELECT 1 AS x) t324 ON t1.x = t324.x JOIN (SELECT 1 AS x) t325 ON t1.x = t325.x JOIN (SELECT 1 AS x) t326 ON t1.x = t326.x JOIN (SELECT 1 AS x) t327 ON t1.x = t327.x JOIN (SELECT 1 AS x) t328 ON t1.x = t328.x JOIN (SELECT 1 AS x) t329 ON t1.x = t329.x JOIN (SELECT 1 AS x) t330 ON t1.x = t330.x JOIN (SELECT 1 AS x) t331 ON t1.x = t331.x JOIN (SELECT 1 AS x) t332 ON t1.x = t332.x JOIN (SELECT 1 AS x) t333 ON t1.x = t333.x JOIN (SELECT 1 AS x) t334 ON t1.x = t334.x JOIN (SELECT 1 AS x) t335 ON t1.x = t335.x JOIN (SELECT 1 AS x) t336 ON t1.x = t336.x JOIN (SELECT 1 AS x) t337 ON t1.x = t337.x JOIN (SELECT 1 AS x) t338 ON t1.x = t338.x JOIN (SELECT 1 AS x) t339 ON t1.x = t339.x JOIN (SELECT 1 AS x) t340 ON t1.x = t340.x JOIN (SELECT 1 AS x) t341 ON t1.x = t341.x JOIN (SELECT 1 AS x) t342 ON t1.x = t342.x JOIN (SELECT 1 AS x) t343 ON t1.x = t343.x JOIN (SELECT 1 AS x) t344 ON t1.x = t344.x JOIN (SELECT 1 AS x) t345 ON t1.x = t345.x JOIN (SELECT 1 AS x) t346 ON t1.x = t346.x JOIN (SELECT 1 AS x) t347 ON t1.x = t347.x JOIN (SELECT 1 AS x) t348 ON t1.x = t348.x JOIN (SELECT 1 AS x) t349 ON t1.x = t349.x JOIN (SELECT 1 AS x) t350 ON t1.x = t350.x JOIN (SELECT 1 AS x) t351 ON t1.x = t351.x JOIN (SELECT 1 AS x) t352 ON t1.x = t352.x JOIN (SELECT 1 AS x) t353 ON t1.x = t353.x JOIN (SELECT 1 AS x) t354 ON t1.x = t354.x JOIN (SELECT 1 AS x) t355 ON t1.x = t355.x JOIN (SELECT 1 AS x) t356 ON t1.x = t356.x JOIN (SELECT 1 AS x) t357 ON t1.x = t357.x JOIN (SELECT 1 AS x) t358 ON t1.x = t358.x JOIN (SELECT 1 AS x) t359 ON t1.x = t359.x JOIN (SELECT 1 AS x) t360 ON t1.x = t360.x JOIN (SELECT 1 AS x) t361 ON t1.x = t361.x JOIN (SELECT 1 AS x) t362 ON t1.x = t362.x JOIN (SELECT 1 AS x) t363 ON t1.x = t363.x JOIN (SELECT 1 AS x) t364 ON t1.x = t364.x JOIN (SELECT 1 AS x) t365 ON t1.x = t365.x JOIN (SELECT 1 AS x) t366 ON t1.x = t366.x JOIN (SELECT 1 AS x) t367 ON t1.x = t367.x JOIN (SELECT 1 AS x) t368 ON t1.x = t368.x JOIN (SELECT 1 AS x) t369 ON t1.x = t369.x JOIN (SELECT 1 AS x) t370 ON t1.x = t370.x JOIN (SELECT 1 AS x) t371 ON t1.x = t371.x JOIN (SELECT 1 AS x) t372 ON t1.x = t372.x JOIN (SELECT 1 AS x) t373 ON t1.x = t373.x JOIN (SELECT 1 AS x) t374 ON t1.x = t374.x JOIN (SELECT 1 AS x) t375 ON t1.x = t375.x JOIN (SELECT 1 AS x) t376 ON t1.x = t376.x JOIN (SELECT 1 AS x) t377 ON t1.x = t377.x JOIN (SELECT 1 AS x) t378 ON t1.x = t378.x JOIN (SELECT 1 AS x) t379 ON t1.x = t379.x JOIN (SELECT 1 AS x) t380 ON t1.x = t380.x JOIN (SELECT 1 AS x) t381 ON t1.x = t381.x JOIN (SELECT 1 AS x) t382 ON t1.x = t382.x JOIN (SELECT 1 AS x) t383 ON t1.x = t383.x JOIN (SELECT 1 AS x) t384 ON t1.x = t384.x JOIN (SELECT 1 AS x) t385 ON t1.x = t385.x JOIN (SELECT 1 AS x) t386 ON t1.x = t386.x JOIN (SELECT 1 AS x) t387 ON t1.x = t387.x JOIN (SELECT 1 AS x) t388 ON t1.x = t388.x JOIN (SELECT 1 AS x) t389 ON t1.x = t389.x JOIN (SELECT 1 AS x) t390 ON t1.x = t390.x JOIN (SELECT 1 AS x) t391 ON t1.x = t391.x JOIN (SELECT 1 AS x) t392 ON t1.x = t392.x JOIN (SELECT 1 AS x) t393 ON t1.x = t393.x JOIN (SELECT 1 AS x) t394 ON t1.x = t394.x JOIN (SELECT 1 AS x) t395 ON t1.x = t395.x JOIN (SELECT 1 AS x) t396 ON t1.x = t396.x JOIN (SELECT 1 AS x) t397 ON t1.x = t397.x JOIN (SELECT 1 AS x) t398 ON t1.x = t398.x JOIN (SELECT 1 AS x) t399 ON t1.x = t399.x JOIN (SELECT 1 AS x) t400 ON t1.x = t400.x JOIN (SELECT 1 AS x) t401 ON t1.x = t401.x JOIN (SELECT 1 AS x) t402 ON t1.x = t402.x JOIN (SELECT 1 AS x) t403 ON t1.x = t403.x JOIN (SELECT 1 AS x) t404 ON t1.x = t404.x JOIN (SELECT 1 AS x) t405 ON t1.x = t405.x JOIN (SELECT 1 AS x) t406 ON t1.x = t406.x JOIN (SELECT 1 AS x) t407 ON t1.x = t407.x JOIN (SELECT 1 AS x) t408 ON t1.x = t408.x JOIN (SELECT 1 AS x) t409 ON t1.x = t409.x JOIN (SELECT 1 AS x) t410 ON t1.x = t410.x JOIN (SELECT 1 AS x) t411 ON t1.x = t411.x JOIN (SELECT 1 AS x) t412 ON t1.x = t412.x JOIN (SELECT 1 AS x) t413 ON t1.x = t413.x JOIN (SELECT 1 AS x) t414 ON t1.x = t414.x JOIN (SELECT 1 AS x) t415 ON t1.x = t415.x JOIN (SELECT 1 AS x) t416 ON t1.x = t416.x JOIN (SELECT 1 AS x) t417 ON t1.x = t417.x JOIN (SELECT 1 AS x) t418 ON t1.x = t418.x JOIN (SELECT 1 AS x) t419 ON t1.x = t419.x JOIN (SELECT 1 AS x) t420 ON t1.x = t420.x JOIN (SELECT 1 AS x) t421 ON t1.x = t421.x JOIN (SELECT 1 AS x) t422 ON t1.x = t422.x JOIN (SELECT 1 AS x) t423 ON t1.x = t423.x JOIN (SELECT 1 AS x) t424 ON t1.x = t424.x JOIN (SELECT 1 AS x) t425 ON t1.x = t425.x JOIN (SELECT 1 AS x) t426 ON t1.x = t426.x JOIN (SELECT 1 AS x) t427 ON t1.x = t427.x JOIN (SELECT 1 AS x) t428 ON t1.x = t428.x JOIN (SELECT 1 AS x) t429 ON t1.x = t429.x JOIN (SELECT 1 AS x) t430 ON t1.x = t430.x JOIN (SELECT 1 AS x) t431 ON t1.x = t431.x JOIN (SELECT 1 AS x) t432 ON t1.x = t432.x JOIN (SELECT 1 AS x) t433 ON t1.x = t433.x JOIN (SELECT 1 AS x) t434 ON t1.x = t434.x JOIN (SELECT 1 AS x) t435 ON t1.x = t435.x JOIN (SELECT 1 AS x) t436 ON t1.x = t436.x JOIN (SELECT 1 AS x) t437 ON t1.x = t437.x JOIN (SELECT 1 AS x) t438 ON t1.x = t438.x JOIN (SELECT 1 AS x) t439 ON t1.x = t439.x JOIN (SELECT 1 AS x) t440 ON t1.x = t440.x JOIN (SELECT 1 AS x) t441 ON t1.x = t441.x JOIN (SELECT 1 AS x) t442 ON t1.x = t442.x JOIN (SELECT 1 AS x) t443 ON t1.x = t443.x JOIN (SELECT 1 AS x) t444 ON t1.x = t444.x JOIN (SELECT 1 AS x) t445 ON t1.x = t445.x JOIN (SELECT 1 AS x) t446 ON t1.x = t446.x JOIN (SELECT 1 AS x) t447 ON t1.x = t447.x JOIN (SELECT 1 AS x) t448 ON t1.x = t448.x JOIN (SELECT 1 AS x) t449 ON t1.x = t449.x JOIN (SELECT 1 AS x) t450 ON t1.x = t450.x JOIN (SELECT 1 AS x) t451 ON t1.x = t451.x JOIN (SELECT 1 AS x) t452 ON t1.x = t452.x JOIN (SELECT 1 AS x) t453 ON t1.x = t453.x JOIN (SELECT 1 AS x) t454 ON t1.x = t454.x JOIN (SELECT 1 AS x) t455 ON t1.x = t455.x JOIN (SELECT 1 AS x) t456 ON t1.x = t456.x JOIN (SELECT 1 AS x) t457 ON t1.x = t457.x JOIN (SELECT 1 AS x) t458 ON t1.x = t458.x JOIN (SELECT 1 AS x) t459 ON t1.x = t459.x JOIN (SELECT 1 AS x) t460 ON t1.x = t460.x JOIN (SELECT 1 AS x) t461 ON t1.x = t461.x JOIN (SELECT 1 AS x) t462 ON t1.x = t462.x JOIN (SELECT 1 AS x) t463 ON t1.x = t463.x JOIN (SELECT 1 AS x) t464 ON t1.x = t464.x JOIN (SELECT 1 AS x) t465 ON t1.x = t465.x JOIN (SELECT 1 AS x) t466 ON t1.x = t466.x JOIN (SELECT 1 AS x) t467 ON t1.x = t467.x JOIN (SELECT 1 AS x) t468 ON t1.x = t468.x JOIN (SELECT 1 AS x) t469 ON t1.x = t469.x JOIN (SELECT 1 AS x) t470 ON t1.x = t470.x JOIN (SELECT 1 AS x) t471 ON t1.x = t471.x JOIN (SELECT 1 AS x) t472 ON t1.x = t472.x JOIN (SELECT 1 AS x) t473 ON t1.x = t473.x JOIN (SELECT 1 AS x) t474 ON t1.x = t474.x JOIN (SELECT 1 AS x) t475 ON t1.x = t475.x JOIN (SELECT 1 AS x) t476 ON t1.x = t476.x JOIN (SELECT 1 AS x) t477 ON t1.x = t477.x JOIN (SELECT 1 AS x) t478 ON t1.x = t478.x JOIN (SELECT 1 AS x) t479 ON t1.x = t479.x JOIN (SELECT 1 AS x) t480 ON t1.x = t480.x JOIN (SELECT 1 AS x) t481 ON t1.x = t481.x JOIN (SELECT 1 AS x) t482 ON t1.x = t482.x JOIN (SELECT 1 AS x) t483 ON t1.x = t483.x JOIN (SELECT 1 AS x) t484 ON t1.x = t484.x JOIN (SELECT 1 AS x) t485 ON t1.x = t485.x JOIN (SELECT 1 AS x) t486 ON t1.x = t486.x JOIN (SELECT 1 AS x) t487 ON t1.x = t487.x JOIN (SELECT 1 AS x) t488 ON t1.x = t488.x JOIN (SELECT 1 AS x) t489 ON t1.x = t489.x JOIN (SELECT 1 AS x) t490 ON t1.x = t490.x JOIN (SELECT 1 AS x) t491 ON t1.x = t491.x JOIN (SELECT 1 AS x) t492 ON t1.x = t492.x JOIN (SELECT 1 AS x) t493 ON t1.x = t493.x JOIN (SELECT 1 AS x) t494 ON t1.x = t494.x JOIN (SELECT 1 AS x) t495 ON t1.x = t495.x JOIN (SELECT 1 AS x) t496 ON t1.x = t496.x JOIN (SELECT 1 AS x) t497 ON t1.x = t497.x JOIN (SELECT 1 AS x) t498 ON t1.x = t498.x JOIN (SELECT 1 AS x) t499 ON t1.x = t499.x JOIN (SELECT 1 AS x) t500 ON t1.x = t500.x JOIN (SELECT 1 AS x) t501 ON t1.x = t501.x JOIN (SELECT 1 AS x) t502 ON t1.x = t502.x JOIN (SELECT 1 AS x) t503 ON t1.x = t503.x JOIN (SELECT 1 AS x) t504 ON t1.x = t504.x JOIN (SELECT 1 AS x) t505 ON t1.x = t505.x JOIN (SELECT 1 AS x) t506 ON t1.x = t506.x JOIN (SELECT 1 AS x) t507 ON t1.x = t507.x JOIN (SELECT 1 AS x) t508 ON t1.x = t508.x JOIN (SELECT 1 AS x) t509 ON t1.x = t509.x JOIN (SELECT 1 AS x) t510 ON t1.x = t510.x JOIN (SELECT 1 AS x) t511 ON t1.x = t511.x JOIN (SELECT 1 AS x) t512 ON t1.x = t512.x JOIN (SELECT 1 AS x) t513 ON t1.x = t513.x JOIN (SELECT 1 AS x) t514 ON t1.x = t514.x JOIN (SELECT 1 AS x) t515 ON t1.x = t515.x JOIN (SELECT 1 AS x) t516 ON t1.x = t516.x JOIN (SELECT 1 AS x) t517 ON t1.x = t517.x JOIN (SELECT 1 AS x) t518 ON t1.x = t518.x JOIN (SELECT 1 AS x) t519 ON t1.x = t519.x JOIN (SELECT 1 AS x) t520 ON t1.x = t520.x JOIN (SELECT 1 AS x) t521 ON t1.x = t521.x JOIN (SELECT 1 AS x) t522 ON t1.x = t522.x JOIN (SELECT 1 AS x) t523 ON t1.x = t523.x JOIN (SELECT 1 AS x) t524 ON t1.x = t524.x JOIN (SELECT 1 AS x) t525 ON t1.x = t525.x JOIN (SELECT 1 AS x) t526 ON t1.x = t526.x JOIN (SELECT 1 AS x) t527 ON t1.x = t527.x JOIN (SELECT 1 AS x) t528 ON t1.x = t528.x JOIN (SELECT 1 AS x) t529 ON t1.x = t529.x JOIN (SELECT 1 AS x) t530 ON t1.x = t530.x JOIN (SELECT 1 AS x) t531 ON t1.x = t531.x JOIN (SELECT 1 AS x) t532 ON t1.x = t532.x JOIN (SELECT 1 AS x) t533 ON t1.x = t533.x JOIN (SELECT 1 AS x) t534 ON t1.x = t534.x JOIN (SELECT 1 AS x) t535 ON t1.x = t535.x JOIN (SELECT 1 AS x) t536 ON t1.x = t536.x JOIN (SELECT 1 AS x) t537 ON t1.x = t537.x JOIN (SELECT 1 AS x) t538 ON t1.x = t538.x JOIN (SELECT 1 AS x) t539 ON t1.x = t539.x JOIN (SELECT 1 AS x) t540 ON t1.x = t540.x JOIN (SELECT 1 AS x) t541 ON t1.x = t541.x JOIN (SELECT 1 AS x) t542 ON t1.x = t542.x JOIN (SELECT 1 AS x) t543 ON t1.x = t543.x JOIN (SELECT 1 AS x) t544 ON t1.x = t544.x JOIN (SELECT 1 AS x) t545 ON t1.x = t545.x JOIN (SELECT 1 AS x) t546 ON t1.x = t546.x JOIN (SELECT 1 AS x) t547 ON t1.x = t547.x JOIN (SELECT 1 AS x) t548 ON t1.x = t548.x JOIN (SELECT 1 AS x) t549 ON t1.x = t549.x JOIN (SELECT 1 AS x) t550 ON t1.x = t550.x JOIN (SELECT 1 AS x) t551 ON t1.x = t551.x JOIN (SELECT 1 AS x) t552 ON t1.x = t552.x JOIN (SELECT 1 AS x) t553 ON t1.x = t553.x JOIN (SELECT 1 AS x) t554 ON t1.x = t554.x JOIN (SELECT 1 AS x) t555 ON t1.x = t555.x JOIN (SELECT 1 AS x) t556 ON t1.x = t556.x JOIN (SELECT 1 AS x) t557 ON t1.x = t557.x JOIN (SELECT 1 AS x) t558 ON t1.x = t558.x JOIN (SELECT 1 AS x) t559 ON t1.x = t559.x JOIN (SELECT 1 AS x) t560 ON t1.x = t560.x JOIN (SELECT 1 AS x) t561 ON t1.x = t561.x JOIN (SELECT 1 AS x) t562 ON t1.x = t562.x JOIN (SELECT 1 AS x) t563 ON t1.x = t563.x JOIN (SELECT 1 AS x) t564 ON t1.x = t564.x JOIN (SELECT 1 AS x) t565 ON t1.x = t565.x JOIN (SELECT 1 AS x) t566 ON t1.x = t566.x JOIN (SELECT 1 AS x) t567 ON t1.x = t567.x JOIN (SELECT 1 AS x) t568 ON t1.x = t568.x JOIN (SELECT 1 AS x) t569 ON t1.x = t569.x JOIN (SELECT 1 AS x) t570 ON t1.x = t570.x JOIN (SELECT 1 AS x) t571 ON t1.x = t571.x JOIN (SELECT 1 AS x) t572 ON t1.x = t572.x JOIN (SELECT 1 AS x) t573 ON t1.x = t573.x JOIN (SELECT 1 AS x) t574 ON t1.x = t574.x JOIN (SELECT 1 AS x) t575 ON t1.x = t575.x JOIN (SELECT 1 AS x) t576 ON t1.x = t576.x JOIN (SELECT 1 AS x) t577 ON t1.x = t577.x JOIN (SELECT 1 AS x) t578 ON t1.x = t578.x JOIN (SELECT 1 AS x) t579 ON t1.x = t579.x JOIN (SELECT 1 AS x) t580 ON t1.x = t580.x JOIN (SELECT 1 AS x) t581 ON t1.x = t581.x JOIN (SELECT 1 AS x) t582 ON t1.x = t582.x JOIN (SELECT 1 AS x) t583 ON t1.x = t583.x JOIN (SELECT 1 AS x) t584 ON t1.x = t584.x JOIN (SELECT 1 AS x) t585 ON t1.x = t585.x JOIN (SELECT 1 AS x) t586 ON t1.x = t586.x JOIN (SELECT 1 AS x) t587 ON t1.x = t587.x JOIN (SELECT 1 AS x) t588 ON t1.x = t588.x JOIN (SELECT 1 AS x) t589 ON t1.x = t589.x JOIN (SELECT 1 AS x) t590 ON t1.x = t590.x JOIN (SELECT 1 AS x) t591 ON t1.x = t591.x JOIN (SELECT 1 AS x) t592 ON t1.x = t592.x JOIN (SELECT 1 AS x) t593 ON t1.x = t593.x JOIN (SELECT 1 AS x) t594 ON t1.x = t594.x JOIN (SELECT 1 AS x) t595 ON t1.x = t595.x JOIN (SELECT 1 AS x) t596 ON t1.x = t596.x JOIN (SELECT 1 AS x) t597 ON t1.x = t597.x JOIN (SELECT 1 AS x) t598 ON t1.x = t598.x JOIN (SELECT 1 AS x) t599 ON t1.x = t599.x JOIN (SELECT 1 AS x) t600 ON t1.x = t600.x JOIN (SELECT 1 AS x) t601 ON t1.x = t601.x JOIN (SELECT 1 AS x) t602 ON t1.x = t602.x JOIN (SELECT 1 AS x) t603 ON t1.x = t603.x JOIN (SELECT 1 AS x) t604 ON t1.x = t604.x JOIN (SELECT 1 AS x) t605 ON t1.x = t605.x JOIN (SELECT 1 AS x) t606 ON t1.x = t606.x JOIN (SELECT 1 AS x) t607 ON t1.x = t607.x JOIN (SELECT 1 AS x) t608 ON t1.x = t608.x JOIN (SELECT 1 AS x) t609 ON t1.x = t609.x JOIN (SELECT 1 AS x) t610 ON t1.x = t610.x JOIN (SELECT 1 AS x) t611 ON t1.x = t611.x JOIN (SELECT 1 AS x) t612 ON t1.x = t612.x JOIN (SELECT 1 AS x) t613 ON t1.x = t613.x JOIN (SELECT 1 AS x) t614 ON t1.x = t614.x JOIN (SELECT 1 AS x) t615 ON t1.x = t615.x JOIN (SELECT 1 AS x) t616 ON t1.x = t616.x JOIN (SELECT 1 AS x) t617 ON t1.x = t617.x JOIN (SELECT 1 AS x) t618 ON t1.x = t618.x JOIN (SELECT 1 AS x) t619 ON t1.x = t619.x JOIN (SELECT 1 AS x) t620 ON t1.x = t620.x JOIN (SELECT 1 AS x) t621 ON t1.x = t621.x JOIN (SELECT 1 AS x) t622 ON t1.x = t622.x JOIN (SELECT 1 AS x) t623 ON t1.x = t623.x JOIN (SELECT 1 AS x) t624 ON t1.x = t624.x JOIN (SELECT 1 AS x) t625 ON t1.x = t625.x JOIN (SELECT 1 AS x) t626 ON t1.x = t626.x JOIN (SELECT 1 AS x) t627 ON t1.x = t627.x JOIN (SELECT 1 AS x) t628 ON t1.x = t628.x JOIN (SELECT 1 AS x) t629 ON t1.x = t629.x JOIN (SELECT 1 AS x) t630 ON t1.x = t630.x JOIN (SELECT 1 AS x) t631 ON t1.x = t631.x JOIN (SELECT 1 AS x) t632 ON t1.x = t632.x JOIN (SELECT 1 AS x) t633 ON t1.x = t633.x JOIN (SELECT 1 AS x) t634 ON t1.x = t634.x JOIN (SELECT 1 AS x) t635 ON t1.x = t635.x JOIN (SELECT 1 AS x) t636 ON t1.x = t636.x JOIN (SELECT 1 AS x) t637 ON t1.x = t637.x JOIN (SELECT 1 AS x) t638 ON t1.x = t638.x JOIN (SELECT 1 AS x) t639 ON t1.x = t639.x JOIN (SELECT 1 AS x) t640 ON t1.x = t640.x JOIN (SELECT 1 AS x) t641 ON t1.x = t641.x JOIN (SELECT 1 AS x) t642 ON t1.x = t642.x JOIN (SELECT 1 AS x) t643 ON t1.x = t643.x JOIN (SELECT 1 AS x) t644 ON t1.x = t644.x JOIN (SELECT 1 AS x) t645 ON t1.x = t645.x JOIN (SELECT 1 AS x) t646 ON t1.x = t646.x JOIN (SELECT 1 AS x) t647 ON t1.x = t647.x JOIN (SELECT 1 AS x) t648 ON t1.x = t648.x JOIN (SELECT 1 AS x) t649 ON t1.x = t649.x JOIN (SELECT 1 AS x) t650 ON t1.x = t650.x JOIN (SELECT 1 AS x) t651 ON t1.x = t651.x JOIN (SELECT 1 AS x) t652 ON t1.x = t652.x JOIN (SELECT 1 AS x) t653 ON t1.x = t653.x JOIN (SELECT 1 AS x) t654 ON t1.x = t654.x JOIN (SELECT 1 AS x) t655 ON t1.x = t655.x JOIN (SELECT 1 AS x) t656 ON t1.x = t656.x JOIN (SELECT 1 AS x) t657 ON t1.x = t657.x JOIN (SELECT 1 AS x) t658 ON t1.x = t658.x JOIN (SELECT 1 AS x) t659 ON t1.x = t659.x JOIN (SELECT 1 AS x) t660 ON t1.x = t660.x JOIN (SELECT 1 AS x) t661 ON t1.x = t661.x JOIN (SELECT 1 AS x) t662 ON t1.x = t662.x JOIN (SELECT 1 AS x) t663 ON t1.x = t663.x JOIN (SELECT 1 AS x) t664 ON t1.x = t664.x JOIN (SELECT 1 AS x) t665 ON t1.x = t665.x JOIN (SELECT 1 AS x) t666 ON t1.x = t666.x From a5e6f948ee7fc654717b8deb96f170838e396f6e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 23 Apr 2024 13:40:10 +0000 Subject: [PATCH 0277/1018] Docs: Fix documented default of cgroup_memory_watcher_soft_limit_ratio --- docs/en/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 24248209025..3a20c7b758b 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -523,7 +523,7 @@ See settings `cgroups_memory_usage_observer_wait_time` and `cgroup_memory_watche Type: Double -Default: 0.95 +Default: 0.9 ## max_table_size_to_drop From f0faac2e8bcef83289e858e4cbfe374f68cf2ce4 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 23 Apr 2024 14:02:26 +0000 Subject: [PATCH 0278/1018] Move MergeTree*BloomFilter into the same header/source file --- .../MergeTreeIndexAggregatorBloomFilter.cpp | 65 -- .../MergeTreeIndexAggregatorBloomFilter.h | 30 - .../MergeTree/MergeTreeIndexBloomFilter.cpp | 866 +++++++++++++++++- .../MergeTree/MergeTreeIndexBloomFilter.h | 121 ++- .../MergeTreeIndexConditionBloomFilter.cpp | 729 --------------- .../MergeTreeIndexConditionBloomFilter.h | 87 -- .../MergeTreeIndexGranuleBloomFilter.cpp | 102 --- .../MergeTreeIndexGranuleBloomFilter.h | 35 - 8 files changed, 979 insertions(+), 1056 deletions(-) delete mode 100644 src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp delete mode 100644 src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.h delete mode 100644 src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp delete mode 100644 src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h delete mode 100644 src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp delete mode 100644 src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.h diff --git a/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp deleted file mode 100644 index c69c54f1c0d..00000000000 --- a/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp +++ /dev/null @@ -1,65 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -MergeTreeIndexAggregatorBloomFilter::MergeTreeIndexAggregatorBloomFilter( - size_t bits_per_row_, size_t hash_functions_, const Names & columns_name_) - : bits_per_row(bits_per_row_), hash_functions(hash_functions_), index_columns_name(columns_name_), column_hashes(columns_name_.size()) -{ - assert(bits_per_row != 0); - assert(hash_functions != 0); -} - -bool MergeTreeIndexAggregatorBloomFilter::empty() const -{ - return !total_rows; -} - -MergeTreeIndexGranulePtr MergeTreeIndexAggregatorBloomFilter::getGranuleAndReset() -{ - const auto granule = std::make_shared(bits_per_row, hash_functions, column_hashes); - total_rows = 0; - column_hashes.clear(); - return granule; -} - -void MergeTreeIndexAggregatorBloomFilter::update(const Block & block, size_t * pos, size_t limit) -{ - if (*pos >= block.rows()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "The provided position is not less than the number of block rows. " - "Position: {}, Block rows: {}.", *pos, block.rows()); - - Block granule_index_block; - size_t max_read_rows = std::min(block.rows() - *pos, limit); - - for (size_t column = 0; column < index_columns_name.size(); ++column) - { - const auto & column_and_type = block.getByName(index_columns_name[column]); - auto index_column = BloomFilterHash::hashWithColumn(column_and_type.type, column_and_type.column, *pos, max_read_rows); - - const auto & index_col = checkAndGetColumn(index_column.get()); - const auto & index_data = index_col->getData(); - for (const auto & hash: index_data) - column_hashes[column].insert(hash); - } - - *pos += max_read_rows; - total_rows += max_read_rows; -} - -} diff --git a/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.h b/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.h deleted file mode 100644 index d20653b7689..00000000000 --- a/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.h +++ /dev/null @@ -1,30 +0,0 @@ -#pragma once - -#include -#include -#include - -namespace DB -{ - -class MergeTreeIndexAggregatorBloomFilter final : public IMergeTreeIndexAggregator -{ -public: - MergeTreeIndexAggregatorBloomFilter(size_t bits_per_row_, size_t hash_functions_, const Names & columns_name_); - - bool empty() const override; - - MergeTreeIndexGranulePtr getGranuleAndReset() override; - - void update(const Block & block, size_t * pos, size_t limit) override; - -private: - size_t bits_per_row; - size_t hash_functions; - const Names index_columns_name; - - std::vector> column_hashes; - size_t total_rows = 0; -}; - -} diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp index dbd33609a00..ed091022a91 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp @@ -1,13 +1,36 @@ #include -#include -#include -#include -#include -#include -#include + +#include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include +#include +#include +#include +#include +#include #include +#include +#include +#include +#include +#include +#include +#include +#include namespace DB @@ -17,8 +40,839 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int INCORRECT_QUERY; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int LOGICAL_ERROR; +} + +MergeTreeIndexGranuleBloomFilter::MergeTreeIndexGranuleBloomFilter(size_t bits_per_row_, size_t hash_functions_, size_t index_columns_) + : bits_per_row(bits_per_row_), hash_functions(hash_functions_), bloom_filters(index_columns_) +{ + total_rows = 0; + for (size_t column = 0; column < index_columns_; ++column) + bloom_filters[column] = std::make_shared(bits_per_row, hash_functions, 0); +} + +MergeTreeIndexGranuleBloomFilter::MergeTreeIndexGranuleBloomFilter( + size_t bits_per_row_, size_t hash_functions_, const std::vector>& column_hashes_) + : bits_per_row(bits_per_row_), hash_functions(hash_functions_), bloom_filters(column_hashes_.size()) +{ + if (column_hashes_.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Granule_index_blocks empty or total_rows is zero."); + + size_t bloom_filter_max_size = 0; + for (const auto & column_hash : column_hashes_) + bloom_filter_max_size = std::max(bloom_filter_max_size, column_hash.size()); + + static size_t atom_size = 8; + + // If multiple columns are given, we will initialize all the bloom filters + // with the size of the highest-cardinality one. This is done for compatibility with + // existing binary serialization format + total_rows = bloom_filter_max_size; + size_t bytes_size = (bits_per_row * total_rows + atom_size - 1) / atom_size; + + for (size_t column = 0, columns = column_hashes_.size(); column < columns; ++column) + { + bloom_filters[column] = std::make_shared(bytes_size, hash_functions, 0); + fillingBloomFilter(bloom_filters[column], column_hashes_[column]); + } +} + +bool MergeTreeIndexGranuleBloomFilter::empty() const +{ + return !total_rows; +} + +void MergeTreeIndexGranuleBloomFilter::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) +{ + if (version != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown index version {}.", version); + + readVarUInt(total_rows, istr); + + static size_t atom_size = 8; + size_t bytes_size = (bits_per_row * total_rows + atom_size - 1) / atom_size; + size_t read_size = bytes_size; + for (auto & filter : bloom_filters) + { + filter->resize(bytes_size); +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + read_size = filter->getFilter().size() * sizeof(BloomFilter::UnderType); +#endif + istr.readStrict(reinterpret_cast(filter->getFilter().data()), read_size); + } +} + +void MergeTreeIndexGranuleBloomFilter::serializeBinary(WriteBuffer & ostr) const +{ + if (empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to write empty bloom filter index."); + + writeVarUInt(total_rows, ostr); + + static size_t atom_size = 8; + size_t write_size = (bits_per_row * total_rows + atom_size - 1) / atom_size; + for (const auto & bloom_filter : bloom_filters) + { +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + write_size = bloom_filter->getFilter().size() * sizeof(BloomFilter::UnderType); +#endif + ostr.write(reinterpret_cast(bloom_filter->getFilter().data()), write_size); + } +} + +void MergeTreeIndexGranuleBloomFilter::fillingBloomFilter(BloomFilterPtr & bf, const HashSet &hashes) const +{ + for (const auto & bf_base_hash : hashes) + for (size_t i = 0; i < hash_functions; ++i) + bf->addHashWithSeed(bf_base_hash.getKey(), BloomFilterHash::bf_hash_seed[i]); +} + +namespace +{ + +ColumnWithTypeAndName getPreparedSetInfo(const ConstSetPtr & prepared_set) +{ + if (prepared_set->getDataTypes().size() == 1) + return {prepared_set->getSetElements()[0], prepared_set->getElementsTypes()[0], "dummy"}; + + Columns set_elements; + for (auto & set_element : prepared_set->getSetElements()) + + set_elements.emplace_back(set_element->convertToFullColumnIfConst()); + + return {ColumnTuple::create(set_elements), std::make_shared(prepared_set->getElementsTypes()), "dummy"}; +} + +bool hashMatchesFilter(const BloomFilterPtr& bloom_filter, UInt64 hash, size_t hash_functions) +{ + return std::all_of(BloomFilterHash::bf_hash_seed, + BloomFilterHash::bf_hash_seed + hash_functions, + [&](const auto &hash_seed) + { + return bloom_filter->findHashWithSeed(hash, + hash_seed); + }); +} + +bool maybeTrueOnBloomFilter(const IColumn * hash_column, const BloomFilterPtr & bloom_filter, size_t hash_functions, bool match_all) +{ + const auto * const_column = typeid_cast(hash_column); + const auto * non_const_column = typeid_cast(hash_column); + + if (!const_column && !non_const_column) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Hash column must be Const or UInt64."); + + if (const_column) + { + return hashMatchesFilter(bloom_filter, + const_column->getValue(), + hash_functions); + } + + const ColumnUInt64::Container & hashes = non_const_column->getData(); + + if (match_all) + { + return std::all_of(hashes.begin(), + hashes.end(), + [&](const auto& hash_row) + { + return hashMatchesFilter(bloom_filter, + hash_row, + hash_functions); + }); + } + else + { + return std::any_of(hashes.begin(), + hashes.end(), + [&](const auto& hash_row) + { + return hashMatchesFilter(bloom_filter, + hash_row, + hash_functions); + }); + } +} + +} + +MergeTreeIndexConditionBloomFilter::MergeTreeIndexConditionBloomFilter( + const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, const Block & header_, size_t hash_functions_) + : WithContext(context_), header(header_), hash_functions(hash_functions_) +{ + if (!filter_actions_dag) + { + rpn.push_back(RPNElement::FUNCTION_UNKNOWN); + return; + } + + RPNBuilder builder( + filter_actions_dag->getOutputs().at(0), + context_, + [&](const RPNBuilderTreeNode & node, RPNElement & out) { return extractAtomFromTree(node, out); }); + rpn = std::move(builder).extractRPN(); +} + +bool MergeTreeIndexConditionBloomFilter::alwaysUnknownOrTrue() const +{ + std::vector rpn_stack; + + for (const auto & element : rpn) + { + if (element.function == RPNElement::FUNCTION_UNKNOWN + || element.function == RPNElement::ALWAYS_TRUE) + { + rpn_stack.push_back(true); + } + else if (element.function == RPNElement::FUNCTION_EQUALS + || element.function == RPNElement::FUNCTION_NOT_EQUALS + || element.function == RPNElement::FUNCTION_HAS + || element.function == RPNElement::FUNCTION_HAS_ANY + || element.function == RPNElement::FUNCTION_HAS_ALL + || element.function == RPNElement::FUNCTION_IN + || element.function == RPNElement::FUNCTION_NOT_IN + || element.function == RPNElement::ALWAYS_FALSE) + { + rpn_stack.push_back(false); + } + else if (element.function == RPNElement::FUNCTION_NOT) + { + // do nothing + } + else if (element.function == RPNElement::FUNCTION_AND) + { + auto arg1 = rpn_stack.back(); + rpn_stack.pop_back(); + auto arg2 = rpn_stack.back(); + rpn_stack.back() = arg1 && arg2; + } + else if (element.function == RPNElement::FUNCTION_OR) + { + auto arg1 = rpn_stack.back(); + rpn_stack.pop_back(); + auto arg2 = rpn_stack.back(); + rpn_stack.back() = arg1 || arg2; + } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected function type in KeyCondition::RPNElement"); + } + + return rpn_stack[0]; +} + +bool MergeTreeIndexConditionBloomFilter::mayBeTrueOnGranule(const MergeTreeIndexGranuleBloomFilter * granule) const +{ + std::vector rpn_stack; + const auto & filters = granule->getFilters(); + + for (const auto & element : rpn) + { + if (element.function == RPNElement::FUNCTION_UNKNOWN) + { + rpn_stack.emplace_back(true, true); + } + else if (element.function == RPNElement::FUNCTION_IN + || element.function == RPNElement::FUNCTION_NOT_IN + || element.function == RPNElement::FUNCTION_EQUALS + || element.function == RPNElement::FUNCTION_NOT_EQUALS + || element.function == RPNElement::FUNCTION_HAS + || element.function == RPNElement::FUNCTION_HAS_ANY + || element.function == RPNElement::FUNCTION_HAS_ALL) + { + bool match_rows = true; + bool match_all = element.function == RPNElement::FUNCTION_HAS_ALL; + const auto & predicate = element.predicate; + for (size_t index = 0; match_rows && index < predicate.size(); ++index) + { + const auto & query_index_hash = predicate[index]; + const auto & filter = filters[query_index_hash.first]; + const ColumnPtr & hash_column = query_index_hash.second; + + match_rows = maybeTrueOnBloomFilter(&*hash_column, + filter, + hash_functions, + match_all); + } + + rpn_stack.emplace_back(match_rows, true); + if (element.function == RPNElement::FUNCTION_NOT_EQUALS || element.function == RPNElement::FUNCTION_NOT_IN) + rpn_stack.back() = !rpn_stack.back(); + } + else if (element.function == RPNElement::FUNCTION_NOT) + { + rpn_stack.back() = !rpn_stack.back(); + } + else if (element.function == RPNElement::FUNCTION_OR) + { + auto arg1 = rpn_stack.back(); + rpn_stack.pop_back(); + auto arg2 = rpn_stack.back(); + rpn_stack.back() = arg1 | arg2; + } + else if (element.function == RPNElement::FUNCTION_AND) + { + auto arg1 = rpn_stack.back(); + rpn_stack.pop_back(); + auto arg2 = rpn_stack.back(); + rpn_stack.back() = arg1 & arg2; + } + else if (element.function == RPNElement::ALWAYS_TRUE) + { + rpn_stack.emplace_back(true, false); + } + else if (element.function == RPNElement::ALWAYS_FALSE) + { + rpn_stack.emplace_back(false, true); + } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected function type in KeyCondition::RPNElement"); + } + + if (rpn_stack.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected stack size in KeyCondition::mayBeTrueInRange"); + + return rpn_stack[0].can_be_true; +} + +bool MergeTreeIndexConditionBloomFilter::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNElement & out) +{ + { + Field const_value; + DataTypePtr const_type; + + if (node.tryGetConstant(const_value, const_type)) + { + if (const_value.getType() == Field::Types::UInt64) + { + out.function = const_value.get() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + return true; + } + + if (const_value.getType() == Field::Types::Int64) + { + out.function = const_value.get() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + return true; + } + + if (const_value.getType() == Field::Types::Float64) + { + out.function = const_value.get() != 0.0 ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + return true; + } + } + } + + return traverseFunction(node, out, nullptr /*parent*/); +} + +bool MergeTreeIndexConditionBloomFilter::traverseFunction(const RPNBuilderTreeNode & node, RPNElement & out, const RPNBuilderTreeNode * parent) +{ + bool maybe_useful = false; + + if (node.isFunction()) + { + const auto function = node.toFunctionNode(); + auto arguments_size = function.getArgumentsSize(); + auto function_name = function.getFunctionName(); + + for (size_t i = 0; i < arguments_size; ++i) + { + auto argument = function.getArgumentAt(i); + if (traverseFunction(argument, out, &node)) + maybe_useful = true; + } + + if (arguments_size != 2) + return false; + + auto lhs_argument = function.getArgumentAt(0); + auto rhs_argument = function.getArgumentAt(1); + + if (functionIsInOrGlobalInOperator(function_name)) + { + if (auto future_set = rhs_argument.tryGetPreparedSet(); future_set) + { + if (auto prepared_set = future_set->buildOrderedSetInplace(rhs_argument.getTreeContext().getQueryContext()); prepared_set) + { + if (prepared_set->hasExplicitSetElements()) + { + const auto prepared_info = getPreparedSetInfo(prepared_set); + if (traverseTreeIn(function_name, lhs_argument, prepared_set, prepared_info.type, prepared_info.column, out)) + maybe_useful = true; + } + } + } + } + else if (function_name == "equals" || + function_name == "notEquals" || + function_name == "has" || + function_name == "mapContains" || + function_name == "indexOf" || + function_name == "hasAny" || + function_name == "hasAll") + { + Field const_value; + DataTypePtr const_type; + + if (rhs_argument.tryGetConstant(const_value, const_type)) + { + if (traverseTreeEquals(function_name, lhs_argument, const_type, const_value, out, parent)) + maybe_useful = true; + } + else if (lhs_argument.tryGetConstant(const_value, const_type)) + { + if (traverseTreeEquals(function_name, rhs_argument, const_type, const_value, out, parent)) + maybe_useful = true; + } + } + } + + return maybe_useful; +} + +bool MergeTreeIndexConditionBloomFilter::traverseTreeIn( + const String & function_name, + const RPNBuilderTreeNode & key_node, + const ConstSetPtr & prepared_set, + const DataTypePtr & type, + const ColumnPtr & column, + RPNElement & out) +{ + auto key_node_column_name = key_node.getColumnName(); + + if (header.has(key_node_column_name)) + { + size_t row_size = column->size(); + size_t position = header.getPositionByName(key_node_column_name); + const DataTypePtr & index_type = header.getByPosition(position).type; + const auto & converted_column = castColumn(ColumnWithTypeAndName{column, type, ""}, index_type); + out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithColumn(index_type, converted_column, 0, row_size))); + + if (function_name == "in" || function_name == "globalIn") + out.function = RPNElement::FUNCTION_IN; + + if (function_name == "notIn" || function_name == "globalNotIn") + out.function = RPNElement::FUNCTION_NOT_IN; + + return true; + } + + if (key_node.isFunction()) + { + auto key_node_function = key_node.toFunctionNode(); + auto key_node_function_name = key_node_function.getFunctionName(); + size_t key_node_function_arguments_size = key_node_function.getArgumentsSize(); + + WhichDataType which(type); + + if (which.isTuple() && key_node_function_name == "tuple") + { + const auto & tuple_column = typeid_cast(column.get()); + const auto & tuple_data_type = typeid_cast(type.get()); + + if (tuple_data_type->getElements().size() != key_node_function_arguments_size || tuple_column->getColumns().size() != key_node_function_arguments_size) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal types of arguments of function {}", function_name); + + bool match_with_subtype = false; + const auto & sub_columns = tuple_column->getColumns(); + const auto & sub_data_types = tuple_data_type->getElements(); + + for (size_t index = 0; index < key_node_function_arguments_size; ++index) + match_with_subtype |= traverseTreeIn(function_name, key_node_function.getArgumentAt(index), nullptr, sub_data_types[index], sub_columns[index], out); + + return match_with_subtype; + } + + if (key_node_function_name == "arrayElement") + { + /** Try to parse arrayElement for mapKeys index. + * It is important to ignore keys like column_map['Key'] IN ('') because if key does not exists in map + * we return default value for arrayElement. + * + * We cannot skip keys that does not exist in map if comparison is with default type value because + * that way we skip necessary granules where map key does not exists. + */ + if (!prepared_set) + return false; + + auto default_column_to_check = type->createColumnConstWithDefaultValue(1)->convertToFullColumnIfConst(); + ColumnWithTypeAndName default_column_with_type_to_check { default_column_to_check, type, "" }; + ColumnsWithTypeAndName default_columns_with_type_to_check = {default_column_with_type_to_check}; + auto set_contains_default_value_predicate_column = prepared_set->execute(default_columns_with_type_to_check, false /*negative*/); + const auto & set_contains_default_value_predicate_column_typed = assert_cast(*set_contains_default_value_predicate_column); + bool set_contain_default_value = set_contains_default_value_predicate_column_typed.getData()[0]; + if (set_contain_default_value) + return false; + + auto first_argument = key_node_function.getArgumentAt(0); + const auto column_name = first_argument.getColumnName(); + auto map_keys_index_column_name = fmt::format("mapKeys({})", column_name); + auto map_values_index_column_name = fmt::format("mapValues({})", column_name); + + if (header.has(map_keys_index_column_name)) + { + /// For mapKeys we serialize key argument with bloom filter + + auto second_argument = key_node_function.getArgumentAt(1); + + Field constant_value; + DataTypePtr constant_type; + + if (second_argument.tryGetConstant(constant_value, constant_type)) + { + size_t position = header.getPositionByName(map_keys_index_column_name); + const DataTypePtr & index_type = header.getByPosition(position).type; + const DataTypePtr actual_type = BloomFilter::getPrimitiveType(index_type); + out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), constant_value))); + } + else + { + return false; + } + } + else if (header.has(map_values_index_column_name)) + { + /// For mapValues we serialize set with bloom filter + + size_t row_size = column->size(); + size_t position = header.getPositionByName(map_values_index_column_name); + const DataTypePtr & index_type = header.getByPosition(position).type; + const auto & array_type = assert_cast(*index_type); + const auto & array_nested_type = array_type.getNestedType(); + const auto & converted_column = castColumn(ColumnWithTypeAndName{column, type, ""}, array_nested_type); + out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithColumn(array_nested_type, converted_column, 0, row_size))); + } + else + { + return false; + } + + if (function_name == "in" || function_name == "globalIn") + out.function = RPNElement::FUNCTION_IN; + + if (function_name == "notIn" || function_name == "globalNotIn") + out.function = RPNElement::FUNCTION_NOT_IN; + + return true; + } + } + + return false; +} + + +static bool indexOfCanUseBloomFilter(const RPNBuilderTreeNode * parent) +{ + if (!parent) + return true; + + if (!parent->isFunction()) + return false; + + auto function = parent->toFunctionNode(); + auto function_name = function.getFunctionName(); + + /// `parent` is a function where `indexOf` is located. + /// Example: `indexOf(arr, x) = 1`, parent is a function named `equals`. + if (function_name == "and") + { + return true; + } + else if (function_name == "equals" /// notEquals is not applicable + || function_name == "greater" || function_name == "greaterOrEquals" + || function_name == "less" || function_name == "lessOrEquals") + { + size_t function_arguments_size = function.getArgumentsSize(); + if (function_arguments_size != 2) + return false; + + /// We don't allow constant expressions like `indexOf(arr, x) = 1 + 0` but it's negligible. + + /// We should return true when the corresponding expression implies that the array contains the element. + /// Example: when `indexOf(arr, x)` > 10 is written, it means that arr definitely should contain the element + /// (at least at 11th position but it does not matter). + + bool reversed = false; + Field constant_value; + DataTypePtr constant_type; + + if (function.getArgumentAt(0).tryGetConstant(constant_value, constant_type)) + { + reversed = true; + } + else if (function.getArgumentAt(1).tryGetConstant(constant_value, constant_type)) + { + } + else + { + return false; + } + + Field zero(0); + bool constant_equal_zero = applyVisitor(FieldVisitorAccurateEquals(), constant_value, zero); + + if (function_name == "equals" && !constant_equal_zero) + { + /// indexOf(...) = c, c != 0 + return true; + } + else if (function_name == "notEquals" && constant_equal_zero) + { + /// indexOf(...) != c, c = 0 + return true; + } + else if (function_name == (reversed ? "less" : "greater") && !applyVisitor(FieldVisitorAccurateLess(), constant_value, zero)) + { + /// indexOf(...) > c, c >= 0 + return true; + } + else if (function_name == (reversed ? "lessOrEquals" : "greaterOrEquals") && applyVisitor(FieldVisitorAccurateLess(), zero, constant_value)) + { + /// indexOf(...) >= c, c > 0 + return true; + } + + return false; + } + + return false; +} + + +bool MergeTreeIndexConditionBloomFilter::traverseTreeEquals( + const String & function_name, + const RPNBuilderTreeNode & key_node, + const DataTypePtr & value_type, + const Field & value_field, + RPNElement & out, + const RPNBuilderTreeNode * parent) +{ + auto key_column_name = key_node.getColumnName(); + + if (header.has(key_column_name)) + { + size_t position = header.getPositionByName(key_column_name); + const DataTypePtr & index_type = header.getByPosition(position).type; + const auto * array_type = typeid_cast(index_type.get()); + + if (function_name == "has" || function_name == "indexOf") + { + if (!array_type) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be an array.", function_name); + + /// We can treat `indexOf` function similar to `has`. + /// But it is little more cumbersome, compare: `has(arr, elem)` and `indexOf(arr, elem) != 0`. + /// The `parent` in this context is expected to be function `!=` (`notEquals`). + if (function_name == "has" || indexOfCanUseBloomFilter(parent)) + { + out.function = RPNElement::FUNCTION_HAS; + const DataTypePtr actual_type = BloomFilter::getPrimitiveType(array_type->getNestedType()); + auto converted_field = convertFieldToType(value_field, *actual_type, value_type.get()); + if (converted_field.isNull()) + return false; + + out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), converted_field))); + } + } + else if (function_name == "hasAny" || function_name == "hasAll") + { + if (!array_type) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be an array.", function_name); + + if (value_field.getType() != Field::Types::Array) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument for function {} must be an array.", function_name); + + const DataTypePtr actual_type = BloomFilter::getPrimitiveType(array_type->getNestedType()); + ColumnPtr column; + { + const bool is_nullable = actual_type->isNullable(); + auto mutable_column = actual_type->createColumn(); + + for (const auto & f : value_field.get()) + { + if ((f.isNull() && !is_nullable) || f.isDecimal(f.getType())) /// NOLINT(readability-static-accessed-through-instance) + return false; + + auto converted = convertFieldToType(f, *actual_type); + if (converted.isNull()) + return false; + + mutable_column->insert(converted); + } + + column = std::move(mutable_column); + } + + out.function = function_name == "hasAny" ? + RPNElement::FUNCTION_HAS_ANY : + RPNElement::FUNCTION_HAS_ALL; + out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithColumn(actual_type, column, 0, column->size()))); + } + else + { + if (array_type) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "An array type of bloom_filter supports only has(), indexOf(), and hasAny() functions."); + + out.function = function_name == "equals" ? RPNElement::FUNCTION_EQUALS : RPNElement::FUNCTION_NOT_EQUALS; + const DataTypePtr actual_type = BloomFilter::getPrimitiveType(index_type); + auto converted_field = convertFieldToType(value_field, *actual_type, value_type.get()); + if (converted_field.isNull()) + return false; + + out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), converted_field))); + } + + return true; + } + + if (function_name == "mapContains" || function_name == "has") + { + auto map_keys_index_column_name = fmt::format("mapKeys({})", key_column_name); + if (!header.has(map_keys_index_column_name)) + return false; + + size_t position = header.getPositionByName(map_keys_index_column_name); + const DataTypePtr & index_type = header.getByPosition(position).type; + const auto * array_type = typeid_cast(index_type.get()); + + if (!array_type) + return false; + + out.function = RPNElement::FUNCTION_HAS; + const DataTypePtr actual_type = BloomFilter::getPrimitiveType(array_type->getNestedType()); + auto converted_field = convertFieldToType(value_field, *actual_type, value_type.get()); + if (converted_field.isNull()) + return false; + + out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), converted_field))); + return true; + } + + if (key_node.isFunction()) + { + WhichDataType which(value_type); + + auto key_node_function = key_node.toFunctionNode(); + auto key_node_function_name = key_node_function.getFunctionName(); + size_t key_node_function_arguments_size = key_node_function.getArgumentsSize(); + + if (which.isTuple() && key_node_function_name == "tuple") + { + const Tuple & tuple = value_field.get(); + const auto * value_tuple_data_type = typeid_cast(value_type.get()); + + if (tuple.size() != key_node_function_arguments_size) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal types of arguments of function {}", function_name); + + bool match_with_subtype = false; + const DataTypes & subtypes = value_tuple_data_type->getElements(); + + for (size_t index = 0; index < tuple.size(); ++index) + match_with_subtype |= traverseTreeEquals(function_name, key_node_function.getArgumentAt(index), subtypes[index], tuple[index], out, &key_node); + + return match_with_subtype; + } + + if (key_node_function_name == "arrayElement" && (function_name == "equals" || function_name == "notEquals")) + { + /** Try to parse arrayElement for mapKeys index. + * It is important to ignore keys like column_map['Key'] = '' because if key does not exists in map + * we return default value for arrayElement. + * + * We cannot skip keys that does not exist in map if comparison is with default type value because + * that way we skip necessary granules where map key does not exists. + */ + if (value_field == value_type->getDefault()) + return false; + + auto first_argument = key_node_function.getArgumentAt(0); + const auto column_name = first_argument.getColumnName(); + + auto map_keys_index_column_name = fmt::format("mapKeys({})", column_name); + auto map_values_index_column_name = fmt::format("mapValues({})", column_name); + + size_t position = 0; + Field const_value = value_field; + DataTypePtr const_type; + + if (header.has(map_keys_index_column_name)) + { + position = header.getPositionByName(map_keys_index_column_name); + auto second_argument = key_node_function.getArgumentAt(1); + + if (!second_argument.tryGetConstant(const_value, const_type)) + return false; + } + else if (header.has(map_values_index_column_name)) + { + position = header.getPositionByName(map_values_index_column_name); + } + else + { + return false; + } + + out.function = function_name == "equals" ? RPNElement::FUNCTION_EQUALS : RPNElement::FUNCTION_NOT_EQUALS; + + const auto & index_type = header.getByPosition(position).type; + const auto actual_type = BloomFilter::getPrimitiveType(index_type); + out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), const_value))); + + return true; + } + } + + return false; +} + +MergeTreeIndexAggregatorBloomFilter::MergeTreeIndexAggregatorBloomFilter( + size_t bits_per_row_, size_t hash_functions_, const Names & columns_name_) + : bits_per_row(bits_per_row_), hash_functions(hash_functions_), index_columns_name(columns_name_), column_hashes(columns_name_.size()) +{ + assert(bits_per_row != 0); + assert(hash_functions != 0); +} + +bool MergeTreeIndexAggregatorBloomFilter::empty() const +{ + return !total_rows; +} + +MergeTreeIndexGranulePtr MergeTreeIndexAggregatorBloomFilter::getGranuleAndReset() +{ + const auto granule = std::make_shared(bits_per_row, hash_functions, column_hashes); + total_rows = 0; + column_hashes.clear(); + return granule; +} + +void MergeTreeIndexAggregatorBloomFilter::update(const Block & block, size_t * pos, size_t limit) +{ + if (*pos >= block.rows()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "The provided position is not less than the number of block rows. " + "Position: {}, Block rows: {}.", *pos, block.rows()); + + Block granule_index_block; + size_t max_read_rows = std::min(block.rows() - *pos, limit); + + for (size_t column = 0; column < index_columns_name.size(); ++column) + { + const auto & column_and_type = block.getByName(index_columns_name[column]); + auto index_column = BloomFilterHash::hashWithColumn(column_and_type.type, column_and_type.column, *pos, max_read_rows); + + const auto & index_col = checkAndGetColumn(index_column.get()); + const auto & index_data = index_col->getData(); + for (const auto & hash: index_data) + column_hashes[column].insert(hash); + } + + *pos += max_read_rows; + total_rows += max_read_rows; } MergeTreeIndexBloomFilter::MergeTreeIndexBloomFilter( diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.h b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.h index d6f4d6f2cf5..eeaa938551c 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.h +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.h @@ -1,13 +1,130 @@ #pragma once +#include +#include #include +#include #include -#include -#include namespace DB { +class MergeTreeIndexGranuleBloomFilter final : public IMergeTreeIndexGranule +{ +public: + MergeTreeIndexGranuleBloomFilter(size_t bits_per_row_, size_t hash_functions_, size_t index_columns_); + + MergeTreeIndexGranuleBloomFilter(size_t bits_per_row_, size_t hash_functions_, const std::vector> & column_hashes); + + bool empty() const override; + + void serializeBinary(WriteBuffer & ostr) const override; + void deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) override; + + const std::vector & getFilters() const { return bloom_filters; } + +private: + const size_t bits_per_row; + const size_t hash_functions; + + size_t total_rows = 0; + std::vector bloom_filters; + + void fillingBloomFilter(BloomFilterPtr & bf, const HashSet & hashes) const; +}; + +class MergeTreeIndexConditionBloomFilter final : public IMergeTreeIndexCondition, WithContext +{ +public: + struct RPNElement + { + enum Function + { + /// Atoms of a Boolean expression. + FUNCTION_EQUALS, + FUNCTION_NOT_EQUALS, + FUNCTION_HAS, + FUNCTION_HAS_ANY, + FUNCTION_HAS_ALL, + FUNCTION_IN, + FUNCTION_NOT_IN, + FUNCTION_UNKNOWN, /// Can take any value. + /// Operators of the logical expression. + FUNCTION_NOT, + FUNCTION_AND, + FUNCTION_OR, + /// Constants + ALWAYS_FALSE, + ALWAYS_TRUE, + }; + + RPNElement(Function function_ = FUNCTION_UNKNOWN) : function(function_) {} /// NOLINT + + Function function = FUNCTION_UNKNOWN; + std::vector> predicate; + }; + + MergeTreeIndexConditionBloomFilter(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, const Block & header_, size_t hash_functions_); + + bool alwaysUnknownOrTrue() const override; + + bool mayBeTrueOnGranule(MergeTreeIndexGranulePtr granule) const override + { + if (const auto & bf_granule = typeid_cast(granule.get())) + return mayBeTrueOnGranule(bf_granule); + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Requires bloom filter index granule."); + } + +private: + const Block & header; + const size_t hash_functions; + std::vector rpn; + + bool mayBeTrueOnGranule(const MergeTreeIndexGranuleBloomFilter * granule) const; + + bool extractAtomFromTree(const RPNBuilderTreeNode & node, RPNElement & out); + + bool traverseFunction(const RPNBuilderTreeNode & node, RPNElement & out, const RPNBuilderTreeNode * parent); + + bool traverseTreeIn( + const String & function_name, + const RPNBuilderTreeNode & key_node, + const ConstSetPtr & prepared_set, + const DataTypePtr & type, + const ColumnPtr & column, + RPNElement & out); + + bool traverseTreeEquals( + const String & function_name, + const RPNBuilderTreeNode & key_node, + const DataTypePtr & value_type, + const Field & value_field, + RPNElement & out, + const RPNBuilderTreeNode * parent); +}; + +class MergeTreeIndexAggregatorBloomFilter final : public IMergeTreeIndexAggregator +{ +public: + MergeTreeIndexAggregatorBloomFilter(size_t bits_per_row_, size_t hash_functions_, const Names & columns_name_); + + bool empty() const override; + + MergeTreeIndexGranulePtr getGranuleAndReset() override; + + void update(const Block & block, size_t * pos, size_t limit) override; + +private: + size_t bits_per_row; + size_t hash_functions; + const Names index_columns_name; + + std::vector> column_hashes; + size_t total_rows = 0; +}; + + class MergeTreeIndexBloomFilter final : public IMergeTreeIndex { public: diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp deleted file mode 100644 index 7ab90dac5b0..00000000000 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ /dev/null @@ -1,729 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int LOGICAL_ERROR; -} - -namespace -{ - -ColumnWithTypeAndName getPreparedSetInfo(const ConstSetPtr & prepared_set) -{ - if (prepared_set->getDataTypes().size() == 1) - return {prepared_set->getSetElements()[0], prepared_set->getElementsTypes()[0], "dummy"}; - - Columns set_elements; - for (auto & set_element : prepared_set->getSetElements()) - - set_elements.emplace_back(set_element->convertToFullColumnIfConst()); - - return {ColumnTuple::create(set_elements), std::make_shared(prepared_set->getElementsTypes()), "dummy"}; -} - -bool hashMatchesFilter(const BloomFilterPtr& bloom_filter, UInt64 hash, size_t hash_functions) -{ - return std::all_of(BloomFilterHash::bf_hash_seed, - BloomFilterHash::bf_hash_seed + hash_functions, - [&](const auto &hash_seed) - { - return bloom_filter->findHashWithSeed(hash, - hash_seed); - }); -} - -bool maybeTrueOnBloomFilter(const IColumn * hash_column, const BloomFilterPtr & bloom_filter, size_t hash_functions, bool match_all) -{ - const auto * const_column = typeid_cast(hash_column); - const auto * non_const_column = typeid_cast(hash_column); - - if (!const_column && !non_const_column) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Hash column must be Const or UInt64."); - - if (const_column) - { - return hashMatchesFilter(bloom_filter, - const_column->getValue(), - hash_functions); - } - - const ColumnUInt64::Container & hashes = non_const_column->getData(); - - if (match_all) - { - return std::all_of(hashes.begin(), - hashes.end(), - [&](const auto& hash_row) - { - return hashMatchesFilter(bloom_filter, - hash_row, - hash_functions); - }); - } - else - { - return std::any_of(hashes.begin(), - hashes.end(), - [&](const auto& hash_row) - { - return hashMatchesFilter(bloom_filter, - hash_row, - hash_functions); - }); - } -} - -} - -MergeTreeIndexConditionBloomFilter::MergeTreeIndexConditionBloomFilter( - const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, const Block & header_, size_t hash_functions_) - : WithContext(context_), header(header_), hash_functions(hash_functions_) -{ - if (!filter_actions_dag) - { - rpn.push_back(RPNElement::FUNCTION_UNKNOWN); - return; - } - - RPNBuilder builder( - filter_actions_dag->getOutputs().at(0), - context_, - [&](const RPNBuilderTreeNode & node, RPNElement & out) { return extractAtomFromTree(node, out); }); - rpn = std::move(builder).extractRPN(); -} - -bool MergeTreeIndexConditionBloomFilter::alwaysUnknownOrTrue() const -{ - std::vector rpn_stack; - - for (const auto & element : rpn) - { - if (element.function == RPNElement::FUNCTION_UNKNOWN - || element.function == RPNElement::ALWAYS_TRUE) - { - rpn_stack.push_back(true); - } - else if (element.function == RPNElement::FUNCTION_EQUALS - || element.function == RPNElement::FUNCTION_NOT_EQUALS - || element.function == RPNElement::FUNCTION_HAS - || element.function == RPNElement::FUNCTION_HAS_ANY - || element.function == RPNElement::FUNCTION_HAS_ALL - || element.function == RPNElement::FUNCTION_IN - || element.function == RPNElement::FUNCTION_NOT_IN - || element.function == RPNElement::ALWAYS_FALSE) - { - rpn_stack.push_back(false); - } - else if (element.function == RPNElement::FUNCTION_NOT) - { - // do nothing - } - else if (element.function == RPNElement::FUNCTION_AND) - { - auto arg1 = rpn_stack.back(); - rpn_stack.pop_back(); - auto arg2 = rpn_stack.back(); - rpn_stack.back() = arg1 && arg2; - } - else if (element.function == RPNElement::FUNCTION_OR) - { - auto arg1 = rpn_stack.back(); - rpn_stack.pop_back(); - auto arg2 = rpn_stack.back(); - rpn_stack.back() = arg1 || arg2; - } - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected function type in KeyCondition::RPNElement"); - } - - return rpn_stack[0]; -} - -bool MergeTreeIndexConditionBloomFilter::mayBeTrueOnGranule(const MergeTreeIndexGranuleBloomFilter * granule) const -{ - std::vector rpn_stack; - const auto & filters = granule->getFilters(); - - for (const auto & element : rpn) - { - if (element.function == RPNElement::FUNCTION_UNKNOWN) - { - rpn_stack.emplace_back(true, true); - } - else if (element.function == RPNElement::FUNCTION_IN - || element.function == RPNElement::FUNCTION_NOT_IN - || element.function == RPNElement::FUNCTION_EQUALS - || element.function == RPNElement::FUNCTION_NOT_EQUALS - || element.function == RPNElement::FUNCTION_HAS - || element.function == RPNElement::FUNCTION_HAS_ANY - || element.function == RPNElement::FUNCTION_HAS_ALL) - { - bool match_rows = true; - bool match_all = element.function == RPNElement::FUNCTION_HAS_ALL; - const auto & predicate = element.predicate; - for (size_t index = 0; match_rows && index < predicate.size(); ++index) - { - const auto & query_index_hash = predicate[index]; - const auto & filter = filters[query_index_hash.first]; - const ColumnPtr & hash_column = query_index_hash.second; - - match_rows = maybeTrueOnBloomFilter(&*hash_column, - filter, - hash_functions, - match_all); - } - - rpn_stack.emplace_back(match_rows, true); - if (element.function == RPNElement::FUNCTION_NOT_EQUALS || element.function == RPNElement::FUNCTION_NOT_IN) - rpn_stack.back() = !rpn_stack.back(); - } - else if (element.function == RPNElement::FUNCTION_NOT) - { - rpn_stack.back() = !rpn_stack.back(); - } - else if (element.function == RPNElement::FUNCTION_OR) - { - auto arg1 = rpn_stack.back(); - rpn_stack.pop_back(); - auto arg2 = rpn_stack.back(); - rpn_stack.back() = arg1 | arg2; - } - else if (element.function == RPNElement::FUNCTION_AND) - { - auto arg1 = rpn_stack.back(); - rpn_stack.pop_back(); - auto arg2 = rpn_stack.back(); - rpn_stack.back() = arg1 & arg2; - } - else if (element.function == RPNElement::ALWAYS_TRUE) - { - rpn_stack.emplace_back(true, false); - } - else if (element.function == RPNElement::ALWAYS_FALSE) - { - rpn_stack.emplace_back(false, true); - } - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected function type in KeyCondition::RPNElement"); - } - - if (rpn_stack.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected stack size in KeyCondition::mayBeTrueInRange"); - - return rpn_stack[0].can_be_true; -} - -bool MergeTreeIndexConditionBloomFilter::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNElement & out) -{ - { - Field const_value; - DataTypePtr const_type; - - if (node.tryGetConstant(const_value, const_type)) - { - if (const_value.getType() == Field::Types::UInt64) - { - out.function = const_value.get() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; - return true; - } - - if (const_value.getType() == Field::Types::Int64) - { - out.function = const_value.get() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; - return true; - } - - if (const_value.getType() == Field::Types::Float64) - { - out.function = const_value.get() != 0.0 ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; - return true; - } - } - } - - return traverseFunction(node, out, nullptr /*parent*/); -} - -bool MergeTreeIndexConditionBloomFilter::traverseFunction(const RPNBuilderTreeNode & node, RPNElement & out, const RPNBuilderTreeNode * parent) -{ - bool maybe_useful = false; - - if (node.isFunction()) - { - const auto function = node.toFunctionNode(); - auto arguments_size = function.getArgumentsSize(); - auto function_name = function.getFunctionName(); - - for (size_t i = 0; i < arguments_size; ++i) - { - auto argument = function.getArgumentAt(i); - if (traverseFunction(argument, out, &node)) - maybe_useful = true; - } - - if (arguments_size != 2) - return false; - - auto lhs_argument = function.getArgumentAt(0); - auto rhs_argument = function.getArgumentAt(1); - - if (functionIsInOrGlobalInOperator(function_name)) - { - if (auto future_set = rhs_argument.tryGetPreparedSet(); future_set) - { - if (auto prepared_set = future_set->buildOrderedSetInplace(rhs_argument.getTreeContext().getQueryContext()); prepared_set) - { - if (prepared_set->hasExplicitSetElements()) - { - const auto prepared_info = getPreparedSetInfo(prepared_set); - if (traverseTreeIn(function_name, lhs_argument, prepared_set, prepared_info.type, prepared_info.column, out)) - maybe_useful = true; - } - } - } - } - else if (function_name == "equals" || - function_name == "notEquals" || - function_name == "has" || - function_name == "mapContains" || - function_name == "indexOf" || - function_name == "hasAny" || - function_name == "hasAll") - { - Field const_value; - DataTypePtr const_type; - - if (rhs_argument.tryGetConstant(const_value, const_type)) - { - if (traverseTreeEquals(function_name, lhs_argument, const_type, const_value, out, parent)) - maybe_useful = true; - } - else if (lhs_argument.tryGetConstant(const_value, const_type)) - { - if (traverseTreeEquals(function_name, rhs_argument, const_type, const_value, out, parent)) - maybe_useful = true; - } - } - } - - return maybe_useful; -} - -bool MergeTreeIndexConditionBloomFilter::traverseTreeIn( - const String & function_name, - const RPNBuilderTreeNode & key_node, - const ConstSetPtr & prepared_set, - const DataTypePtr & type, - const ColumnPtr & column, - RPNElement & out) -{ - auto key_node_column_name = key_node.getColumnName(); - - if (header.has(key_node_column_name)) - { - size_t row_size = column->size(); - size_t position = header.getPositionByName(key_node_column_name); - const DataTypePtr & index_type = header.getByPosition(position).type; - const auto & converted_column = castColumn(ColumnWithTypeAndName{column, type, ""}, index_type); - out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithColumn(index_type, converted_column, 0, row_size))); - - if (function_name == "in" || function_name == "globalIn") - out.function = RPNElement::FUNCTION_IN; - - if (function_name == "notIn" || function_name == "globalNotIn") - out.function = RPNElement::FUNCTION_NOT_IN; - - return true; - } - - if (key_node.isFunction()) - { - auto key_node_function = key_node.toFunctionNode(); - auto key_node_function_name = key_node_function.getFunctionName(); - size_t key_node_function_arguments_size = key_node_function.getArgumentsSize(); - - WhichDataType which(type); - - if (which.isTuple() && key_node_function_name == "tuple") - { - const auto & tuple_column = typeid_cast(column.get()); - const auto & tuple_data_type = typeid_cast(type.get()); - - if (tuple_data_type->getElements().size() != key_node_function_arguments_size || tuple_column->getColumns().size() != key_node_function_arguments_size) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal types of arguments of function {}", function_name); - - bool match_with_subtype = false; - const auto & sub_columns = tuple_column->getColumns(); - const auto & sub_data_types = tuple_data_type->getElements(); - - for (size_t index = 0; index < key_node_function_arguments_size; ++index) - match_with_subtype |= traverseTreeIn(function_name, key_node_function.getArgumentAt(index), nullptr, sub_data_types[index], sub_columns[index], out); - - return match_with_subtype; - } - - if (key_node_function_name == "arrayElement") - { - /** Try to parse arrayElement for mapKeys index. - * It is important to ignore keys like column_map['Key'] IN ('') because if key does not exists in map - * we return default value for arrayElement. - * - * We cannot skip keys that does not exist in map if comparison is with default type value because - * that way we skip necessary granules where map key does not exists. - */ - if (!prepared_set) - return false; - - auto default_column_to_check = type->createColumnConstWithDefaultValue(1)->convertToFullColumnIfConst(); - ColumnWithTypeAndName default_column_with_type_to_check { default_column_to_check, type, "" }; - ColumnsWithTypeAndName default_columns_with_type_to_check = {default_column_with_type_to_check}; - auto set_contains_default_value_predicate_column = prepared_set->execute(default_columns_with_type_to_check, false /*negative*/); - const auto & set_contains_default_value_predicate_column_typed = assert_cast(*set_contains_default_value_predicate_column); - bool set_contain_default_value = set_contains_default_value_predicate_column_typed.getData()[0]; - if (set_contain_default_value) - return false; - - auto first_argument = key_node_function.getArgumentAt(0); - const auto column_name = first_argument.getColumnName(); - auto map_keys_index_column_name = fmt::format("mapKeys({})", column_name); - auto map_values_index_column_name = fmt::format("mapValues({})", column_name); - - if (header.has(map_keys_index_column_name)) - { - /// For mapKeys we serialize key argument with bloom filter - - auto second_argument = key_node_function.getArgumentAt(1); - - Field constant_value; - DataTypePtr constant_type; - - if (second_argument.tryGetConstant(constant_value, constant_type)) - { - size_t position = header.getPositionByName(map_keys_index_column_name); - const DataTypePtr & index_type = header.getByPosition(position).type; - const DataTypePtr actual_type = BloomFilter::getPrimitiveType(index_type); - out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), constant_value))); - } - else - { - return false; - } - } - else if (header.has(map_values_index_column_name)) - { - /// For mapValues we serialize set with bloom filter - - size_t row_size = column->size(); - size_t position = header.getPositionByName(map_values_index_column_name); - const DataTypePtr & index_type = header.getByPosition(position).type; - const auto & array_type = assert_cast(*index_type); - const auto & array_nested_type = array_type.getNestedType(); - const auto & converted_column = castColumn(ColumnWithTypeAndName{column, type, ""}, array_nested_type); - out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithColumn(array_nested_type, converted_column, 0, row_size))); - } - else - { - return false; - } - - if (function_name == "in" || function_name == "globalIn") - out.function = RPNElement::FUNCTION_IN; - - if (function_name == "notIn" || function_name == "globalNotIn") - out.function = RPNElement::FUNCTION_NOT_IN; - - return true; - } - } - - return false; -} - - -static bool indexOfCanUseBloomFilter(const RPNBuilderTreeNode * parent) -{ - if (!parent) - return true; - - if (!parent->isFunction()) - return false; - - auto function = parent->toFunctionNode(); - auto function_name = function.getFunctionName(); - - /// `parent` is a function where `indexOf` is located. - /// Example: `indexOf(arr, x) = 1`, parent is a function named `equals`. - if (function_name == "and") - { - return true; - } - else if (function_name == "equals" /// notEquals is not applicable - || function_name == "greater" || function_name == "greaterOrEquals" - || function_name == "less" || function_name == "lessOrEquals") - { - size_t function_arguments_size = function.getArgumentsSize(); - if (function_arguments_size != 2) - return false; - - /// We don't allow constant expressions like `indexOf(arr, x) = 1 + 0` but it's negligible. - - /// We should return true when the corresponding expression implies that the array contains the element. - /// Example: when `indexOf(arr, x)` > 10 is written, it means that arr definitely should contain the element - /// (at least at 11th position but it does not matter). - - bool reversed = false; - Field constant_value; - DataTypePtr constant_type; - - if (function.getArgumentAt(0).tryGetConstant(constant_value, constant_type)) - { - reversed = true; - } - else if (function.getArgumentAt(1).tryGetConstant(constant_value, constant_type)) - { - } - else - { - return false; - } - - Field zero(0); - bool constant_equal_zero = applyVisitor(FieldVisitorAccurateEquals(), constant_value, zero); - - if (function_name == "equals" && !constant_equal_zero) - { - /// indexOf(...) = c, c != 0 - return true; - } - else if (function_name == "notEquals" && constant_equal_zero) - { - /// indexOf(...) != c, c = 0 - return true; - } - else if (function_name == (reversed ? "less" : "greater") && !applyVisitor(FieldVisitorAccurateLess(), constant_value, zero)) - { - /// indexOf(...) > c, c >= 0 - return true; - } - else if (function_name == (reversed ? "lessOrEquals" : "greaterOrEquals") && applyVisitor(FieldVisitorAccurateLess(), zero, constant_value)) - { - /// indexOf(...) >= c, c > 0 - return true; - } - - return false; - } - - return false; -} - - -bool MergeTreeIndexConditionBloomFilter::traverseTreeEquals( - const String & function_name, - const RPNBuilderTreeNode & key_node, - const DataTypePtr & value_type, - const Field & value_field, - RPNElement & out, - const RPNBuilderTreeNode * parent) -{ - auto key_column_name = key_node.getColumnName(); - - if (header.has(key_column_name)) - { - size_t position = header.getPositionByName(key_column_name); - const DataTypePtr & index_type = header.getByPosition(position).type; - const auto * array_type = typeid_cast(index_type.get()); - - if (function_name == "has" || function_name == "indexOf") - { - if (!array_type) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be an array.", function_name); - - /// We can treat `indexOf` function similar to `has`. - /// But it is little more cumbersome, compare: `has(arr, elem)` and `indexOf(arr, elem) != 0`. - /// The `parent` in this context is expected to be function `!=` (`notEquals`). - if (function_name == "has" || indexOfCanUseBloomFilter(parent)) - { - out.function = RPNElement::FUNCTION_HAS; - const DataTypePtr actual_type = BloomFilter::getPrimitiveType(array_type->getNestedType()); - auto converted_field = convertFieldToType(value_field, *actual_type, value_type.get()); - if (converted_field.isNull()) - return false; - - out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), converted_field))); - } - } - else if (function_name == "hasAny" || function_name == "hasAll") - { - if (!array_type) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be an array.", function_name); - - if (value_field.getType() != Field::Types::Array) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument for function {} must be an array.", function_name); - - const DataTypePtr actual_type = BloomFilter::getPrimitiveType(array_type->getNestedType()); - ColumnPtr column; - { - const bool is_nullable = actual_type->isNullable(); - auto mutable_column = actual_type->createColumn(); - - for (const auto & f : value_field.get()) - { - if ((f.isNull() && !is_nullable) || f.isDecimal(f.getType())) /// NOLINT(readability-static-accessed-through-instance) - return false; - - auto converted = convertFieldToType(f, *actual_type); - if (converted.isNull()) - return false; - - mutable_column->insert(converted); - } - - column = std::move(mutable_column); - } - - out.function = function_name == "hasAny" ? - RPNElement::FUNCTION_HAS_ANY : - RPNElement::FUNCTION_HAS_ALL; - out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithColumn(actual_type, column, 0, column->size()))); - } - else - { - if (array_type) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "An array type of bloom_filter supports only has(), indexOf(), and hasAny() functions."); - - out.function = function_name == "equals" ? RPNElement::FUNCTION_EQUALS : RPNElement::FUNCTION_NOT_EQUALS; - const DataTypePtr actual_type = BloomFilter::getPrimitiveType(index_type); - auto converted_field = convertFieldToType(value_field, *actual_type, value_type.get()); - if (converted_field.isNull()) - return false; - - out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), converted_field))); - } - - return true; - } - - if (function_name == "mapContains" || function_name == "has") - { - auto map_keys_index_column_name = fmt::format("mapKeys({})", key_column_name); - if (!header.has(map_keys_index_column_name)) - return false; - - size_t position = header.getPositionByName(map_keys_index_column_name); - const DataTypePtr & index_type = header.getByPosition(position).type; - const auto * array_type = typeid_cast(index_type.get()); - - if (!array_type) - return false; - - out.function = RPNElement::FUNCTION_HAS; - const DataTypePtr actual_type = BloomFilter::getPrimitiveType(array_type->getNestedType()); - auto converted_field = convertFieldToType(value_field, *actual_type, value_type.get()); - if (converted_field.isNull()) - return false; - - out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), converted_field))); - return true; - } - - if (key_node.isFunction()) - { - WhichDataType which(value_type); - - auto key_node_function = key_node.toFunctionNode(); - auto key_node_function_name = key_node_function.getFunctionName(); - size_t key_node_function_arguments_size = key_node_function.getArgumentsSize(); - - if (which.isTuple() && key_node_function_name == "tuple") - { - const Tuple & tuple = value_field.get(); - const auto * value_tuple_data_type = typeid_cast(value_type.get()); - - if (tuple.size() != key_node_function_arguments_size) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal types of arguments of function {}", function_name); - - bool match_with_subtype = false; - const DataTypes & subtypes = value_tuple_data_type->getElements(); - - for (size_t index = 0; index < tuple.size(); ++index) - match_with_subtype |= traverseTreeEquals(function_name, key_node_function.getArgumentAt(index), subtypes[index], tuple[index], out, &key_node); - - return match_with_subtype; - } - - if (key_node_function_name == "arrayElement" && (function_name == "equals" || function_name == "notEquals")) - { - /** Try to parse arrayElement for mapKeys index. - * It is important to ignore keys like column_map['Key'] = '' because if key does not exists in map - * we return default value for arrayElement. - * - * We cannot skip keys that does not exist in map if comparison is with default type value because - * that way we skip necessary granules where map key does not exists. - */ - if (value_field == value_type->getDefault()) - return false; - - auto first_argument = key_node_function.getArgumentAt(0); - const auto column_name = first_argument.getColumnName(); - - auto map_keys_index_column_name = fmt::format("mapKeys({})", column_name); - auto map_values_index_column_name = fmt::format("mapValues({})", column_name); - - size_t position = 0; - Field const_value = value_field; - DataTypePtr const_type; - - if (header.has(map_keys_index_column_name)) - { - position = header.getPositionByName(map_keys_index_column_name); - auto second_argument = key_node_function.getArgumentAt(1); - - if (!second_argument.tryGetConstant(const_value, const_type)) - return false; - } - else if (header.has(map_values_index_column_name)) - { - position = header.getPositionByName(map_values_index_column_name); - } - else - { - return false; - } - - out.function = function_name == "equals" ? RPNElement::FUNCTION_EQUALS : RPNElement::FUNCTION_NOT_EQUALS; - - const auto & index_type = header.getByPosition(position).type; - const auto actual_type = BloomFilter::getPrimitiveType(index_type); - out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), const_value))); - - return true; - } - } - - return false; -} - -} diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h deleted file mode 100644 index 8029d6d405b..00000000000 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h +++ /dev/null @@ -1,87 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -class MergeTreeIndexConditionBloomFilter final : public IMergeTreeIndexCondition, WithContext -{ -public: - struct RPNElement - { - enum Function - { - /// Atoms of a Boolean expression. - FUNCTION_EQUALS, - FUNCTION_NOT_EQUALS, - FUNCTION_HAS, - FUNCTION_HAS_ANY, - FUNCTION_HAS_ALL, - FUNCTION_IN, - FUNCTION_NOT_IN, - FUNCTION_UNKNOWN, /// Can take any value. - /// Operators of the logical expression. - FUNCTION_NOT, - FUNCTION_AND, - FUNCTION_OR, - /// Constants - ALWAYS_FALSE, - ALWAYS_TRUE, - }; - - RPNElement(Function function_ = FUNCTION_UNKNOWN) : function(function_) {} /// NOLINT - - Function function = FUNCTION_UNKNOWN; - std::vector> predicate; - }; - - MergeTreeIndexConditionBloomFilter(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, const Block & header_, size_t hash_functions_); - - bool alwaysUnknownOrTrue() const override; - - bool mayBeTrueOnGranule(MergeTreeIndexGranulePtr granule) const override - { - if (const auto & bf_granule = typeid_cast(granule.get())) - return mayBeTrueOnGranule(bf_granule); - - throw Exception(ErrorCodes::LOGICAL_ERROR, "Requires bloom filter index granule."); - } - -private: - const Block & header; - const size_t hash_functions; - std::vector rpn; - - bool mayBeTrueOnGranule(const MergeTreeIndexGranuleBloomFilter * granule) const; - - bool extractAtomFromTree(const RPNBuilderTreeNode & node, RPNElement & out); - - bool traverseFunction(const RPNBuilderTreeNode & node, RPNElement & out, const RPNBuilderTreeNode * parent); - - bool traverseTreeIn( - const String & function_name, - const RPNBuilderTreeNode & key_node, - const ConstSetPtr & prepared_set, - const DataTypePtr & type, - const ColumnPtr & column, - RPNElement & out); - - bool traverseTreeEquals( - const String & function_name, - const RPNBuilderTreeNode & key_node, - const DataTypePtr & value_type, - const Field & value_field, - RPNElement & out, - const RPNBuilderTreeNode * parent); -}; - -} diff --git a/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp deleted file mode 100644 index 8355cac8033..00000000000 --- a/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp +++ /dev/null @@ -1,102 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -MergeTreeIndexGranuleBloomFilter::MergeTreeIndexGranuleBloomFilter(size_t bits_per_row_, size_t hash_functions_, size_t index_columns_) - : bits_per_row(bits_per_row_), hash_functions(hash_functions_), bloom_filters(index_columns_) -{ - total_rows = 0; - for (size_t column = 0; column < index_columns_; ++column) - bloom_filters[column] = std::make_shared(bits_per_row, hash_functions, 0); -} - -MergeTreeIndexGranuleBloomFilter::MergeTreeIndexGranuleBloomFilter( - size_t bits_per_row_, size_t hash_functions_, const std::vector>& column_hashes_) - : bits_per_row(bits_per_row_), hash_functions(hash_functions_), bloom_filters(column_hashes_.size()) -{ - if (column_hashes_.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Granule_index_blocks empty or total_rows is zero."); - - size_t bloom_filter_max_size = 0; - for (const auto & column_hash : column_hashes_) - bloom_filter_max_size = std::max(bloom_filter_max_size, column_hash.size()); - - static size_t atom_size = 8; - - // If multiple columns are given, we will initialize all the bloom filters - // with the size of the highest-cardinality one. This is done for compatibility with - // existing binary serialization format - total_rows = bloom_filter_max_size; - size_t bytes_size = (bits_per_row * total_rows + atom_size - 1) / atom_size; - - for (size_t column = 0, columns = column_hashes_.size(); column < columns; ++column) - { - bloom_filters[column] = std::make_shared(bytes_size, hash_functions, 0); - fillingBloomFilter(bloom_filters[column], column_hashes_[column]); - } -} - -bool MergeTreeIndexGranuleBloomFilter::empty() const -{ - return !total_rows; -} - -void MergeTreeIndexGranuleBloomFilter::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) -{ - if (version != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown index version {}.", version); - - readVarUInt(total_rows, istr); - - static size_t atom_size = 8; - size_t bytes_size = (bits_per_row * total_rows + atom_size - 1) / atom_size; - size_t read_size = bytes_size; - for (auto & filter : bloom_filters) - { - filter->resize(bytes_size); -#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - read_size = filter->getFilter().size() * sizeof(BloomFilter::UnderType); -#endif - istr.readStrict(reinterpret_cast(filter->getFilter().data()), read_size); - } -} - -void MergeTreeIndexGranuleBloomFilter::serializeBinary(WriteBuffer & ostr) const -{ - if (empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to write empty bloom filter index."); - - writeVarUInt(total_rows, ostr); - - static size_t atom_size = 8; - size_t write_size = (bits_per_row * total_rows + atom_size - 1) / atom_size; - for (const auto & bloom_filter : bloom_filters) - { -#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - write_size = bloom_filter->getFilter().size() * sizeof(BloomFilter::UnderType); -#endif - ostr.write(reinterpret_cast(bloom_filter->getFilter().data()), write_size); - } -} - -void MergeTreeIndexGranuleBloomFilter::fillingBloomFilter(BloomFilterPtr & bf, const HashSet &hashes) const -{ - for (const auto & bf_base_hash : hashes) - for (size_t i = 0; i < hash_functions; ++i) - bf->addHashWithSeed(bf_base_hash.getKey(), BloomFilterHash::bf_hash_seed[i]); -} - -} diff --git a/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.h b/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.h deleted file mode 100644 index a3434daa5a4..00000000000 --- a/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.h +++ /dev/null @@ -1,35 +0,0 @@ -#pragma once - -#include -#include -#include - -namespace DB -{ - -class MergeTreeIndexGranuleBloomFilter final : public IMergeTreeIndexGranule -{ -public: - MergeTreeIndexGranuleBloomFilter(size_t bits_per_row_, size_t hash_functions_, size_t index_columns_); - - MergeTreeIndexGranuleBloomFilter(size_t bits_per_row_, size_t hash_functions_, const std::vector> & column_hashes); - - bool empty() const override; - - void serializeBinary(WriteBuffer & ostr) const override; - void deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) override; - - const std::vector & getFilters() const { return bloom_filters; } - -private: - const size_t bits_per_row; - const size_t hash_functions; - - size_t total_rows = 0; - std::vector bloom_filters; - - void fillingBloomFilter(BloomFilterPtr & bf, const HashSet & hashes) const; -}; - - -} From 0ae422d33c0538f759c64444cc8b7a6f8f2a5b63 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 23 Apr 2024 14:11:12 +0000 Subject: [PATCH 0279/1018] Rename MergeTreeIndexFullText --> MergeTreeIndexBloomFilterText --- src/Interpreters/GinFilter.cpp | 2 +- .../MergeTree/MergeTreeIndexBloomFilter.cpp | 4 +- ....cpp => MergeTreeIndexBloomFilterText.cpp} | 56 +++++++++---------- ...Text.h => MergeTreeIndexBloomFilterText.h} | 28 +++++----- .../MergeTree/MergeTreeIndexInverted.h | 1 - src/Storages/MergeTree/MergeTreeIndices.cpp | 12 ++-- src/Storages/MergeTree/MergeTreeIndices.h | 6 +- .../tests/gtest_SplitTokenExtractor.cpp | 2 +- 8 files changed, 55 insertions(+), 56 deletions(-) rename src/Storages/MergeTree/{MergeTreeIndexFullText.cpp => MergeTreeIndexBloomFilterText.cpp} (92%) rename src/Storages/MergeTree/{MergeTreeIndexFullText.h => MergeTreeIndexBloomFilterText.h} (84%) diff --git a/src/Interpreters/GinFilter.cpp b/src/Interpreters/GinFilter.cpp index 5d823318313..1ce26ed1fd1 100644 --- a/src/Interpreters/GinFilter.cpp +++ b/src/Interpreters/GinFilter.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp index ed091022a91..4f25a014382 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp @@ -921,7 +921,7 @@ static void assertIndexColumnsType(const Block & header) } } -MergeTreeIndexPtr bloomFilterIndexCreatorNew( +MergeTreeIndexPtr bloomFilterIndexCreator( const IndexDescription & index) { double max_conflict_probability = 0.025; @@ -938,7 +938,7 @@ MergeTreeIndexPtr bloomFilterIndexCreatorNew( index, bits_per_row_and_size_of_hash_functions.first, bits_per_row_and_size_of_hash_functions.second); } -void bloomFilterIndexValidatorNew(const IndexDescription & index, bool attach) +void bloomFilterIndexValidator(const IndexDescription & index, bool attach) { assertIndexColumnsType(index.sample_block); diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.cpp similarity index 92% rename from src/Storages/MergeTree/MergeTreeIndexFullText.cpp rename to src/Storages/MergeTree/MergeTreeIndexBloomFilterText.cpp index 4cd616513ac..826b149cf01 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include @@ -32,7 +32,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -MergeTreeIndexGranuleFullText::MergeTreeIndexGranuleFullText( +MergeTreeIndexGranuleBloomFilterText::MergeTreeIndexGranuleBloomFilterText( const String & index_name_, size_t columns_number, const BloomFilterParameters & params_) @@ -44,7 +44,7 @@ MergeTreeIndexGranuleFullText::MergeTreeIndexGranuleFullText( { } -void MergeTreeIndexGranuleFullText::serializeBinary(WriteBuffer & ostr) const +void MergeTreeIndexGranuleBloomFilterText::serializeBinary(WriteBuffer & ostr) const { if (empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to write empty fulltext index {}.", backQuote(index_name)); @@ -53,7 +53,7 @@ void MergeTreeIndexGranuleFullText::serializeBinary(WriteBuffer & ostr) const ostr.write(reinterpret_cast(bloom_filter.getFilter().data()), params.filter_size); } -void MergeTreeIndexGranuleFullText::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) +void MergeTreeIndexGranuleBloomFilterText::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) { if (version != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown index version {}.", version); @@ -66,7 +66,7 @@ void MergeTreeIndexGranuleFullText::deserializeBinary(ReadBuffer & istr, MergeTr } -MergeTreeIndexAggregatorFullText::MergeTreeIndexAggregatorFullText( +MergeTreeIndexAggregatorBloomFilterText::MergeTreeIndexAggregatorBloomFilterText( const Names & index_columns_, const String & index_name_, const BloomFilterParameters & params_, @@ -76,20 +76,20 @@ MergeTreeIndexAggregatorFullText::MergeTreeIndexAggregatorFullText( , params(params_) , token_extractor(token_extractor_) , granule( - std::make_shared( + std::make_shared( index_name, index_columns.size(), params)) { } -MergeTreeIndexGranulePtr MergeTreeIndexAggregatorFullText::getGranuleAndReset() +MergeTreeIndexGranulePtr MergeTreeIndexAggregatorBloomFilterText::getGranuleAndReset() { - auto new_granule = std::make_shared( + auto new_granule = std::make_shared( index_name, index_columns.size(), params); new_granule.swap(granule); return new_granule; } -void MergeTreeIndexAggregatorFullText::update(const Block & block, size_t * pos, size_t limit) +void MergeTreeIndexAggregatorBloomFilterText::update(const Block & block, size_t * pos, size_t limit) { if (*pos >= block.rows()) throw Exception(ErrorCodes::LOGICAL_ERROR, "The provided position is not less than the number of block rows. " @@ -137,7 +137,7 @@ void MergeTreeIndexAggregatorFullText::update(const Block & block, size_t * pos, *pos += rows_read; } -MergeTreeConditionFullText::MergeTreeConditionFullText( +MergeTreeConditionBloomFilterText::MergeTreeConditionBloomFilterText( const ActionsDAGPtr & filter_actions_dag, ContextPtr context, const Block & index_sample_block, @@ -162,7 +162,7 @@ MergeTreeConditionFullText::MergeTreeConditionFullText( } /// Keep in-sync with MergeTreeConditionGinFilter::alwaysUnknownOrTrue -bool MergeTreeConditionFullText::alwaysUnknownOrTrue() const +bool MergeTreeConditionBloomFilterText::alwaysUnknownOrTrue() const { /// Check like in KeyCondition. std::vector rpn_stack; @@ -212,10 +212,10 @@ bool MergeTreeConditionFullText::alwaysUnknownOrTrue() const } /// Keep in-sync with MergeTreeIndexConditionGin::mayBeTrueOnTranuleInPart -bool MergeTreeConditionFullText::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const +bool MergeTreeConditionBloomFilterText::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const { - std::shared_ptr granule - = std::dynamic_pointer_cast(idx_granule); + std::shared_ptr granule + = std::dynamic_pointer_cast(idx_granule); if (!granule) throw Exception(ErrorCodes::LOGICAL_ERROR, "BloomFilter index condition got a granule with the wrong type."); @@ -323,13 +323,13 @@ bool MergeTreeConditionFullText::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx return rpn_stack[0].can_be_true; } -std::optional MergeTreeConditionFullText::getKeyIndex(const std::string & key_column_name) +std::optional MergeTreeConditionBloomFilterText::getKeyIndex(const std::string & key_column_name) { const auto it = std::ranges::find(index_columns, key_column_name); return it == index_columns.end() ? std::nullopt : std::make_optional(std::ranges::distance(index_columns.cbegin(), it)); } -bool MergeTreeConditionFullText::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNElement & out) +bool MergeTreeConditionBloomFilterText::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNElement & out) { { Field const_value; @@ -419,7 +419,7 @@ bool MergeTreeConditionFullText::extractAtomFromTree(const RPNBuilderTreeNode & return false; } -bool MergeTreeConditionFullText::traverseTreeEquals( +bool MergeTreeConditionBloomFilterText::traverseTreeEquals( const String & function_name, const RPNBuilderTreeNode & key_node, const DataTypePtr & value_type, @@ -638,7 +638,7 @@ bool MergeTreeConditionFullText::traverseTreeEquals( return false; } -bool MergeTreeConditionFullText::tryPrepareSetBloomFilter( +bool MergeTreeConditionBloomFilterText::tryPrepareSetBloomFilter( const RPNBuilderTreeNode & left_argument, const RPNBuilderTreeNode & right_argument, RPNElement & out) @@ -714,23 +714,23 @@ bool MergeTreeConditionFullText::tryPrepareSetBloomFilter( return true; } -MergeTreeIndexGranulePtr MergeTreeIndexFullText::createIndexGranule() const +MergeTreeIndexGranulePtr MergeTreeIndexBloomFilterText::createIndexGranule() const { - return std::make_shared(index.name, index.column_names.size(), params); + return std::make_shared(index.name, index.column_names.size(), params); } -MergeTreeIndexAggregatorPtr MergeTreeIndexFullText::createIndexAggregator(const MergeTreeWriterSettings & /*settings*/) const +MergeTreeIndexAggregatorPtr MergeTreeIndexBloomFilterText::createIndexAggregator(const MergeTreeWriterSettings & /*settings*/) const { - return std::make_shared(index.column_names, index.name, params, token_extractor.get()); + return std::make_shared(index.column_names, index.name, params, token_extractor.get()); } -MergeTreeIndexConditionPtr MergeTreeIndexFullText::createIndexCondition( +MergeTreeIndexConditionPtr MergeTreeIndexBloomFilterText::createIndexCondition( const ActionsDAGPtr & filter_dag, ContextPtr context) const { - return std::make_shared(filter_dag, context, index.sample_block, params, token_extractor.get()); + return std::make_shared(filter_dag, context, index.sample_block, params, token_extractor.get()); } -MergeTreeIndexPtr bloomFilterIndexCreator( +MergeTreeIndexPtr bloomFilterIndexTextCreator( const IndexDescription & index) { if (index.type == NgramTokenExtractor::getName()) @@ -743,7 +743,7 @@ MergeTreeIndexPtr bloomFilterIndexCreator( auto tokenizer = std::make_unique(n); - return std::make_shared(index, params, std::move(tokenizer)); + return std::make_shared(index, params, std::move(tokenizer)); } else if (index.type == SplitTokenExtractor::getName()) { @@ -754,7 +754,7 @@ MergeTreeIndexPtr bloomFilterIndexCreator( auto tokenizer = std::make_unique(); - return std::make_shared(index, params, std::move(tokenizer)); + return std::make_shared(index, params, std::move(tokenizer)); } else { @@ -762,7 +762,7 @@ MergeTreeIndexPtr bloomFilterIndexCreator( } } -void bloomFilterIndexValidator(const IndexDescription & index, bool /*attach*/) +void bloomFilterIndexTextValidator(const IndexDescription & index, bool /*attach*/) { for (const auto & index_data_type : index.data_types) { diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.h b/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.h similarity index 84% rename from src/Storages/MergeTree/MergeTreeIndexFullText.h rename to src/Storages/MergeTree/MergeTreeIndexBloomFilterText.h index e66f498ce1d..6fd969030df 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.h +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.h @@ -11,14 +11,14 @@ namespace DB { -struct MergeTreeIndexGranuleFullText final : public IMergeTreeIndexGranule +struct MergeTreeIndexGranuleBloomFilterText final : public IMergeTreeIndexGranule { - explicit MergeTreeIndexGranuleFullText( + explicit MergeTreeIndexGranuleBloomFilterText( const String & index_name_, size_t columns_number, const BloomFilterParameters & params_); - ~MergeTreeIndexGranuleFullText() override = default; + ~MergeTreeIndexGranuleBloomFilterText() override = default; void serializeBinary(WriteBuffer & ostr) const override; void deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) override; @@ -32,17 +32,17 @@ struct MergeTreeIndexGranuleFullText final : public IMergeTreeIndexGranule bool has_elems; }; -using MergeTreeIndexGranuleFullTextPtr = std::shared_ptr; +using MergeTreeIndexGranuleBloomFilterTextPtr = std::shared_ptr; -struct MergeTreeIndexAggregatorFullText final : IMergeTreeIndexAggregator +struct MergeTreeIndexAggregatorBloomFilterText final : IMergeTreeIndexAggregator { - explicit MergeTreeIndexAggregatorFullText( + explicit MergeTreeIndexAggregatorBloomFilterText( const Names & index_columns_, const String & index_name_, const BloomFilterParameters & params_, TokenExtractorPtr token_extractor_); - ~MergeTreeIndexAggregatorFullText() override = default; + ~MergeTreeIndexAggregatorBloomFilterText() override = default; bool empty() const override { return !granule || granule->empty(); } MergeTreeIndexGranulePtr getGranuleAndReset() override; @@ -54,21 +54,21 @@ struct MergeTreeIndexAggregatorFullText final : IMergeTreeIndexAggregator BloomFilterParameters params; TokenExtractorPtr token_extractor; - MergeTreeIndexGranuleFullTextPtr granule; + MergeTreeIndexGranuleBloomFilterTextPtr granule; }; -class MergeTreeConditionFullText final : public IMergeTreeIndexCondition +class MergeTreeConditionBloomFilterText final : public IMergeTreeIndexCondition { public: - MergeTreeConditionFullText( + MergeTreeConditionBloomFilterText( const ActionsDAGPtr & filter_actions_dag, ContextPtr context, const Block & index_sample_block, const BloomFilterParameters & params_, TokenExtractorPtr token_extactor_); - ~MergeTreeConditionFullText() override = default; + ~MergeTreeConditionBloomFilterText() override = default; bool alwaysUnknownOrTrue() const override; bool mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const override; @@ -146,10 +146,10 @@ private: RPN rpn; }; -class MergeTreeIndexFullText final : public IMergeTreeIndex +class MergeTreeIndexBloomFilterText final : public IMergeTreeIndex { public: - MergeTreeIndexFullText( + MergeTreeIndexBloomFilterText( const IndexDescription & index_, const BloomFilterParameters & params_, std::unique_ptr && token_extractor_) @@ -157,7 +157,7 @@ public: , params(params_) , token_extractor(std::move(token_extractor_)) {} - ~MergeTreeIndexFullText() override = default; + ~MergeTreeIndexBloomFilterText() override = default; MergeTreeIndexGranulePtr createIndexGranule() const override; MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override; diff --git a/src/Storages/MergeTree/MergeTreeIndexInverted.h b/src/Storages/MergeTree/MergeTreeIndexInverted.h index f3c1f37e364..bab4e122aa6 100644 --- a/src/Storages/MergeTree/MergeTreeIndexInverted.h +++ b/src/Storages/MergeTree/MergeTreeIndexInverted.h @@ -5,7 +5,6 @@ #include #include #include -#include #include namespace DB diff --git a/src/Storages/MergeTree/MergeTreeIndices.cpp b/src/Storages/MergeTree/MergeTreeIndices.cpp index 322cdd35afe..be8b4c795f0 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.cpp +++ b/src/Storages/MergeTree/MergeTreeIndices.cpp @@ -115,14 +115,14 @@ MergeTreeIndexFactory::MergeTreeIndexFactory() registerCreator("set", setIndexCreator); registerValidator("set", setIndexValidator); - registerCreator("ngrambf_v1", bloomFilterIndexCreator); - registerValidator("ngrambf_v1", bloomFilterIndexValidator); + registerCreator("ngrambf_v1", bloomFilterIndexTextCreator); + registerValidator("ngrambf_v1", bloomFilterIndexTextValidator); - registerCreator("tokenbf_v1", bloomFilterIndexCreator); - registerValidator("tokenbf_v1", bloomFilterIndexValidator); + registerCreator("tokenbf_v1", bloomFilterIndexTextCreator); + registerValidator("tokenbf_v1", bloomFilterIndexTextValidator); - registerCreator("bloom_filter", bloomFilterIndexCreatorNew); - registerValidator("bloom_filter", bloomFilterIndexValidatorNew); + registerCreator("bloom_filter", bloomFilterIndexCreator); + registerValidator("bloom_filter", bloomFilterIndexValidator); registerCreator("hypothesis", hypothesisIndexCreator); registerValidator("hypothesis", hypothesisIndexValidator); diff --git a/src/Storages/MergeTree/MergeTreeIndices.h b/src/Storages/MergeTree/MergeTreeIndices.h index 8fdadb4e5eb..900e6b6658c 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.h +++ b/src/Storages/MergeTree/MergeTreeIndices.h @@ -221,12 +221,12 @@ void minmaxIndexValidator(const IndexDescription & index, bool attach); MergeTreeIndexPtr setIndexCreator(const IndexDescription & index); void setIndexValidator(const IndexDescription & index, bool attach); +MergeTreeIndexPtr bloomFilterIndexTextCreator(const IndexDescription & index); +void bloomFilterIndexTextValidator(const IndexDescription & index, bool attach); + MergeTreeIndexPtr bloomFilterIndexCreator(const IndexDescription & index); void bloomFilterIndexValidator(const IndexDescription & index, bool attach); -MergeTreeIndexPtr bloomFilterIndexCreatorNew(const IndexDescription & index); -void bloomFilterIndexValidatorNew(const IndexDescription & index, bool attach); - MergeTreeIndexPtr hypothesisIndexCreator(const IndexDescription & index); void hypothesisIndexValidator(const IndexDescription & index, bool attach); diff --git a/src/Storages/tests/gtest_SplitTokenExtractor.cpp b/src/Storages/tests/gtest_SplitTokenExtractor.cpp index 62389639c11..e01673359bd 100644 --- a/src/Storages/tests/gtest_SplitTokenExtractor.cpp +++ b/src/Storages/tests/gtest_SplitTokenExtractor.cpp @@ -1,4 +1,4 @@ -#include +#include #include From ca4e2f581ab2985f4e26a8894f1b6cf9a1737d9b Mon Sep 17 00:00:00 2001 From: Alexey Katsman Date: Mon, 5 Feb 2024 23:43:39 +0100 Subject: [PATCH 0280/1018] Add columns for values used in the formatting message into system.text_log --- base/poco/Foundation/include/Poco/Message.h | 10 ++++- base/poco/Foundation/src/Message.cpp | 32 ++++++++++++--- src/Access/AccessControl.cpp | 3 +- src/Common/Exception.cpp | 6 ++- src/Common/Exception.h | 33 ++++++++------- src/Common/LoggingFormatStringHelpers.h | 40 +++++++++++++++++-- src/Common/logger_useful.h | 8 ++-- src/Functions/FunctionsConversion.cpp | 2 +- src/Interpreters/QueryLog.h | 1 + src/Interpreters/TextLog.cpp | 20 ++++++++++ src/Interpreters/TextLog.h | 10 +++++ src/Interpreters/TreeRewriter.cpp | 2 +- src/Interpreters/executeQuery.cpp | 3 ++ src/Loggers/OwnSplitChannel.cpp | 15 +++++++ ...log_format_string_args_not_empty.reference | 3 ++ ..._text_log_format_string_args_not_empty.sql | 13 ++++++ 16 files changed, 167 insertions(+), 34 deletions(-) create mode 100644 tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.reference create mode 100644 tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql diff --git a/base/poco/Foundation/include/Poco/Message.h b/base/poco/Foundation/include/Poco/Message.h index 282c7fb5fd1..9068e56a93c 100644 --- a/base/poco/Foundation/include/Poco/Message.h +++ b/base/poco/Foundation/include/Poco/Message.h @@ -66,9 +66,11 @@ public: /// The thread and process ids are set. Message( - const std::string & source, const std::string & text, Priority prio, const char * file, int line, std::string_view fmt_str = {}); + const std::string & source, const std::string & text, Priority prio, const char * file, int line, + std::string_view fmt_str = {}, const std::vector & fmt_str_args = {}); Message( - std::string && source, std::string && text, Priority prio, const char * file, int line, std::string_view fmt_str); + std::string && source, std::string && text, Priority prio, const char * file, int line, + std::string_view fmt_str, std::vector && fmt_str_args); /// Creates a Message with the given source, text, priority, /// source file path and line. /// @@ -161,6 +163,9 @@ public: std::string_view getFormatString() const; void setFormatString(std::string_view fmt_str); + const std::vector & getFormatStringArgs() const; + void setFormatStringArgs(const std::vector & fmt_str_args); + int getSourceLine() const; /// Returns the source file line of the statement /// generating the log message. May be 0 @@ -210,6 +215,7 @@ private: int _line; StringMap * _pMap; std::string_view _fmt_str; + std::vector _fmt_str_args; }; diff --git a/base/poco/Foundation/src/Message.cpp b/base/poco/Foundation/src/Message.cpp index 54118cc0fc5..9e9835ac585 100644 --- a/base/poco/Foundation/src/Message.cpp +++ b/base/poco/Foundation/src/Message.cpp @@ -46,7 +46,9 @@ Message::Message(const std::string& source, const std::string& text, Priority pr } -Message::Message(const std::string& source, const std::string& text, Priority prio, const char* file, int line, std::string_view fmt_str): +Message::Message( + const std::string& source, const std::string& text, Priority prio, const char* file, int line, + std::string_view fmt_str, const std::vector& fmt_str_args): _source(source), _text(text), _prio(prio), @@ -54,13 +56,16 @@ Message::Message(const std::string& source, const std::string& text, Priority pr _file(file), _line(line), _pMap(0), - _fmt_str(fmt_str) + _fmt_str(fmt_str), + _fmt_str_args(fmt_str_args) { init(); } -Message::Message(std::string && source, std::string && text, Priority prio, const char * file, int line, std::string_view fmt_str): +Message::Message( + std::string && source, std::string && text, Priority prio, const char * file, int line, + std::string_view fmt_str, std::vector && fmt_str_args): _source(std::move(source)), _text(std::move(text)), _prio(prio), @@ -68,7 +73,8 @@ Message::Message(std::string && source, std::string && text, Priority prio, cons _file(file), _line(line), _pMap(0), - _fmt_str(fmt_str) + _fmt_str(fmt_str), + _fmt_str_args(std::move(fmt_str_args)) { init(); } @@ -83,7 +89,8 @@ Message::Message(const Message& msg): _pid(msg._pid), _file(msg._file), _line(msg._line), - _fmt_str(msg._fmt_str) + _fmt_str(msg._fmt_str), + _fmt_str_args(msg._fmt_str_args) { if (msg._pMap) _pMap = new StringMap(*msg._pMap); @@ -102,7 +109,8 @@ Message::Message(const Message& msg, const std::string& text): _pid(msg._pid), _file(msg._file), _line(msg._line), - _fmt_str(msg._fmt_str) + _fmt_str(msg._fmt_str), + _fmt_str_args(msg._fmt_str_args) { if (msg._pMap) _pMap = new StringMap(*msg._pMap); @@ -154,6 +162,7 @@ void Message::swap(Message& msg) swap(_line, msg._line); swap(_pMap, msg._pMap); swap(_fmt_str, msg._fmt_str); + swap(_fmt_str_args, msg._fmt_str_args); } @@ -227,6 +236,17 @@ void Message::setFormatString(std::string_view fmt_str) } +const std::vector& Message::getFormatStringArgs() const +{ + return _fmt_str_args; +} + +void Message::setFormatStringArgs(const std::vector& fmt_str_args) +{ + _fmt_str_args = fmt_str_args; +} + + bool Message::has(const std::string& param) const { return _pMap && (_pMap->find(param) != _pMap->end()); diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index da047d1cb1d..aca0c4d2d53 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -607,7 +607,8 @@ AuthResult AccessControl::authenticate(const Credentials & credentials, const Po /// We use the same message for all authentication failures because we don't want to give away any unnecessary information for security reasons, /// only the log will show the exact reason. throw Exception(PreformattedMessage{message.str(), - "{}: Authentication failed: password is incorrect, or there is no user with such name.{}"}, + "{}: Authentication failed: password is incorrect, or there is no user with such name.{}", + std::vector{credentials.getUserName()}}, ErrorCodes::AUTHENTICATION_FAILED); } } diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index 5d936bbb88c..1f4b0aea8f2 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -391,6 +391,7 @@ PreformattedMessage getCurrentExceptionMessageAndPattern(bool with_stacktrace, b { WriteBufferFromOwnString stream; std::string_view message_format_string; + std::vector message_format_string_args; try { @@ -402,6 +403,7 @@ PreformattedMessage getCurrentExceptionMessageAndPattern(bool with_stacktrace, b << (with_extra_info ? getExtraExceptionInfo(e) : "") << " (version " << VERSION_STRING << VERSION_OFFICIAL << ")"; message_format_string = e.tryGetMessageFormatString(); + message_format_string_args = e.getMessageFormatStringArgs(); } catch (const Poco::Exception & e) { @@ -462,7 +464,7 @@ PreformattedMessage getCurrentExceptionMessageAndPattern(bool with_stacktrace, b catch (...) {} // NOLINT(bugprone-empty-catch) } - return PreformattedMessage{stream.str(), message_format_string}; + return PreformattedMessage{stream.str(), message_format_string, message_format_string_args}; } @@ -581,7 +583,7 @@ PreformattedMessage getExceptionMessageAndPattern(const Exception & e, bool with } catch (...) {} // NOLINT(bugprone-empty-catch) - return PreformattedMessage{stream.str(), e.tryGetMessageFormatString()}; + return PreformattedMessage{stream.str(), e.tryGetMessageFormatString(), e.getMessageFormatStringArgs()}; } std::string getExceptionMessage(std::exception_ptr e, bool with_stacktrace) diff --git a/src/Common/Exception.h b/src/Common/Exception.h index 1b4dabec113..73c50debf33 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -13,6 +13,7 @@ #include #include +#include #include #include @@ -59,6 +60,7 @@ public: std::terminate(); capture_thread_frame_pointers = thread_frame_pointers; message_format_string = msg.format_string; + message_format_string_args = msg.format_string_args; } Exception(PreformattedMessage && msg, int code): Exception(std::move(msg.text), code) @@ -67,6 +69,7 @@ public: std::terminate(); capture_thread_frame_pointers = thread_frame_pointers; message_format_string = msg.format_string; + message_format_string_args = msg.format_string_args; } /// Collect call stacks of all previous jobs' schedulings leading to this thread job's execution @@ -107,12 +110,7 @@ public: // Format message with fmt::format, like the logging functions. template - Exception(int code, FormatStringHelper fmt, Args &&... args) - : Exception(fmt::format(fmt.fmt_str, std::forward(args)...), code) - { - capture_thread_frame_pointers = thread_frame_pointers; - message_format_string = fmt.message_format_string; - } + Exception(int code, FormatStringHelper fmt, Args &&... args) : Exception(fmt.format(std::forward(args)...), code) {} struct CreateFromPocoTag {}; struct CreateFromSTDTag {}; @@ -152,6 +150,8 @@ public: std::string_view tryGetMessageFormatString() const { return message_format_string; } + std::vector getMessageFormatStringArgs() const { return message_format_string_args; } + private: #ifndef STD_EXCEPTION_HAS_STACK_TRACE StackTrace trace; @@ -162,6 +162,7 @@ private: protected: std::string_view message_format_string; + std::vector message_format_string_args; /// Local copy of static per-thread thread_frame_pointers, should be mutable to be unpoisoned on printout mutable std::vector capture_thread_frame_pointers; }; @@ -193,26 +194,29 @@ public: // Format message with fmt::format, like the logging functions. template ErrnoException(int code, FormatStringHelper fmt, Args &&... args) - : Exception(fmt::format(fmt.fmt_str, std::forward(args)...), code), saved_errno(errno) + : Exception(fmt.format(std::forward(args)...), code), saved_errno(errno) + { + addMessage(", {}", errnoToString(saved_errno)); + } + + template + ErrnoException(int code, int with_errno, FormatStringHelper fmt, Args &&... args) + : Exception(fmt.format(std::forward(args)...), code), saved_errno(with_errno) { - capture_thread_frame_pointers = thread_frame_pointers; - message_format_string = fmt.message_format_string; addMessage(", {}", errnoToString(saved_errno)); } template [[noreturn]] static void throwWithErrno(int code, int with_errno, FormatStringHelper fmt, Args &&... args) { - auto e = ErrnoException(fmt::format(fmt.fmt_str, std::forward(args)...), code, with_errno); - e.message_format_string = fmt.message_format_string; + auto e = ErrnoException(code, with_errno, std::move(fmt), std::forward(args)...); throw e; /// NOLINT } template [[noreturn]] static void throwFromPath(int code, const std::string & path, FormatStringHelper fmt, Args &&... args) { - auto e = ErrnoException(fmt::format(fmt.fmt_str, std::forward(args)...), code, errno); - e.message_format_string = fmt.message_format_string; + auto e = ErrnoException(code, errno, std::move(fmt), std::forward(args)...); e.path = path; throw e; /// NOLINT } @@ -221,8 +225,7 @@ public: [[noreturn]] static void throwFromPathWithErrno(int code, const std::string & path, int with_errno, FormatStringHelper fmt, Args &&... args) { - auto e = ErrnoException(fmt::format(fmt.fmt_str, std::forward(args)...), code, with_errno); - e.message_format_string = fmt.message_format_string; + auto e = ErrnoException(code, with_errno, std::move(fmt), std::forward(args)...); e.path = path; throw e; /// NOLINT } diff --git a/src/Common/LoggingFormatStringHelpers.h b/src/Common/LoggingFormatStringHelpers.h index 678948424da..3e64c8dfb8f 100644 --- a/src/Common/LoggingFormatStringHelpers.h +++ b/src/Common/LoggingFormatStringHelpers.h @@ -2,8 +2,11 @@ #include #include +#include +#include #include #include +#include #include #include #include @@ -14,6 +17,10 @@ struct PreformattedMessage; consteval void formatStringCheckArgsNumImpl(std::string_view str, size_t nargs); template constexpr std::string_view tryGetStaticFormatString(T && x); +[[maybe_unused]] inline void tryGetFormattedArgs(std::vector&) {}; +template [[maybe_unused]] inline void tryGetFormattedArgs(std::vector&, T &&, Ts && ...); +template inline std::string tryGetArgsAndFormat(std::vector&, fmt::format_string, Args && ...); + /// Extract format string from a string literal and constructs consteval fmt::format_string template struct FormatStringHelperImpl @@ -39,6 +46,7 @@ struct PreformattedMessage { std::string text; std::string_view format_string; + std::vector format_string_args; template static PreformattedMessage create(FormatStringHelper fmt, Args &&... args); @@ -47,22 +55,26 @@ struct PreformattedMessage operator std::string () && { return std::move(text); } /// NOLINT operator fmt::format_string<> () const { UNREACHABLE(); } /// NOLINT - void apply(std::string & out_text, std::string_view & out_format_string) const & + void apply(std::string & out_text, std::string_view & out_format_string, std::vector & out_format_string_args) const & { out_text = text; out_format_string = format_string; + out_format_string_args = format_string_args; } - void apply(std::string & out_text, std::string_view & out_format_string) && + void apply(std::string & out_text, std::string_view & out_format_string, std::vector & out_format_string_args) && { out_text = std::move(text); out_format_string = format_string; + out_format_string_args = std::move(format_string_args); } }; template PreformattedMessage FormatStringHelperImpl::format(Args && ...args) const { - return PreformattedMessage{fmt::format(fmt_str, std::forward(args)...), message_format_string}; + std::vector out_format_string_args; + std::string msg_text = tryGetArgsAndFormat(out_format_string_args, fmt_str, std::forward(args)...); + return PreformattedMessage{msg_text, message_format_string, out_format_string_args}; } template @@ -113,12 +125,23 @@ template constexpr std::string_view tryGetStaticFormatString(T && x } } +template void tryGetFormattedArgs(std::vector& out, T && x, Ts && ...rest) +{ + if constexpr (std::is_base_of_v>) + out.push_back(fmt::format("{}", std::remove_reference_t(x))); + else + out.push_back(fmt::format("{}", std::forward(x))); + + tryGetFormattedArgs(out, std::forward(rest)...); +} + /// Constexpr ifs are not like ifdefs, and compiler still checks that unneeded code can be compiled /// This template is useful to avoid compilation failures when condition of some "constexpr if" is false template struct ConstexprIfsAreNotIfdefs { template constexpr static std::string_view getStaticFormatString(T &&) { return {}; } template static PreformattedMessage getPreformatted(T &&) { return {}; } + template static std::string getArgsAndFormat(std::vector&, fmt::format_string, Args &&...) { return {}; } }; template<> struct ConstexprIfsAreNotIfdefs @@ -133,8 +156,19 @@ template<> struct ConstexprIfsAreNotIfdefs } template static T && getPreformatted(T && x) { return std::forward(x); } + + template static std::string getArgsAndFormat(std::vector& out, fmt::format_string fmt_str, Args && ...args) + { + return tryGetArgsAndFormat(out, std::move(fmt_str), std::forward(args)...); + } }; +template inline std::string tryGetArgsAndFormat(std::vector& out, fmt::format_string fmt_str, Args && ...args) +{ + tryGetFormattedArgs(out, args...); + return fmt::format(fmt_str, std::forward(args)...); +} + template constexpr size_t numArgs(Ts &&...) { return sizeof...(Ts); } template constexpr auto firstArg(T && x, Ts &&...) { return std::forward(x); } /// For implicit conversion of fmt::basic_runtime<> to char* for std::string ctor diff --git a/src/Common/logger_useful.h b/src/Common/logger_useful.h index 9d6ebaddcc6..eaae7e69dbc 100644 --- a/src/Common/logger_useful.h +++ b/src/Common/logger_useful.h @@ -2,6 +2,7 @@ /// Macros for convenient usage of Poco logger. #include +#include #include #include #include @@ -80,6 +81,7 @@ namespace impl \ std::string_view _format_string; \ std::string _formatted_message; \ + std::vector _format_string_args; \ \ if constexpr (LogTypeInfo::is_static) \ { \ @@ -91,17 +93,17 @@ namespace impl if constexpr (is_preformatted_message) \ { \ static_assert(_nargs == 1 || !is_preformatted_message); \ - ConstexprIfsAreNotIfdefs::getPreformatted(LOG_IMPL_FIRST_ARG(__VA_ARGS__)).apply(_formatted_message, _format_string); \ + ConstexprIfsAreNotIfdefs::getPreformatted(LOG_IMPL_FIRST_ARG(__VA_ARGS__)).apply(_formatted_message, _format_string, _format_string_args); \ } \ else \ { \ - _formatted_message = _nargs == 1 ? firstArg(__VA_ARGS__) : fmt::format(__VA_ARGS__); \ + _formatted_message = _nargs == 1 ? firstArg(__VA_ARGS__) : ConstexprIfsAreNotIfdefs::getArgsAndFormat(_format_string_args, __VA_ARGS__); \ } \ \ std::string _file_function = __FILE__ "; "; \ _file_function += __PRETTY_FUNCTION__; \ Poco::Message _poco_message(_logger->name(), std::move(_formatted_message), \ - (PRIORITY), _file_function.c_str(), __LINE__, _format_string); \ + (PRIORITY), _file_function.c_str(), __LINE__, _format_string, _format_string_args); \ _channel->log(_poco_message); \ } \ catch (const Poco::Exception & logger_exception) \ diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 5e072d406ad..a16ce136b9a 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -794,7 +794,7 @@ inline bool tryParseImpl(DataTypeIPv6::FieldType & x, ReadBuffer & if (isNativeNumber(result_type) && !(result_type.getName() == "IPv4" || result_type.getName() == "IPv6")) message_buf << ". Note: there are to" << result_type.getName() << "OrZero and to" << result_type.getName() << "OrNull functions, which returns zero/NULL instead of throwing exception."; - throw Exception(PreformattedMessage{message_buf.str(), "Cannot parse string {} as {}: syntax error {}"}, ErrorCodes::CANNOT_PARSE_TEXT); + throw Exception(PreformattedMessage{message_buf.str(), "Cannot parse string {} as {}: syntax error {}", {String(read_buffer.buffer().begin(), read_buffer.buffer().size()), result_type.getName()}}, ErrorCodes::CANNOT_PARSE_TEXT); } diff --git a/src/Interpreters/QueryLog.h b/src/Interpreters/QueryLog.h index f9ed8a2fb9e..5072d220160 100644 --- a/src/Interpreters/QueryLog.h +++ b/src/Interpreters/QueryLog.h @@ -86,6 +86,7 @@ struct QueryLogElement String exception; String stack_trace; std::string_view exception_format_string{}; + std::vector exception_format_string_args{}; ClientInfo client_info; diff --git a/src/Interpreters/TextLog.cpp b/src/Interpreters/TextLog.cpp index d6971bbac54..3c3eff600e4 100644 --- a/src/Interpreters/TextLog.cpp +++ b/src/Interpreters/TextLog.cpp @@ -53,6 +53,16 @@ ColumnsDescription TextLogElement::getColumnsDescription() {"source_line", std::make_shared(), "Source line from which the logging was done."}, {"message_format_string", std::make_shared(std::make_shared()), "A format string that was used to format the message."}, + {"value1", std::make_shared(), "Argument 1 that was used to format the message."}, + {"value2", std::make_shared(), "Argument 2 that was used to format the message."}, + {"value3", std::make_shared(), "Argument 3 that was used to format the message."}, + {"value4", std::make_shared(), "Argument 4 that was used to format the message."}, + {"value5", std::make_shared(), "Argument 5 that was used to format the message."}, + {"value6", std::make_shared(), "Argument 6 that was used to format the message."}, + {"value7", std::make_shared(), "Argument 7 that was used to format the message."}, + {"value8", std::make_shared(), "Argument 8 that was used to format the message."}, + {"value9", std::make_shared(), "Argument 9 that was used to format the message."}, + {"value10", std::make_shared(), "Argument 10 that was used to format the message."}, }; } @@ -79,6 +89,16 @@ void TextLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(source_line); columns[i++]->insert(message_format_string); + columns[i++]->insert(value1); + columns[i++]->insert(value2); + columns[i++]->insert(value3); + columns[i++]->insert(value4); + columns[i++]->insert(value5); + columns[i++]->insert(value6); + columns[i++]->insert(value7); + columns[i++]->insert(value8); + columns[i++]->insert(value9); + columns[i++]->insert(value10); } TextLog::TextLog(ContextPtr context_, diff --git a/src/Interpreters/TextLog.h b/src/Interpreters/TextLog.h index 90fc4ef36a0..78153b9f81e 100644 --- a/src/Interpreters/TextLog.h +++ b/src/Interpreters/TextLog.h @@ -29,6 +29,16 @@ struct TextLogElement UInt64 source_line{}; std::string_view message_format_string; + String value1; + String value2; + String value3; + String value4; + String value5; + String value6; + String value7; + String value8; + String value9; + String value10; static std::string name() { return "TextLog"; } static ColumnsDescription getColumnsDescription(); diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 914b3c3037d..50c28fbc8b2 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1249,7 +1249,7 @@ bool TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select if (no_throw) return false; - throw Exception(PreformattedMessage{ss.str(), format_string}, ErrorCodes::UNKNOWN_IDENTIFIER); + throw Exception(PreformattedMessage{ss.str(), format_string, std::vector{}}, ErrorCodes::UNKNOWN_IDENTIFIER); } required_source_columns.swap(source_columns); diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index db8753000d8..09c8c459f0a 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -200,6 +200,7 @@ static void logException(ContextPtr context, QueryLogElement & elem, bool log_er /// so we pass elem.exception_format_string as format string instead. PreformattedMessage message; message.format_string = elem.exception_format_string; + message.format_string_args = elem.exception_format_string_args; if (elem.stack_trace.empty() || !log_error) message.text = fmt::format("{} (from {}){} (in query: {})", elem.exception, @@ -504,6 +505,7 @@ void logQueryException( auto exception_message = getCurrentExceptionMessageAndPattern(/* with_stacktrace */ false); elem.exception = std::move(exception_message.text); elem.exception_format_string = exception_message.format_string; + elem.exception_format_string_args = exception_message.format_string_args; QueryStatusPtr process_list_elem = context->getProcessListElement(); @@ -597,6 +599,7 @@ void logExceptionBeforeStart( auto exception_message = getCurrentExceptionMessageAndPattern(/* with_stacktrace */ false); elem.exception = std::move(exception_message.text); elem.exception_format_string = exception_message.format_string; + elem.exception_format_string_args = exception_message.format_string_args; elem.client_info = context->getClientInfo(); diff --git a/src/Loggers/OwnSplitChannel.cpp b/src/Loggers/OwnSplitChannel.cpp index 68fad2fbaf6..fee33781c27 100644 --- a/src/Loggers/OwnSplitChannel.cpp +++ b/src/Loggers/OwnSplitChannel.cpp @@ -131,6 +131,21 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg) elem.source_line = msg.getSourceLine(); elem.message_format_string = msg.getFormatString(); +#define SET_VALUE_IF_EXISTS(INDEX) if ((INDEX) <= msg.getFormatStringArgs().size()) (elem.value##INDEX) = msg.getFormatStringArgs()[(INDEX) - 1] + + SET_VALUE_IF_EXISTS(1); + SET_VALUE_IF_EXISTS(2); + SET_VALUE_IF_EXISTS(3); + SET_VALUE_IF_EXISTS(4); + SET_VALUE_IF_EXISTS(5); + SET_VALUE_IF_EXISTS(6); + SET_VALUE_IF_EXISTS(7); + SET_VALUE_IF_EXISTS(8); + SET_VALUE_IF_EXISTS(9); + SET_VALUE_IF_EXISTS(10); + +#undef SET_VALUE_IF_EXISTS + std::shared_ptr> text_log_locked{}; text_log_locked = text_log.lock(); if (text_log_locked) diff --git a/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.reference b/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.reference new file mode 100644 index 00000000000..e8183f05f5d --- /dev/null +++ b/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.reference @@ -0,0 +1,3 @@ +1 +1 +1 diff --git a/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql b/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql new file mode 100644 index 00000000000..0f4a217a4ae --- /dev/null +++ b/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql @@ -0,0 +1,13 @@ +set allow_experimental_analyzer = true; + +select count; -- { serverError 47 } + +select conut(); -- { serverError 46 } + +system flush logs; + +select count() > 0 from system.text_log where message_format_string = 'Peak memory usage{}: {}.' and value1 is not null and value2 like '% MiB'; + +select count() > 0 from system.text_log where level = 'Error' and message_format_string = 'Unknown {}{} identifier \'{}\' in scope {}{}' and value1 = 'expression' and value3 = 'count' and value4 = 'SELECT count'; + +select count() > 0 from system.text_log where level = 'Error' and message_format_string = 'Function with name \'{}\' does not exists. In scope {}{}' and value1 = 'conut' and value2 = 'SELECT conut()' and value3 ilike '%\'count\'%'; From 7490e7b2b507e4b1f168c307dcd27a9ed811242f Mon Sep 17 00:00:00 2001 From: Alexey Katsman Date: Mon, 5 Feb 2024 23:44:55 +0100 Subject: [PATCH 0281/1018] Add docs for columns with values used in the formatting message in system.text_log --- docs/en/operations/system-tables/text_log.md | 20 ++++++++++++++++++++ docs/ru/operations/system-tables/text_log.md | 20 ++++++++++++++++++++ 2 files changed, 40 insertions(+) diff --git a/docs/en/operations/system-tables/text_log.md b/docs/en/operations/system-tables/text_log.md index 0c39499e190..2f0e1c6bf50 100644 --- a/docs/en/operations/system-tables/text_log.md +++ b/docs/en/operations/system-tables/text_log.md @@ -30,6 +30,16 @@ Columns: - `source_file` (LowCardinality(String)) — Source file from which the logging was done. - `source_line` (UInt64) — Source line from which the logging was done. - `message_format_string` (LowCardinality(String)) — A format string that was used to format the message. +- `value1` (String) - Argument 1 that was used to format the message. +- `value2` (String) - Argument 2 that was used to format the message. +- `value3` (String) - Argument 3 that was used to format the message. +- `value4` (String) - Argument 4 that was used to format the message. +- `value5` (String) - Argument 5 that was used to format the message. +- `value6` (String) - Argument 6 that was used to format the message. +- `value7` (String) - Argument 7 that was used to format the message. +- `value8` (String) - Argument 8 that was used to format the message. +- `value9` (String) - Argument 9 that was used to format the message. +- `value10` (String) - Argument 10 that was used to format the message. **Example** @@ -55,4 +65,14 @@ revision: 54440 source_file: /ClickHouse/src/Interpreters/DNSCacheUpdater.cpp; void DB::DNSCacheUpdater::start() source_line: 45 message_format_string: Update period {} seconds +value1: 15 +value2: +value3: +value4: +value5: +value6: +value7: +value8: +value9: +value10: ``` diff --git a/docs/ru/operations/system-tables/text_log.md b/docs/ru/operations/system-tables/text_log.md index 59ae804d85f..a6ab6f7d268 100644 --- a/docs/ru/operations/system-tables/text_log.md +++ b/docs/ru/operations/system-tables/text_log.md @@ -29,6 +29,16 @@ slug: /ru/operations/system-tables/text_log - `source_file` (LowCardinality(String)) — исходный файл, из которого была сделана запись. - `source_line` (UInt64) — исходная строка, из которой была сделана запись. - `message_format_string` (LowCardinality(String)) — форматная строка, с помощью которой было отформатировано сообщение. +- `value1` (String) - аргумент 1, который использовался для форматирования сообщения. +- `value2` (String) - аргумент 2, который использовался для форматирования сообщения. +- `value3` (String) - аргумент 3, который использовался для форматирования сообщения. +- `value4` (String) - аргумент 4, который использовался для форматирования сообщения. +- `value5` (String) - аргумент 5, который использовался для форматирования сообщения. +- `value6` (String) - аргумент 6, который использовался для форматирования сообщения. +- `value7` (String) - аргумент 7, который использовался для форматирования сообщения. +- `value8` (String) - аргумент 8, который использовался для форматирования сообщения. +- `value9` (String) - аргумент 9, который использовался для форматирования сообщения. +- `value10` (String) - аргумент 10, который использовался для форматирования сообщения. **Пример** @@ -53,4 +63,14 @@ revision: 54440 source_file: /ClickHouse/src/Interpreters/DNSCacheUpdater.cpp; void DB::DNSCacheUpdater::start() source_line: 45 message_format_string: Update period {} seconds +value1: 15 +value2: +value3: +value4: +value5: +value6: +value7: +value8: +value9: +value10: ``` From 01434974604d02c65853772135a51688819b026a Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 23 Apr 2024 17:29:10 +0100 Subject: [PATCH 0282/1018] fix tidy --- src/Compression/CompressionCodecGCD.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Compression/CompressionCodecGCD.cpp b/src/Compression/CompressionCodecGCD.cpp index 02864f2fcee..59e11f896f0 100644 --- a/src/Compression/CompressionCodecGCD.cpp +++ b/src/Compression/CompressionCodecGCD.cpp @@ -211,7 +211,7 @@ void CompressionCodecGCD::doDecompressData(const char * source, UInt32 source_si throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress GCD-encoded data. File has wrong header"); UInt8 bytes_to_skip = uncompressed_size % bytes_size; - chassert(bytes_to_skip == source[1]); + chassert(bytes_to_skip == static_cast(source[1])); UInt32 output_size = uncompressed_size - bytes_to_skip; From 5dfd7bfd2d4a6bc4596d6e0910b4672e2f5fd3f0 Mon Sep 17 00:00:00 2001 From: Jhonso7393 <167781426+Jhonso7393@users.noreply.github.com> Date: Tue, 23 Apr 2024 19:48:59 +0300 Subject: [PATCH 0283/1018] Update README.md Fixing a minor typo --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 2b97bd25d70..84859ac78d2 100644 --- a/README.md +++ b/README.md @@ -39,7 +39,7 @@ Every month we get together with the community (users, contributors, customers, ## Upcoming Events -Keep an eye out for upcoming meetups and eventsaround the world. Somewhere else you want us to be? Please feel free to reach out to tyler `` clickhouse `` com. You can also peruse [ClickHouse Events](https://clickhouse.com/company/news-events) for a list of all upcoming trainings, meetups, speaking engagements, etc. +Keep an eye out for upcoming meetups and events around the world. Somewhere else you want us to be? Please feel free to reach out to tyler `` clickhouse `` com. You can also peruse [ClickHouse Events](https://clickhouse.com/company/news-events) for a list of all upcoming trainings, meetups, speaking engagements, etc. * [ClickHouse Meetup in Bellevue](https://www.meetup.com/clickhouse-seattle-user-group/events/298650371/) - Mar 11 * [ClickHouse Meetup at Ramp's Offices in NYC](https://www.meetup.com/clickhouse-new-york-user-group/events/298640542/) - Mar 19 From 2dcd1b673154d678a6e6c04db6f8ff89e3bd2d88 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 23 Apr 2024 19:30:04 +0200 Subject: [PATCH 0284/1018] Update README.md --- README.md | 7 ------- 1 file changed, 7 deletions(-) diff --git a/README.md b/README.md index 84859ac78d2..0d50813e30a 100644 --- a/README.md +++ b/README.md @@ -41,13 +41,6 @@ Every month we get together with the community (users, contributors, customers, Keep an eye out for upcoming meetups and events around the world. Somewhere else you want us to be? Please feel free to reach out to tyler `` clickhouse `` com. You can also peruse [ClickHouse Events](https://clickhouse.com/company/news-events) for a list of all upcoming trainings, meetups, speaking engagements, etc. -* [ClickHouse Meetup in Bellevue](https://www.meetup.com/clickhouse-seattle-user-group/events/298650371/) - Mar 11 -* [ClickHouse Meetup at Ramp's Offices in NYC](https://www.meetup.com/clickhouse-new-york-user-group/events/298640542/) - Mar 19 -* [ClickHouse Melbourne Meetup](https://www.meetup.com/clickhouse-australia-user-group/events/299479750/) - Mar 20 -* [ClickHouse Meetup in Paris](https://www.meetup.com/clickhouse-france-user-group/events/298997115/) - Mar 21 -* [ClickHouse Meetup in Bengaluru](https://www.meetup.com/clickhouse-bangalore-user-group/events/299479850/) - Mar 23 -* [ClickHouse Meetup in Zurich](https://www.meetup.com/clickhouse-switzerland-meetup-group/events/299628922/) - Apr 16 -* [ClickHouse Meetup in Copenhagen](https://www.meetup.com/clickhouse-denmark-meetup-group/events/299629133/) - Apr 23 * [ClickHouse Meetup in Dubai](https://www.meetup.com/clickhouse-dubai-meetup-group/events/299629189/) - May 28 From 69b259b40936fc1c03d545d0aa54bb48228d6578 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Tue, 23 Apr 2024 16:34:00 -0700 Subject: [PATCH 0285/1018] Fix build for openssl dynamic linking --- contrib/openssl-cmake/CMakeLists.txt | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/contrib/openssl-cmake/CMakeLists.txt b/contrib/openssl-cmake/CMakeLists.txt index ddbe9f06b1d..14528fbc16f 100644 --- a/contrib/openssl-cmake/CMakeLists.txt +++ b/contrib/openssl-cmake/CMakeLists.txt @@ -93,6 +93,7 @@ enable_language(ASM) if(COMPILER_CLANG) add_definitions(-Wno-unused-command-line-argument) + set(CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=lld") endif() if(ARCH_AMD64) @@ -960,11 +961,6 @@ set(CRYPTO_SRC ${OPENSSL_SOURCE_DIR}/crypto/x509/x_req.c ${OPENSSL_SOURCE_DIR}/crypto/x509/x_x509.c ${OPENSSL_SOURCE_DIR}/crypto/x509/x_x509a.c - ${OPENSSL_SOURCE_DIR}/engines/e_capi.c - ${OPENSSL_SOURCE_DIR}/engines/e_dasync.c - ${OPENSSL_SOURCE_DIR}/engines/e_loader_attic.c - ${OPENSSL_SOURCE_DIR}/engines/e_ossltest.c - ${OPENSSL_SOURCE_DIR}/engines/e_padlock.c ${OPENSSL_SOURCE_DIR}/providers/baseprov.c ${OPENSSL_SOURCE_DIR}/providers/common/bio_prov.c ${OPENSSL_SOURCE_DIR}/providers/common/capabilities.c @@ -985,8 +981,6 @@ set(CRYPTO_SRC ${OPENSSL_SOURCE_DIR}/providers/common/securitycheck.c ${OPENSSL_SOURCE_DIR}/providers/common/securitycheck_default.c ${OPENSSL_SOURCE_DIR}/providers/defltprov.c - ${OPENSSL_SOURCE_DIR}/providers/fips/fips_entry.c - ${OPENSSL_SOURCE_DIR}/providers/fips/fipsprov.c ${OPENSSL_SOURCE_DIR}/providers/implementations/asymciphers/rsa_enc.c ${OPENSSL_SOURCE_DIR}/providers/implementations/asymciphers/sm2_enc.c ${OPENSSL_SOURCE_DIR}/providers/implementations/ciphers/cipher_aes.c @@ -1145,9 +1139,10 @@ set(CRYPTO_SRC ${OPENSSL_SOURCE_DIR}/providers/implementations/signature/sm2_sig.c ${OPENSSL_SOURCE_DIR}/providers/implementations/storemgmt/file_store.c ${OPENSSL_SOURCE_DIR}/providers/implementations/storemgmt/file_store_any2obj.c - ${OPENSSL_SOURCE_DIR}/providers/legacyprov.c ${OPENSSL_SOURCE_DIR}/providers/nullprov.c ${OPENSSL_SOURCE_DIR}/providers/prov_running.c + ${OPENSSL_SOURCE_DIR}/ssl/record/methods/tls_pad.c + ${OPENSSL_SOURCE_DIR}/ssl/record/methods/ssl3_cbc.c ) if(ARCH_AMD64) From 7c24d4f48de6b62faf76445e729eea536dbec010 Mon Sep 17 00:00:00 2001 From: Alexey Petrunyaka <167422282+pet74alex@users.noreply.github.com> Date: Wed, 24 Apr 2024 03:33:53 +0300 Subject: [PATCH 0286/1018] Update Russian version of uuid-functions.md --- .../sql-reference/functions/uuid-functions.md | 238 ++++++++++++++++++ 1 file changed, 238 insertions(+) diff --git a/docs/ru/sql-reference/functions/uuid-functions.md b/docs/ru/sql-reference/functions/uuid-functions.md index 65d13079ee8..8f41d2ab6f4 100644 --- a/docs/ru/sql-reference/functions/uuid-functions.md +++ b/docs/ru/sql-reference/functions/uuid-functions.md @@ -51,6 +51,166 @@ SELECT generateUUIDv4(1), generateUUIDv4(2) └──────────────────────────────────────┴──────────────────────────────────────┘ ``` +## generateUUIDv7 {#uuidv7-function-generate} + +Генерирует идентификатор [UUID версии 7](https://datatracker.ietf.org/doc/html/draft-peabody-dispatch-new-uuid-format-04). Генерируемый UUID состоит из 48-битной временной метки (Unix time в миллисекундах), маркеров версии 7 и варианта 2, и случайных данных в следующей последовательности: +``` + 0 1 2 3 + 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| unix_ts_ms | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| unix_ts_ms | ver | rand_a | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +|var| rand_b | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| rand_b | +└─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┘ +``` + +**Синтаксис** + +``` sql +generateUUIDv7([x]) +``` + +**Аргументы** + +- `x` — [выражение](../syntax.md#syntax-expressions), возвращающее значение одного из [поддерживаемых типов данных](../data-types/index.md#data_types). Значение используется, чтобы избежать [склейки одинаковых выражений](index.md#common-subexpression-elimination), если функция вызывается несколько раз в одном запросе. Необязательный параметр. + +**Возвращаемое значение** + +Значение типа [UUID](../../sql-reference/functions/uuid-functions.md). + +**Пример использования** + +Этот пример демонстрирует, как создать таблицу с UUID-колонкой и добавить в нее сгенерированный UUIDv7. + +``` sql +CREATE TABLE t_uuid (x UUID) ENGINE=TinyLog + +INSERT INTO t_uuid SELECT generateUUIDv7() + +SELECT * FROM t_uuid +``` + +``` text +┌────────────────────────────────────x─┐ +│ 018f05af-f4a8-778f-beee-1bedbc95c93b │ +└──────────────────────────────────────┘ +``` + +**Пример использования, для генерации нескольких значений в одной строке** + +```sql +SELECT generateUUIDv7(1), generateUUIDv7(7) +┌─generateUUIDv7(1)────────────────────┬─generateUUIDv7(2)────────────────────┐ +│ 018f05b1-8c2e-7567-a988-48d09606ae8c │ 018f05b1-8c2e-7946-895b-fcd7635da9a0 │ +└──────────────────────────────────────┴──────────────────────────────────────┘ +``` + +## generateUUIDv7WithCounter {#uuidv7withcounter-function-generate} + +Генерирует идентификатор [UUID версии 7](https://datatracker.ietf.org/doc/html/draft-peabody-dispatch-new-uuid-format-04). Генерируемый UUID состоит из 48-битной временной метки (Unix time в миллисекундах), маркеров версии 7 и варианта 2, монотонно возрастающего счётчика для данной временной метки и случайных данных в указанной ниже последовательности. Для каждой новой временной метки счётчик стартует с нового случайного значения, а для следующих UUIDv7 он увеличивается на единицу. В случае переполнения счётчика временная метка принудительно увеличивается на 1, и счётчик снова стартует со случайного значения. Монотонность возрастания счётчика для каждой временной метки гарантируется между всеми одновременно работающими функциями `generateUUIDv7WithCounter`. +``` + 0 1 2 3 + 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| unix_ts_ms | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| unix_ts_ms | ver | counter_high_bits | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +|var| counter_low_bits | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| rand_b | +└─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┘ +``` +**Синтаксис** + +``` sql +generateUUIDv7WithCounter([x]) +``` + +**Аргументы** + +- `x` — [выражение](../syntax.md#syntax-expressions), возвращающее значение одного из [поддерживаемых типов данных](../data-types/index.md#data_types). Значение используется, чтобы избежать [склейки одинаковых выражений](index.md#common-subexpression-elimination), если функция вызывается несколько раз в одном запросе. Необязательный параметр. + +**Возвращаемое значение** + +Значение типа [UUID](../../sql-reference/functions/uuid-functions.md). + +**Пример использования** + +Этот пример демонстрирует, как создать таблицу с UUID-колонкой и добавить в нее сгенерированный UUIDv7. + +``` sql +CREATE TABLE t_uuid (x UUID) ENGINE=TinyLog + +INSERT INTO t_uuid SELECT generateUUIDv7WithCounter() + +SELECT * FROM t_uuid +``` + +``` text +┌────────────────────────────────────x─┐ +│ 018f05c7-56e3-7ac3-93e9-1d93c4218e0e │ +└──────────────────────────────────────┘ +``` + +**Пример использования, для генерации нескольких значений в одной строке** + +```sql +SELECT generateUUIDv7WithCounter(1), generateUUIDv7WithCounter(7) +┌─generateUUIDv7WithCounter(1)─────────┬─generateUUIDv7WithCounter(2)─────────┐ +│ 018f05c9-4ab8-7b86-b64e-c9f03fbd45d1 │ 018f05c9-4ab8-7b86-b64e-c9f12efb7e16 │ +└──────────────────────────────────────┴──────────────────────────────────────┘ +``` + +## generateUUIDv7WithFastCounter {#uuidv7withfastcounter-function-generate} + +Генерирует идентификатор [UUID версии 7](https://datatracker.ietf.org/doc/html/draft-peabody-dispatch-new-uuid-format-04). Данная функция является ускоренным аналогом функции `generateUUIDv7WithCounter` за счёт потери гарантии монотонности счётчика при одной и той же метке времени между одновременно исполняемыми разными запросами. Монотонность счётчика гарантируется только в пределах одного треда, исполняющего данную функцию для генерации нескольких UUID. + +**Синтаксис** + +``` sql +generateUUIDv7WithFastCounter([x]) +``` + +**Аргументы** + +- `x` — [выражение](../syntax.md#syntax-expressions), возвращающее значение одного из [поддерживаемых типов данных](../data-types/index.md#data_types). Значение используется, чтобы избежать [склейки одинаковых выражений](index.md#common-subexpression-elimination), если функция вызывается несколько раз в одном запросе. Необязательный параметр. + +**Возвращаемое значение** + +Значение типа [UUID](../../sql-reference/functions/uuid-functions.md). + +**Пример использования** + +Этот пример демонстрирует, как создать таблицу с UUID-колонкой и добавить в нее сгенерированный UUIDv7. + +``` sql +CREATE TABLE t_uuid (x UUID) ENGINE=TinyLog + +INSERT INTO t_uuid SELECT generateUUIDv7WithFastCounter() + +SELECT * FROM t_uuid +``` + +``` text +┌────────────────────────────────────x─┐ +│ 018f05e2-e3b2-70cb-b8be-64b09b626d32 │ +└──────────────────────────────────────┘ +``` + +**Пример использования, для генерации нескольких значений в одной строке** + +```sql +SELECT generateUUIDv7WithFastCounter(1), generateUUIDv7WithFastCounter(7) +┌─generateUUIDv7WithFastCounter(1)─────┬─generateUUIDv7WithFastCounter(2)─────┐ +│ 018f05e1-14ee-7bc5-9906-207153b400b1 │ 018f05e1-14ee-7bc5-9906-2072b8e96758 │ +└──────────────────────────────────────┴──────────────────────────────────────┘ +``` + ## empty {#empty} Проверяет, является ли входной UUID пустым. @@ -259,6 +419,84 @@ SELECT └──────────────────┴──────────────────────────────────────┘ ``` +## UUIDToNum {#uuidtonum} + +Принимает UUID и возвращает в виде набора байт в [FixedString(16)](../../sql-reference/functions/uuid-functions.md). Также принимает необязательный второй параметр - вариант представления UUID, по умолчанию 1 - `Big-endian` (2 означает представление в формате `Microsoft`). Данная функция заменяет последовательность из двух отдельных функций `UUIDStringToNum(toString(uuid))`, так что промежуточная конвертация из UUID в String для извлечения набора байт из UUID не требуется. + +``` sql +UUIDToNum(UUID[, variant = 1]) +``` + +**Возвращаемое значение** + +FixedString(16) + +**Примеры использования** + +``` sql +SELECT + toUUID('612f3c40-5d3b-217e-707b-6a546a3d7b29') AS uuid, + UUIDToNum(uuid) AS bytes +``` + +``` text +┌─uuid─────────────────────────────────┬─bytes────────────┐ +│ 612f3c40-5d3b-217e-707b-6a546a3d7b29 │ a/<@];!~p{jTj={) │ +└──────────────────────────────────────┴──────────────────┘ +``` +``` sql +SELECT + toUUID('612f3c40-5d3b-217e-707b-6a546a3d7b29') AS uuid, + UUIDToNum(uuid, 2) AS bytes +``` + +```text +┌─uuid─────────────────────────────────┬─bytes────────────┐ +│ 612f3c40-5d3b-217e-707b-6a546a3d7b29 │ @ Date: Wed, 24 Apr 2024 02:50:04 +0200 Subject: [PATCH 0287/1018] Update run.sh --- docker/test/stateless/run.sh | 2 -- 1 file changed, 2 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 9497b7ecc19..55dfe5e2e08 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -44,8 +44,6 @@ source /utils.lib if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then echo "Azure is disabled" -elif [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then - echo "Azure is disabled" else azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --debug /azurite_log & fi From 56e32e0f996d8a0838ef9f89f7a1db0bc5b14a38 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 23 Apr 2024 14:19:56 +0000 Subject: [PATCH 0288/1018] Rename MergeTreeIndexInverted* to MergeTreeIndexFullText* --- .../mergetree-family/invertedindexes.md | 46 ++++++------- src/Core/Settings.h | 2 +- src/Interpreters/GinFilter.cpp | 2 +- .../MergeTreeDataPartWriterOnDisk.cpp | 8 +-- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 6 +- .../MergeTree/MergeTreeIndexBloomFilter.h | 5 ++ ...nverted.cpp => MergeTreeIndexFullText.cpp} | 66 +++++++++---------- ...dexInverted.h => MergeTreeIndexFullText.h} | 28 ++++---- src/Storages/MergeTree/MergeTreeIndices.cpp | 6 +- src/Storages/MergeTree/MergeTreeIndices.h | 4 +- src/Storages/MergeTree/MutateTask.cpp | 4 +- ...> 02346_fulltext_index_bug47393.reference} | 0 ....sql => 02346_fulltext_index_bug47393.sql} | 2 +- ...> 02346_fulltext_index_bug52019.reference} | 0 ....sql => 02346_fulltext_index_bug52019.sql} | 2 +- ...> 02346_fulltext_index_bug59039.reference} | 0 ....sql => 02346_fulltext_index_bug59039.sql} | 2 +- ...46_fulltext_index_detach_attach.reference} | 0 ...=> 02346_fulltext_index_detach_attach.sql} | 2 +- ..._fulltext_index_match_predicate.reference} | 0 ... 02346_fulltext_index_match_predicate.sql} | 2 +- ... => 02346_fulltext_index_search.reference} | 20 +++--- ...ch.sql => 02346_fulltext_index_search.sql} | 64 +++++++++--------- 23 files changed, 138 insertions(+), 133 deletions(-) rename src/Storages/MergeTree/{MergeTreeIndexInverted.cpp => MergeTreeIndexFullText.cpp} (93%) rename src/Storages/MergeTree/{MergeTreeIndexInverted.h => MergeTreeIndexFullText.h} (87%) rename tests/queries/0_stateless/{02346_inverted_index_bug47393.reference => 02346_fulltext_index_bug47393.reference} (100%) rename tests/queries/0_stateless/{02346_inverted_index_bug47393.sql => 02346_fulltext_index_bug47393.sql} (93%) rename tests/queries/0_stateless/{02346_inverted_index_bug52019.reference => 02346_fulltext_index_bug52019.reference} (100%) rename tests/queries/0_stateless/{02346_inverted_index_bug52019.sql => 02346_fulltext_index_bug52019.sql} (91%) rename tests/queries/0_stateless/{02346_inverted_index_bug59039.reference => 02346_fulltext_index_bug59039.reference} (100%) rename tests/queries/0_stateless/{02346_inverted_index_bug59039.sql => 02346_fulltext_index_bug59039.sql} (93%) rename tests/queries/0_stateless/{02346_inverted_index_detach_attach.reference => 02346_fulltext_index_detach_attach.reference} (100%) rename tests/queries/0_stateless/{02346_inverted_index_detach_attach.sql => 02346_fulltext_index_detach_attach.sql} (82%) rename tests/queries/0_stateless/{02346_inverted_index_match_predicate.reference => 02346_fulltext_index_match_predicate.reference} (100%) rename tests/queries/0_stateless/{02346_inverted_index_match_predicate.sql => 02346_fulltext_index_match_predicate.sql} (97%) rename tests/queries/0_stateless/{02346_inverted_index_search.reference => 02346_fulltext_index_search.reference} (74%) rename tests/queries/0_stateless/{02346_inverted_index_search.sql => 02346_fulltext_index_search.sql} (87%) diff --git a/docs/en/engines/table-engines/mergetree-family/invertedindexes.md b/docs/en/engines/table-engines/mergetree-family/invertedindexes.md index 7e5140b4c4f..9374f6a3ac1 100644 --- a/docs/en/engines/table-engines/mergetree-family/invertedindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/invertedindexes.md @@ -1,19 +1,19 @@ --- slug: /en/engines/table-engines/mergetree-family/invertedindexes -sidebar_label: Inverted Indexes +sidebar_label: Full-text Indexes description: Quickly find search terms in text. keywords: [full-text search, text search, inverted, index, indices] --- -# Full-text Search using Inverted Indexes [experimental] +# Full-text Search using Full-text Indexes [experimental] -Inverted indexes are an experimental type of [secondary indexes](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#available-types-of-indices) which provide fast text search +Full-text indexes are an experimental type of [secondary indexes](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#available-types-of-indices) which provide fast text search capabilities for [String](/docs/en/sql-reference/data-types/string.md) or [FixedString](/docs/en/sql-reference/data-types/fixedstring.md) -columns. The main idea of an inverted index is to store a mapping from "terms" to the rows which contain these terms. "Terms" are +columns. The main idea of a full-text index is to store a mapping from "terms" to the rows which contain these terms. "Terms" are tokenized cells of the string column. For example, the string cell "I will be a little late" is by default tokenized into six terms "I", "will", "be", "a", "little" and "late". Another kind of tokenizer is n-grams. For example, the result of 3-gram tokenization will be 21 terms "I w", " wi", "wil", "ill", "ll ", "l b", " be" etc. The more fine-granular the input strings are tokenized, the bigger but also the more -useful the resulting inverted index will be. +useful the resulting full-text index will be.
-Syntax to create an Annoy index over an [Array](../../../sql-reference/data-types/array.md) column: +Syntax to create an Annoy index over an [Array(Float32)](../../../sql-reference/data-types/array.md) column: ```sql CREATE TABLE table_with_annoy_index @@ -177,19 +163,6 @@ ENGINE = MergeTree ORDER BY id; ``` -Syntax to create an ANN index over a [Tuple](../../../sql-reference/data-types/tuple.md) column: - -```sql -CREATE TABLE table_with_annoy_index -( - id Int64, - vectors Tuple(Float32[, Float32[, ...]]), - INDEX [ann_index_name] vectors TYPE annoy([Distance[, NumTrees]]) [GRANULARITY N] -) -ENGINE = MergeTree -ORDER BY id; -``` - Annoy currently supports two distance functions: - `L2Distance`, also called Euclidean distance, is the length of a line segment between two points in Euclidean space ([Wikipedia](https://en.wikipedia.org/wiki/Euclidean_distance)). @@ -203,10 +176,9 @@ Parameter `NumTrees` is the number of trees which the algorithm creates (default more accurate search results but slower index creation / query times (approximately linearly) as well as larger index sizes. :::note -Indexes over columns of type `Array` will generally work faster than indexes on `Tuple` columns. All arrays must have same length. To avoid -errors, you can use a [CONSTRAINT](/docs/en/sql-reference/statements/create/table.md#constraints), for example, `CONSTRAINT -constraint_name_1 CHECK length(vectors) = 256`. Also, empty `Arrays` and unspecified `Array` values in INSERT statements (i.e. default -values) are not supported. +All arrays must have same length. To avoid errors, you can use a +[CONSTRAINT](/docs/en/sql-reference/statements/create/table.md#constraints), for example, `CONSTRAINT constraint_name_1 CHECK +length(vectors) = 256`. Also, empty `Arrays` and unspecified `Array` values in INSERT statements (i.e. default values) are not supported. ::: The creation of Annoy indexes (whenever a new part is build, e.g. at the end of a merge) is a relatively slow process. You can increase @@ -264,19 +236,6 @@ ENGINE = MergeTree ORDER BY id; ``` -Syntax to create an ANN index over a [Tuple](../../../sql-reference/data-types/tuple.md) column: - -```sql -CREATE TABLE table_with_usearch_index -( - id Int64, - vectors Tuple(Float32[, Float32[, ...]]), - INDEX [ann_index_name] vectors TYPE usearch([Distance[, ScalarKind]]) [GRANULARITY N] -) -ENGINE = MergeTree -ORDER BY id; -``` - USearch currently supports two distance functions: - `L2Distance`, also called Euclidean distance, is the length of a line segment between two points in Euclidean space ([Wikipedia](https://en.wikipedia.org/wiki/Euclidean_distance)). From 9d55bc82d6609633a135a5044f05aadeaff21755 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 6 May 2024 16:19:57 +0200 Subject: [PATCH 0838/1018] Revert "Revert "Do not remove server constants from GROUP BY key for secondary query."" --- src/Planner/PlannerExpressionAnalysis.cpp | 6 ++++-- .../03095_group_by_server_constants_bug.reference | 1 + .../0_stateless/03095_group_by_server_constants_bug.sql | 5 +++++ 3 files changed, 10 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03095_group_by_server_constants_bug.reference create mode 100644 tests/queries/0_stateless/03095_group_by_server_constants_bug.sql diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index ad8db83d66c..d7fa270a643 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -85,6 +85,8 @@ std::optional analyzeAggregation(const QueryTreeNodeP bool group_by_use_nulls = planner_context->getQueryContext()->getSettingsRef().group_by_use_nulls && (query_node.isGroupByWithGroupingSets() || query_node.isGroupByWithRollup() || query_node.isGroupByWithCube()); + bool is_secondary_query = planner_context->getQueryContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY; + if (query_node.hasGroupBy()) { if (query_node.isGroupByWithGroupingSets()) @@ -100,7 +102,7 @@ std::optional analyzeAggregation(const QueryTreeNodeP auto is_constant_key = grouping_set_key_node->as() != nullptr; group_by_with_constant_keys |= is_constant_key; - if (is_constant_key && !aggregates_descriptions.empty()) + if (!is_secondary_query && is_constant_key && !aggregates_descriptions.empty()) continue; auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions, grouping_set_key_node); @@ -152,7 +154,7 @@ std::optional analyzeAggregation(const QueryTreeNodeP auto is_constant_key = group_by_key_node->as() != nullptr; group_by_with_constant_keys |= is_constant_key; - if (is_constant_key && !aggregates_descriptions.empty()) + if (!is_secondary_query && is_constant_key && !aggregates_descriptions.empty()) continue; auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions, group_by_key_node); diff --git a/tests/queries/0_stateless/03095_group_by_server_constants_bug.reference b/tests/queries/0_stateless/03095_group_by_server_constants_bug.reference new file mode 100644 index 00000000000..80ab3c879bb --- /dev/null +++ b/tests/queries/0_stateless/03095_group_by_server_constants_bug.reference @@ -0,0 +1 @@ +r1 2 diff --git a/tests/queries/0_stateless/03095_group_by_server_constants_bug.sql b/tests/queries/0_stateless/03095_group_by_server_constants_bug.sql new file mode 100644 index 00000000000..9f9fda1ef62 --- /dev/null +++ b/tests/queries/0_stateless/03095_group_by_server_constants_bug.sql @@ -0,0 +1,5 @@ +SELECT serverUUID() AS s, count() FROM remote('127.0.0.{1,2}', system.one) GROUP BY s format Null; + +select getMacro('replica') as s, count() from remote('127.0.0.{1,2}', system.one) group by s; + +select uptime() as s, count() FROM remote('127.0.0.{1,2}', system.one) group by s format Null; From 81398da882a19d70138d003824c0cb855f7faf29 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Mon, 6 May 2024 16:20:51 +0200 Subject: [PATCH 0839/1018] Update function order Moved `sigmoid` function from the end of the file to after `sign` function. --- .../sql-reference/functions/math-functions.md | 67 +++++++++---------- 1 file changed, 33 insertions(+), 34 deletions(-) diff --git a/docs/en/sql-reference/functions/math-functions.md b/docs/en/sql-reference/functions/math-functions.md index 2feb91cfce4..945166056af 100644 --- a/docs/en/sql-reference/functions/math-functions.md +++ b/docs/en/sql-reference/functions/math-functions.md @@ -792,6 +792,39 @@ Result: │ -1 │ └──────────┘ ``` +## sigmoid + +Returns the [sigmoid function](https://en.wikipedia.org/wiki/Sigmoid_function). + +**Syntax** + +```sql +sigmoid(x) +``` + +**Parameters** + +- `x` — input value. Values from the interval: `-∞ < x < +∞`. [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). + +**Returned value** + +- Corresponding value along the sigmoid curve between 0 and 1. [Float64](../../sql-reference/data-types/float.md). + +**Example** + +Query: + +``` sql +SELECT round(sigmoid(x), 5) FROM (SELECT arrayJoin([-1, 0, 1]) AS x); +``` + +Result: + +```result +0.26894 +0.5 +0.73106 +``` ## degrees @@ -914,37 +947,3 @@ Result: │ 11 │ └──────────────────────────────────┘ ``` - -## sigmoid - -Returns the [sigmoid function](https://en.wikipedia.org/wiki/Sigmoid_function). - -**Syntax** - -```sql -sigmoid(x) -``` - -**Parameters** - -- `x` — input value. Values from the interval: `-∞ < x < +∞`. [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). - -**Returned value** - -- Corresponding value along the sigmoid curve between 0 and 1. [Float64](../../sql-reference/data-types/float.md). - -**Example** - -Query: - -``` sql -SELECT round(sigmoid(x), 5) FROM (SELECT arrayJoin([-1, 0, 1]) AS x); -``` - -Result: - -```result -0.26894 -0.5 -0.73106 -``` From 87176fdd131d97aec5f694fc1fabfb12521727e3 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Mon, 6 May 2024 16:37:30 +0200 Subject: [PATCH 0840/1018] Update date-time-functions.md - Change parameter `x` to `num` to be more idiomatic. - Correct issues with casing in types. --- .../functions/date-time-functions.md | 92 +++++++++---------- 1 file changed, 46 insertions(+), 46 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 5a6ab02efc7..cee905db4b9 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -2588,16 +2588,16 @@ Subtracts a specified number of years from a date, a date with time or a string- **Syntax** ```sql -subtractYears(date, x) +subtractYears(date, num) ``` **Parameters** -- `date`: Date / date with time to subtract specified number of years from. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md), [String](../data-types/string.md). -- `x`: Number of years to subtract. [(U)Int*](../data-types/int-uint.md), [float*](../data-types/float.md). +- `date`: Date / date with time to subtract specified number of years from. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md), [String](../data-types/string.md). +- `num`: Number of years to subtract. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md). **Returned value** -- Returns `date` minus `x` years. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md). +- Returns `date` minus `num` years. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md). **Example** @@ -2625,16 +2625,16 @@ Subtracts a specified number of quarters from a date, a date with time or a stri **Syntax** ```sql -subtractQuarters(date, x) +subtractQuarters(date, num) ``` **Parameters** -- `date`: Date / date with time to subtract specified number of quarters from. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md), [String](../data-types/string.md). -- `x`: Number of quarters to subtract. [(U)Int*](../data-types/int-uint.md), [float*](../data-types/float.md). +- `date`: Date / date with time to subtract specified number of quarters from. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md), [String](../data-types/string.md). +- `num`: Number of quarters to subtract. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md). **Returned value** -- Returns `date` minus `x` quarters. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md). +- Returns `date` minus `num` quarters. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md). **Example** @@ -2662,16 +2662,16 @@ Subtracts a specified number of months from a date, a date with time or a string **Syntax** ```sql -subtractMonths(date, x) +subtractMonths(date, num) ``` **Parameters** -- `date`: Date / date with time to subtract specified number of months from. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md), [String](../data-types/string.md). -- `x`: Number of months to subtract. [(U)Int*](../data-types/int-uint.md), [float*](../data-types/float.md). +- `date`: Date / date with time to subtract specified number of months from. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md), [String](../data-types/string.md). +- `num`: Number of months to subtract. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md). **Returned value** -- Returns `date` minus `x` months. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md). +- Returns `date` minus `num` months. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md). **Example** @@ -2699,16 +2699,16 @@ Subtracts a specified number of weeks from a date, a date with time or a string- **Syntax** ```sql -subtractWeeks(date, x) +subtractWeeks(date, num) ``` **Parameters** -- `date`: Date / date with time to subtract specified number of weeks from. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md), [String](../data-types/string.md). -- `x`: Number of weeks to subtract. [(U)Int*](../data-types/int-uint.md), [float*](../data-types/float.md). +- `date`: Date / date with time to subtract specified number of weeks from. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md), [String](../data-types/string.md). +- `num`: Number of weeks to subtract. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md). **Returned value** -- Returns `date` minus `x` weeks. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md). +- Returns `date` minus `num` weeks. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md). **Example** @@ -2736,16 +2736,16 @@ Subtracts a specified number of days from a date, a date with time or a string-e **Syntax** ```sql -subtractDays(date, x) +subtractDays(date, num) ``` **Parameters** -- `date`: Date / date with time to subtract specified number of days from. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md), [String](../data-types/string.md). -- `x`: Number of days to subtract. [(U)Int*](../data-types/int-uint.md), [float*](../data-types/float.md). +- `date`: Date / date with time to subtract specified number of days from. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md), [String](../data-types/string.md). +- `num`: Number of days to subtract. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md). **Returned value** -- Returns `date` minus `x` days. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md). +- Returns `date` minus `num` days. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md). **Example** @@ -2773,16 +2773,16 @@ Subtracts a specified number of hours from a date, a date with time or a string- **Syntax** ```sql -subtractHours(date, x) +subtractHours(date, num) ``` **Parameters** -- `date`: Date / date with time to subtract specified number of hours from. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md), [String](../data-types/string.md). -- `x`: Number of hours to subtract. [(U)Int*](../data-types/int-uint.md), [float*](../data-types/float.md). +- `date`: Date / date with time to subtract specified number of hours from. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[Datetime](../data-types/datetime.md)/[Datetime64](../data-types/datetime64.md), [String](../data-types/string.md). +- `num`: Number of hours to subtract. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md). **Returned value** -- Returns `date` minus `x` hours. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md). +- Returns `date` minus `num` hours. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[Datetime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md). **Example** @@ -2810,16 +2810,16 @@ Subtracts a specified number of minutes from a date, a date with time or a strin **Syntax** ```sql -subtractMinutes(date, x) +subtractMinutes(date, num) ``` **Parameters** -- `date`: Date / date with time to subtract specified number of minutes from. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md), [String](../data-types/string.md). -- `x`: Number of minutes to subtract. [(U)Int*](../data-types/int-uint.md), [float*](../data-types/float.md). +- `date`: Date / date with time to subtract specified number of minutes from. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md), [String](../data-types/string.md). +- `num`: Number of minutes to subtract. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md). **Returned value** -- Returns `date` minus `x` minutes. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md). +- Returns `date` minus `num` minutes. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md). **Example** @@ -2847,16 +2847,16 @@ Subtracts a specified number of seconds from a date, a date with time or a strin **Syntax** ```sql -subtractSeconds(date, x) +subtractSeconds(date, num) ``` **Parameters** -- `date`: Date / date with time to subtract specified number of seconds from. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md), [String](../data-types/string.md). -- `x`: Number of seconds to subtract. [(U)Int*](../data-types/int-uint.md), [float*](../data-types/float.md). +- `date`: Date / date with time to subtract specified number of seconds from. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md), [String](../data-types/string.md). +- `num`: Number of seconds to subtract. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md). **Returned value** -- Returns `date` minus `x` seconds. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md). +- Returns `date` minus `num` seconds. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md). **Example** @@ -2884,16 +2884,16 @@ Subtracts a specified number of milliseconds from a date with time or a string-e **Syntax** ```sql -subtractMilliseconds(date_time, x) +subtractMilliseconds(date_time, num) ``` **Parameters** -- `date_time`: Date with time to subtract specified number of milliseconds from. [datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md), [String](../data-types/string.md). -- `x`: Number of milliseconds to subtract. [(U)Int*](../data-types/int-uint.md), [float*](../data-types/float.md). +- `date_time`: Date with time to subtract specified number of milliseconds from. [DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md), [String](../data-types/string.md). +- `num`: Number of milliseconds to subtract. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md). **Returned value** -- Returns `date_time` minus `x` milliseconds. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md). +- Returns `date_time` minus `num` milliseconds. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md). **Example** @@ -2919,16 +2919,16 @@ Subtracts a specified number of microseconds from a date with time or a string-e **Syntax** ```sql -subtractMicroseconds(date_time, x) +subtractMicroseconds(date_time, num) ``` **Parameters** -- `date_time`: Date with time to subtract specified number of microseconds from. [datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md), [String](../data-types/string.md). -- `x`: Number of microseconds to subtract. [(U)Int*](../data-types/int-uint.md), [float*](../data-types/float.md). +- `date_time`: Date with time to subtract specified number of microseconds from. [DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md), [String](../data-types/string.md). +- `num`: Number of microseconds to subtract. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md). **Returned value** -- Returns `date_time` minus `x` microseconds. [datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md). +- Returns `date_time` minus `num` microseconds. [DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md). **Example** @@ -2954,16 +2954,16 @@ Subtracts a specified number of nanoseconds from a date with time or a string-en **Syntax** ```sql -subtractNanoseconds(date_time, x) +subtractNanoseconds(date_time, num) ``` **Parameters** -- `date_time`: Date with time to subtract specified number of nanoseconds from. [datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md), [String](../data-types/string.md). -- `x`: Number of nanoseconds to subtract. [(U)Int*](../data-types/int-uint.md), [float*](../data-types/float.md). +- `date_time`: Date with time to subtract specified number of nanoseconds from. [DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md), [String](../data-types/string.md). +- `num`: Number of nanoseconds to subtract. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md). **Returned value** -- Returns `date_time` minus `x` nanoseconds. [datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md). +- Returns `date_time` minus `num` nanoseconds. [DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md). **Example** @@ -3040,11 +3040,11 @@ subtractTupleOfIntervals(interval_1, interval_2) **Parameters** -- `date`: First interval or interval of tuples. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md). +- `date`: First interval or interval of tuples. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md). - `intervals`: Tuple of intervals to subtract from `date`. [tuple](../data-types/tuple.md)([interval](../data-types/special-data-types/interval.md)). **Returned value** -- Returns `date` with subtracted `intervals`. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md). +- Returns `date` with subtracted `intervals`. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md). **Example** From c0d2a99a9e5cb02de730fc237166c41b79d84034 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Mon, 6 May 2024 16:51:23 +0200 Subject: [PATCH 0841/1018] Update date-time-functions.md --- .../functions/date-time-functions.md | 88 +++++++++---------- 1 file changed, 44 insertions(+), 44 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 629b3ca9305..493cc9b7648 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -2564,16 +2564,16 @@ Adds a specified number of years to a date, a date with time or a string-encoded **Syntax** ```sql -addYears(date, x) +addYears(date, num) ``` **Parameters** -- `date`: Date / date with time to add specified number of years to. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md), [String](../data-types/string.md). -- `x`: Number of years to add. [(U)Int*](../data-types/int-uint.md), [float*](../data-types/float.md). +- `date`: Date / date with time to add specified number of years to. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md), [String](../data-types/string.md). +- `num`: Number of years to add. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md). **Returned value** -- Returns `date` plus `x` years. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md). +- Returns `date` plus `num` years. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md). **Example** @@ -2601,16 +2601,16 @@ Adds a specified number of quarters to a date, a date with time or a string-enco **Syntax** ```sql -addQuarters(date, x) +addQuarters(date, num) ``` **Parameters** -- `date`: Date / date with time to add specified number of quarters to. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md), [String](../data-types/string.md). -- `x`: Number of quarters to add. [(U)Int*](../data-types/int-uint.md), [float*](../data-types/float.md). +- `date`: Date / date with time to add specified number of quarters to. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md), [String](../data-types/string.md). +- `num`: Number of quarters to add. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md). **Returned value** -- Returns `date` plus `x` quarters. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md). +- Returns `date` plus `num` quarters. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md). **Example** @@ -2638,16 +2638,16 @@ Adds a specified number of months to a date, a date with time or a string-encode **Syntax** ```sql -addMonths(date, x) +addMonths(date, num) ``` **Parameters** -- `date`: Date / date with time to add specified number of months to. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md), [String](../data-types/string.md). -- `x`: Number of months to add. [(U)Int*](../data-types/int-uint.md), [float*](../data-types/float.md). +- `date`: Date / date with time to add specified number of months to. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md), [String](../data-types/string.md). +- `num`: Number of months to add. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md). **Returned value** -- Returns `date` plus `x` months. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md). +- Returns `date` plus `num` months. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md). **Example** @@ -2675,16 +2675,16 @@ Adds a specified number of weeks to a date, a date with time or a string-encoded **Syntax** ```sql -addWeeks(date, x) +addWeeks(date, num) ``` **Parameters** -- `date`: Date / date with time to add specified number of weeks to. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md), [String](../data-types/string.md). -- `x`: Number of weeks to add. [(U)Int*](../data-types/int-uint.md), [float*](../data-types/float.md). +- `date`: Date / date with time to add specified number of weeks to. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md), [String](../data-types/string.md). +- `num`: Number of weeks to add. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md). **Returned value** -- Returns `date` plus `x` weeks. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md). +- Returns `date` plus `num` weeks. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md). **Example** @@ -2712,16 +2712,16 @@ Adds a specified number of days to a date, a date with time or a string-encoded **Syntax** ```sql -addDays(date, x) +addDays(date, num) ``` **Parameters** -- `date`: Date / date with time to add specified number of days to. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md), [String](../data-types/string.md). -- `x`: Number of days to add. [(U)Int*](../data-types/int-uint.md), [float*](../data-types/float.md). +- `date`: Date / date with time to add specified number of days to. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md), [String](../data-types/string.md). +- `num`: Number of days to add. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md). **Returned value** -- Returns `date` plus `x` days. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md). +- Returns `date` plus `num` days. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md). **Example** @@ -2749,16 +2749,16 @@ Adds a specified number of days to a date, a date with time or a string-encoded **Syntax** ```sql -addHours(date, x) +addHours(date, num) ``` **Parameters** -- `date`: Date / date with time to add specified number of hours to. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md), [String](../data-types/string.md). -- `x`: Number of hours to add. [(U)Int*](../data-types/int-uint.md), [float*](../data-types/float.md). +- `date`: Date / date with time to add specified number of hours to. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md), [String](../data-types/string.md). +- `num`: Number of hours to add. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md). **Returned value** -- Returns `date` plus `x` hours. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md). +- Returns `date` plus `num` hours. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md). **Example** @@ -2786,16 +2786,16 @@ Adds a specified number of minutes to a date, a date with time or a string-encod **Syntax** ```sql -addMinutes(date, x) +addMinutes(date, num) ``` **Parameters** -- `date`: Date / date with time to add specified number of minutes to. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md), [String](../data-types/string.md). -- `x`: Number of minutes to add. [(U)Int*](../data-types/int-uint.md), [float*](../data-types/float.md). +- `date`: Date / date with time to add specified number of minutes to. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md), [String](../data-types/string.md). +- `num`: Number of minutes to add. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md). **Returned value** -- Returns `date` plus `x` minutes. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md). +- Returns `date` plus `num` minutes. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md). **Example** @@ -2823,16 +2823,16 @@ Adds a specified number of seconds to a date, a date with time or a string-encod **Syntax** ```sql -addSeconds(date, x) +addSeconds(date, num) ``` **Parameters** -- `date`: Date / date with time to add specified number of seconds to. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md), [String](../data-types/string.md). -- `x`: Number of seconds to add. [(U)Int*](../data-types/int-uint.md), [float*](../data-types/float.md). +- `date`: Date / date with time to add specified number of seconds to. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md), [String](../data-types/string.md). +- `num`: Number of seconds to add. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md). **Returned value** -- Returns `date` plus `x` seconds. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md). +- Returns `date` plus `num` seconds. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md). **Example** @@ -2860,16 +2860,16 @@ Adds a specified number of milliseconds to a date with time or a string-encoded **Syntax** ```sql -addMilliseconds(date_time, x) +addMilliseconds(date_time, num) ``` **Parameters** -- `date_time`: Date with time to add specified number of milliseconds to. [datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md), [String](../data-types/string.md). -- `x`: Number of milliseconds to add. [(U)Int*](../data-types/int-uint.md), [float*](../data-types/float.md). +- `date_time`: Date with time to add specified number of milliseconds to. [DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md), [String](../data-types/string.md). +- `num`: Number of milliseconds to add. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md). **Returned value** -- Returns `date_time` plus `x` milliseconds. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md). +- Returns `date_time` plus `num` milliseconds. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md). **Example** @@ -2895,16 +2895,16 @@ Adds a specified number of microseconds to a date with time or a string-encoded **Syntax** ```sql -addMicroseconds(date_time, x) +addMicroseconds(date_time, num) ``` **Parameters** -- `date_time`: Date with time to add specified number of microseconds to. [datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md), [String](../data-types/string.md). -- `x`: Number of microseconds to add. [(U)Int*](../data-types/int-uint.md), [float*](../data-types/float.md). +- `date_time`: Date with time to add specified number of microseconds to. [DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md), [String](../data-types/string.md). +- `num`: Number of microseconds to add. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md). **Returned value** -- Returns `date_time` plus `x` microseconds. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md). +- Returns `date_time` plus `num` microseconds. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md). **Example** @@ -2930,16 +2930,16 @@ Adds a specified number of microseconds to a date with time or a string-encoded **Syntax** ```sql -addNanoseconds(date_time, x) +addNanoseconds(date_time, num) ``` **Parameters** -- `date_time`: Date with time to add specified number of nanoseconds to. [datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md), [String](../data-types/string.md). -- `x`: Number of nanoseconds to add. [(U)Int*](../data-types/int-uint.md), [float*](../data-types/float.md). +- `date_time`: Date with time to add specified number of nanoseconds to. [DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md), [String](../data-types/string.md). +- `num`: Number of nanoseconds to add. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md). **Returned value** -- Returns `date_time` plus `x` nanoseconds. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md). +- Returns `date_time` plus `num` nanoseconds. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md). **Example** From c3abf7f462985ec4e05fbac076373d787f04721c Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Mon, 6 May 2024 14:55:34 +0000 Subject: [PATCH 0842/1018] propagate session_timezone to result_details when query fails --- src/Interpreters/executeQuery.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 1923e8ae78a..c471035df1d 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1522,6 +1522,9 @@ void executeQuery( if (output_format) handle_exception_in_output_format(*output_format, format_name, context, output_format_settings); } + /// The timezone was already set before query was processed, + /// But `session_timezone` setting could be modified in the query itself, so we update the value. + result_details.timezone = DateLUT::instance().getTimeZone(); throw; } From 962ebd1a9b8ac03e7874edb15ced7037792a2ba7 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Mon, 6 May 2024 15:17:46 +0000 Subject: [PATCH 0843/1018] add test for failing query --- .../00265_http_content_type_format_timezone.reference | 1 + .../0_stateless/00265_http_content_type_format_timezone.sh | 2 ++ 2 files changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/00265_http_content_type_format_timezone.reference b/tests/queries/0_stateless/00265_http_content_type_format_timezone.reference index 39bc7e8cab8..cd44e2d1aa0 100644 --- a/tests/queries/0_stateless/00265_http_content_type_format_timezone.reference +++ b/tests/queries/0_stateless/00265_http_content_type_format_timezone.reference @@ -23,3 +23,4 @@ < X-ClickHouse-Timezone: Africa/Cairo < X-ClickHouse-Timezone: Europe/Berlin < X-ClickHouse-Timezone: America/Denver +< X-ClickHouse-Timezone: Europe/Lisbon diff --git a/tests/queries/0_stateless/00265_http_content_type_format_timezone.sh b/tests/queries/0_stateless/00265_http_content_type_format_timezone.sh index fb5ea456ed7..86e76de133d 100755 --- a/tests/queries/0_stateless/00265_http_content_type_format_timezone.sh +++ b/tests/queries/0_stateless/00265_http_content_type_format_timezone.sh @@ -22,3 +22,5 @@ CLICKHOUSE_URL_WO_SESSION_TZ=$(echo "${CLICKHOUSE_URL}" |sed 's/\&session_timezo ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL_WO_SESSION_TZ}&session_timezone=Europe/Berlin&query=SELECT+timezone()" 2>&1 | grep '< X-ClickHouse-Timezone' | grep -v 'GET' | tr -d '\r'; ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL_WO_SESSION_TZ}&session_timezone=America/Denver&query=SELECT+timezone()" 2>&1 | grep '< X-ClickHouse-Timezone' | grep -v 'GET' | tr -d '\r'; +# check that proper X-ClickHouse-Timezone returned on query fail +${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL_WO_SESSION_TZ}&session_timezone=UTC&query=SELECT+intDiv(1,+(3600-timeZoneOffset(now())))+SETTINGS+session_timezone+=+'Europe/Lisbon'" 2>&1 | grep '< X-ClickHouse-Timezone' | grep -v 'GET' | tr -d '\r'; From 34a9ec3bb1d3f6d54b6f2737a28dc75dc7849e99 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 6 May 2024 17:25:06 +0200 Subject: [PATCH 0844/1018] empty commit From 8e51419998f6e12a2eeb3f84c1270b67d59670e9 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 6 May 2024 17:25:23 +0200 Subject: [PATCH 0845/1018] Hardcode DateTime instead of now() to make test stable --- .../0_stateless/00265_http_content_type_format_timezone.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00265_http_content_type_format_timezone.sh b/tests/queries/0_stateless/00265_http_content_type_format_timezone.sh index 86e76de133d..813d839a2ad 100755 --- a/tests/queries/0_stateless/00265_http_content_type_format_timezone.sh +++ b/tests/queries/0_stateless/00265_http_content_type_format_timezone.sh @@ -23,4 +23,4 @@ CLICKHOUSE_URL_WO_SESSION_TZ=$(echo "${CLICKHOUSE_URL}" |sed 's/\&session_timezo ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL_WO_SESSION_TZ}&session_timezone=Europe/Berlin&query=SELECT+timezone()" 2>&1 | grep '< X-ClickHouse-Timezone' | grep -v 'GET' | tr -d '\r'; ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL_WO_SESSION_TZ}&session_timezone=America/Denver&query=SELECT+timezone()" 2>&1 | grep '< X-ClickHouse-Timezone' | grep -v 'GET' | tr -d '\r'; # check that proper X-ClickHouse-Timezone returned on query fail -${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL_WO_SESSION_TZ}&session_timezone=UTC&query=SELECT+intDiv(1,+(3600-timeZoneOffset(now())))+SETTINGS+session_timezone+=+'Europe/Lisbon'" 2>&1 | grep '< X-ClickHouse-Timezone' | grep -v 'GET' | tr -d '\r'; +${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL_WO_SESSION_TZ}&session_timezone=UTC&query=SELECT+intDiv(1,+(3600-timeZoneOffset('2024-05-06+12:00:00'::DateTime)))+SETTINGS+session_timezone+=+'Europe/Lisbon'" 2>&1 | grep '< X-ClickHouse-Timezone' | grep -v 'GET' | tr -d '\r'; From c8e761116a9d6fead08472e232438695082d7649 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Mon, 6 May 2024 15:51:16 +0000 Subject: [PATCH 0846/1018] add more tests for session_timezone --- .../00265_http_content_type_format_timezone.reference | 1 + .../0_stateless/00265_http_content_type_format_timezone.sh | 2 ++ tests/queries/0_stateless/02737_session_timezone.reference | 1 + tests/queries/0_stateless/02737_session_timezone.sql | 3 +++ 4 files changed, 7 insertions(+) diff --git a/tests/queries/0_stateless/00265_http_content_type_format_timezone.reference b/tests/queries/0_stateless/00265_http_content_type_format_timezone.reference index cd44e2d1aa0..caf1c25eb6c 100644 --- a/tests/queries/0_stateless/00265_http_content_type_format_timezone.reference +++ b/tests/queries/0_stateless/00265_http_content_type_format_timezone.reference @@ -24,3 +24,4 @@ < X-ClickHouse-Timezone: Europe/Berlin < X-ClickHouse-Timezone: America/Denver < X-ClickHouse-Timezone: Europe/Lisbon +< X-ClickHouse-Timezone: Europe/Lisbon diff --git a/tests/queries/0_stateless/00265_http_content_type_format_timezone.sh b/tests/queries/0_stateless/00265_http_content_type_format_timezone.sh index 813d839a2ad..9ca801a1417 100755 --- a/tests/queries/0_stateless/00265_http_content_type_format_timezone.sh +++ b/tests/queries/0_stateless/00265_http_content_type_format_timezone.sh @@ -24,3 +24,5 @@ ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL_WO_SESSION_TZ}&session_timezone=Europe ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL_WO_SESSION_TZ}&session_timezone=America/Denver&query=SELECT+timezone()" 2>&1 | grep '< X-ClickHouse-Timezone' | grep -v 'GET' | tr -d '\r'; # check that proper X-ClickHouse-Timezone returned on query fail ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL_WO_SESSION_TZ}&session_timezone=UTC&query=SELECT+intDiv(1,+(3600-timeZoneOffset('2024-05-06+12:00:00'::DateTime)))+SETTINGS+session_timezone+=+'Europe/Lisbon'" 2>&1 | grep '< X-ClickHouse-Timezone' | grep -v 'GET' | tr -d '\r'; +# main query's session_timezone shall be set in header +${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL_WO_SESSION_TZ}&session_timezone=America/New_York&query=SELECT+1,(SELECT+1+SETTINGS+session_timezone='UTC')+SETTINGS+session_timezone='Europe/Lisbon'" 2>&1 | grep '< X-ClickHouse-Timezone' | grep -v 'GET' | tr -d '\r'; diff --git a/tests/queries/0_stateless/02737_session_timezone.reference b/tests/queries/0_stateless/02737_session_timezone.reference index 578aec4e316..6c6fc2aa93a 100644 --- a/tests/queries/0_stateless/02737_session_timezone.reference +++ b/tests/queries/0_stateless/02737_session_timezone.reference @@ -2,6 +2,7 @@ Pacific/Pitcairn Pacific/Pitcairn Asia/Novosibirsk Asia/Novosibirsk 2022-12-12 17:23:23 2022-12-13 07:23:23.123 +2022-12-13 07:23:23 2022-12-13 07:23:23 2002-12-12 23:23:23 2002-12-12 23:23:23 2002-12-12 23:23:23.123 2002-12-12 23:23:23.123 2000-01-01 01:00:00 diff --git a/tests/queries/0_stateless/02737_session_timezone.sql b/tests/queries/0_stateless/02737_session_timezone.sql index 8543e2a2c82..1afadbde6df 100644 --- a/tests/queries/0_stateless/02737_session_timezone.sql +++ b/tests/queries/0_stateless/02737_session_timezone.sql @@ -9,6 +9,9 @@ SELECT timezone(), timezoneOf(now()); SELECT toDateTime(toDateTime('2022-12-12 23:23:23'), 'Europe/Zurich'); SELECT toDateTime64(toDateTime64('2022-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS session_timezone = 'America/Denver'; +-- subquery shall use main query's session_timezone +SELECT toDateTime(toDateTime('2022-12-12 23:23:23'), 'Europe/Zurich'), (SELECT toDateTime(toDateTime('2022-12-12 23:23:23'), 'Europe/Zurich') SETTINGS session_timezone = 'Europe/Helsinki') SETTINGS session_timezone = 'America/Denver'; + -- test proper serialization SELECT toDateTime('2002-12-12 23:23:23') AS dt, toString(dt) SETTINGS session_timezone = 'Asia/Phnom_Penh'; SELECT toDateTime64('2002-12-12 23:23:23.123', 3) AS dt64, toString(dt64) SETTINGS session_timezone = 'Asia/Phnom_Penh'; From 0f87653fef1319db4ea54111383ff90f3adf90a5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 2 May 2024 07:21:35 +0000 Subject: [PATCH 0847/1018] Rename "inverted" to "fulltext" --- .../mergetree-family/invertedindexes.md | 2 +- .../mergetree-family/mergetree.md | 52 +++++++++---------- src/Common/FST.cpp | 2 +- src/Core/Settings.h | 2 +- src/Interpreters/GinFilter.cpp | 2 +- src/Interpreters/GinFilter.h | 7 ++- src/Interpreters/InterpreterCreateQuery.cpp | 9 ++-- src/Storages/AlterCommands.cpp | 4 +- src/Storages/AlterCommands.h | 4 +- src/Storages/MergeTree/GinIndexStore.cpp | 2 +- src/Storages/MergeTree/GinIndexStore.h | 4 +- src/Storages/MergeTree/MergeTreeData.cpp | 4 +- .../MergeTree/MergeTreeDataPartChecksum.cpp | 4 +- .../MergeTreeDataPartWriterOnDisk.cpp | 2 +- src/Storages/MergeTree/MutateTask.cpp | 4 +- src/Storages/MergeTree/checkDataPart.cpp | 2 +- ...02346_inverted_index_experimental_flag.sql | 2 +- 17 files changed, 52 insertions(+), 56 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/invertedindexes.md b/docs/en/engines/table-engines/mergetree-family/invertedindexes.md index 832ea1e3793..f58a06464b2 100644 --- a/docs/en/engines/table-engines/mergetree-family/invertedindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/invertedindexes.md @@ -2,7 +2,7 @@ slug: /en/engines/table-engines/mergetree-family/invertedindexes sidebar_label: Full-text Indexes description: Quickly find search terms in text. -keywords: [full-text search, text search, inverted, index, indices] +keywords: [full-text search, text search, index, indices] --- # Full-text Search using Full-text Indexes [experimental] diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 886c29e755e..7862eef69f8 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -494,7 +494,7 @@ Syntax: `tokenbf_v1(size_of_bloom_filter_in_bytes, number_of_hash_functions, ran #### Special-purpose - Experimental indexes to support approximate nearest neighbor (ANN) search. See [here](annindexes.md) for details. -- An experimental inverted index to support full-text search. See [here](invertedindexes.md) for details. +- An experimental full-text index to support full-text search. See [here](invertedindexes.md) for details. ### Functions Support {#functions-support} @@ -502,31 +502,31 @@ Conditions in the `WHERE` clause contains calls of the functions that operate wi Indexes of type `set` can be utilized by all functions. The other index types are supported as follows: -| Function (operator) / Index | primary key | minmax | ngrambf_v1 | tokenbf_v1 | bloom_filter | inverted | -|------------------------------------------------------------------------------------------------------------|-------------|--------|------------|------------|--------------|----------| -| [equals (=, ==)](/docs/en/sql-reference/functions/comparison-functions.md/#equals) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | -| [notEquals(!=, <>)](/docs/en/sql-reference/functions/comparison-functions.md/#notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | -| [like](/docs/en/sql-reference/functions/string-search-functions.md/#like) | ✔ | ✔ | ✔ | ✔ | ✗ | ✔ | -| [notLike](/docs/en/sql-reference/functions/string-search-functions.md/#notlike) | ✔ | ✔ | ✔ | ✔ | ✗ | ✔ | -| [match](/docs/en/sql-reference/functions/string-search-functions.md/#match) | ✗ | ✗ | ✔ | ✔ | ✗ | ✔ | -| [startsWith](/docs/en/sql-reference/functions/string-functions.md/#startswith) | ✔ | ✔ | ✔ | ✔ | ✗ | ✔ | -| [endsWith](/docs/en/sql-reference/functions/string-functions.md/#endswith) | ✗ | ✗ | ✔ | ✔ | ✗ | ✔ | -| [multiSearchAny](/docs/en/sql-reference/functions/string-search-functions.md/#multisearchany) | ✗ | ✗ | ✔ | ✗ | ✗ | ✔ | -| [in](/docs/en/sql-reference/functions/in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | -| [notIn](/docs/en/sql-reference/functions/in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | -| [less (<)](/docs/en/sql-reference/functions/comparison-functions.md/#less) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | -| [greater (>)](/docs/en/sql-reference/functions/comparison-functions.md/#greater) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | -| [lessOrEquals (<=)](/docs/en/sql-reference/functions/comparison-functions.md/#lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | -| [greaterOrEquals (>=)](/docs/en/sql-reference/functions/comparison-functions.md/#greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | -| [empty](/docs/en/sql-reference/functions/array-functions/#empty) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | -| [notEmpty](/docs/en/sql-reference/functions/array-functions/#notempty) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | -| [has](/docs/en/sql-reference/functions/array-functions/#has) | ✗ | ✗ | ✔ | ✔ | ✔ | ✔ | -| [hasAny](/docs/en/sql-reference/functions/array-functions/#hasany) | ✗ | ✗ | ✔ | ✔ | ✔ | ✗ | -| [hasAll](/docs/en/sql-reference/functions/array-functions/#hasall) | ✗ | ✗ | ✗ | ✗ | ✔ | ✗ | -| hasToken | ✗ | ✗ | ✗ | ✔ | ✗ | ✔ | -| hasTokenOrNull | ✗ | ✗ | ✗ | ✔ | ✗ | ✔ | -| hasTokenCaseInsensitive (*) | ✗ | ✗ | ✗ | ✔ | ✗ | ✗ | -| hasTokenCaseInsensitiveOrNull (*) | ✗ | ✗ | ✗ | ✔ | ✗ | ✗ | +| Function (operator) / Index | primary key | minmax | ngrambf_v1 | tokenbf_v1 | bloom_filter | full_text | +|------------------------------------------------------------------------------------------------------------|-------------|--------|------------|------------|--------------|-----------| +| [equals (=, ==)](/docs/en/sql-reference/functions/comparison-functions.md/#equals) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | +| [notEquals(!=, <>)](/docs/en/sql-reference/functions/comparison-functions.md/#notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | +| [like](/docs/en/sql-reference/functions/string-search-functions.md/#like) | ✔ | ✔ | ✔ | ✔ | ✗ | ✔ | +| [notLike](/docs/en/sql-reference/functions/string-search-functions.md/#notlike) | ✔ | ✔ | ✔ | ✔ | ✗ | ✔ | +| [match](/docs/en/sql-reference/functions/string-search-functions.md/#match) | ✗ | ✗ | ✔ | ✔ | ✗ | ✔ | +| [startsWith](/docs/en/sql-reference/functions/string-functions.md/#startswith) | ✔ | ✔ | ✔ | ✔ | ✗ | ✔ | +| [endsWith](/docs/en/sql-reference/functions/string-functions.md/#endswith) | ✗ | ✗ | ✔ | ✔ | ✗ | ✔ | +| [multiSearchAny](/docs/en/sql-reference/functions/string-search-functions.md/#multisearchany) | ✗ | ✗ | ✔ | ✗ | ✗ | ✔ | +| [in](/docs/en/sql-reference/functions/in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | +| [notIn](/docs/en/sql-reference/functions/in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | +| [less (<)](/docs/en/sql-reference/functions/comparison-functions.md/#less) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | +| [greater (>)](/docs/en/sql-reference/functions/comparison-functions.md/#greater) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | +| [lessOrEquals (<=)](/docs/en/sql-reference/functions/comparison-functions.md/#lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | +| [greaterOrEquals (>=)](/docs/en/sql-reference/functions/comparison-functions.md/#greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | +| [empty](/docs/en/sql-reference/functions/array-functions/#empty) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | +| [notEmpty](/docs/en/sql-reference/functions/array-functions/#notempty) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ | +| [has](/docs/en/sql-reference/functions/array-functions/#has) | ✗ | ✗ | ✔ | ✔ | ✔ | ✔ | +| [hasAny](/docs/en/sql-reference/functions/array-functions/#hasany) | ✗ | ✗ | ✔ | ✔ | ✔ | ✗ | +| [hasAll](/docs/en/sql-reference/functions/array-functions/#hasall) | ✗ | ✗ | ✗ | ✗ | ✔ | ✗ | +| hasToken | ✗ | ✗ | ✗ | ✔ | ✗ | ✔ | +| hasTokenOrNull | ✗ | ✗ | ✗ | ✔ | ✗ | ✔ | +| hasTokenCaseInsensitive (*) | ✗ | ✗ | ✗ | ✔ | ✗ | ✗ | +| hasTokenCaseInsensitiveOrNull (*) | ✗ | ✗ | ✗ | ✔ | ✗ | ✗ | Functions with a constant argument that is less than ngram size can’t be used by `ngrambf_v1` for query optimization. diff --git a/src/Common/FST.cpp b/src/Common/FST.cpp index af2acc859e7..9f4e2f1f9e0 100644 --- a/src/Common/FST.cpp +++ b/src/Common/FST.cpp @@ -306,7 +306,7 @@ void FstBuilder::add(std::string_view current_word, Output current_output) size_t current_word_len = current_word.size(); if (current_word_len > MAX_TERM_LENGTH) - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Cannot build inverted index: The maximum term length is {}, this is exceeded by term {}", MAX_TERM_LENGTH, current_word_len); + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Cannot build full-text index: The maximum term length is {}, this is exceeded by term {}", MAX_TERM_LENGTH, current_word_len); size_t prefix_length_plus1 = getCommonPrefixLength(current_word, previous_word) + 1; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 04029983d84..b4313d9af56 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -235,7 +235,7 @@ class IColumn; M(Bool, do_not_merge_across_partitions_select_final, false, "Merge parts only in one partition in select final", 0) \ M(Bool, split_parts_ranges_into_intersecting_and_non_intersecting_final, true, "Split parts ranges into intersecting and non intersecting during FINAL optimization", 0) \ M(Bool, split_intersecting_parts_ranges_into_layers_final, true, "Split intersecting parts ranges into layers during FINAL optimization", 0) \ - M(Bool, allow_experimental_inverted_index, false, "If it is set to true, allow to use experimental fulltext (inverted) index.", 0) \ + M(Bool, allow_experimental_inverted_index, false, "If it is set to true, allow to use experimental full-text index.", 0) \ \ M(UInt64, mysql_max_rows_to_insert, 65536, "The maximum number of rows in MySQL batch insertion of the MySQL storage engine", 0) \ M(Bool, mysql_map_string_to_text_in_show_columns, true, "If enabled, String type will be mapped to TEXT in SHOW [FULL] COLUMNS, BLOB otherwise. Has an effect only when the connection is made through the MySQL wire protocol.", 0) \ diff --git a/src/Interpreters/GinFilter.cpp b/src/Interpreters/GinFilter.cpp index dd13a264d96..b9b5697d2ee 100644 --- a/src/Interpreters/GinFilter.cpp +++ b/src/Interpreters/GinFilter.cpp @@ -29,7 +29,7 @@ GinFilterParameters::GinFilterParameters(size_t ngrams_, UInt64 max_rows_per_pos max_rows_per_postings_list = std::numeric_limits::max(); if (ngrams > 8) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "The size of inverted index filter cannot be greater than 8"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The size of full-text index filter cannot be greater than 8"); } GinFilter::GinFilter(const GinFilterParameters & params_) diff --git a/src/Interpreters/GinFilter.h b/src/Interpreters/GinFilter.h index 7924bd4ce3d..5d8c631be17 100644 --- a/src/Interpreters/GinFilter.h +++ b/src/Interpreters/GinFilter.h @@ -2,12 +2,11 @@ #include #include -#include namespace DB { -static inline constexpr auto INVERTED_INDEX_NAME = "inverted"; +static inline constexpr auto FULL_TEXT_INDEX_NAME = "full_text"; static inline constexpr UInt64 UNLIMITED_ROWS_PER_POSTINGS_LIST = 0; static inline constexpr UInt64 MIN_ROWS_PER_POSTINGS_LIST = 8 * 1024; static inline constexpr UInt64 DEFAULT_MAX_ROWS_PER_POSTINGS_LIST = 64 * 1024; @@ -34,7 +33,7 @@ struct GinSegmentWithRowIdRange using GinSegmentWithRowIdRangeVector = std::vector; -/// GinFilter provides underlying functionalities for building inverted index and also +/// GinFilter provides underlying functionalities for building full-text index and also /// it does filtering the unmatched rows according to its query string. /// It also builds and uses skipping index which stores (segmentID, RowIDStart, RowIDEnd) triples. class GinFilter @@ -44,7 +43,7 @@ public: explicit GinFilter(const GinFilterParameters & params_); /// Add term (located at 'data' with length 'len') and its row ID to the postings list builder - /// for building inverted index for the given store. + /// for building full-text index for the given store. void add(const char * data, size_t len, UInt32 rowID, GinIndexStorePtr & store) const; /// Accumulate (segmentID, RowIDStart, RowIDEnd) for building skipping index diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 82d2c3fd5f7..604344df177 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -748,15 +748,12 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti IndexDescription index_desc = IndexDescription::getIndexFromAST(index->clone(), properties.columns, getContext()); if (properties.indices.has(index_desc.name)) throw Exception(ErrorCodes::ILLEGAL_INDEX, "Duplicated index name {} is not allowed. Please use different index names.", backQuoteIfNeed(index_desc.name)); + const auto & settings = getContext()->getSettingsRef(); - if (index_desc.type == INVERTED_INDEX_NAME && !settings.allow_experimental_inverted_index) - { - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, - "Experimental Inverted Index feature is not enabled (the setting 'allow_experimental_inverted_index')"); - } + if (index_desc.type == FULL_TEXT_INDEX_NAME && !settings.allow_experimental_inverted_index) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental full-text index feature is not enabled (the setting 'allow_experimental_inverted_index')"); if (index_desc.type == "annoy" && !settings.allow_experimental_annoy_index) throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index is disabled. Turn on allow_experimental_annoy_index"); - if (index_desc.type == "usearch" && !settings.allow_experimental_usearch_index) throw Exception(ErrorCodes::INCORRECT_QUERY, "USearch index is disabled. Turn on allow_experimental_usearch_index"); diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 281fc72dfc4..6f9161c0c0a 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -1100,11 +1100,11 @@ std::optional AlterCommand::tryConvertToMutationCommand(Storage return result; } -bool AlterCommands::hasInvertedIndex(const StorageInMemoryMetadata & metadata) +bool AlterCommands::hasFullTextIndex(const StorageInMemoryMetadata & metadata) { for (const auto & index : metadata.secondary_indices) { - if (index.type == INVERTED_INDEX_NAME) + if (index.type == FULL_TEXT_INDEX_NAME) return true; } return false; diff --git a/src/Storages/AlterCommands.h b/src/Storages/AlterCommands.h index b1b6c8308f9..55cba505584 100644 --- a/src/Storages/AlterCommands.h +++ b/src/Storages/AlterCommands.h @@ -234,8 +234,8 @@ public: /// additional mutation command (MATERIALIZE_TTL) will be returned. MutationCommands getMutationCommands(StorageInMemoryMetadata metadata, bool materialize_ttl, ContextPtr context, bool with_alters=false) const; - /// Check if commands have any inverted index - static bool hasInvertedIndex(const StorageInMemoryMetadata & metadata); + /// Check if commands have any full-text index + static bool hasFullTextIndex(const StorageInMemoryMetadata & metadata); }; } diff --git a/src/Storages/MergeTree/GinIndexStore.cpp b/src/Storages/MergeTree/GinIndexStore.cpp index f05e8288719..9e24d9f018d 100644 --- a/src/Storages/MergeTree/GinIndexStore.cpp +++ b/src/Storages/MergeTree/GinIndexStore.cpp @@ -240,7 +240,7 @@ UInt32 GinIndexStore::getNumOfSegments() readBinary(version, *istr); if (version > static_cast>(CURRENT_GIN_FILE_FORMAT_VERSION)) - throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unsupported inverted index version {}", version); + throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unsupported full-text index version {}", version); readVarUInt(result, *istr); } diff --git a/src/Storages/MergeTree/GinIndexStore.h b/src/Storages/MergeTree/GinIndexStore.h index ad14a142318..94c880211df 100644 --- a/src/Storages/MergeTree/GinIndexStore.h +++ b/src/Storages/MergeTree/GinIndexStore.h @@ -13,8 +13,8 @@ #include #include -/// GinIndexStore manages the generalized inverted index ("gin") for a data part, and it is made up of one or more immutable -/// index segments. +/// GinIndexStore manages the generalized inverted index ("gin") (full-text index )for a data part, and it is made up of one or more +/// immutable index segments. /// /// There are 4 types of index files in a store: /// 1. Segment ID file(.gin_sid): it contains one byte for version followed by the next available segment ID. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 9350b24c96a..af7e12eca06 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3008,9 +3008,9 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context commands.apply(new_metadata, local_context); - if (AlterCommands::hasInvertedIndex(new_metadata) && !settings.allow_experimental_inverted_index) + if (AlterCommands::hasFullTextIndex(new_metadata) && !settings.allow_experimental_inverted_index) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, - "Experimental Inverted Index feature is not enabled (turn on setting 'allow_experimental_inverted_index')"); + "Experimental full-text index feature is not enabled (turn on setting 'allow_experimental_inverted_index')"); for (const auto & disk : getDisks()) if (!disk->supportsHardLinks()) diff --git a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp index 7c9e4a371ab..f33f4293023 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp @@ -61,7 +61,7 @@ void MergeTreeDataPartChecksum::checkEqual(const MergeTreeDataPartChecksum & rhs void MergeTreeDataPartChecksum::checkSize(const IDataPartStorage & storage, const String & name) const { - /// Skip inverted index files, these have a default MergeTreeDataPartChecksum with file_size == 0 + /// Skip full-text index files, these have a default MergeTreeDataPartChecksum with file_size == 0 if (isGinFile(name)) return; @@ -88,7 +88,7 @@ void MergeTreeDataPartChecksums::checkEqual(const MergeTreeDataPartChecksums & r for (const auto & [name, checksum] : files) { - /// Exclude files written by inverted index from check. No correct checksums are available for them currently. + /// Exclude files written by full-text index from check. No correct checksums are available for them currently. if (name.ends_with(".gin_dict") || name.ends_with(".gin_post") || name.ends_with(".gin_seg") || name.ends_with(".gin_sid")) continue; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index 441447dcaba..491d2399b82 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -468,7 +468,7 @@ void MergeTreeDataPartWriterOnDisk::fillSkipIndicesChecksums(MergeTreeData::Data if (!skip_indices_aggregators[i]->empty()) skip_indices_aggregators[i]->getGranuleAndReset()->serializeBinary(stream.compressed_hashing); - /// Register additional files written only by the inverted index. Required because otherwise DROP TABLE complains about unknown + /// Register additional files written only by the full-text index. Required because otherwise DROP TABLE complains about unknown /// files. Note that the provided actual checksums are bogus. The problem is that at this point the file writes happened already and /// we'd need to re-open + hash the files (fixing this is TODO). For now, CHECK TABLE skips these four files. if (typeid_cast(&*skip_indices[i]) != nullptr) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index e11ff87d1b2..55d845dfbb9 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -652,7 +652,7 @@ static NameSet collectFilesToSkip( files_to_skip.insert(index->getFileName() + index->getSerializedFileExtension()); files_to_skip.insert(index->getFileName() + mrk_extension); - // Skip all inverted index files, for they will be rebuilt + // Skip all full-text index files, for they will be rebuilt if (dynamic_cast(index.get())) { auto index_filename = index->getFileName(); @@ -731,7 +731,7 @@ static NameToNameVector collectFilesForRenames( if (command.type == MutationCommand::Type::DROP_INDEX) { static const std::array suffixes = {".idx2", ".idx"}; - static const std::array gin_suffixes = {".gin_dict", ".gin_post", ".gin_seg", ".gin_sid"}; /// .gin_* is inverted index + static const std::array gin_suffixes = {".gin_dict", ".gin_post", ".gin_seg", ".gin_sid"}; /// .gin_* means generalized inverted index (aka. full-text-index) for (const auto & suffix : suffixes) { diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index b4d32e71d0d..5834147d191 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -250,7 +250,7 @@ static IMergeTreeDataPart::Checksums checkDataPart( continue; } - /// Exclude files written by inverted index from check. No correct checksums are available for them currently. + /// Exclude files written by full-text index from check. No correct checksums are available for them currently. if (isGinFile(file_name)) continue; diff --git a/tests/queries/0_stateless/02346_inverted_index_experimental_flag.sql b/tests/queries/0_stateless/02346_inverted_index_experimental_flag.sql index bf89265372e..edc562a5c57 100644 --- a/tests/queries/0_stateless/02346_inverted_index_experimental_flag.sql +++ b/tests/queries/0_stateless/02346_inverted_index_experimental_flag.sql @@ -11,6 +11,6 @@ CREATE TABLE tab ENGINE = MergeTree ORDER BY key; -ALTER TABLE tab ADD INDEX inv_idx(str) TYPE inverted(0); -- { serverError SUPPORT_IS_DISABLED } +ALTER TABLE tab ADD INDEX inv_idx(str) TYPE full_text(0); -- { serverError SUPPORT_IS_DISABLED } DROP TABLE tab; From c348e62a2188119d54c0e0e691745fa608680759 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 6 May 2024 17:50:47 +0000 Subject: [PATCH 0848/1018] Add test for issue #47862 --- tests/queries/0_stateless/03146_bug47862.reference | 1 + tests/queries/0_stateless/03146_bug47862.sql | 12 ++++++++++++ 2 files changed, 13 insertions(+) create mode 100644 tests/queries/0_stateless/03146_bug47862.reference create mode 100644 tests/queries/0_stateless/03146_bug47862.sql diff --git a/tests/queries/0_stateless/03146_bug47862.reference b/tests/queries/0_stateless/03146_bug47862.reference new file mode 100644 index 00000000000..bc6298e80ad --- /dev/null +++ b/tests/queries/0_stateless/03146_bug47862.reference @@ -0,0 +1 @@ +110 diff --git a/tests/queries/0_stateless/03146_bug47862.sql b/tests/queries/0_stateless/03146_bug47862.sql new file mode 100644 index 00000000000..918f2316bea --- /dev/null +++ b/tests/queries/0_stateless/03146_bug47862.sql @@ -0,0 +1,12 @@ +SELECT toInt64(lookup_res) AS cast_res +FROM ( + SELECT + indexOf(field_id, 10) AS val_idx, + ['110'][val_idx] AS lookup_res + FROM ( + SELECT arrayJoin([[10], [15]]) AS field_id + ) + WHERE val_idx != 0 +) +WHERE cast_res > 0 +SETTINGS allow_experimental_analyzer = 1; From 216cd83b8ed6da5506d315b643a23f9e7b56b8bc Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 6 May 2024 17:53:08 +0000 Subject: [PATCH 0849/1018] fix parsing of CREATE INDEX query --- src/Parsers/ParserCreateIndexQuery.cpp | 46 +++++++++++++++---- ...03146_create_index_compatibility.reference | 1 + .../03146_create_index_compatibility.sql | 17 +++++++ 3 files changed, 54 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/03146_create_index_compatibility.reference create mode 100644 tests/queries/0_stateless/03146_create_index_compatibility.sql diff --git a/src/Parsers/ParserCreateIndexQuery.cpp b/src/Parsers/ParserCreateIndexQuery.cpp index fd2bbbab177..2fa34696c58 100644 --- a/src/Parsers/ParserCreateIndexQuery.cpp +++ b/src/Parsers/ParserCreateIndexQuery.cpp @@ -17,9 +17,9 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected { ParserKeyword s_type(Keyword::TYPE); ParserKeyword s_granularity(Keyword::GRANULARITY); - ParserToken open(TokenType::OpeningRoundBracket); - ParserToken close(TokenType::ClosingRoundBracket); - ParserOrderByExpressionList order_list; + ParserToken open_p(TokenType::OpeningRoundBracket); + ParserToken close_p(TokenType::ClosingRoundBracket); + ParserOrderByExpressionList order_list_p; ParserDataType data_type_p; ParserExpression expression_p; @@ -29,17 +29,41 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected ASTPtr type; ASTPtr granularity; - /// Skip name parser for SQL-standard CREATE INDEX - if (expression_p.parse(pos, expr, expected)) + if (open_p.ignore(pos, expected)) { - } - else if (open.ignore(pos, expected)) - { - if (!order_list.parse(pos, expr, expected)) + ASTPtr order_list; + if (!order_list_p.parse(pos, order_list, expected)) return false; - if (!close.ignore(pos, expected)) + if (!close_p.ignore(pos, expected)) return false; + + if (order_list->children.empty()) + return false; + + /// CREATE INDEX with ASC, DESC is implemented only for SQL compatibility. + /// ASC and DESC modifiers are not supported and are ignored further. + if (order_list->children.size() == 1) + { + auto order_by_elem = order_list->children[0]; + expr = order_by_elem->children[0]; + } + else + { + auto tuple_func = makeASTFunction("tuple"); + tuple_func->arguments = std::make_shared(); + + for (const auto & order_by_elem : order_list->children) + { + auto elem_expr = order_by_elem->children[0]; + tuple_func->arguments->children.push_back(std::move(elem_expr)); + } + expr = std::move(tuple_func); + } + } + else if (!expression_p.parse(pos, expr, expected)) + { + return false; } if (s_type.ignore(pos, expected)) @@ -59,7 +83,9 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected index->part_of_create_index_query = true; if (granularity) + { index->granularity = granularity->as().value.safeGet(); + } else { auto index_type = index->getType(); diff --git a/tests/queries/0_stateless/03146_create_index_compatibility.reference b/tests/queries/0_stateless/03146_create_index_compatibility.reference new file mode 100644 index 00000000000..64f1d1382ee --- /dev/null +++ b/tests/queries/0_stateless/03146_create_index_compatibility.reference @@ -0,0 +1 @@ +CREATE TABLE default.t_index_3146\n(\n `a` UInt64,\n `b` UInt64,\n INDEX i1 a TYPE minmax GRANULARITY 1,\n INDEX i2 (a, b) TYPE minmax GRANULARITY 1,\n INDEX i3 (a, b) TYPE minmax GRANULARITY 1,\n INDEX i4 a TYPE minmax GRANULARITY 1\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/03146_create_index_compatibility.sql b/tests/queries/0_stateless/03146_create_index_compatibility.sql new file mode 100644 index 00000000000..ede5bc0567a --- /dev/null +++ b/tests/queries/0_stateless/03146_create_index_compatibility.sql @@ -0,0 +1,17 @@ +DROP TABLE IF EXISTS t_index_3146; + +CREATE TABLE t_index_3146 (a UInt64, b UInt64) ENGINE = MergeTree ORDER BY tuple(); + +SET allow_create_index_without_type = 1; + +CREATE INDEX i1 ON t_index_3146 (a) TYPE minmax; +CREATE INDEX i2 ON t_index_3146 (a, b) TYPE minmax; +CREATE INDEX i3 ON t_index_3146 (a DESC, b ASC) TYPE minmax; +CREATE INDEX i4 ON t_index_3146 a TYPE minmax; +CREATE INDEX i5 ON t_index_3146 (a); -- ignored +CREATE INDEX i6 ON t_index_3146 (a DESC, b ASC); -- ignored +CREATE INDEX i7 ON t_index_3146; -- { clientError SYNTAX_ERROR } +CREATE INDEX i8 ON t_index_3146 a, b TYPE minmax; -- { clientError SYNTAX_ERROR } + +SHOW CREATE TABLE t_index_3146; +DROP TABLE t_index_3146; From ba4f5f9b3fe74d44295633995033c9d484b8a9c6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 6 May 2024 19:46:57 +0200 Subject: [PATCH 0850/1018] Fix backup of parts with projections but without projections in metadata --- src/Storages/MergeTree/MergeTreeData.cpp | 42 +++++++++++++--- ...145_non_loaded_projection_backup.reference | 7 +++ .../03145_non_loaded_projection_backup.sh | 49 +++++++++++++++++++ 3 files changed, 92 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/03145_non_loaded_projection_backup.reference create mode 100755 tests/queries/0_stateless/03145_non_loaded_projection_backup.sh diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 9350b24c96a..fbe79a32b8e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5361,20 +5361,50 @@ MergeTreeData::PartsBackupEntries MergeTreeData::backupParts( &temp_dirs, false, false); - auto projection_parts = part->getProjectionParts(); - for (const auto & [projection_name, projection_part] : projection_parts) + auto backup_projection = [&](IDataPartStorage & storage, IMergeTreeDataPart & projection_part) { - projection_part->getDataPartStorage().backup( - projection_part->checksums, - projection_part->getFileNamesWithoutChecksums(), + storage.backup( + projection_part.checksums, + projection_part.getFileNamesWithoutChecksums(), fs::path{data_path_in_backup} / part->name, backup_settings, read_settings, make_temporary_hard_links, backup_entries_from_part, &temp_dirs, - projection_part->is_broken, + projection_part.is_broken, backup_settings.allow_backup_broken_projections); + }; + + auto projection_parts = part->getProjectionParts(); + std::string proj_suffix = ".proj"; + std::unordered_set defined_projections; + + for (const auto & [projection_name, projection_part] : projection_parts) + { + defined_projections.emplace(projection_name); + backup_projection(projection_part->getDataPartStorage(), *projection_part); + } + + /// It is possible that the part has a written but not loaded projection, + /// e.g. it is written to parent part's checksums.txt and exists on disk, + /// but does not exist in table's projections definition. + /// Such a part can appear server was restarted after DROP PROJECTION but before old part was removed. + /// In this case, the old part will load only projections from metadata. + /// See 031145_non_loaded_projection_backup.sh. + for (const auto & [name, _] : part->checksums.files) + { + auto projection_name = fs::path(name).stem().string(); + if (endsWith(name, proj_suffix) && !defined_projections.contains(projection_name)) + { + auto projection_storage = part->getDataPartStorage().getProjection(projection_name + proj_suffix); + if (projection_storage->exists("checksums.txt")) + { + auto projection_part = const_cast(*part).getProjectionPartBuilder( + projection_name, /* is_temp_projection */false).withPartFormatFromDisk().build(); + backup_projection(projection_part->getDataPartStorage(), *projection_part); + } + } } if (hold_storage_and_part_ptrs) diff --git a/tests/queries/0_stateless/03145_non_loaded_projection_backup.reference b/tests/queries/0_stateless/03145_non_loaded_projection_backup.reference new file mode 100644 index 00000000000..a11ee210e62 --- /dev/null +++ b/tests/queries/0_stateless/03145_non_loaded_projection_backup.reference @@ -0,0 +1,7 @@ +7 +Found unexpected projection directories: pp.proj +BACKUP_CREATED +RESTORED +7 +Found unexpected projection directories: pp.proj +0 diff --git a/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh b/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh new file mode 100755 index 00000000000..721ed784fc2 --- /dev/null +++ b/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh @@ -0,0 +1,49 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -nm -q " +drop table if exists tp_1; +create table tp_1 (x Int32, y Int32, projection p (select x, y order by x)) engine = MergeTree order by y partition by intDiv(y, 100); +insert into tp_1 select number, number from numbers(3); + +set mutations_sync = 2; + +alter table tp_1 add projection pp (select x, count() group by x); +insert into tp_1 select number, number from numbers(4); +select count() from tp_1; + +-- Here we have a part with written projection pp +alter table tp_1 detach partition '0'; +-- Move part to detached +alter table tp_1 clear projection pp; +-- Remove projection from table metadata +alter table tp_1 drop projection pp; +-- Now, we don't load projection pp for attached part, but it is written on disk +alter table tp_1 attach partition '0'; +" + +$CLICKHOUSE_CLIENT -nm -q " +set send_logs_level='fatal'; +check table tp_1 settings check_query_single_value_result = 0;" | grep -o "Found unexpected projection directories: pp.proj" + +backup_id="$CLICKHOUSE_TEST_UNIQUE_NAME" +$CLICKHOUSE_CLIENT -q " +backup table tp_1 to Disk('backups', '$backup_id'); +" | grep -o "BACKUP_CREATED" + +$CLICKHOUSE_CLIENT -nm -q " +drop table tp_1; +restore table tp_1 from Disk('backups', '$backup_id'); +" | grep -o "RESTORED" + +$CLICKHOUSE_CLIENT -q "select count() from tp_1;" +$CLICKHOUSE_CLIENT -nm -q " +set send_logs_level='fatal'; +check table tp_1 settings check_query_single_value_result = 0;" | grep -o "Found unexpected projection directories: pp.proj" +$CLICKHOUSE_CLIENT -nm -q " +set send_logs_level='fatal'; +check table tp_1" +$CLICKHOUSE_CLIENT -q "drop table tp_1 sync" From 0b0e97917e3e4ab27a17cbf14d9e73163a20adbe Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 6 May 2024 21:03:04 +0200 Subject: [PATCH 0851/1018] Do in reverse order --- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index eb757e1d8c7..492e4065502 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -1007,6 +1007,10 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar filterColumns(columns, read_result.final_filter); } + /// If columns not empty, then apply on-fly alter conversions if any required + if (!prewhere_info || prewhere_info->perform_alter_conversions) + merge_tree_reader->performRequiredConversions(columns); + /// If some columns absent in part, then evaluate default values if (should_evaluate_missing_defaults) { @@ -1018,9 +1022,6 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar merge_tree_reader->evaluateMissingDefaults(additional_columns, columns); } - /// If columns not empty, then apply on-fly alter conversions if any required - if (!prewhere_info || prewhere_info->perform_alter_conversions) - merge_tree_reader->performRequiredConversions(columns); } read_result.columns.reserve(read_result.columns.size() + columns.size()); @@ -1046,14 +1047,14 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar bool should_evaluate_missing_defaults; merge_tree_reader->fillMissingColumns(columns, should_evaluate_missing_defaults, read_result.num_rows); - /// If some columns absent in part, then evaluate default values - if (should_evaluate_missing_defaults) - merge_tree_reader->evaluateMissingDefaults({}, columns); - /// If result not empty, then apply on-fly alter conversions if any required if (!prewhere_info || prewhere_info->perform_alter_conversions) merge_tree_reader->performRequiredConversions(columns); + /// If some columns absent in part, then evaluate default values + if (should_evaluate_missing_defaults) + merge_tree_reader->evaluateMissingDefaults({}, columns); + for (size_t i = 0; i < columns.size(); ++i) read_result.columns[i] = std::move(columns[i]); } From 670c8c11886ee5763639bccb28cf45f74f683c05 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 6 May 2024 21:42:25 +0200 Subject: [PATCH 0852/1018] fix exception --- src/AggregateFunctions/Combinators/AggregateFunctionArray.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/Combinators/AggregateFunctionArray.cpp b/src/AggregateFunctions/Combinators/AggregateFunctionArray.cpp index 6b60faadf07..16646c702bf 100644 --- a/src/AggregateFunctions/Combinators/AggregateFunctionArray.cpp +++ b/src/AggregateFunctions/Combinators/AggregateFunctionArray.cpp @@ -26,7 +26,7 @@ public: DataTypes transformArguments(const DataTypes & arguments) const override { if (arguments.empty()) - throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "-Array aggregate functions require at least one argument"); + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Array aggregate functions require at least one argument"); DataTypes nested_arguments; for (const auto & type : arguments) From ac7e6fc69136153c8c99e1d7f76c884fb500815a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 6 May 2024 23:00:05 +0200 Subject: [PATCH 0853/1018] Properly send client info in Suggest --- src/Client/ClientBase.cpp | 2 +- src/Client/Suggest.cpp | 14 +++++++++----- src/Client/Suggest.h | 5 +++-- 3 files changed, 13 insertions(+), 8 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 46887d6cefe..402195ca944 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2616,7 +2616,7 @@ void ClientBase::runInteractive() { // If a separate connection loading suggestions failed to open a new session, // use the main session to receive them. - suggest->load(*connection, connection_parameters.timeouts, config().getInt("suggestion_limit")); + suggest->load(*connection, connection_parameters.timeouts, config().getInt("suggestion_limit"), global_context->getClientInfo()); } try diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index 736967f4a68..487271a2cf9 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -99,7 +99,10 @@ void Suggest::load(ContextPtr context, const ConnectionParameters & connection_p try { auto connection = ConnectionType::createConnection(connection_parameters, my_context); - fetch(*connection, connection_parameters.timeouts, getLoadSuggestionQuery(suggestion_limit, std::is_same_v)); + fetch(*connection, + connection_parameters.timeouts, + getLoadSuggestionQuery(suggestion_limit, std::is_same_v), + my_context->getClientInfo()); } catch (const Exception & e) { @@ -138,11 +141,12 @@ void Suggest::load(ContextPtr context, const ConnectionParameters & connection_p void Suggest::load(IServerConnection & connection, const ConnectionTimeouts & timeouts, - Int32 suggestion_limit) + Int32 suggestion_limit, + const ClientInfo & client_info) { try { - fetch(connection, timeouts, getLoadSuggestionQuery(suggestion_limit, true)); + fetch(connection, timeouts, getLoadSuggestionQuery(suggestion_limit, true), client_info); } catch (...) { @@ -151,10 +155,10 @@ void Suggest::load(IServerConnection & connection, } } -void Suggest::fetch(IServerConnection & connection, const ConnectionTimeouts & timeouts, const std::string & query) +void Suggest::fetch(IServerConnection & connection, const ConnectionTimeouts & timeouts, const std::string & query, const ClientInfo & client_info) { connection.sendQuery( - timeouts, query, {} /* query_parameters */, "" /* query_id */, QueryProcessingStage::Complete, nullptr, nullptr, false, {}); + timeouts, query, {} /* query_parameters */, "" /* query_id */, QueryProcessingStage::Complete, nullptr, &client_info, false, {}); while (true) { diff --git a/src/Client/Suggest.h b/src/Client/Suggest.h index d04910f4d00..f1ba36c7f1c 100644 --- a/src/Client/Suggest.h +++ b/src/Client/Suggest.h @@ -31,7 +31,8 @@ public: void load(IServerConnection & connection, const ConnectionTimeouts & timeouts, - Int32 suggestion_limit); + Int32 suggestion_limit, + const ClientInfo & client_info); /// Older server versions cannot execute the query loading suggestions. static constexpr int MIN_SERVER_REVISION = DBMS_MIN_PROTOCOL_VERSION_WITH_VIEW_IF_PERMITTED; @@ -39,7 +40,7 @@ public: int getLastError() const { return last_error.load(); } private: - void fetch(IServerConnection & connection, const ConnectionTimeouts & timeouts, const std::string & query); + void fetch(IServerConnection & connection, const ConnectionTimeouts & timeouts, const std::string & query, const ClientInfo & client_info); void fillWordsFromBlock(const Block & block); From 9fcb5d2ebfb3e2105b5d6171a1a53ff75c0023f4 Mon Sep 17 00:00:00 2001 From: Yohann Jardin Date: Mon, 6 May 2024 23:44:27 +0200 Subject: [PATCH 0854/1018] Move ErrorCodes inside DB namespace --- .../AggregateFunctionMannWhitney.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionMannWhitney.cpp b/src/AggregateFunctions/AggregateFunctionMannWhitney.cpp index d185058dbd0..8434f11eae6 100644 --- a/src/AggregateFunctions/AggregateFunctionMannWhitney.cpp +++ b/src/AggregateFunctions/AggregateFunctionMannWhitney.cpp @@ -17,6 +17,11 @@ #include +namespace DB +{ + +struct Settings; + namespace ErrorCodes { extern const int NOT_IMPLEMENTED; @@ -25,11 +30,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -namespace DB -{ - -struct Settings; - namespace { From 313843331b588b052372770505bd07805b16c376 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 3 May 2024 19:59:03 +0000 Subject: [PATCH 0855/1018] Fix AST parsing of invalid type names --- src/Interpreters/executeQuery.cpp | 1 + src/Parsers/ParserDataType.cpp | 8 ++++++++ .../0_stateless/03144_fuzz_quoted_type_name.reference | 0 tests/queries/0_stateless/03144_fuzz_quoted_type_name.sql | 7 +++++++ 4 files changed, 16 insertions(+) create mode 100644 tests/queries/0_stateless/03144_fuzz_quoted_type_name.reference create mode 100644 tests/queries/0_stateless/03144_fuzz_quoted_type_name.sql diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index c785de61a18..365dadf2f5d 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -783,6 +783,7 @@ static std::tuple executeQueryImpl( catch (const Exception & e) { if (e.code() == ErrorCodes::SYNTAX_ERROR) + /// Don't print the original query text because it may contain sensitive data. throw Exception(ErrorCodes::LOGICAL_ERROR, "Inconsistent AST formatting: the query:\n{}\ncannot parse.", formatted1); diff --git a/src/Parsers/ParserDataType.cpp b/src/Parsers/ParserDataType.cpp index 58eb00519d8..fdd712f2e68 100644 --- a/src/Parsers/ParserDataType.cpp +++ b/src/Parsers/ParserDataType.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -57,6 +58,13 @@ bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; tryGetIdentifierNameInto(identifier, type_name); + /// Don't accept things like Array(`x.y`). + if (!std::all_of(type_name.begin(), type_name.end(), [](char c) { return isWordCharASCII(c) || c == '$'; })) + { + expected.add(pos, "type name"); + return false; + } + String type_name_upper = Poco::toUpper(type_name); String type_name_suffix; diff --git a/tests/queries/0_stateless/03144_fuzz_quoted_type_name.reference b/tests/queries/0_stateless/03144_fuzz_quoted_type_name.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03144_fuzz_quoted_type_name.sql b/tests/queries/0_stateless/03144_fuzz_quoted_type_name.sql new file mode 100644 index 00000000000..746d15285a2 --- /dev/null +++ b/tests/queries/0_stateless/03144_fuzz_quoted_type_name.sql @@ -0,0 +1,7 @@ +create table t (x 123) engine Memory; -- { clientError 62 } +create table t (x `a.b`) engine Memory; -- { clientError 62 } +create table t (x Array(`a.b`)) engine Memory; -- { clientError 62 } + +create table t (x Array(`ab`)) engine Memory; -- { serverError 50 } +create table t (x `ab`) engine Memory; -- { serverError 50 } +create table t (x `Int64`) engine Memory; \ No newline at end of file From 01046cba3139cee8b0a14b63610698c8b9046649 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 6 May 2024 21:55:54 +0000 Subject: [PATCH 0856/1018] Error names --- .../0_stateless/03144_fuzz_quoted_type_name.sql | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/03144_fuzz_quoted_type_name.sql b/tests/queries/0_stateless/03144_fuzz_quoted_type_name.sql index 746d15285a2..4f6cc6c86ba 100644 --- a/tests/queries/0_stateless/03144_fuzz_quoted_type_name.sql +++ b/tests/queries/0_stateless/03144_fuzz_quoted_type_name.sql @@ -1,7 +1,7 @@ -create table t (x 123) engine Memory; -- { clientError 62 } -create table t (x `a.b`) engine Memory; -- { clientError 62 } -create table t (x Array(`a.b`)) engine Memory; -- { clientError 62 } +create table t (x 123) engine Memory; -- { clientError SYNTAX_ERROR } +create table t (x `a.b`) engine Memory; -- { clientError SYNTAX_ERROR } +create table t (x Array(`a.b`)) engine Memory; -- { clientError SYNTAX_ERROR } -create table t (x Array(`ab`)) engine Memory; -- { serverError 50 } -create table t (x `ab`) engine Memory; -- { serverError 50 } +create table t (x Array(`ab`)) engine Memory; -- { serverError UNKNOWN_TYPE } +create table t (x `ab`) engine Memory; -- { serverError UNKNOWN_TYPE } create table t (x `Int64`) engine Memory; \ No newline at end of file From 4703eb943b9586be7cab2429a170d9cf9131558f Mon Sep 17 00:00:00 2001 From: Yohann Jardin Date: Mon, 6 May 2024 23:59:59 +0200 Subject: [PATCH 0857/1018] Move ErrorCodes inside DB namespace --- src/AggregateFunctions/AggregateFunctionWelchTTest.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionWelchTTest.cpp b/src/AggregateFunctions/AggregateFunctionWelchTTest.cpp index 7f1c4861fbc..7efd6154227 100644 --- a/src/AggregateFunctions/AggregateFunctionWelchTTest.cpp +++ b/src/AggregateFunctions/AggregateFunctionWelchTTest.cpp @@ -4,16 +4,16 @@ #include +namespace DB +{ +struct Settings; + namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; } -namespace DB -{ -struct Settings; - namespace { From d14fc62d4d4b597435e482f60a2f7c83412e53cc Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 29 Mar 2024 08:59:07 +0000 Subject: [PATCH 0858/1018] Avoid crashing on column type mismatch in a few dozen places --- src/Columns/ColumnArray.cpp | 2 +- src/Columns/ColumnLowCardinality.cpp | 2 +- src/Columns/ColumnUnique.h | 4 +- src/Columns/FilterDescription.cpp | 4 +- src/Columns/IColumn.h | 8 +++- src/Columns/MaskOperations.cpp | 4 +- src/Common/ColumnsHashing.h | 12 +++--- src/Common/ColumnsHashingImpl.h | 2 +- src/Core/DecimalComparison.h | 16 ++++---- src/DataTypes/ObjectUtils.cpp | 2 +- .../Serializations/SerializationBool.cpp | 4 +- .../Serializations/SerializationInterval.cpp | 2 +- .../SerializationLowCardinality.cpp | 2 +- .../HierarchyDictionariesUtils.cpp | 8 ++-- src/Formats/insertNullAsDefaultIfNeeded.cpp | 40 +++++++++---------- src/Functions/FunctionBase64Conversion.h | 4 +- src/Functions/FunctionBinaryArithmetic.h | 20 +++++----- src/Functions/FunctionHelpers.cpp | 10 ++--- src/Functions/FunctionHelpers.h | 25 +++++++++++- src/Functions/FunctionUnixTimestamp64.h | 2 +- src/Functions/FunctionsBitmap.h | 4 +- src/Functions/FunctionsCodingIP.cpp | 4 +- src/Functions/FunctionsConversion.cpp | 2 +- src/Functions/FunctionsRound.h | 18 ++++----- src/Functions/FunctionsStringHash.h | 12 +++--- src/Functions/Kusto/KqlArraySort.cpp | 4 +- src/Functions/MultiMatchAllIndicesImpl.h | 4 +- src/Functions/MultiMatchAnyImpl.h | 4 +- src/Functions/MultiSearchAllPositionsImpl.h | 4 +- src/Functions/MultiSearchFirstIndexImpl.h | 4 +- src/Functions/MultiSearchFirstPositionImpl.h | 4 +- src/Functions/MultiSearchImpl.h | 4 +- .../URL/FirstSignificantSubdomainCustomImpl.h | 2 +- src/Functions/array/FunctionArrayMapped.h | 2 +- src/Functions/array/arrayCompact.cpp | 2 +- src/Functions/array/arrayDistinct.cpp | 8 ++-- src/Functions/array/arrayElement.cpp | 6 +-- src/Functions/array/arrayEnumerateExtended.h | 2 +- src/Functions/array/arrayIndex.h | 10 ++--- src/Functions/array/arrayJaccardIndex.cpp | 8 ++-- src/Functions/array/arrayUniq.cpp | 2 +- src/Functions/array/emptyArrayToSingle.cpp | 2 +- src/Functions/array/range.cpp | 6 +-- src/Functions/arrayStringConcat.cpp | 2 +- src/Functions/assumeNotNull.cpp | 2 +- src/Functions/coalesce.cpp | 10 ++--- src/Functions/fromModifiedJulianDay.cpp | 4 +- src/Functions/grouping.h | 4 +- src/Functions/hasColumnInTable.cpp | 4 +- src/Functions/if.cpp | 18 ++++----- src/Functions/isNotNull.cpp | 14 +++---- src/Functions/isNull.cpp | 14 +++---- src/Functions/minSampleSize.cpp | 8 ++-- src/Functions/multiIf.cpp | 2 +- src/Functions/readWkt.cpp | 4 +- src/Functions/repeat.cpp | 8 ++-- src/Functions/seriesOutliersDetectTukey.cpp | 6 +-- src/Functions/seriesPeriodDetectFFT.cpp | 6 +-- src/Functions/space.cpp | 6 +-- src/Functions/toStartOfInterval.cpp | 6 +-- src/Functions/ztest.cpp | 12 +++--- src/Interpreters/BloomFilterHash.h | 2 +- src/Interpreters/HashJoin.cpp | 2 +- src/Interpreters/InterpreterCheckQuery.cpp | 6 +-- src/Interpreters/JoinUtils.cpp | 20 +++++----- src/Interpreters/MergeJoin.cpp | 8 ++-- src/Interpreters/NullableUtils.cpp | 4 +- src/Interpreters/SetVariants.cpp | 2 +- src/Interpreters/SetVariants.h | 2 +- .../Transforms/CheckConstraintsTransform.cpp | 2 +- .../Transforms/MergeJoinTransform.cpp | 8 ++-- src/Processors/Transforms/WindowTransform.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 2 +- 73 files changed, 253 insertions(+), 228 deletions(-) diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 7b268b80116..8d5c246c48c 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -1283,7 +1283,7 @@ ColumnPtr ColumnArray::replicateTuple(const Offsets & replicate_offsets) const size_t ColumnArray::getNumberOfDimensions() const { - const auto * nested_array = checkAndGetColumn(*data); + const auto * nested_array = checkAndGetColumn(&*data); if (!nested_array) return 1; return 1 + nested_array->getNumberOfDimensions(); /// Every modern C++ compiler optimizes tail recursion. diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index a3696c78669..a032c2b25b7 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -903,7 +903,7 @@ ColumnPtr ColumnLowCardinality::cloneWithDefaultOnNull() const bool isColumnLowCardinalityNullable(const IColumn & column) { - if (const auto * lc_column = checkAndGetColumn(column)) + if (const auto * lc_column = checkAndGetColumn(&column)) return lc_column->nestedIsNullable(); return false; } diff --git a/src/Columns/ColumnUnique.h b/src/Columns/ColumnUnique.h index a8873140817..0311efd4c83 100644 --- a/src/Columns/ColumnUnique.h +++ b/src/Columns/ColumnUnique.h @@ -376,7 +376,7 @@ size_t ColumnUnique::uniqueInsertFrom(const IColumn & src, size_t n) if (is_nullable && src.isNullAt(n)) return getNullValueIndex(); - if (const auto * nullable = checkAndGetColumn(src)) + if (const auto * nullable = checkAndGetColumn(&src)) return uniqueInsertFrom(nullable->getNestedColumn(), n); auto ref = src.getDataAt(n); @@ -569,7 +569,7 @@ MutableColumnPtr ColumnUnique::uniqueInsertRangeImpl( return nullptr; }; - if (const auto * nullable_column = checkAndGetColumn(src)) + if (const auto * nullable_column = checkAndGetColumn(&src)) { src_column = typeid_cast(&nullable_column->getNestedColumn()); null_map = &nullable_column->getNullMapData(); diff --git a/src/Columns/FilterDescription.cpp b/src/Columns/FilterDescription.cpp index 62d2babe17a..56c99a5b49c 100644 --- a/src/Columns/FilterDescription.cpp +++ b/src/Columns/FilterDescription.cpp @@ -32,7 +32,7 @@ ConstantFilterDescription::ConstantFilterDescription(const IColumn & column) if (!typeid_cast(column_nested.get())) { - const ColumnNullable * column_nested_nullable = checkAndGetColumn(*column_nested); + const ColumnNullable * column_nested_nullable = checkAndGetColumn(&*column_nested); if (!column_nested_nullable || !typeid_cast(&column_nested_nullable->getNestedColumn())) { throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, @@ -66,7 +66,7 @@ FilterDescription::FilterDescription(const IColumn & column_) return; } - if (const auto * nullable_column = checkAndGetColumn(column)) + if (const auto * nullable_column = checkAndGetColumn(&column)) { ColumnPtr nested_column = nullable_column->getNestedColumnPtr(); MutableColumnPtr mutable_holder = IColumn::mutate(std::move(nested_column)); diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index cea8d7c9f55..cf2693e008c 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -640,12 +640,16 @@ template <> struct IsMutableColumns<> { static const bool value = true; }; +/// Throws LOGICAL_ERROR if the type doesn't match. template -const Type * checkAndGetColumn(const IColumn & column) +const Type & checkAndGetColumn(const IColumn & column) { - return typeid_cast(&column); + return typeid_cast(column); } +/// Returns nullptr if the type doesn't match. +/// If you're going to dereference the returned pointer without checking for null, use the +/// `const IColumn &` overload above instead. template const Type * checkAndGetColumn(const IColumn * column) { diff --git a/src/Columns/MaskOperations.cpp b/src/Columns/MaskOperations.cpp index 8f5299cf6e8..2c54a416850 100644 --- a/src/Columns/MaskOperations.cpp +++ b/src/Columns/MaskOperations.cpp @@ -205,10 +205,10 @@ static MaskInfo extractMaskImpl( auto column = col->convertToFullColumnIfLowCardinality(); /// Special implementation for Null and Const columns. - if (column->onlyNull() || checkAndGetColumn(*column)) + if (column->onlyNull() || checkAndGetColumn(&*column)) return extractMaskFromConstOrNull(mask, column, null_value, nulls); - if (const auto * nullable_column = checkAndGetColumn(*column)) + if (const auto * nullable_column = checkAndGetColumn(&*column)) { const PaddedPODArray & null_map = nullable_column->getNullMapData(); return extractMaskImpl(mask, nullable_column->getNestedColumnPtr(), null_value, &null_map, nulls); diff --git a/src/Common/ColumnsHashing.h b/src/Common/ColumnsHashing.h index 25d2ddc684d..bd3880eb83b 100644 --- a/src/Common/ColumnsHashing.h +++ b/src/Common/ColumnsHashing.h @@ -44,8 +44,8 @@ struct HashMethodOneNumber { if constexpr (nullable) { - const auto * null_column = checkAndGetColumn(key_columns[0]); - vec = null_column->getNestedColumnPtr()->getRawData().data(); + const auto & null_column = checkAndGetColumn(*key_columns[0]); + vec = null_column.getNestedColumnPtr()->getRawData().data(); } else { @@ -57,8 +57,8 @@ struct HashMethodOneNumber { if constexpr (nullable) { - const auto * null_column = checkAndGetColumn(column); - vec = null_column->getNestedColumnPtr()->getRawData().data(); + const auto & null_column = checkAndGetColumn(*column); + vec = null_column.getNestedColumnPtr()->getRawData().data(); } else { @@ -105,7 +105,7 @@ struct HashMethodString const IColumn * column; if constexpr (nullable) { - column = checkAndGetColumn(key_columns[0])->getNestedColumnPtr().get(); + column = checkAndGetColumn(*key_columns[0]).getNestedColumnPtr().get(); } else { @@ -153,7 +153,7 @@ struct HashMethodFixedString const IColumn * column; if constexpr (nullable) { - column = checkAndGetColumn(key_columns[0])->getNestedColumnPtr().get(); + column = checkAndGetColumn(*key_columns[0]).getNestedColumnPtr().get(); } else { diff --git a/src/Common/ColumnsHashingImpl.h b/src/Common/ColumnsHashingImpl.h index 6b3cc24d870..f74a56292ae 100644 --- a/src/Common/ColumnsHashingImpl.h +++ b/src/Common/ColumnsHashingImpl.h @@ -305,7 +305,7 @@ protected: } if constexpr (nullable) - null_map = &checkAndGetColumn(column)->getNullMapColumn(); + null_map = &checkAndGetColumn(*column).getNullMapColumn(); } template diff --git a/src/Core/DecimalComparison.h b/src/Core/DecimalComparison.h index 08569aa534c..63817e77805 100644 --- a/src/Core/DecimalComparison.h +++ b/src/Core/DecimalComparison.h @@ -170,11 +170,11 @@ private: if (c0_is_const && c1_is_const) { - const ColumnConst * c0_const = checkAndGetColumnConst(c0.get()); - const ColumnConst * c1_const = checkAndGetColumnConst(c1.get()); + const ColumnConst & c0_const = checkAndGetColumnConst(*c0); + const ColumnConst & c1_const = checkAndGetColumnConst(*c1); - A a = c0_const->template getValue(); - B b = c1_const->template getValue(); + A a = c0_const.template getValue(); + B b = c1_const.template getValue(); UInt8 res = apply(a, b, scale); return DataTypeUInt8().createColumnConst(c0->size(), toField(res)); } @@ -184,8 +184,8 @@ private: if (c0_is_const) { - const ColumnConst * c0_const = checkAndGetColumnConst(c0.get()); - A a = c0_const->template getValue(); + const ColumnConst & c0_const = checkAndGetColumnConst(*c0); + A a = c0_const.template getValue(); if (const ColVecB * c1_vec = checkAndGetColumn(c1.get())) constantVector(a, c1_vec->getData(), vec_res, scale); else @@ -193,8 +193,8 @@ private: } else if (c1_is_const) { - const ColumnConst * c1_const = checkAndGetColumnConst(c1.get()); - B b = c1_const->template getValue(); + const ColumnConst & c1_const = checkAndGetColumnConst(*c1); + B b = c1_const.template getValue(); if (const ColVecA * c0_vec = checkAndGetColumn(c0.get())) vectorConstant(c0_vec->getData(), b, vec_res, scale); else diff --git a/src/DataTypes/ObjectUtils.cpp b/src/DataTypes/ObjectUtils.cpp index 99cf092e6cd..7c671fcf44f 100644 --- a/src/DataTypes/ObjectUtils.cpp +++ b/src/DataTypes/ObjectUtils.cpp @@ -47,7 +47,7 @@ size_t getNumberOfDimensions(const IDataType & type) size_t getNumberOfDimensions(const IColumn & column) { - if (const auto * column_array = checkAndGetColumn(column)) + if (const auto * column_array = checkAndGetColumn(&column)) return column_array->getNumberOfDimensions(); return 0; } diff --git a/src/DataTypes/Serializations/SerializationBool.cpp b/src/DataTypes/Serializations/SerializationBool.cpp index d6a74e5cb8f..b63f25ddc35 100644 --- a/src/DataTypes/Serializations/SerializationBool.cpp +++ b/src/DataTypes/Serializations/SerializationBool.cpp @@ -28,7 +28,7 @@ constexpr char str_false[6] = "false"; const ColumnUInt8 * checkAndGetSerializeColumnType(const IColumn & column) { const auto * col = checkAndGetColumn(&column); - if (!checkAndGetColumn(&column)) + if (!col) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Bool type can only serialize columns of type UInt8.{}", column.getName()); return col; } @@ -36,7 +36,7 @@ const ColumnUInt8 * checkAndGetSerializeColumnType(const IColumn & column) ColumnUInt8 * checkAndGetDeserializeColumnType(IColumn & column) { auto * col = typeid_cast(&column); - if (!checkAndGetColumn(&column)) + if (!col) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Bool type can only deserialize columns of type UInt8.{}", column.getName()); return col; diff --git a/src/DataTypes/Serializations/SerializationInterval.cpp b/src/DataTypes/Serializations/SerializationInterval.cpp index 59086d8aef3..c4ef34b4325 100644 --- a/src/DataTypes/Serializations/SerializationInterval.cpp +++ b/src/DataTypes/Serializations/SerializationInterval.cpp @@ -17,7 +17,7 @@ namespace ErrorCodes void SerializationKustoInterval::serializeText( const IColumn & column, const size_t row, WriteBuffer & ostr, const FormatSettings &) const { - const auto * interval_column = checkAndGetColumn(column); + const auto * interval_column = checkAndGetColumn(&column); if (!interval_column) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Expected column of underlying type of Interval"); diff --git a/src/DataTypes/Serializations/SerializationLowCardinality.cpp b/src/DataTypes/Serializations/SerializationLowCardinality.cpp index 9efe05042ed..2d2be195098 100644 --- a/src/DataTypes/Serializations/SerializationLowCardinality.cpp +++ b/src/DataTypes/Serializations/SerializationLowCardinality.cpp @@ -477,7 +477,7 @@ void SerializationLowCardinality::serializeBinaryBulkWithMultipleStreams( settings.low_cardinality_max_dictionary_size); } - if (const auto * nullable_keys = checkAndGetColumn(*keys)) + if (const auto * nullable_keys = checkAndGetColumn(&*keys)) keys = nullable_keys->getNestedColumnPtr(); bool need_additional_keys = !keys->empty(); diff --git a/src/Dictionaries/HierarchyDictionariesUtils.cpp b/src/Dictionaries/HierarchyDictionariesUtils.cpp index fd59a0c37db..e1119982a34 100644 --- a/src/Dictionaries/HierarchyDictionariesUtils.cpp +++ b/src/Dictionaries/HierarchyDictionariesUtils.cpp @@ -95,7 +95,7 @@ namespace parent_key_column_non_null = parent_key_column_typed->getNestedColumnPtr(); } - const auto * parent_key_column_typed = checkAndGetColumn>(*parent_key_column_non_null); + const auto * parent_key_column_typed = checkAndGetColumn>(&*parent_key_column_non_null); if (!parent_key_column_typed) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Parent key column should be UInt64. Actual {}", @@ -166,7 +166,7 @@ ColumnPtr getKeysHierarchyDefaultImplementation( valid_keys = 0; key_column = key_column->convertToFullColumnIfConst(); - const auto * key_column_typed = checkAndGetColumn>(*key_column); + const auto * key_column_typed = checkAndGetColumn>(&*key_column); if (!key_column_typed) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Key column should be UInt64"); @@ -224,11 +224,11 @@ ColumnUInt8::Ptr getKeysIsInHierarchyDefaultImplementation( key_column = key_column->convertToFullColumnIfConst(); in_key_column = in_key_column->convertToFullColumnIfConst(); - const auto * key_column_typed = checkAndGetColumn>(*key_column); + const auto * key_column_typed = checkAndGetColumn>(&*key_column); if (!key_column_typed) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Key column should be UInt64"); - const auto * in_key_column_typed = checkAndGetColumn>(*in_key_column); + const auto * in_key_column_typed = checkAndGetColumn>(&*in_key_column); if (!in_key_column_typed) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Key column should be UInt64"); diff --git a/src/Formats/insertNullAsDefaultIfNeeded.cpp b/src/Formats/insertNullAsDefaultIfNeeded.cpp index c42b8c54d73..ff40d036fb5 100644 --- a/src/Formats/insertNullAsDefaultIfNeeded.cpp +++ b/src/Formats/insertNullAsDefaultIfNeeded.cpp @@ -19,45 +19,45 @@ bool insertNullAsDefaultIfNeeded(ColumnWithTypeAndName & input_column, const Col if (isArray(input_column.type) && isArray(header_column.type)) { ColumnWithTypeAndName nested_input_column; - const auto * array_input_column = checkAndGetColumn(input_column.column.get()); - nested_input_column.column = array_input_column->getDataPtr(); + const auto & array_input_column = checkAndGetColumn(*input_column.column); + nested_input_column.column = array_input_column.getDataPtr(); nested_input_column.type = checkAndGetDataType(input_column.type.get())->getNestedType(); ColumnWithTypeAndName nested_header_column; - nested_header_column.column = checkAndGetColumn(header_column.column.get())->getDataPtr(); + nested_header_column.column = checkAndGetColumn(*header_column.column).getDataPtr(); nested_header_column.type = checkAndGetDataType(header_column.type.get())->getNestedType(); if (!insertNullAsDefaultIfNeeded(nested_input_column, nested_header_column, 0, nullptr)) return false; - input_column.column = ColumnArray::create(nested_input_column.column, array_input_column->getOffsetsPtr()); + input_column.column = ColumnArray::create(nested_input_column.column, array_input_column.getOffsetsPtr()); input_column.type = std::make_shared(std::move(nested_input_column.type)); return true; } if (isTuple(input_column.type) && isTuple(header_column.type)) { - const auto * tuple_input_column = checkAndGetColumn(input_column.column.get()); - const auto * tuple_input_type = checkAndGetDataType(input_column.type.get()); - const auto * tuple_header_column = checkAndGetColumn(header_column.column.get()); - const auto * tuple_header_type = checkAndGetDataType(header_column.type.get()); + const auto & tuple_input_column = checkAndGetColumn(*input_column.column); + const auto & tuple_input_type = checkAndGetDataType(*input_column.type); + const auto & tuple_header_column = checkAndGetColumn(*header_column.column); + const auto & tuple_header_type = checkAndGetDataType(*header_column.type); - if (tuple_input_type->getElements().size() != tuple_header_type->getElements().size()) + if (tuple_input_type.getElements().size() != tuple_header_type.getElements().size()) return false; Columns nested_input_columns; - nested_input_columns.reserve(tuple_input_type->getElements().size()); + nested_input_columns.reserve(tuple_input_type.getElements().size()); DataTypes nested_input_types; - nested_input_types.reserve(tuple_input_type->getElements().size()); + nested_input_types.reserve(tuple_input_type.getElements().size()); bool changed = false; - for (size_t i = 0; i != tuple_input_type->getElements().size(); ++i) + for (size_t i = 0; i != tuple_input_type.getElements().size(); ++i) { ColumnWithTypeAndName nested_input_column; - nested_input_column.column = tuple_input_column->getColumnPtr(i); - nested_input_column.type = tuple_input_type->getElement(i); + nested_input_column.column = tuple_input_column.getColumnPtr(i); + nested_input_column.type = tuple_input_type.getElement(i); ColumnWithTypeAndName nested_header_column; - nested_header_column.column = tuple_header_column->getColumnPtr(i); - nested_header_column.type = tuple_header_type->getElement(i); + nested_header_column.column = tuple_header_column.getColumnPtr(i); + nested_header_column.type = tuple_header_type.getElement(i); changed |= insertNullAsDefaultIfNeeded(nested_input_column, nested_header_column, 0, nullptr); nested_input_columns.push_back(std::move(nested_input_column.column)); nested_input_types.push_back(std::move(nested_input_column.type)); @@ -74,12 +74,12 @@ bool insertNullAsDefaultIfNeeded(ColumnWithTypeAndName & input_column, const Col if (isMap(input_column.type) && isMap(header_column.type)) { ColumnWithTypeAndName nested_input_column; - nested_input_column.column = checkAndGetColumn(input_column.column.get())->getNestedColumnPtr(); - nested_input_column.type = checkAndGetDataType(input_column.type.get())->getNestedType(); + nested_input_column.column = checkAndGetColumn(*input_column.column).getNestedColumnPtr(); + nested_input_column.type = checkAndGetDataType(*input_column.type).getNestedType(); ColumnWithTypeAndName nested_header_column; - nested_header_column.column = checkAndGetColumn(header_column.column.get())->getNestedColumnPtr(); - nested_header_column.type = checkAndGetDataType(header_column.type.get())->getNestedType(); + nested_header_column.column = checkAndGetColumn(*header_column.column).getNestedColumnPtr(); + nested_header_column.type = checkAndGetDataType(*header_column.type).getNestedType(); if (!insertNullAsDefaultIfNeeded(nested_input_column, nested_header_column, 0, nullptr)) return false; diff --git a/src/Functions/FunctionBase64Conversion.h b/src/Functions/FunctionBase64Conversion.h index 979c589c64b..3906563a254 100644 --- a/src/Functions/FunctionBase64Conversion.h +++ b/src/Functions/FunctionBase64Conversion.h @@ -111,9 +111,9 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { const auto & input_column = arguments[0].column; - if (const auto * src_column_as_fixed_string = checkAndGetColumn(*input_column)) + if (const auto * src_column_as_fixed_string = checkAndGetColumn(&*input_column)) return execute(*src_column_as_fixed_string, input_rows_count); - else if (const auto * src_column_as_string = checkAndGetColumn(*input_column)) + else if (const auto * src_column_as_string = checkAndGetColumn(&*input_column)) return execute(*src_column_as_string, input_rows_count); throw Exception( diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 89ff63995b1..8ac602faed9 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -1767,8 +1767,8 @@ public: { if (const auto * col_right_const = checkAndGetColumnConst(col_right_raw)) { - const auto * col_left = checkAndGetColumn(col_left_const->getDataColumn()); - const auto * col_right = checkAndGetColumn(col_right_const->getDataColumn()); + const auto * col_left = &checkAndGetColumn(col_left_const->getDataColumn()); + const auto * col_right = &checkAndGetColumn(col_right_const->getDataColumn()); if (col_left->getN() != col_right->getN()) return nullptr; @@ -1805,11 +1805,11 @@ public: const auto * col_left = is_left_column_const ? checkAndGetColumn( - checkAndGetColumnConst(col_left_raw)->getDataColumn()) + &checkAndGetColumnConst(col_left_raw)->getDataColumn()) : checkAndGetColumn(col_left_raw); const auto * col_right = is_right_column_const ? checkAndGetColumn( - checkAndGetColumnConst(col_right_raw)->getDataColumn()) + &checkAndGetColumnConst(col_right_raw)->getDataColumn()) : checkAndGetColumn(col_right_raw); if (col_left && col_right) @@ -1881,8 +1881,8 @@ public: { if (const auto * col_right_const = checkAndGetColumnConst(col_right_raw)) { - const auto * col_left = checkAndGetColumn(col_left_const->getDataColumn()); - const auto * col_right = checkAndGetColumn(col_right_const->getDataColumn()); + const auto * col_left = &checkAndGetColumn(col_left_const->getDataColumn()); + const auto * col_right = &checkAndGetColumn(col_right_const->getDataColumn()); std::string_view a = col_left->getDataAt(0).toView(); std::string_view b = col_right->getDataAt(0).toView(); @@ -1897,10 +1897,10 @@ public: const bool is_right_column_const = checkAndGetColumnConst(col_right_raw) != nullptr; const auto * col_left = is_left_column_const - ? checkAndGetColumn(checkAndGetColumnConst(col_left_raw)->getDataColumn()) + ? &checkAndGetColumn(checkAndGetColumnConst(col_left_raw)->getDataColumn()) : checkAndGetColumn(col_left_raw); const auto * col_right = is_right_column_const - ? checkAndGetColumn(checkAndGetColumnConst(col_right_raw)->getDataColumn()) + ? &checkAndGetColumn(checkAndGetColumnConst(col_right_raw)->getDataColumn()) : checkAndGetColumn(col_right_raw); if (col_left && col_right) @@ -1948,7 +1948,7 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A const ColumnConst * const col_left_const = checkAndGetColumnConst(col_left_raw); - const auto * col_left = col_left_const ? checkAndGetColumn(col_left_const->getDataColumn()) + const auto * col_left = col_left_const ? &checkAndGetColumn(col_left_const->getDataColumn()) : checkAndGetColumn(col_left_raw); if (!col_left) @@ -2231,7 +2231,7 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A bool is_const = checkColumnConst(right_argument.column.get()); const ColumnNullable * nullable_column = is_const ? checkAndGetColumnConstData(right_argument.column.get()) - : checkAndGetColumn(*right_argument.column); + : checkAndGetColumn(right_argument.column.get()); const auto & null_bytemap = nullable_column->getNullMapData(); auto res = executeImpl2(createBlockWithNestedColumns(arguments), removeNullable(result_type), input_rows_count, &null_bytemap); diff --git a/src/Functions/FunctionHelpers.cpp b/src/Functions/FunctionHelpers.cpp index ce83a489a3d..048a601de81 100644 --- a/src/Functions/FunctionHelpers.cpp +++ b/src/Functions/FunctionHelpers.cpp @@ -58,14 +58,14 @@ ColumnWithTypeAndName columnGetNested(const ColumnWithTypeAndName & col) { return ColumnWithTypeAndName{nullptr, nested_type, col.name}; } - else if (const auto * nullable = checkAndGetColumn(*col.column)) + else if (const auto * nullable = checkAndGetColumn(&*col.column)) { const auto & nested_col = nullable->getNestedColumnPtr(); return ColumnWithTypeAndName{nested_col, nested_type, col.name}; } - else if (const auto * const_column = checkAndGetColumn(*col.column)) + else if (const auto * const_column = checkAndGetColumn(&*col.column)) { - const auto * nullable_column = checkAndGetColumn(const_column->getDataColumn()); + const auto * nullable_column = checkAndGetColumn(&const_column->getDataColumn()); ColumnPtr nullable_res; if (nullable_column) @@ -226,7 +226,7 @@ ColumnPtr wrapInNullable(const ColumnPtr & src, const ColumnsWithTypeAndName & a if (src->onlyNull()) return src; - else if (const auto * nullable = checkAndGetColumn(*src)) + else if (const auto * nullable = checkAndGetColumn(&*src)) { src_not_nullable = nullable->getNestedColumnPtr(); result_null_map_column = nullable->getNullMapColumnPtr(); @@ -247,7 +247,7 @@ ColumnPtr wrapInNullable(const ColumnPtr & src, const ColumnsWithTypeAndName & a if (isColumnConst(*elem.column)) continue; - if (const auto * nullable = checkAndGetColumn(*elem.column)) + if (const auto * nullable = checkAndGetColumn(&*elem.column)) { const ColumnPtr & null_map_column = nullable->getNullMapColumnPtr(); if (!result_null_map_column) diff --git a/src/Functions/FunctionHelpers.h b/src/Functions/FunctionHelpers.h index 9f44d3e95c2..89b224e79ef 100644 --- a/src/Functions/FunctionHelpers.h +++ b/src/Functions/FunctionHelpers.h @@ -25,6 +25,13 @@ const Type * checkAndGetDataType(const IDataType * data_type) return typeid_cast(data_type); } +/// Throws on mismatch. +template +const Type & checkAndGetDataType(const IDataType & data_type) +{ + return typeid_cast(data_type); +} + template bool checkDataTypes(const IDataType * data_type) { @@ -34,10 +41,12 @@ bool checkDataTypes(const IDataType * data_type) template const ColumnConst * checkAndGetColumnConst(const IColumn * column) { - if (!column || !isColumnConst(*column)) + if (!column) return {}; - const ColumnConst * res = assert_cast(column); + const ColumnConst * res = checkAndGetColumn(column); + if (!res) + return {}; if (!checkColumn(&res->getDataColumn())) return {}; @@ -45,6 +54,18 @@ const ColumnConst * checkAndGetColumnConst(const IColumn * column) return res; } +template +const ColumnConst & checkAndGetColumnConst(const IColumn & column) +{ + const ColumnConst & res = checkAndGetColumn(column); + + const auto & data_column = res.getDataColumn(); + if (!checkColumn(&data_column)) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Unexpected const column type: expected {}, got {}", demangle(typeid(Type).name()), demangle(typeid(data_column).name())); + + return res; +} + template const Type * checkAndGetColumnConstData(const IColumn * column) { diff --git a/src/Functions/FunctionUnixTimestamp64.h b/src/Functions/FunctionUnixTimestamp64.h index 53421a565cb..c418163343b 100644 --- a/src/Functions/FunctionUnixTimestamp64.h +++ b/src/Functions/FunctionUnixTimestamp64.h @@ -140,7 +140,7 @@ public: const auto & src = arguments[0]; const auto & col = *src.column; - if (!checkAndGetColumn>(col)) + if (!checkAndGetColumn>(&col)) return false; auto & result_data = result_column->getData(); diff --git a/src/Functions/FunctionsBitmap.h b/src/Functions/FunctionsBitmap.h index 22d46fa7728..92ec71a3118 100644 --- a/src/Functions/FunctionsBitmap.h +++ b/src/Functions/FunctionsBitmap.h @@ -193,8 +193,8 @@ private: const ColumnArray * array = typeid_cast(arguments[0].column.get()); const ColumnPtr & mapped = array->getDataPtr(); const ColumnArray::Offsets & offsets = array->getOffsets(); - const ColumnVector * column = checkAndGetColumn>(&*mapped); - const typename ColumnVector::Container & input_data = column->getData(); + const ColumnVector & column = checkAndGetColumn>(*mapped); + const typename ColumnVector::Container & input_data = column.getData(); // output data Array params_row; diff --git a/src/Functions/FunctionsCodingIP.cpp b/src/Functions/FunctionsCodingIP.cpp index 7bdbac6531d..54f7b6dd1f4 100644 --- a/src/Functions/FunctionsCodingIP.cpp +++ b/src/Functions/FunctionsCodingIP.cpp @@ -536,7 +536,7 @@ public: const auto & col_type_name = arguments[0]; const ColumnPtr & column = col_type_name.column; - if (const auto * col_in = checkAndGetColumn(*column)) + if (const auto * col_in = checkAndGetColumn(&*column)) { auto col_res = ColumnIPv6::create(); @@ -551,7 +551,7 @@ public: return col_res; } - if (const auto * col_in = checkAndGetColumn(*column)) + if (const auto * col_in = checkAndGetColumn(&*column)) { auto col_res = ColumnFixedString::create(IPV6_BINARY_LENGTH); diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index a16ce136b9a..02da450e0c2 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -3302,7 +3302,7 @@ private: /// both columns have type UInt8, but we shouldn't use identity wrapper, /// because Bool column can contain only 0 and 1. auto res_column = to_type->createColumn(); - const auto & data_from = checkAndGetColumn(arguments[0].column.get())->getData(); + const auto & data_from = checkAndGetColumn(*arguments[0].column).getData(); auto & data_to = assert_cast(res_column.get())->getData(); data_to.resize(data_from.size()); for (size_t i = 0; i != data_from.size(); ++i) diff --git a/src/Functions/FunctionsRound.h b/src/Functions/FunctionsRound.h index 3d1028c6d35..24ffb5eb0c1 100644 --- a/src/Functions/FunctionsRound.h +++ b/src/Functions/FunctionsRound.h @@ -467,28 +467,28 @@ struct Dispatcher static ColumnPtr apply(const IColumn * col_general, Scale scale_arg) { - const auto * const col = checkAndGetColumn>(col_general); + const auto & col = checkAndGetColumn>(*col_general); auto col_res = ColumnVector::create(); typename ColumnVector::Container & vec_res = col_res->getData(); - vec_res.resize(col->getData().size()); + vec_res.resize(col.getData().size()); if (!vec_res.empty()) { if (scale_arg == 0) { size_t scale = 1; - FunctionRoundingImpl::apply(col->getData(), scale, vec_res); + FunctionRoundingImpl::apply(col.getData(), scale, vec_res); } else if (scale_arg > 0) { size_t scale = intExp10(scale_arg); - FunctionRoundingImpl::apply(col->getData(), scale, vec_res); + FunctionRoundingImpl::apply(col.getData(), scale, vec_res); } else { size_t scale = intExp10(-scale_arg); - FunctionRoundingImpl::apply(col->getData(), scale, vec_res); + FunctionRoundingImpl::apply(col.getData(), scale, vec_res); } } @@ -502,14 +502,14 @@ struct Dispatcher public: static ColumnPtr apply(const IColumn * col_general, Scale scale_arg) { - const auto * const col = checkAndGetColumn>(col_general); - const typename ColumnDecimal::Container & vec_src = col->getData(); + const auto & col = checkAndGetColumn>(*col_general); + const typename ColumnDecimal::Container & vec_src = col.getData(); - auto col_res = ColumnDecimal::create(vec_src.size(), col->getScale()); + auto col_res = ColumnDecimal::create(vec_src.size(), col.getScale()); auto & vec_res = col_res->getData(); if (!vec_res.empty()) - DecimalRoundingImpl::apply(col->getData(), col->getScale(), vec_res, scale_arg); + DecimalRoundingImpl::apply(col.getData(), col.getScale(), vec_res, scale_arg); return col_res; } diff --git a/src/Functions/FunctionsStringHash.h b/src/Functions/FunctionsStringHash.h index d951e77395e..fcd4c970a47 100644 --- a/src/Functions/FunctionsStringHash.h +++ b/src/Functions/FunctionsStringHash.h @@ -153,8 +153,8 @@ public: auto col_res = ColumnVector::create(); auto & vec_res = col_res->getData(); vec_res.resize(column->size()); - const ColumnString * col_str_vector = checkAndGetColumn(&*column); - Impl::apply(col_str_vector->getChars(), col_str_vector->getOffsets(), shingle_size, vec_res); + const ColumnString & col_str_vector = checkAndGetColumn(*column); + Impl::apply(col_str_vector.getChars(), col_str_vector.getOffsets(), shingle_size, vec_res); return col_res; } else if constexpr (is_arg) // Min hash arg @@ -170,8 +170,8 @@ public: auto min_tuple = ColumnTuple::create(std::move(min_columns)); auto max_tuple = ColumnTuple::create(std::move(max_columns)); - const ColumnString * col_str_vector = checkAndGetColumn(&*column); - Impl::apply(col_str_vector->getChars(), col_str_vector->getOffsets(), shingle_size, num_hashes, nullptr, nullptr, min_tuple.get(), max_tuple.get()); + const ColumnString & col_str_vector = checkAndGetColumn(*column); + Impl::apply(col_str_vector.getChars(), col_str_vector.getOffsets(), shingle_size, num_hashes, nullptr, nullptr, min_tuple.get(), max_tuple.get()); MutableColumns tuple_columns; tuple_columns.emplace_back(std::move(min_tuple)); @@ -186,8 +186,8 @@ public: auto & vec_h2 = col_h2->getData(); vec_h1.resize(column->size()); vec_h2.resize(column->size()); - const ColumnString * col_str_vector = checkAndGetColumn(&*column); - Impl::apply(col_str_vector->getChars(), col_str_vector->getOffsets(), shingle_size, num_hashes, &vec_h1, &vec_h2, nullptr, nullptr); + const ColumnString & col_str_vector = checkAndGetColumn(*column); + Impl::apply(col_str_vector.getChars(), col_str_vector.getOffsets(), shingle_size, num_hashes, &vec_h1, &vec_h2, nullptr, nullptr); MutableColumns tuple_columns; tuple_columns.emplace_back(std::move(col_h1)); tuple_columns.emplace_back(std::move(col_h2)); diff --git a/src/Functions/Kusto/KqlArraySort.cpp b/src/Functions/Kusto/KqlArraySort.cpp index 5be36328cc3..22544f4302c 100644 --- a/src/Functions/Kusto/KqlArraySort.cpp +++ b/src/Functions/Kusto/KqlArraySort.cpp @@ -158,12 +158,12 @@ public: auto out_tmp = ColumnArray::create(nested_types[i]->createColumn()); size_t array_size = tuple_coulmn->size(); - const auto * arr = checkAndGetColumn(tuple_coulmn.get()); + const auto & arr = checkAndGetColumn(*tuple_coulmn); for (size_t j = 0; j < array_size; ++j) { Field arr_field; - arr->get(j, arr_field); + arr.get(j, arr_field); out_tmp->insert(arr_field); } diff --git a/src/Functions/MultiMatchAllIndicesImpl.h b/src/Functions/MultiMatchAllIndicesImpl.h index d655311f532..3e9c8fba215 100644 --- a/src/Functions/MultiMatchAllIndicesImpl.h +++ b/src/Functions/MultiMatchAllIndicesImpl.h @@ -185,7 +185,7 @@ struct MultiMatchAllIndicesImpl size_t prev_haystack_offset = 0; size_t prev_needles_offset = 0; - const ColumnString * needles_data_string = checkAndGetColumn(&needles_data); + const ColumnString & needles_data_string = checkAndGetColumn(needles_data); std::vector needles; @@ -195,7 +195,7 @@ struct MultiMatchAllIndicesImpl for (size_t j = prev_needles_offset; j < needles_offsets[i]; ++j) { - needles.emplace_back(needles_data_string->getDataAt(j).toView()); + needles.emplace_back(needles_data_string.getDataAt(j).toView()); } if (needles.empty()) diff --git a/src/Functions/MultiMatchAnyImpl.h b/src/Functions/MultiMatchAnyImpl.h index 0b9eee2382f..000c334f6c1 100644 --- a/src/Functions/MultiMatchAnyImpl.h +++ b/src/Functions/MultiMatchAnyImpl.h @@ -212,7 +212,7 @@ struct MultiMatchAnyImpl size_t prev_haystack_offset = 0; size_t prev_needles_offset = 0; - const ColumnString * needles_data_string = checkAndGetColumn(&needles_data); + const ColumnString & needles_data_string = checkAndGetColumn(needles_data); std::vector needles; @@ -221,7 +221,7 @@ struct MultiMatchAnyImpl needles.reserve(needles_offsets[i] - prev_needles_offset); for (size_t j = prev_needles_offset; j < needles_offsets[i]; ++j) - needles.emplace_back(needles_data_string->getDataAt(j).toView()); + needles.emplace_back(needles_data_string.getDataAt(j).toView()); if (needles.empty()) { diff --git a/src/Functions/MultiSearchAllPositionsImpl.h b/src/Functions/MultiSearchAllPositionsImpl.h index 6e1f13d87b6..cfe60e51bcd 100644 --- a/src/Functions/MultiSearchAllPositionsImpl.h +++ b/src/Functions/MultiSearchAllPositionsImpl.h @@ -89,7 +89,7 @@ struct MultiSearchAllPositionsImpl offsets_res.reserve(haystack_offsets.size()); - const ColumnString * needles_data_string = checkAndGetColumn(&needles_data); + const ColumnString & needles_data_string = checkAndGetColumn(needles_data); std::vector needles; @@ -99,7 +99,7 @@ struct MultiSearchAllPositionsImpl for (size_t j = prev_needles_offset; j < needles_offsets[i]; ++j) { - needles.emplace_back(needles_data_string->getDataAt(j).toView()); + needles.emplace_back(needles_data_string.getDataAt(j).toView()); } const size_t needles_size = needles.size(); diff --git a/src/Functions/MultiSearchFirstIndexImpl.h b/src/Functions/MultiSearchFirstIndexImpl.h index 73f3c92adfb..36a5fd514d9 100644 --- a/src/Functions/MultiSearchFirstIndexImpl.h +++ b/src/Functions/MultiSearchFirstIndexImpl.h @@ -88,7 +88,7 @@ struct MultiSearchFirstIndexImpl size_t prev_haystack_offset = 0; size_t prev_needles_offset = 0; - const ColumnString * needles_data_string = checkAndGetColumn(&needles_data); + const ColumnString & needles_data_string = checkAndGetColumn(needles_data); std::vector needles; @@ -98,7 +98,7 @@ struct MultiSearchFirstIndexImpl for (size_t j = prev_needles_offset; j < needles_offsets[i]; ++j) { - needles.emplace_back(needles_data_string->getDataAt(j).toView()); + needles.emplace_back(needles_data_string.getDataAt(j).toView()); } auto searcher = Impl::createMultiSearcherInBigHaystack(needles); // sub-optimal diff --git a/src/Functions/MultiSearchFirstPositionImpl.h b/src/Functions/MultiSearchFirstPositionImpl.h index 99dd3f9d394..8b137bb67d1 100644 --- a/src/Functions/MultiSearchFirstPositionImpl.h +++ b/src/Functions/MultiSearchFirstPositionImpl.h @@ -97,7 +97,7 @@ struct MultiSearchFirstPositionImpl size_t prev_haystack_offset = 0; size_t prev_needles_offset = 0; - const ColumnString * needles_data_string = checkAndGetColumn(&needles_data); + const ColumnString & needles_data_string = checkAndGetColumn(needles_data); std::vector needles; @@ -112,7 +112,7 @@ struct MultiSearchFirstPositionImpl for (size_t j = prev_needles_offset; j < needles_offsets[i]; ++j) { - needles.emplace_back(needles_data_string->getDataAt(j).toView()); + needles.emplace_back(needles_data_string.getDataAt(j).toView()); } auto searcher = Impl::createMultiSearcherInBigHaystack(needles); // sub-optimal diff --git a/src/Functions/MultiSearchImpl.h b/src/Functions/MultiSearchImpl.h index fb7d56f302a..494eb323639 100644 --- a/src/Functions/MultiSearchImpl.h +++ b/src/Functions/MultiSearchImpl.h @@ -87,7 +87,7 @@ struct MultiSearchImpl size_t prev_haystack_offset = 0; size_t prev_needles_offset = 0; - const ColumnString * needles_data_string = checkAndGetColumn(&needles_data); + const ColumnString & needles_data_string = checkAndGetColumn(needles_data); std::vector needles; @@ -97,7 +97,7 @@ struct MultiSearchImpl for (size_t j = prev_needles_offset; j < needles_offsets[i]; ++j) { - needles.emplace_back(needles_data_string->getDataAt(j).toView()); + needles.emplace_back(needles_data_string.getDataAt(j).toView()); } const auto * const haystack = &haystack_data[prev_haystack_offset]; diff --git a/src/Functions/URL/FirstSignificantSubdomainCustomImpl.h b/src/Functions/URL/FirstSignificantSubdomainCustomImpl.h index 93691e35741..68582198ea3 100644 --- a/src/Functions/URL/FirstSignificantSubdomainCustomImpl.h +++ b/src/Functions/URL/FirstSignificantSubdomainCustomImpl.h @@ -69,7 +69,7 @@ public: const ColumnConst * column_tld_list_name = checkAndGetColumnConstStringOrFixedString(arguments[1].column.get()); FirstSignificantSubdomainCustomLookup tld_lookup(column_tld_list_name->getValue()); - if (const ColumnString * col = checkAndGetColumn(*arguments[0].column)) + if (const ColumnString * col = checkAndGetColumn(&*arguments[0].column)) { auto col_res = ColumnString::create(); vector(tld_lookup, col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets()); diff --git a/src/Functions/array/FunctionArrayMapped.h b/src/Functions/array/FunctionArrayMapped.h index 5d6d70521b0..3c8dc04de46 100644 --- a/src/Functions/array/FunctionArrayMapped.h +++ b/src/Functions/array/FunctionArrayMapped.h @@ -317,7 +317,7 @@ public: ErrorCodes::ILLEGAL_COLUMN, "Expected Array column, found {}", column_array_ptr->getName()); column_array_ptr = recursiveRemoveLowCardinality(column_const_array->convertToFullColumn()); - column_array = checkAndGetColumn(column_array_ptr.get()); + column_array = &checkAndGetColumn(*column_array_ptr); } if (!array_type) diff --git a/src/Functions/array/arrayCompact.cpp b/src/Functions/array/arrayCompact.cpp index 7d09d1078d5..fe870bebd84 100644 --- a/src/Functions/array/arrayCompact.cpp +++ b/src/Functions/array/arrayCompact.cpp @@ -34,7 +34,7 @@ struct ArrayCompactImpl using ColVecType = ColumnVectorOrDecimal; const ColVecType * check_values_column = checkAndGetColumn(mapped.get()); - const ColVecType * src_values_column = checkAndGetColumn(array.getData()); + const ColVecType * src_values_column = checkAndGetColumn(&array.getData()); if (!src_values_column || !check_values_column) return false; diff --git a/src/Functions/array/arrayDistinct.cpp b/src/Functions/array/arrayDistinct.cpp index ea331d6bdad..2344742e4fd 100644 --- a/src/Functions/array/arrayDistinct.cpp +++ b/src/Functions/array/arrayDistinct.cpp @@ -89,20 +89,20 @@ private: ColumnPtr FunctionArrayDistinct::executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/) const { ColumnPtr array_ptr = arguments[0].column; - const ColumnArray * array = checkAndGetColumn(array_ptr.get()); + const ColumnArray & array = checkAndGetColumn(*array_ptr); const auto & return_type = result_type; auto res_ptr = return_type->createColumn(); ColumnArray & res = assert_cast(*res_ptr); - const IColumn & src_data = array->getData(); - const ColumnArray::Offsets & offsets = array->getOffsets(); + const IColumn & src_data = array.getData(); + const ColumnArray::Offsets & offsets = array.getOffsets(); IColumn & res_data = res.getData(); ColumnArray::Offsets & res_offsets = res.getOffsets(); - const ColumnNullable * nullable_col = checkAndGetColumn(src_data); + const ColumnNullable * nullable_col = checkAndGetColumn(&src_data); const IColumn * inner_col; diff --git a/src/Functions/array/arrayElement.cpp b/src/Functions/array/arrayElement.cpp index 8669fd1f3a7..227b29d5d9f 100644 --- a/src/Functions/array/arrayElement.cpp +++ b/src/Functions/array/arrayElement.cpp @@ -1538,9 +1538,9 @@ ColumnPtr FunctionArrayElement::executeMap2(const ColumnsWithTypeAndName & argum return nullptr; const ColumnArray * col_map_nested = &col_map->getNestedColumn(); - const ColumnTuple * col_map_kv = checkAndGetColumn(col_map_nested->getDataPtr().get()); - ColumnPtr col_map_keys = col_map_kv->getColumnPtr(0); - ColumnPtr col_map_values = col_map_kv->getColumnPtr(1); + const ColumnTuple & col_map_kv = checkAndGetColumn(*col_map_nested->getDataPtr()); + ColumnPtr col_map_keys = col_map_kv.getColumnPtr(0); + ColumnPtr col_map_values = col_map_kv.getColumnPtr(1); const DataTypeMap & map_type = typeid_cast(*typeid_cast(*arguments[0].type).getNestedType()); diff --git a/src/Functions/array/arrayEnumerateExtended.h b/src/Functions/array/arrayEnumerateExtended.h index cf38afcfa5a..62850a1cbf0 100644 --- a/src/Functions/array/arrayEnumerateExtended.h +++ b/src/Functions/array/arrayEnumerateExtended.h @@ -165,7 +165,7 @@ ColumnPtr FunctionArrayEnumerateExtended::executeImpl(const ColumnsWith for (size_t i = 0; i < num_arguments; ++i) { - if (const auto * nullable_col = checkAndGetColumn(*data_columns[i])) + if (const auto * nullable_col = checkAndGetColumn(data_columns[i])) { if (num_arguments == 1) data_columns[i] = &nullable_col->getNestedColumn(); diff --git a/src/Functions/array/arrayIndex.h b/src/Functions/array/arrayIndex.h index cd537763b4a..c0b1c773387 100644 --- a/src/Functions/array/arrayIndex.h +++ b/src/Functions/array/arrayIndex.h @@ -506,10 +506,10 @@ private: const ColumnNullable * nullable = nullptr; if (col_array) - nullable = checkAndGetColumn(col_array->getData()); + nullable = checkAndGetColumn(&col_array->getData()); const auto & arg_column = arguments[1].column; - const ColumnNullable * arg_nullable = checkAndGetColumn(*arg_column); + const ColumnNullable * arg_nullable = checkAndGetColumn(&*arg_column); if (!nullable && !arg_nullable) { @@ -738,7 +738,7 @@ private: const auto [null_map_data, null_map_item] = getNullMaps(arguments); - if (const ColumnConst * col_arg_const = checkAndGetColumn(*arguments[1].column)) + if (const ColumnConst * col_arg_const = checkAndGetColumn(&*arguments[1].column)) { const IColumnUnique & col_lc_dict = col_lc->getDictionary(); @@ -754,7 +754,7 @@ private: if (!col_arg_cloned->isNullAt(0)) { if (col_arg_cloned->isNullable()) - col_arg_cloned = checkAndGetColumn(*col_arg_cloned)->getNestedColumnPtr(); + col_arg_cloned = checkAndGetColumn(*col_arg_cloned).getNestedColumnPtr(); StringRef elem = col_arg_cloned->getDataAt(0); @@ -786,7 +786,7 @@ private: else if (col_lc->nestedIsNullable()) // LowCardinality(Nullable(T)) and U { const ColumnPtr left_casted = col_lc->convertToFullColumnIfLowCardinality(); // Nullable(T) - const ColumnNullable& left_nullable = *checkAndGetColumn(left_casted.get()); + const ColumnNullable & left_nullable = checkAndGetColumn(*left_casted); const NullMap * const null_map_left_casted = &left_nullable.getNullMapColumn().getData(); diff --git a/src/Functions/array/arrayJaccardIndex.cpp b/src/Functions/array/arrayJaccardIndex.cpp index 9cb74a7aa62..87f3390ac73 100644 --- a/src/Functions/array/arrayJaccardIndex.cpp +++ b/src/Functions/array/arrayJaccardIndex.cpp @@ -97,8 +97,8 @@ public: { if (const ColumnConst * col_const = typeid_cast(col.column.get())) { - const ColumnArray * col_const_array = checkAndGetColumn(col_const->getDataColumnPtr().get()); - return {col_const_array, true}; + const ColumnArray & col_const_array = checkAndGetColumn(*col_const->getDataColumnPtr()); + return {&col_const_array, true}; } else if (const ColumnArray * col_non_const_array = checkAndGetColumn(col.column.get())) return {col_non_const_array, false}; @@ -128,8 +128,8 @@ public: vectorWithEmptyIntersect(left_array->getOffsets(), right_array->getOffsets(), vec_res); \ else \ { \ - const ColumnArray * intersect_column_array = checkAndGetColumn(intersect_column.column.get()); \ - vector(intersect_column_array->getOffsets(), left_array->getOffsets(), right_array->getOffsets(), vec_res); \ + const ColumnArray & intersect_column_array = checkAndGetColumn(*intersect_column.column); \ + vector(intersect_column_array.getOffsets(), left_array->getOffsets(), right_array->getOffsets(), vec_res); \ } if (!left_is_const && !right_is_const) diff --git a/src/Functions/array/arrayUniq.cpp b/src/Functions/array/arrayUniq.cpp index 81ba5b62094..f92daf384d2 100644 --- a/src/Functions/array/arrayUniq.cpp +++ b/src/Functions/array/arrayUniq.cpp @@ -162,7 +162,7 @@ ColumnPtr FunctionArrayUniq::executeImpl(const ColumnsWithTypeAndName & argument for (size_t i = 0; i < num_arguments; ++i) { - if (const auto * nullable_col = checkAndGetColumn(*data_columns[i])) + if (const auto * nullable_col = checkAndGetColumn(data_columns[i])) { if (num_arguments == 1) data_columns[i] = &nullable_col->getNestedColumn(); diff --git a/src/Functions/array/emptyArrayToSingle.cpp b/src/Functions/array/emptyArrayToSingle.cpp index 86d4c32265a..2071abf9911 100644 --- a/src/Functions/array/emptyArrayToSingle.cpp +++ b/src/Functions/array/emptyArrayToSingle.cpp @@ -391,7 +391,7 @@ ColumnPtr FunctionEmptyArrayToSingle::executeImpl(const ColumnsWithTypeAndName & const IColumn * inner_col; IColumn * inner_res_col; - const auto * nullable_col = checkAndGetColumn(src_data); + const auto * nullable_col = checkAndGetColumn(&src_data); if (nullable_col) { inner_col = &nullable_col->getNestedColumn(); diff --git a/src/Functions/array/range.cpp b/src/Functions/array/range.cpp index 57679ccb180..f939ea7b462 100644 --- a/src/Functions/array/range.cpp +++ b/src/Functions/array/range.cpp @@ -404,7 +404,7 @@ private: { if (!col.type->isNullable()) return; - const ColumnNullable * nullable_col = checkAndGetColumn(*col.column); + const ColumnNullable * nullable_col = checkAndGetColumn(col.column.get()); if (!nullable_col) nullable_col = checkAndGetColumnConstData(col.column.get()); if (!nullable_col) @@ -421,8 +421,8 @@ private: const auto * col = arguments[0].column.get(); if (arguments[0].type->isNullable()) { - const auto * nullable = checkAndGetColumn(*arguments[0].column); - col = nullable->getNestedColumnPtr().get(); + const auto & nullable = checkAndGetColumn(*arguments[0].column); + col = nullable.getNestedColumnPtr().get(); } if (!((res = executeInternal(col)) || (res = executeInternal(col)) || (res = executeInternal(col)) diff --git a/src/Functions/arrayStringConcat.cpp b/src/Functions/arrayStringConcat.cpp index c186c0ca7e6..b787feeeca1 100644 --- a/src/Functions/arrayStringConcat.cpp +++ b/src/Functions/arrayStringConcat.cpp @@ -183,7 +183,7 @@ public: const ColumnString & col_string = assert_cast(*str_subcolumn.get()); auto col_res = ColumnString::create(); - if (const ColumnNullable * col_nullable = checkAndGetColumn(col_arr.getData())) + if (const ColumnNullable * col_nullable = checkAndGetColumn(&col_arr.getData())) executeInternal(col_string, col_arr, delimiter, *col_res, col_nullable->getNullMapData().data()); else executeInternal(col_string, col_arr, delimiter, *col_res); diff --git a/src/Functions/assumeNotNull.cpp b/src/Functions/assumeNotNull.cpp index 4dd88163ecb..be9d38f5af3 100644 --- a/src/Functions/assumeNotNull.cpp +++ b/src/Functions/assumeNotNull.cpp @@ -54,7 +54,7 @@ public: if (arguments[0].type->onlyNull() && !col->empty()) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot create non-empty column with type Nothing"); - if (const auto * nullable_col = checkAndGetColumn(*col)) + if (const auto * nullable_col = checkAndGetColumn(&*col)) return nullable_col->getNestedColumnPtr(); else return col; diff --git a/src/Functions/coalesce.cpp b/src/Functions/coalesce.cpp index 4ae90a9db13..722f32af523 100644 --- a/src/Functions/coalesce.cpp +++ b/src/Functions/coalesce.cpp @@ -157,12 +157,12 @@ public: /// if last argument is not nullable, result should be also not nullable if (!multi_if_args.back().column->isNullable() && res->isNullable()) { - if (const auto * column_lc = checkAndGetColumn(*res)) - res = checkAndGetColumn(*column_lc->convertToFullColumn())->getNestedColumnPtr(); - else if (const auto * column_const = checkAndGetColumn(*res)) - res = checkAndGetColumn(column_const->getDataColumn())->getNestedColumnPtr(); + if (const auto * column_lc = checkAndGetColumn(&*res)) + res = checkAndGetColumn(*column_lc->convertToFullColumn()).getNestedColumnPtr(); + else if (const auto * column_const = checkAndGetColumn(&*res)) + res = checkAndGetColumn(column_const->getDataColumn()).getNestedColumnPtr(); else - res = checkAndGetColumn(*res)->getNestedColumnPtr(); + res = checkAndGetColumn(&*res)->getNestedColumnPtr(); } return res; diff --git a/src/Functions/fromModifiedJulianDay.cpp b/src/Functions/fromModifiedJulianDay.cpp index 695d1b7d63c..a1a3102c70e 100644 --- a/src/Functions/fromModifiedJulianDay.cpp +++ b/src/Functions/fromModifiedJulianDay.cpp @@ -34,8 +34,8 @@ namespace DB ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { using ColVecType = typename FromDataType::ColumnType; - const ColVecType * col_from = checkAndGetColumn(arguments[0].column.get()); - const typename ColVecType::Container & vec_from = col_from->getData(); + const ColVecType & col_from = checkAndGetColumn(*arguments[0].column); + const typename ColVecType::Container & vec_from = col_from.getData(); auto col_to = ColumnString::create(); ColumnString::Chars & data_to = col_to->getChars(); diff --git a/src/Functions/grouping.h b/src/Functions/grouping.h index 830c509f1f5..85ba1967909 100644 --- a/src/Functions/grouping.h +++ b/src/Functions/grouping.h @@ -55,7 +55,7 @@ public: template ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, size_t input_rows_count, AggregationKeyChecker checker) const { - const auto * grouping_set_column = checkAndGetColumn(arguments[0].column.get()); + const auto & grouping_set_column = checkAndGetColumn(*arguments[0].column); auto result = ColumnUInt64::create(); auto & result_data = result->getData(); @@ -64,7 +64,7 @@ public: const auto * result_table = likely(force_compatibility) ? COMPATIBLE_MODE : INCOMPATIBLE_MODE; for (size_t i = 0; i < input_rows_count; ++i) { - UInt64 set_index = grouping_set_column->getElement(i); + UInt64 set_index = grouping_set_column.getElement(i); UInt64 value = 0; for (auto index : arguments_indexes) diff --git a/src/Functions/hasColumnInTable.cpp b/src/Functions/hasColumnInTable.cpp index 48783a672e2..8ea16f688ee 100644 --- a/src/Functions/hasColumnInTable.cpp +++ b/src/Functions/hasColumnInTable.cpp @@ -88,8 +88,8 @@ ColumnPtr FunctionHasColumnInTable::executeImpl(const ColumnsWithTypeAndName & a { auto get_string_from_columns = [&](const ColumnWithTypeAndName & column) -> String { - const ColumnConst * const_column = checkAndGetColumnConst(column.column.get()); - return const_column->getValue(); + const ColumnConst & const_column = checkAndGetColumnConst(*column.column); + return const_column.getValue(); }; size_t arg = 0; diff --git a/src/Functions/if.cpp b/src/Functions/if.cpp index 4f75042ad8d..9766f34edca 100644 --- a/src/Functions/if.cpp +++ b/src/Functions/if.cpp @@ -948,12 +948,12 @@ private: bool cond_is_const = false; bool cond_is_true = false; bool cond_is_false = false; - if (const auto * const_arg = checkAndGetColumn(*arg_cond.column)) + if (const auto * const_arg = checkAndGetColumn(&*arg_cond.column)) { cond_is_const = true; not_const_condition = const_arg->getDataColumnPtr(); ColumnPtr data_column = const_arg->getDataColumnPtr(); - if (const auto * const_nullable_arg = checkAndGetColumn(*data_column)) + if (const auto * const_nullable_arg = checkAndGetColumn(&*data_column)) { data_column = const_nullable_arg->getNestedColumnPtr(); if (!data_column->empty()) @@ -962,7 +962,7 @@ private: if (!data_column->empty()) { - cond_is_true = !cond_is_null && checkAndGetColumn(*data_column)->getBool(0); + cond_is_true = !cond_is_null && checkAndGetColumn(*data_column).getBool(0); cond_is_false = !cond_is_null && !cond_is_true; } } @@ -975,12 +975,12 @@ private: else if (cond_is_false || cond_is_null) return castColumn(column2, result_type); - if (const auto * nullable = checkAndGetColumn(*not_const_condition)) + if (const auto * nullable = checkAndGetColumn(&*not_const_condition)) { ColumnPtr new_cond_column = nullable->getNestedColumnPtr(); size_t column_size = arg_cond.column->size(); - if (checkAndGetColumn(*new_cond_column)) + if (checkAndGetColumn(&*new_cond_column)) { auto nested_column_copy = new_cond_column->cloneResized(new_cond_column->size()); typeid_cast(nested_column_copy.get())->applyZeroMap(nullable->getNullMapData()); @@ -1027,12 +1027,12 @@ private: /// Const(size = 0, Int32(size = 1)) static ColumnPtr recursiveGetNestedColumnWithoutNullable(const ColumnPtr & column) { - if (const auto * nullable = checkAndGetColumn(*column)) + if (const auto * nullable = checkAndGetColumn(&*column)) { /// Nullable cannot contain Nullable return nullable->getNestedColumnPtr(); } - else if (const auto * column_const = checkAndGetColumn(*column)) + else if (const auto * column_const = checkAndGetColumn(&*column)) { /// Save Constant, but remove Nullable return ColumnConst::create(recursiveGetNestedColumnWithoutNullable(column_const->getDataColumnPtr()), column->size()); @@ -1051,8 +1051,8 @@ private: const ColumnWithTypeAndName & arg_then = arguments[1]; const ColumnWithTypeAndName & arg_else = arguments[2]; - const auto * then_is_nullable = checkAndGetColumn(*arg_then.column); - const auto * else_is_nullable = checkAndGetColumn(*arg_else.column); + const auto * then_is_nullable = checkAndGetColumn(&*arg_then.column); + const auto * else_is_nullable = checkAndGetColumn(&*arg_else.column); if (!then_is_nullable && !else_is_nullable) return nullptr; diff --git a/src/Functions/isNotNull.cpp b/src/Functions/isNotNull.cpp index dd5182aeade..dd53c700221 100644 --- a/src/Functions/isNotNull.cpp +++ b/src/Functions/isNotNull.cpp @@ -46,7 +46,7 @@ public: if (isVariant(elem.type)) { - const auto & discriminators = checkAndGetColumn(*elem.column)->getLocalDiscriminators(); + const auto & discriminators = checkAndGetColumn(*elem.column).getLocalDiscriminators(); auto res = DataTypeUInt8().createColumn(); auto & data = typeid_cast(*res).getData(); data.resize(discriminators.size()); @@ -57,17 +57,17 @@ public: if (elem.type->isLowCardinalityNullable()) { - const auto * low_cardinality_column = checkAndGetColumn(*elem.column); - const size_t null_index = low_cardinality_column->getDictionary().getNullValueIndex(); + const auto & low_cardinality_column = checkAndGetColumn(*elem.column); + const size_t null_index = low_cardinality_column.getDictionary().getNullValueIndex(); auto res = DataTypeUInt8().createColumn(); auto & data = typeid_cast(*res).getData(); - data.resize(low_cardinality_column->size()); - for (size_t i = 0; i != low_cardinality_column->size(); ++i) - data[i] = (low_cardinality_column->getIndexAt(i) != null_index); + data.resize(low_cardinality_column.size()); + for (size_t i = 0; i != low_cardinality_column.size(); ++i) + data[i] = (low_cardinality_column.getIndexAt(i) != null_index); return res; } - if (const auto * nullable = checkAndGetColumn(*elem.column)) + if (const auto * nullable = checkAndGetColumn(&*elem.column)) { /// Return the negated null map. auto res_column = ColumnUInt8::create(input_rows_count); diff --git a/src/Functions/isNull.cpp b/src/Functions/isNull.cpp index 4bf4e44f866..7a6dabab7af 100644 --- a/src/Functions/isNull.cpp +++ b/src/Functions/isNull.cpp @@ -48,7 +48,7 @@ public: if (isVariant(elem.type)) { - const auto & discriminators = checkAndGetColumn(*elem.column)->getLocalDiscriminators(); + const auto & discriminators = checkAndGetColumn(*elem.column).getLocalDiscriminators(); auto res = DataTypeUInt8().createColumn(); auto & data = typeid_cast(*res).getData(); data.reserve(discriminators.size()); @@ -59,17 +59,17 @@ public: if (elem.type->isLowCardinalityNullable()) { - const auto * low_cardinality_column = checkAndGetColumn(*elem.column); - size_t null_index = low_cardinality_column->getDictionary().getNullValueIndex(); + const auto & low_cardinality_column = checkAndGetColumn(*elem.column); + size_t null_index = low_cardinality_column.getDictionary().getNullValueIndex(); auto res = DataTypeUInt8().createColumn(); auto & data = typeid_cast(*res).getData(); - data.reserve(low_cardinality_column->size()); - for (size_t i = 0; i != low_cardinality_column->size(); ++i) - data.push_back(low_cardinality_column->getIndexAt(i) == null_index); + data.reserve(low_cardinality_column.size()); + for (size_t i = 0; i != low_cardinality_column.size(); ++i) + data.push_back(low_cardinality_column.getIndexAt(i) == null_index); return res; } - if (const auto * nullable = checkAndGetColumn(*elem.column)) + if (const auto * nullable = checkAndGetColumn(&*elem.column)) { /// Merely return the embedded null map. return nullable->getNullMapColumnPtr(); diff --git a/src/Functions/minSampleSize.cpp b/src/Functions/minSampleSize.cpp index a5826ef5c0e..f37b030c85a 100644 --- a/src/Functions/minSampleSize.cpp +++ b/src/Functions/minSampleSize.cpp @@ -102,14 +102,14 @@ struct ContinuousImpl auto baseline_argument = arguments[0]; baseline_argument.column = baseline_argument.column->convertToFullColumnIfConst(); auto baseline_column_untyped = castColumnAccurate(baseline_argument, float_64_type); - const auto * baseline_column = checkAndGetColumn>(*baseline_column_untyped); - const auto & baseline_column_data = baseline_column->getData(); + const auto & baseline_column = checkAndGetColumn>(*baseline_column_untyped); + const auto & baseline_column_data = baseline_column.getData(); auto sigma_argument = arguments[1]; sigma_argument.column = sigma_argument.column->convertToFullColumnIfConst(); auto sigma_column_untyped = castColumnAccurate(sigma_argument, float_64_type); - const auto * sigma_column = checkAndGetColumn>(*sigma_column_untyped); - const auto & sigma_column_data = sigma_column->getData(); + const auto & sigma_column = checkAndGetColumn>(*sigma_column_untyped); + const auto & sigma_column_data = sigma_column.getData(); const IColumn & col_mde = *arguments[2].column; const IColumn & col_power = *arguments[3].column; diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index 49c45d0c0be..8ea2a91f2de 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -198,7 +198,7 @@ public: if (cond_col->onlyNull()) continue; - if (const auto * column_const = checkAndGetColumn(*cond_col)) + if (const auto * column_const = checkAndGetColumn(&*cond_col)) { Field value = column_const->getField(); diff --git a/src/Functions/readWkt.cpp b/src/Functions/readWkt.cpp index 8dff297bcb1..ddc847b1ca5 100644 --- a/src/Functions/readWkt.cpp +++ b/src/Functions/readWkt.cpp @@ -51,14 +51,14 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override { - const auto * column_string = checkAndGetColumn(arguments[0].column.get()); + const auto & column_string = checkAndGetColumn(*arguments[0].column); Serializer serializer; Geometry geometry; for (size_t i = 0; i < input_rows_count; ++i) { - const auto & str = column_string->getDataAt(i).toString(); + const auto & str = column_string.getDataAt(i).toString(); boost::geometry::read_wkt(str, geometry); serializer.add(geometry); } diff --git a/src/Functions/repeat.cpp b/src/Functions/repeat.cpp index 6f2078b7e48..84597f4eadc 100644 --- a/src/Functions/repeat.cpp +++ b/src/Functions/repeat.cpp @@ -238,9 +238,9 @@ public: { using DataType = std::decay_t; using T = typename DataType::FieldType; - const ColumnVector * column = checkAndGetColumn>(col_num.get()); + const ColumnVector & column = checkAndGetColumn>(*col_num); auto col_res = ColumnString::create(); - RepeatImpl::vectorStrVectorRepeat(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), column->getData()); + RepeatImpl::vectorStrVectorRepeat(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), column.getData()); res = std::move(col_res); return true; })) @@ -258,9 +258,9 @@ public: { using DataType = std::decay_t; using T = typename DataType::FieldType; - const ColumnVector * column = checkAndGetColumn>(col_num.get()); + const ColumnVector & column = checkAndGetColumn>(*col_num); auto col_res = ColumnString::create(); - RepeatImpl::constStrVectorRepeat(copy_str, col_res->getChars(), col_res->getOffsets(), column->getData()); + RepeatImpl::constStrVectorRepeat(copy_str, col_res->getChars(), col_res->getOffsets(), column.getData()); res = std::move(col_res); return true; })) diff --git a/src/Functions/seriesOutliersDetectTukey.cpp b/src/Functions/seriesOutliersDetectTukey.cpp index da04d3b78d3..81fc904e16e 100644 --- a/src/Functions/seriesOutliersDetectTukey.cpp +++ b/src/Functions/seriesOutliersDetectTukey.cpp @@ -61,10 +61,10 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { ColumnPtr col = arguments[0].column; - const ColumnArray * col_arr = checkAndGetColumn(col.get()); + const ColumnArray & col_arr = checkAndGetColumn(*col); - const IColumn & arr_data = col_arr->getData(); - const ColumnArray::Offsets & arr_offsets = col_arr->getOffsets(); + const IColumn & arr_data = col_arr.getData(); + const ColumnArray::Offsets & arr_offsets = col_arr.getOffsets(); ColumnPtr col_res; if (input_rows_count == 0) diff --git a/src/Functions/seriesPeriodDetectFFT.cpp b/src/Functions/seriesPeriodDetectFFT.cpp index fbaa2b14e64..e85b3a97c67 100644 --- a/src/Functions/seriesPeriodDetectFFT.cpp +++ b/src/Functions/seriesPeriodDetectFFT.cpp @@ -61,10 +61,10 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { ColumnPtr array_ptr = arguments[0].column; - const ColumnArray * array = checkAndGetColumn(array_ptr.get()); + const ColumnArray & array = checkAndGetColumn(*array_ptr); - const IColumn & src_data = array->getData(); - const ColumnArray::Offsets & offsets = array->getOffsets(); + const IColumn & src_data = array.getData(); + const ColumnArray::Offsets & offsets = array.getOffsets(); auto res = ColumnFloat64::create(input_rows_count); auto & res_data = res->getData(); diff --git a/src/Functions/space.cpp b/src/Functions/space.cpp index 03dc0d06719..4cfa629aa33 100644 --- a/src/Functions/space.cpp +++ b/src/Functions/space.cpp @@ -57,14 +57,14 @@ public: template bool executeConstant(ColumnPtr col_times, ColumnString::Offsets & res_offsets, ColumnString::Chars & res_chars) const { - const ColumnConst * col_times_const = checkAndGetColumn(col_times.get()); + const ColumnConst & col_times_const = checkAndGetColumn(*col_times); - const ColumnPtr & col_times_const_internal = col_times_const->getDataColumnPtr(); + const ColumnPtr & col_times_const_internal = col_times_const.getDataColumnPtr(); if (!checkAndGetColumn(col_times_const_internal.get())) return false; using T = typename DataType::FieldType; - T times = col_times_const->getValue(); + T times = col_times_const.getValue(); if (times < 1) times = 0; diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 7f25a317466..54aa1205a35 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -164,7 +164,7 @@ private: if (isDateTime64(time_column_type)) { - const auto * time_column_vec = checkAndGetColumn(time_column_col); + const auto * time_column_vec = checkAndGetColumn(&time_column_col); auto scale = assert_cast(time_column_type).getScale(); if (time_column_vec) @@ -172,13 +172,13 @@ private: } else if (isDateTime(time_column_type)) { - const auto * time_column_vec = checkAndGetColumn(time_column_col); + const auto * time_column_vec = checkAndGetColumn(&time_column_col); if (time_column_vec) return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, result_type, time_zone); } else if (isDate(time_column_type)) { - const auto * time_column_vec = checkAndGetColumn(time_column_col); + const auto * time_column_vec = checkAndGetColumn(&time_column_col); if (time_column_vec) return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, result_type, time_zone); } diff --git a/src/Functions/ztest.cpp b/src/Functions/ztest.cpp index 55e1b59a897..c2579263674 100644 --- a/src/Functions/ztest.cpp +++ b/src/Functions/ztest.cpp @@ -98,23 +98,23 @@ public: static const auto uint64_data_type = std::make_shared>(); auto column_successes_x = castColumnAccurate(arguments[0], uint64_data_type); - const auto & data_successes_x = checkAndGetColumn>(column_successes_x.get())->getData(); + const auto & data_successes_x = checkAndGetColumn>(*column_successes_x).getData(); auto column_successes_y = castColumnAccurate(arguments[1], uint64_data_type); - const auto & data_successes_y = checkAndGetColumn>(column_successes_y.get())->getData(); + const auto & data_successes_y = checkAndGetColumn>(*column_successes_y).getData(); auto column_trials_x = castColumnAccurate(arguments[2], uint64_data_type); - const auto & data_trials_x = checkAndGetColumn>(column_trials_x.get())->getData(); + const auto & data_trials_x = checkAndGetColumn>(*column_trials_x).getData(); auto column_trials_y = castColumnAccurate(arguments[3], uint64_data_type); - const auto & data_trials_y = checkAndGetColumn>(column_trials_y.get())->getData(); + const auto & data_trials_y = checkAndGetColumn>(*column_trials_y).getData(); static const auto float64_data_type = std::make_shared>(); auto column_confidence_level = castColumnAccurate(arguments[4], float64_data_type); - const auto & data_confidence_level = checkAndGetColumn>(column_confidence_level.get())->getData(); + const auto & data_confidence_level = checkAndGetColumn>(*column_confidence_level).getData(); - String usevar = checkAndGetColumnConst(arguments[5].column.get())->getValue(); + String usevar = checkAndGetColumnConst(*arguments[5].column).getValue(); if (usevar != UNPOOLED && usevar != POOLED) throw Exception{ErrorCodes::BAD_ARGUMENTS, diff --git a/src/Interpreters/BloomFilterHash.h b/src/Interpreters/BloomFilterHash.h index 45098ecff99..8248e9e4469 100644 --- a/src/Interpreters/BloomFilterHash.h +++ b/src/Interpreters/BloomFilterHash.h @@ -108,7 +108,7 @@ struct BloomFilterHash { const auto * array_col = typeid_cast(column.get()); - if (checkAndGetColumn(array_col->getData())) + if (checkAndGetColumn(&array_col->getData())) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected type {} of bloom filter index.", data_type->getName()); const auto & offsets = array_col->getOffsets(); diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 9b05edbce36..0db998c14fc 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -216,7 +216,7 @@ static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nulla { /// We have to replace values masked by NULLs with defaults. if (column.column) - if (const auto * nullable_column = checkAndGetColumn(*column.column)) + if (const auto * nullable_column = checkAndGetColumn(&*column.column)) column.column = JoinCommon::filterWithBlanks(column.column, nullable_column->getNullMapColumn().getData(), true); JoinCommon::removeColumnNullability(column); diff --git a/src/Interpreters/InterpreterCheckQuery.cpp b/src/Interpreters/InterpreterCheckQuery.cpp index ae8cef3f102..4a84a7bf570 100644 --- a/src/Interpreters/InterpreterCheckQuery.cpp +++ b/src/Interpreters/InterpreterCheckQuery.cpp @@ -334,10 +334,10 @@ public: if ((columns.size() != 3 && columns.size() != 5) || column_position_to_check >= columns.size()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong number of columns: {}, position {}", columns.size(), column_position_to_check); - const auto * col = checkAndGetColumn(columns[column_position_to_check].get()); - for (size_t i = 0; i < col->size(); ++i) + const auto & col = checkAndGetColumn(*columns[column_position_to_check]); + for (size_t i = 0; i < col.size(); ++i) { - if (col->getElement(i) == 0) + if (col.getElement(i) == 0) { result_value = 0; return; diff --git a/src/Interpreters/JoinUtils.cpp b/src/Interpreters/JoinUtils.cpp index 0aee96ee9c4..1788c9aca48 100644 --- a/src/Interpreters/JoinUtils.cpp +++ b/src/Interpreters/JoinUtils.cpp @@ -162,7 +162,7 @@ static ColumnPtr tryConvertColumnToNullable(ColumnPtr col) return col_lc.cloneNullable(); } } - else if (const ColumnConst * col_const = checkAndGetColumn(*col)) + else if (const ColumnConst * col_const = checkAndGetColumn(&*col)) { const auto & nested = col_const->getDataColumnPtr(); if (nested->isNullable() || nested->canBeInsideNullable()) @@ -232,7 +232,7 @@ void removeColumnNullability(ColumnWithTypeAndName & column) if (column.column && column.column->isNullable()) { column.column = column.column->convertToFullColumnIfConst(); - const auto * nullable_col = checkAndGetColumn(*column.column); + const auto * nullable_col = checkAndGetColumn(column.column.get()); if (!nullable_col) { throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Column '{}' is expected to be nullable", column.dumpStructure()); @@ -258,11 +258,11 @@ void changeColumnRepresentation(const ColumnPtr & src_column, ColumnPtr & dst_co if (nullable_src && !nullable_dst) { - const auto * nullable = checkAndGetColumn(*src_column); + const auto & nullable = checkAndGetColumn(*src_column); if (change_lowcard) - dst_column = changeLowCardinality(nullable->getNestedColumnPtr(), dst_column); + dst_column = changeLowCardinality(nullable.getNestedColumnPtr(), dst_column); else - dst_column = nullable->getNestedColumnPtr(); + dst_column = nullable.getNestedColumnPtr(); } else if (!nullable_src && nullable_dst) { @@ -275,7 +275,7 @@ void changeColumnRepresentation(const ColumnPtr & src_column, ColumnPtr & dst_co { if (change_lowcard) { - if (const auto * nullable = checkAndGetColumn(*src_column)) + if (const auto * nullable = checkAndGetColumn(&*src_column)) { dst_column = makeNullable(changeLowCardinality(nullable->getNestedColumnPtr(), dst_not_null)); assert_cast(*dst_column->assumeMutable()).applyNullMap(nullable->getNullMapColumn()); @@ -291,7 +291,7 @@ void changeColumnRepresentation(const ColumnPtr & src_column, ColumnPtr & dst_co ColumnPtr emptyNotNullableClone(const ColumnPtr & column) { if (column->isNullable()) - return checkAndGetColumn(*column)->getNestedColumnPtr()->cloneEmpty(); + return checkAndGetColumn(*column).getNestedColumnPtr()->cloneEmpty(); return column->cloneEmpty(); } @@ -374,10 +374,10 @@ ColumnRawPtrs extractKeysForJoin(const Block & block_keys, const Names & key_nam key_columns[i] = block_keys.getByName(column_name).column.get(); /// We will join only keys, where all components are not NULL. - if (const auto * nullable = checkAndGetColumn(*key_columns[i])) + if (const auto * nullable = checkAndGetColumn(&*key_columns[i])) key_columns[i] = &nullable->getNestedColumn(); - if (const auto * sparse = checkAndGetColumn(*key_columns[i])) + if (const auto * sparse = checkAndGetColumn(&*key_columns[i])) key_columns[i] = &sparse->getValuesColumn(); } @@ -490,7 +490,7 @@ JoinMask getColumnAsMask(const Block & block, const String & column_name) if (isNothing(col_type)) return JoinMask(false, block.rows()); - if (const auto * const_cond = checkAndGetColumn(*src_col.column)) + if (const auto * const_cond = checkAndGetColumn(&*src_col.column)) { return JoinMask(const_cond->getBool(0), block.rows()); } diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 5bd49b3c971..c6df06cfac6 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -80,8 +80,8 @@ int nullableCompareAt(const IColumn & left_column, const IColumn & right_column, if constexpr (has_left_nulls && has_right_nulls) { - const auto * left_nullable = checkAndGetColumn(left_column); - const auto * right_nullable = checkAndGetColumn(right_column); + const auto * left_nullable = checkAndGetColumn(&left_column); + const auto * right_nullable = checkAndGetColumn(&right_column); if (left_nullable && right_nullable) { @@ -99,7 +99,7 @@ int nullableCompareAt(const IColumn & left_column, const IColumn & right_column, if constexpr (has_left_nulls) { - if (const auto * left_nullable = checkAndGetColumn(left_column)) + if (const auto * left_nullable = checkAndGetColumn(&left_column)) { if (left_column.isNullAt(lhs_pos)) return null_direction_hint; @@ -109,7 +109,7 @@ int nullableCompareAt(const IColumn & left_column, const IColumn & right_column, if constexpr (has_right_nulls) { - if (const auto * right_nullable = checkAndGetColumn(right_column)) + if (const auto * right_nullable = checkAndGetColumn(&right_column)) { if (right_column.isNullAt(rhs_pos)) return -null_direction_hint; diff --git a/src/Interpreters/NullableUtils.cpp b/src/Interpreters/NullableUtils.cpp index ce681b1d569..fa0ddae8c90 100644 --- a/src/Interpreters/NullableUtils.cpp +++ b/src/Interpreters/NullableUtils.cpp @@ -12,7 +12,7 @@ ColumnPtr extractNestedColumnsAndNullMap(ColumnRawPtrs & key_columns, ConstNullM if (key_columns.size() == 1) { auto & column = key_columns[0]; - if (const auto * column_nullable = checkAndGetColumn(*column)) + if (const auto * column_nullable = checkAndGetColumn(&*column)) { null_map_holder = column_nullable->getNullMapColumnPtr(); null_map = &column_nullable->getNullMapData(); @@ -23,7 +23,7 @@ ColumnPtr extractNestedColumnsAndNullMap(ColumnRawPtrs & key_columns, ConstNullM { for (auto & column : key_columns) { - if (const auto * column_nullable = checkAndGetColumn(*column)) + if (const auto * column_nullable = checkAndGetColumn(&*column)) { column = &column_nullable->getNestedColumn(); diff --git a/src/Interpreters/SetVariants.cpp b/src/Interpreters/SetVariants.cpp index 0fb2e5189d4..64796a013f1 100644 --- a/src/Interpreters/SetVariants.cpp +++ b/src/Interpreters/SetVariants.cpp @@ -74,7 +74,7 @@ typename SetVariantsTemplate::Type SetVariantsTemplate::choose for (const auto & col : key_columns) { - if (const auto * nullable = checkAndGetColumn(*col)) + if (const auto * nullable = checkAndGetColumn(&*col)) { nested_key_columns.push_back(&nullable->getNestedColumn()); has_nullable_key = true; diff --git a/src/Interpreters/SetVariants.h b/src/Interpreters/SetVariants.h index ff527102080..f6eac517349 100644 --- a/src/Interpreters/SetVariants.h +++ b/src/Interpreters/SetVariants.h @@ -80,7 +80,7 @@ protected: for (const auto & col : key_columns) { - if (const auto * nullable = checkAndGetColumn(*col)) + if (const auto * nullable = checkAndGetColumn(&*col)) { actual_columns.push_back(&nullable->getNestedColumn()); null_maps.push_back(&nullable->getNullMapColumn()); diff --git a/src/Processors/Transforms/CheckConstraintsTransform.cpp b/src/Processors/Transforms/CheckConstraintsTransform.cpp index 3a6595ea4fb..e43aa6028da 100644 --- a/src/Processors/Transforms/CheckConstraintsTransform.cpp +++ b/src/Processors/Transforms/CheckConstraintsTransform.cpp @@ -57,7 +57,7 @@ void CheckConstraintsTransform::onConsume(Chunk chunk) auto result_column = res_column.column->convertToFullColumnIfConst()->convertToFullColumnIfLowCardinality(); - if (const auto * column_nullable = checkAndGetColumn(*result_column)) + if (const auto * column_nullable = checkAndGetColumn(&*result_column)) { const auto & nested_column = column_nullable->getNestedColumnPtr(); diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 584125b046f..92f4110e9ed 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -48,8 +48,8 @@ int nullableCompareAt(const IColumn & left_column, const IColumn & right_column, { if constexpr (has_left_nulls && has_right_nulls) { - const auto * left_nullable = checkAndGetColumn(left_column); - const auto * right_nullable = checkAndGetColumn(right_column); + const auto * left_nullable = checkAndGetColumn(&left_column); + const auto * right_nullable = checkAndGetColumn(&right_column); if (left_nullable && right_nullable) { @@ -67,7 +67,7 @@ int nullableCompareAt(const IColumn & left_column, const IColumn & right_column, if constexpr (has_left_nulls) { - if (const auto * left_nullable = checkAndGetColumn(left_column)) + if (const auto * left_nullable = checkAndGetColumn(&left_column)) { if (left_nullable->isNullAt(lhs_pos)) return null_direction_hint; @@ -77,7 +77,7 @@ int nullableCompareAt(const IColumn & left_column, const IColumn & right_column, if constexpr (has_right_nulls) { - if (const auto * right_nullable = checkAndGetColumn(right_column)) + if (const auto * right_nullable = checkAndGetColumn(&right_column)) { if (right_nullable->isNullAt(rhs_pos)) return -null_direction_hint; diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index f43b9a2e794..cf6a84db9e6 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -2516,7 +2516,7 @@ struct WindowFunctionNonNegativeDerivative final : public StatefulWindowFunction if (ts_scale_multiplier) { const auto & column = transform->blockAt(transform->current_row.block).input_columns[workspace.argument_column_indices[ARGUMENT_TIMESTAMP]]; - const auto & curr_timestamp = checkAndGetColumn(column.get())->getInt(transform->current_row.row); + const auto & curr_timestamp = checkAndGetColumn(*column).getInt(transform->current_row.row); Float64 time_elapsed = curr_timestamp - state.previous_timestamp; result = (time_elapsed > 0) ? (metric_diff * ts_scale_multiplier / time_elapsed * interval_duration) : 0; diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index dba2bc1e56c..1bd42518fdd 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -305,7 +305,7 @@ bool MergeTreeIndexConditionSet::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx const NullMap * null_map = nullptr; - if (const auto * col_nullable = checkAndGetColumn(*column)) + if (const auto * col_nullable = checkAndGetColumn(&*column)) { col_uint8 = typeid_cast(&col_nullable->getNestedColumn()); null_map = &col_nullable->getNullMapData(); From 694a8ca81992210dabfcf2da32004899263ae284 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 29 Mar 2024 23:33:12 +0000 Subject: [PATCH 0859/1018] Style --- src/Functions/FunctionHelpers.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Functions/FunctionHelpers.h b/src/Functions/FunctionHelpers.h index 89b224e79ef..9eabb9a0370 100644 --- a/src/Functions/FunctionHelpers.h +++ b/src/Functions/FunctionHelpers.h @@ -15,6 +15,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + class IFunction; /// Methods, that helps dispatching over real column types. From 4b9819a8f5d6b5407486a8d25bcf24dfdee8950f Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 6 May 2024 22:25:32 +0000 Subject: [PATCH 0860/1018] Update call sites added since last commit --- src/Functions/UTCTimestampTransform.cpp | 12 ++++++------ src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp | 4 ++-- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Functions/UTCTimestampTransform.cpp b/src/Functions/UTCTimestampTransform.cpp index 0fcba91f49f..6d301270d8e 100644 --- a/src/Functions/UTCTimestampTransform.cpp +++ b/src/Functions/UTCTimestampTransform.cpp @@ -80,14 +80,14 @@ namespace const DateLUTImpl & utc_time_zone = DateLUT::instance("UTC"); if (WhichDataType(arg1.type).isDateTime()) { - const auto * date_time_col = checkAndGetColumn(arg1.column.get()); - size_t col_size = date_time_col->size(); + const auto & date_time_col = checkAndGetColumn(*arg1.column); + size_t col_size = date_time_col.size(); using ColVecTo = DataTypeDateTime::ColumnType; typename ColVecTo::MutablePtr result_column = ColVecTo::create(col_size); typename ColVecTo::Container & result_data = result_column->getData(); for (size_t i = 0; i < col_size; ++i) { - UInt32 date_time_val = date_time_col->getElement(i); + UInt32 date_time_val = date_time_col.getElement(i); LocalDateTime date_time(date_time_val, Name::to ? utc_time_zone : DateLUT::instance(time_zone_val)); time_t time_val = date_time.to_time_t(Name::from ? utc_time_zone : DateLUT::instance(time_zone_val)); result_data[i] = static_cast(time_val); @@ -96,8 +96,8 @@ namespace } else if (WhichDataType(arg1.type).isDateTime64()) { - const auto * date_time_col = checkAndGetColumn(arg1.column.get()); - size_t col_size = date_time_col->size(); + const auto & date_time_col = checkAndGetColumn(*arg1.column); + size_t col_size = date_time_col.size(); const DataTypeDateTime64 * date_time_type = static_cast(arg1.type.get()); UInt32 col_scale = date_time_type->getScale(); Int64 scale_multiplier = DecimalUtils::scaleMultiplier(col_scale); @@ -106,7 +106,7 @@ namespace typename ColDecimalTo::Container & result_data = result_column->getData(); for (size_t i = 0; i < col_size; ++i) { - DateTime64 date_time_val = date_time_col->getElement(i); + DateTime64 date_time_val = date_time_col.getElement(i); Int64 seconds = date_time_val.value / scale_multiplier; Int64 micros = date_time_val.value % scale_multiplier; LocalDateTime date_time(seconds, Name::to ? utc_time_zone : DateLUT::instance(time_zone_val)); diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp index 4f25a014382..4b164f5ac42 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp @@ -865,8 +865,8 @@ void MergeTreeIndexAggregatorBloomFilter::update(const Block & block, size_t * p const auto & column_and_type = block.getByName(index_columns_name[column]); auto index_column = BloomFilterHash::hashWithColumn(column_and_type.type, column_and_type.column, *pos, max_read_rows); - const auto & index_col = checkAndGetColumn(index_column.get()); - const auto & index_data = index_col->getData(); + const auto & index_col = checkAndGetColumn(*index_column); + const auto & index_data = index_col.getData(); for (const auto & hash: index_data) column_hashes[column].insert(hash); } From a20ef2a3d07343ff914f01db39b8dbb7e02d7584 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=9A=D0=B8=D1=80=D0=B8=D0=BB=D0=BB=20=D0=93=D0=B0=D1=80?= =?UTF-8?q?=D0=B1=D0=B0=D1=80?= Date: Tue, 7 May 2024 02:06:13 +0300 Subject: [PATCH 0861/1018] Add allow without connection setting to MaterializedMySQLSettings --- .../database-engines/materialized-mysql.md | 3 +++ src/Databases/MySQL/DatabaseMaterializedMySQL.cpp | 6 +++++- src/Databases/MySQL/MaterializedMySQLSettings.h | 1 + .../materialized_with_ddl.py | 15 ++++++++++++++- 4 files changed, 23 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index f32698f84f6..2b4d5fe04aa 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -51,6 +51,9 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo ### allows_query_when_mysql_lost `allows_query_when_mysql_lost` — Allows to query a materialized table when MySQL is lost. Default: `0` (`false`). +### allow_startup_database_without_connection_to_mysql +`allow_startup_database_without_connection_to_mysql` — Allow to create and attach database without available connection to MySQL. Default: `0` (`false`). + ### materialized_mysql_tables_list `materialized_mysql_tables_list` — a comma-separated list of mysql database tables, which will be replicated by MaterializedMySQL database engine. Default value: empty list — means whole tables will be replicated. diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp index 6d89cc23590..c81fe1b2b26 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp @@ -81,9 +81,13 @@ LoadTaskPtr DatabaseMaterializedMySQL::startupDatabaseAsync(AsyncLoader & async_ base->goals(), TablesLoaderBackgroundStartupPoolId, fmt::format("startup MaterializedMySQL database {}", getDatabaseName()), - [this] (AsyncLoader &, const LoadJobPtr &) + [this, mode] (AsyncLoader &, const LoadJobPtr &) { LOG_TRACE(log, "Starting MaterializeMySQL database"); + if (!settings->allow_startup_database_without_connection_to_mysql + && mode < LoadingStrictnessLevel::FORCE_ATTACH) + materialize_thread.assertMySQLAvailable(); + materialize_thread.startSynchronization(); started_up = true; }); diff --git a/src/Databases/MySQL/MaterializedMySQLSettings.h b/src/Databases/MySQL/MaterializedMySQLSettings.h index 557d48be85b..b481846afc1 100644 --- a/src/Databases/MySQL/MaterializedMySQLSettings.h +++ b/src/Databases/MySQL/MaterializedMySQLSettings.h @@ -22,6 +22,7 @@ class ASTStorage; M(UInt64, max_milliseconds_to_wait_in_binlog_queue, 10000, "Max milliseconds to wait when max bytes exceeded in a binlog queue.", 0) \ M(UInt64, max_bytes_in_binlog_dispatcher_buffer, DBMS_DEFAULT_BUFFER_SIZE, "Max bytes in the binlog dispatcher's buffer before it is flushed to attached binlogs.", 0) \ M(UInt64, max_flush_milliseconds_in_binlog_dispatcher, 1000, "Max milliseconds in the binlog dispatcher's buffer to wait before it is flushed to attached binlogs.", 0) \ + M(Bool, allow_startup_database_without_connection_to_mysql, false, "Allow to create and attach database without available connection to MySQL.", 0) \ DECLARE_SETTINGS_TRAITS(MaterializedMySQLSettingsTraits, LIST_OF_MATERIALIZE_MODE_SETTINGS) diff --git a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py index dcb2546bad3..86000799ae4 100644 --- a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py @@ -3425,8 +3425,21 @@ def mysql_create_database_without_connection(clickhouse_node, mysql_node, servic clickhouse_node.cluster.pause_container(service_name) + assert "ConnectionFailed:" in clickhouse_node.query_and_get_error( + """ + CREATE DATABASE create_without_connection + ENGINE = MaterializedMySQL('{}:3306', 'create_without_connection', 'root', 'clickhouse') + """.format( + service_name + ) + ) + clickhouse_node.query( - "CREATE DATABASE create_without_connection ENGINE = MaterializedMySQL('{}:3306', 'create_without_connection', 'root', 'clickhouse') SETTINGS max_wait_time_when_mysql_unavailable=-1".format( + """ + CREATE DATABASE create_without_connection + ENGINE = MaterializedMySQL('{}:3306', 'create_without_connection', 'root', 'clickhouse') + SETTINGS allow_startup_database_without_connection_to_mysql=1 + """.format( service_name ) ) From c53c8eb6d178329be8b7f11bfa09521545624384 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 7 May 2024 01:36:20 +0200 Subject: [PATCH 0862/1018] Fix slow suggest --- src/Access/ContextAccess.cpp | 8 +---- src/Interpreters/Context.cpp | 6 ++-- src/Storages/System/StorageSystemColumns.cpp | 18 ++++++---- .../System/StorageSystemDatabases.cpp | 4 +-- src/Storages/System/StorageSystemTables.cpp | 6 ++-- ...147_system_columns_access_checks.reference | 2 ++ .../03147_system_columns_access_checks.sh | 36 +++++++++++++++++++ 7 files changed, 58 insertions(+), 22 deletions(-) create mode 100644 tests/queries/0_stateless/03147_system_columns_access_checks.reference create mode 100755 tests/queries/0_stateless/03147_system_columns_access_checks.sh diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 2736d13e751..2a658d7aaa2 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -570,11 +570,8 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg if (params.full_access) return true; - auto access_granted = [&] + auto access_granted = [] { - if (trace_log) - LOG_TRACE(trace_log, "Access granted: {}{}", (AccessRightsElement{flags, args...}.toStringWithoutOptions()), - (grant_option ? " WITH GRANT OPTION" : "")); return true; }; @@ -582,9 +579,6 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg FormatStringHelper fmt_string [[maybe_unused]], FmtArgs && ...fmt_args [[maybe_unused]]) { - if (trace_log) - LOG_TRACE(trace_log, "Access denied: {}{}", (AccessRightsElement{flags, args...}.toStringWithoutOptions()), - (grant_option ? " WITH GRANT OPTION" : "")); if constexpr (throw_if_denied) throw Exception(error_code, std::move(fmt_string), getUserName(), std::forward(fmt_args)...); return false; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 44d36e94441..d22cdccc722 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2498,7 +2498,7 @@ AsyncLoader & Context::getAsyncLoader() const shared->async_loader = std::make_unique(std::vector{ // IMPORTANT: Pool declaration order should match the order in `PoolId.h` to get the indices right. { // TablesLoaderForegroundPoolId - "FgLoad", + "ForegroundLoad", CurrentMetrics::TablesLoaderForegroundThreads, CurrentMetrics::TablesLoaderForegroundThreadsActive, CurrentMetrics::TablesLoaderForegroundThreadsScheduled, @@ -2506,7 +2506,7 @@ AsyncLoader & Context::getAsyncLoader() const TablesLoaderForegroundPriority }, { // TablesLoaderBackgroundLoadPoolId - "BgLoad", + "BackgroundLoad", CurrentMetrics::TablesLoaderBackgroundThreads, CurrentMetrics::TablesLoaderBackgroundThreadsActive, CurrentMetrics::TablesLoaderBackgroundThreadsScheduled, @@ -2514,7 +2514,7 @@ AsyncLoader & Context::getAsyncLoader() const TablesLoaderBackgroundLoadPriority }, { // TablesLoaderBackgroundStartupPoolId - "BgStartup", + "BackgroundStartup", CurrentMetrics::TablesLoaderBackgroundThreads, CurrentMetrics::TablesLoaderBackgroundThreadsActive, CurrentMetrics::TablesLoaderBackgroundThreadsScheduled, diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 8c6d29a3b70..15b73fd1884 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -88,6 +88,7 @@ public: , total_tables(tables->size()), access(context->getAccess()) , query_id(context->getCurrentQueryId()), lock_acquire_timeout(context->getSettingsRef().lock_acquire_timeout) { + need_to_check_access_for_tables = !access->isGranted(AccessType::SHOW_COLUMNS); } String getName() const override { return "Columns"; } @@ -101,8 +102,6 @@ protected: MutableColumns res_columns = getPort().getHeader().cloneEmptyColumns(); size_t rows_count = 0; - const bool check_access_for_tables = !access->isGranted(AccessType::SHOW_COLUMNS); - while (rows_count < max_block_size && db_table_num < total_tables) { const std::string database_name = (*databases)[db_table_num].get(); @@ -138,13 +137,17 @@ protected: column_sizes = storage->getColumnSizes(); } - bool check_access_for_columns = check_access_for_tables && !access->isGranted(AccessType::SHOW_COLUMNS, database_name, table_name); + /// A shortcut: if we don't allow to list this table in SHOW TABLES, also exclude it from system.columns. + if (need_to_check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name)) + continue; + + bool need_to_check_access_for_columns = need_to_check_access_for_tables && !access->isGranted(AccessType::SHOW_COLUMNS, database_name, table_name); size_t position = 0; for (const auto & column : columns) { ++position; - if (check_access_for_columns && !access->isGranted(AccessType::SHOW_COLUMNS, database_name, table_name, column.name)) + if (need_to_check_access_for_columns && !access->isGranted(AccessType::SHOW_COLUMNS, database_name, table_name, column.name)) continue; size_t src_index = 0; @@ -296,6 +299,7 @@ private: size_t db_table_num = 0; size_t total_tables; std::shared_ptr access; + bool need_to_check_access_for_tables; String query_id; std::chrono::milliseconds lock_acquire_timeout; }; @@ -358,7 +362,6 @@ void StorageSystemColumns::read( auto [columns_mask, header] = getQueriedColumnsMaskAndHeader(sample_block, column_names); - auto this_ptr = std::static_pointer_cast(shared_from_this()); auto reading = std::make_unique( @@ -416,9 +419,10 @@ void ReadFromSystemColumns::initializePipeline(QueryPipelineBuilder & pipeline, /// Add `table` column. MutableColumnPtr table_column_mut = ColumnString::create(); - IColumn::Offsets offsets(database_column->size()); + size_t num_databases = database_column->size(); + IColumn::Offsets offsets(num_databases); - for (size_t i = 0; i < database_column->size(); ++i) + for (size_t i = 0; i < num_databases; ++i) { const std::string database_name = (*database_column)[i].get(); if (database_name.empty()) diff --git a/src/Storages/System/StorageSystemDatabases.cpp b/src/Storages/System/StorageSystemDatabases.cpp index 2351c3c6a2a..1dbb187c418 100644 --- a/src/Storages/System/StorageSystemDatabases.cpp +++ b/src/Storages/System/StorageSystemDatabases.cpp @@ -102,7 +102,7 @@ static ColumnPtr getFilteredDatabases(const Databases & databases, const Actions void StorageSystemDatabases::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector columns_mask) const { const auto access = context->getAccess(); - const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_DATABASES); + const bool need_to_check_access_for_databases = !access->isGranted(AccessType::SHOW_DATABASES); const auto databases = DatabaseCatalog::instance().getDatabases(); ColumnPtr filtered_databases_column = getFilteredDatabases(databases, predicate, context); @@ -111,7 +111,7 @@ void StorageSystemDatabases::fillData(MutableColumns & res_columns, ContextPtr c { auto database_name = filtered_databases_column->getDataAt(i).toString(); - if (check_access_for_databases && !access->isGranted(AccessType::SHOW_DATABASES, database_name)) + if (need_to_check_access_for_databases && !access->isGranted(AccessType::SHOW_DATABASES, database_name)) continue; if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 9bd7ff945ad..d428d6bd6d0 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -224,7 +224,7 @@ protected: MutableColumns res_columns = getPort().getHeader().cloneEmptyColumns(); const auto access = context->getAccess(); - const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); + const bool need_to_check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); size_t rows_count = 0; while (rows_count < max_block_size) @@ -348,7 +348,7 @@ protected: return Chunk(std::move(res_columns), num_rows); } - const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name); + const bool need_to_check_access_for_tables = need_to_check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name); if (!tables_it || !tables_it->isValid()) tables_it = database->getTablesIterator(context); @@ -361,7 +361,7 @@ protected: if (!tables.contains(table_name)) continue; - if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name)) + if (need_to_check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name)) continue; StoragePtr table = nullptr; diff --git a/tests/queries/0_stateless/03147_system_columns_access_checks.reference b/tests/queries/0_stateless/03147_system_columns_access_checks.reference new file mode 100644 index 00000000000..35438f11b31 --- /dev/null +++ b/tests/queries/0_stateless/03147_system_columns_access_checks.reference @@ -0,0 +1,2 @@ +........................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................ +end diff --git a/tests/queries/0_stateless/03147_system_columns_access_checks.sh b/tests/queries/0_stateless/03147_system_columns_access_checks.sh new file mode 100755 index 00000000000..b204e7b28d8 --- /dev/null +++ b/tests/queries/0_stateless/03147_system_columns_access_checks.sh @@ -0,0 +1,36 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-parallel + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# Create many tables in the database +NUM_TABLES=1000 +NUM_COLUMNS=1000 +THREADS=$(nproc) + +COLUMNS=$(seq 1 $NUM_COLUMNS | sed -r -e 's/(.+)/c\1 UInt8, /' | tr -d '\n') + +seq 1 $NUM_TABLES | xargs -P "${THREADS}" -I{} bash -c " + echo -n '.' + $CLICKHOUSE_CLIENT --query 'CREATE OR REPLACE TABLE test{} (${COLUMNS} end String) ENGINE = Memory' +" +echo + +$CLICKHOUSE_CLIENT --multiquery " +DROP USER IF EXISTS test_03147; +CREATE USER test_03147; +GRANT SELECT (end) ON ${CLICKHOUSE_DATABASE}.test1 TO test_03147; +" + +# This query was slow in previous ClickHouse versions for several reasons: +# - tables and databases without SHOW TABLES access were still checked for SHOW COLUMNS access for every column in every table; +# - excessive logging of "access granted" and "access denied" + +# The test could succeed even on the previous version, but it will show up as being too slow. +$CLICKHOUSE_CLIENT --user test_03147 --query "SELECT name FROM system.columns" + +$CLICKHOUSE_CLIENT --multiquery " +DROP USER test_03147; +" From a38ea6c8cd3aa706171fa15e053670a1c49c0d5d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 7 May 2024 02:54:25 +0200 Subject: [PATCH 0863/1018] Thread names --- src/Interpreters/Context.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d22cdccc722..df49b822aaa 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2514,7 +2514,7 @@ AsyncLoader & Context::getAsyncLoader() const TablesLoaderBackgroundLoadPriority }, { // TablesLoaderBackgroundStartupPoolId - "BackgroundStartup", + "BackgrndStartup", CurrentMetrics::TablesLoaderBackgroundThreads, CurrentMetrics::TablesLoaderBackgroundThreadsActive, CurrentMetrics::TablesLoaderBackgroundThreadsScheduled, From e33d8272b03fec5ff72b098c1177130d73a8d6f2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 7 May 2024 03:31:00 +0200 Subject: [PATCH 0864/1018] Fix style --- tests/queries/0_stateless/03147_system_columns_access_checks.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03147_system_columns_access_checks.sh b/tests/queries/0_stateless/03147_system_columns_access_checks.sh index b204e7b28d8..70693e0985f 100755 --- a/tests/queries/0_stateless/03147_system_columns_access_checks.sh +++ b/tests/queries/0_stateless/03147_system_columns_access_checks.sh @@ -29,7 +29,7 @@ GRANT SELECT (end) ON ${CLICKHOUSE_DATABASE}.test1 TO test_03147; # - excessive logging of "access granted" and "access denied" # The test could succeed even on the previous version, but it will show up as being too slow. -$CLICKHOUSE_CLIENT --user test_03147 --query "SELECT name FROM system.columns" +$CLICKHOUSE_CLIENT --user test_03147 --query "SELECT name FROM system.columns WHERE database = currentDatabase()" $CLICKHOUSE_CLIENT --multiquery " DROP USER test_03147; From e4864d6fd698c6b56fe4f6591cb01ff67d20f234 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 7 May 2024 04:43:06 +0300 Subject: [PATCH 0865/1018] Update UTF8Helpers.cpp --- src/Common/UTF8Helpers.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/UTF8Helpers.cpp b/src/Common/UTF8Helpers.cpp index b68665f3277..be1f222dc96 100644 --- a/src/Common/UTF8Helpers.cpp +++ b/src/Common/UTF8Helpers.cpp @@ -150,7 +150,7 @@ size_t computeWidthImpl(const UInt8 * data, size_t size, size_t prefix, size_t l { if (!isEscapeSequence) ++width; - else if (isCSIFinalByte(data[i]) && data[i-1]!='\x1b') + else if (isCSIFinalByte(data[i]) && data[i - 1] != '\x1b') isEscapeSequence = false; /// end of CSI escape sequence reached ++i; } From d37590aed68e12c5fd7664b1a21138dd428d1482 Mon Sep 17 00:00:00 2001 From: unashi Date: Tue, 7 May 2024 10:36:44 +0800 Subject: [PATCH 0866/1018] [update] add test for RawWithNames, RawWithNamesAndTypes and *WithNames, *WithNamesAndTypes; add changelog --- CHANGELOG.md | 1 + .../00397_tsv_format_synonym.reference | 27 +++++++++++++++++++ .../0_stateless/00397_tsv_format_synonym.sql | 8 ++++++ 3 files changed, 36 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index f40c42c4462..955e2f5b72f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -33,6 +33,7 @@ * A mode for `topK`/`topkWeighed` support mode, which return count of values and its error. [#54508](https://github.com/ClickHouse/ClickHouse/pull/54508) ([UnamedRus](https://github.com/UnamedRus)). * Added function `toMillisecond` which returns the millisecond component for values of type`DateTime` or `DateTime64`. [#60281](https://github.com/ClickHouse/ClickHouse/pull/60281) ([Shaun Struwig](https://github.com/Blargian)). * Allow configuring HTTP redirect handlers for clickhouse-server. For example, you can make `/` redirect to the Play UI. [#60390](https://github.com/ClickHouse/ClickHouse/pull/60390) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Allow Raw as a synonym for TSVRaw. [#63394](https://github.com/ClickHouse/ClickHouse/pull/63394) ([Unalian](https://github.com/Unalian)) #### Performance Improvement * Optimized function `dotProduct` to omit unnecessary and expensive memory copies. [#60928](https://github.com/ClickHouse/ClickHouse/pull/60928) ([Robert Schulze](https://github.com/rschu1ze)). diff --git a/tests/queries/0_stateless/00397_tsv_format_synonym.reference b/tests/queries/0_stateless/00397_tsv_format_synonym.reference index c91169a06fa..3326b039b8d 100644 --- a/tests/queries/0_stateless/00397_tsv_format_synonym.reference +++ b/tests/queries/0_stateless/00397_tsv_format_synonym.reference @@ -31,3 +31,30 @@ UInt8 String String 1 hello world 2 hello world 3 hello world +arr s1 s2 +1 hello world +2 hello world +3 hello world +arr s1 s2 +1 hello world +2 hello world +3 hello world +arr s1 s2 +1 hello world +2 hello world +3 hello world +arr s1 s2 +UInt8 String String +1 hello world +2 hello world +3 hello world +arr s1 s2 +UInt8 String String +1 hello world +2 hello world +3 hello world +arr s1 s2 +UInt8 String String +1 hello world +2 hello world +3 hello world diff --git a/tests/queries/0_stateless/00397_tsv_format_synonym.sql b/tests/queries/0_stateless/00397_tsv_format_synonym.sql index 51283c6ced9..b3b231fbf3f 100644 --- a/tests/queries/0_stateless/00397_tsv_format_synonym.sql +++ b/tests/queries/0_stateless/00397_tsv_format_synonym.sql @@ -10,3 +10,11 @@ SELECT arrayJoin([1, 2, 3]) AS arr, 'hello' AS s1, 'world' AS s2 FORMAT TSVWithN SELECT arrayJoin([1, 2, 3]) AS arr, 'hello' AS s1, 'world' AS s2 FORMAT TabSeparatedRaw; SELECT arrayJoin([1, 2, 3]) AS arr, 'hello' AS s1, 'world' AS s2 FORMAT TSVRaw; SELECT arrayJoin([1, 2, 3]) AS arr, 'hello' AS s1, 'world' AS s2 FORMAT Raw; + +SELECT arrayJoin([1, 2, 3]) AS arr, 'hello' AS s1, 'world' AS s2 FORMAT TabSeparatedRawWithNames; +SELECT arrayJoin([1, 2, 3]) AS arr, 'hello' AS s1, 'world' AS s2 FORMAT TSVRawWithNames; +SELECT arrayJoin([1, 2, 3]) AS arr, 'hello' AS s1, 'world' AS s2 FORMAT RawWithNames; + +SELECT arrayJoin([1, 2, 3]) AS arr, 'hello' AS s1, 'world' AS s2 FORMAT TabSeparatedRawWithNamesAndTypes; +SELECT arrayJoin([1, 2, 3]) AS arr, 'hello' AS s1, 'world' AS s2 FORMAT TSVRawWithNamesAndTypes; +SELECT arrayJoin([1, 2, 3]) AS arr, 'hello' AS s1, 'world' AS s2 FORMAT RawWithNamesAndTypes; From a8ae0074aa5563b8e65ae110fa5dc71313a81a77 Mon Sep 17 00:00:00 2001 From: unashi Date: Tue, 7 May 2024 10:40:46 +0800 Subject: [PATCH 0867/1018] [fix] name->names --- docs/en/interfaces/formats.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 937dfb52609..0b108edc17f 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -206,7 +206,7 @@ SELECT * FROM nestedt FORMAT TSV Differs from `TabSeparated` format in that the rows are written without escaping. When parsing with this format, tabs or linefeeds are not allowed in each field. -This format is also available under the name `TSVRaw`, `Raw`. +This format is also available under the names `TSVRaw`, `Raw`. ## TabSeparatedWithNames {#tabseparatedwithnames} @@ -241,14 +241,14 @@ This format is also available under the name `TSVWithNamesAndTypes`. Differs from `TabSeparatedWithNames` format in that the rows are written without escaping. When parsing with this format, tabs or linefeeds are not allowed in each field. -This format is also available under the name `TSVRawWithNames`, `RawWithNames`. +This format is also available under the names `TSVRawWithNames`, `RawWithNames`. ## TabSeparatedRawWithNamesAndTypes {#tabseparatedrawwithnamesandtypes} Differs from `TabSeparatedWithNamesAndTypes` format in that the rows are written without escaping. When parsing with this format, tabs or linefeeds are not allowed in each field. -This format is also available under the name `TSVRawWithNamesAndNames`, `RawWithNamesAndNames`. +This format is also available under the names `TSVRawWithNamesAndNames`, `RawWithNamesAndNames`. ## Template {#format-template} From 98fb9f36a90fdc7220557177977e70897d1f0e88 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 7 May 2024 05:35:04 +0200 Subject: [PATCH 0868/1018] We are using Shared Catalog in the CI Logs cluster --- docker/test/base/setup_export_logs.sh | 3 --- 1 file changed, 3 deletions(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index 8858e12c50e..dc91a5c927d 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -127,9 +127,6 @@ function setup_logs_replication echo 'Create all configured system logs' clickhouse-client --query "SYSTEM FLUSH LOGS" - # It's doesn't make sense to try creating tables if SYNC fails - echo "SYSTEM SYNC DATABASE REPLICA default" | clickhouse-client "${CONNECTION_ARGS[@]}" || return 0 - debug_or_sanitizer_build=$(clickhouse-client -q "WITH ((SELECT value FROM system.build_options WHERE name='BUILD_TYPE') AS build, (SELECT value FROM system.build_options WHERE name='CXX_FLAGS') as flags) SELECT build='Debug' OR flags LIKE '%fsanitize%'") echo "Build is debug or sanitizer: $debug_or_sanitizer_build" From 0a1d852dfd52cc88502a7699d249328edb041976 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Thu, 2 May 2024 21:35:32 +0000 Subject: [PATCH 0869/1018] Enable plain_rewritable metadata for local and azure Enable plain_rewritable support for local and azure (azure_blob_storage) metadata type. - HDFS object storage currently does not support iteration and does not implement listObjects method. It's a blocker for enabling plain_rewritable metadata type with HDFS. - StaticWeb object storage is read-only and works with its dedicated metadata type. --- .../ObjectStorages/ObjectStorageFactory.cpp | 14 +++++-- .../PlainRewritableObjectStorage.h | 39 ++++++++++++++++++- .../ObjectStorages/S3/S3ObjectStorage.cpp | 7 ---- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 3 -- .../03008_local_plain_rewritable.reference | 22 +++++++++++ .../03008_local_plain_rewritable.sh | 35 +++++++++++++++++ 6 files changed, 106 insertions(+), 14 deletions(-) create mode 100644 tests/queries/0_stateless/03008_local_plain_rewritable.reference create mode 100755 tests/queries/0_stateless/03008_local_plain_rewritable.sh diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index 7b949db268b..264ec2b258e 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -73,9 +73,17 @@ ObjectStoragePtr createObjectStorage( return std::make_shared>(std::forward(args)...); else if (isPlainRewritableStorage(type, config, config_prefix)) { - /// TODO(jkartseva@): Test support for generic disk type - if (type != ObjectStorageType::S3) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "plain_rewritable metadata storage support is implemented only for S3"); + /// HDFS object storage currently does not support iteration and does not implement listObjects method. + /// StaticWeb object storage is read-only and works with its dedicated metadata type. + constexpr auto supported_object_storage_types + = std::array{ObjectStorageType::S3, ObjectStorageType::Local, ObjectStorageType::Azure}; + if (std::find(supported_object_storage_types.begin(), supported_object_storage_types.end(), type) + == supported_object_storage_types.end()) + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, + "plain_rewritable metadata storage support is not implemented for '{}' object storage", + DataSourceDescription{DataSourceType::ObjectStorage, type, MetadataStorageType::PlainRewritable, /*description*/ ""} + .toString()); return std::make_shared>(std::forward(args)...); } diff --git a/src/Disks/ObjectStorages/PlainRewritableObjectStorage.h b/src/Disks/ObjectStorages/PlainRewritableObjectStorage.h index d71e995b490..2b116cff443 100644 --- a/src/Disks/ObjectStorages/PlainRewritableObjectStorage.h +++ b/src/Disks/ObjectStorages/PlainRewritableObjectStorage.h @@ -1,16 +1,26 @@ #pragma once #include +#include +#include "CommonPathPrefixKeyGenerator.h" namespace DB { +namespace ErrorCodes +{ +extern const int LOGICAL_ERROR; +} template class PlainRewritableObjectStorage : public BaseObjectStorage { public: template - explicit PlainRewritableObjectStorage(Args &&... args) : BaseObjectStorage(std::forward(args)...) + explicit PlainRewritableObjectStorage(Args &&... args) + : BaseObjectStorage(std::forward(args)...) + /// A basic key generator is required for checking S3 capabilities, + /// it will be reset later by metadata storage. + , key_generator(createObjectStorageKeysGeneratorAsIsWithPrefix(BaseObjectStorage::getCommonKeyPrefix())) { } @@ -19,6 +29,33 @@ public: bool isWriteOnce() const override { return false; } bool isPlain() const override { return true; } + + ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override; + + ObjectStorageKey generateObjectKeyPrefixForDirectoryPath(const std::string & path) const override; + + void setKeysGenerator(ObjectStorageKeysGeneratorPtr gen) override { key_generator = gen; } + +private: + ObjectStorageKeysGeneratorPtr key_generator; }; + +template +ObjectStorageKey PlainRewritableObjectStorage::generateObjectKeyForPath(const std::string & path) const +{ + if (!key_generator) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Key generator is not set"); + + return key_generator->generate(path, /* is_directory */ false); +} + +template +ObjectStorageKey PlainRewritableObjectStorage::generateObjectKeyPrefixForDirectoryPath(const std::string & path) const +{ + if (!key_generator) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Key generator is not set"); + + return key_generator->generate(path, /* is_directory */ true); +} } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 2eae8877f87..a58b37f1df9 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -574,13 +574,6 @@ ObjectStorageKey S3ObjectStorage::generateObjectKeyForPath(const std::string & p return key_generator->generate(path, /* is_directory */ false); } -ObjectStorageKey S3ObjectStorage::generateObjectKeyPrefixForDirectoryPath(const std::string & path) const -{ - if (!key_generator) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Key generator is not set"); - - return key_generator->generate(path, /* is_directory */ true); -} } #endif diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index ff66b00e47c..5eaab4b585c 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -159,12 +159,9 @@ public: bool supportParallelWrite() const override { return true; } ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override; - ObjectStorageKey generateObjectKeyPrefixForDirectoryPath(const std::string & path) const override; bool isReadOnly() const override { return s3_settings.get()->read_only; } - void setKeysGenerator(ObjectStorageKeysGeneratorPtr gen) override { key_generator = gen; } - private: void setNewSettings(std::unique_ptr && s3_settings_); diff --git a/tests/queries/0_stateless/03008_local_plain_rewritable.reference b/tests/queries/0_stateless/03008_local_plain_rewritable.reference new file mode 100644 index 00000000000..10fc932ca4d --- /dev/null +++ b/tests/queries/0_stateless/03008_local_plain_rewritable.reference @@ -0,0 +1,22 @@ +10006 +0 0 0 +1 1 1 +1 2 0 +2 2 2 +2 2 2 +3 1 9 +3 3 3 +4 4 4 +4 7 7 +5 5 5 +10006 +0 0 0 +1 1 1 +1 2 0 +2 2 2 +2 2 2 +3 1 9 +3 3 3 +4 4 4 +4 7 7 +5 5 5 diff --git a/tests/queries/0_stateless/03008_local_plain_rewritable.sh b/tests/queries/0_stateless/03008_local_plain_rewritable.sh new file mode 100755 index 00000000000..07fd013c911 --- /dev/null +++ b/tests/queries/0_stateless/03008_local_plain_rewritable.sh @@ -0,0 +1,35 @@ +#!/usr/bin/env bash +# Tags: no-random-settings, no-replicated-database, no-shared-merge-tree +# Tag no-random-settings: enable after root causing flakiness + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query "drop table if exists test_mt sync" + +${CLICKHOUSE_CLIENT} -nm --query " +create table test_mt (a Int32, b Int64, c Int64) engine = MergeTree() partition by intDiv(a, 1000) order by tuple(a, b) +settings disk = disk( + type = object_storage, + object_storage_type = local, + metadata_type = plain_rewritable, + path = '/var/lib/clickhouse/disks/local_plain_rewritable/') +" + +${CLICKHOUSE_CLIENT} -nm --query " +insert into test_mt (*) values (1, 2, 0), (2, 2, 2), (3, 1, 9), (4, 7, 7), (5, 10, 2), (6, 12, 5); +insert into test_mt (*) select number, number, number from numbers_mt(10000); +" + +${CLICKHOUSE_CLIENT} -nm --query " +select count(*) from test_mt; +select (*) from test_mt order by tuple(a, b) limit 10; +" + +${CLICKHOUSE_CLIENT} --query "optimize table test_mt final" + +${CLICKHOUSE_CLIENT} -nm --query " +select count(*) from test_mt; +select (*) from test_mt order by tuple(a, b) limit 10; +" From c6f17b25e47ffcf96ff49f869f5ecd6b67b910b8 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Fri, 3 May 2024 03:59:50 +0000 Subject: [PATCH 0870/1018] plain_rewritable: add integration test for Azure --- .../__init__.py | 0 .../test.py | 153 ++++++++++++++++++ 2 files changed, 153 insertions(+) create mode 100644 tests/integration/test_azure_blob_storage_plain_rewritable/__init__.py create mode 100644 tests/integration/test_azure_blob_storage_plain_rewritable/test.py diff --git a/tests/integration/test_azure_blob_storage_plain_rewritable/__init__.py b/tests/integration/test_azure_blob_storage_plain_rewritable/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_azure_blob_storage_plain_rewritable/test.py b/tests/integration/test_azure_blob_storage_plain_rewritable/test.py new file mode 100644 index 00000000000..96d116ec6a2 --- /dev/null +++ b/tests/integration/test_azure_blob_storage_plain_rewritable/test.py @@ -0,0 +1,153 @@ +import logging +import os +import random +import string + +import pytest + +from helpers.cluster import ClickHouseCluster +from azure.storage.blob import BlobServiceClient +from test_storage_azure_blob_storage.test import azure_query + +NODE_NAME = "node" + + +def generate_cluster_def(port): + path = os.path.join( + os.path.dirname(os.path.realpath(__file__)), + "./_gen/disk_storage_conf.xml", + ) + os.makedirs(os.path.dirname(path), exist_ok=True) + with open(path, "w") as f: + f.write( + f""" + + + + object_storage + azure_blob_storage + plain_rewritable + http://azurite1:{port}/devstoreaccount1 + cont + true + devstoreaccount1 + Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw== + 100000 + 100000 + 10 + 10 + + + + + +
+ blob_storage_disk +
+
+
+
+
+
+""" + ) + return path + + +insert_values = [ + "(0,'data'),(1,'data')", + ",".join( + f"({i},'{''.join(random.choices(string.ascii_lowercase, k=5))}')" + for i in range(10) + ), +] + + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + port = cluster.azurite_port + path = generate_cluster_def(port) + cluster.add_instance( + NODE_NAME, + main_configs=[ + path, + ], + with_azurite=True, + stay_alive=True, + ) + logging.info("Starting cluster...") + cluster.start() + logging.info("Cluster started") + + yield cluster + finally: + cluster.shutdown() + + +def test_insert_select(cluster): + node = cluster.instances[NODE_NAME] + + for index, value in enumerate(insert_values): + azure_query( + node, + """ + CREATE TABLE test_{} ( + id Int64, + data String + ) ENGINE=MergeTree() + ORDER BY id + SETTINGS storage_policy='blob_storage_policy' + """.format( + index + ), + ) + + azure_query(node, "INSERT INTO test_{} VALUES {}".format(index, value)) + assert ( + azure_query( + node, "SELECT * FROM test_{} ORDER BY id FORMAT Values".format(index) + ) + == value + ) + + +def test_restart_server(cluster): + node = cluster.instances[NODE_NAME] + + for index, value in enumerate(insert_values): + assert ( + azure_query( + node, "SELECT * FROM test_{} ORDER BY id FORMAT Values".format(index) + ) + == value + ) + node.restart_clickhouse() + + for index, value in enumerate(insert_values): + assert ( + azure_query( + node, "SELECT * FROM test_{} ORDER BY id FORMAT Values".format(index) + ) + == value + ) + + +def test_drop_table(cluster): + node = cluster.instances[NODE_NAME] + + for index, value in enumerate(insert_values): + node.query("DROP TABLE IF EXISTS test_{} SYNC".format(index)) + + port = cluster.env_variables["AZURITE_PORT"] + connection_string = ( + f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" + f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" + f"BlobEndpoint=http://127.0.0.1:{port}/devstoreaccount1;" + ) + blob_service_client = BlobServiceClient.from_connection_string(connection_string) + containers = blob_service_client.list_containers() + for container in containers: + container_client = blob_service_client.get_container_client(container) + assert len(list(container_client.list_blobs())) == 0 From fcad15ffc2b7c5d4d1c9e9ce201ba9eb86d4a3d4 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Sat, 4 May 2024 04:26:48 +0000 Subject: [PATCH 0871/1018] plain_rewritable: update docs --- docs/en/operations/storing-data.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index 389c917d427..7005783dd60 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -371,6 +371,8 @@ is equal to ``` +Starting from `24.5` it is possible configure any object storage disk (`s3`, `azure`, `local`) using `plain_rewritable` metadata type. + ### Using Azure Blob Storage {#azure-blob-storage} `MergeTree` family table engines can store data to [Azure Blob Storage](https://azure.microsoft.com/en-us/services/storage/blobs/) using a disk with type `azure_blob_storage`. From a81f21ea8c8f4545e4c08bbe93d0e5d02229bcf6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 7 May 2024 06:12:11 +0200 Subject: [PATCH 0872/1018] Fix it --- src/Functions/FunctionsConversion.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index a16ce136b9a..0da84d6d224 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -4853,7 +4853,7 @@ FunctionBasePtr createFunctionBaseCast( DataTypeUInt8, DataTypeUInt16, DataTypeUInt32, DataTypeUInt64, DataTypeUInt128, DataTypeUInt256, DataTypeInt8, DataTypeInt16, DataTypeInt32, DataTypeInt64, DataTypeInt128, DataTypeInt256, DataTypeFloat32, DataTypeFloat64, - DataTypeDate, DataTypeDate32, DataTypeDateTime, + DataTypeDate, DataTypeDate32, DataTypeDateTime, DataTypeDateTime64, DataTypeString>(return_type.get(), [&](auto & type) { monotonicity = FunctionTo>::Type::Monotonic::get; From e7211357771e15e3cbb81346fd8cfaff67e40bb6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 7 May 2024 06:16:22 +0200 Subject: [PATCH 0873/1018] Add a test --- ...3147_datetime64_constant_index_analysis.reference | 6 ++++++ .../03147_datetime64_constant_index_analysis.sql | 12 ++++++++++++ 2 files changed, 18 insertions(+) create mode 100644 tests/queries/0_stateless/03147_datetime64_constant_index_analysis.reference create mode 100644 tests/queries/0_stateless/03147_datetime64_constant_index_analysis.sql diff --git a/tests/queries/0_stateless/03147_datetime64_constant_index_analysis.reference b/tests/queries/0_stateless/03147_datetime64_constant_index_analysis.reference new file mode 100644 index 00000000000..9abcce11136 --- /dev/null +++ b/tests/queries/0_stateless/03147_datetime64_constant_index_analysis.reference @@ -0,0 +1,6 @@ +7385 +7385 +7385 +7385 +86401 +86401 diff --git a/tests/queries/0_stateless/03147_datetime64_constant_index_analysis.sql b/tests/queries/0_stateless/03147_datetime64_constant_index_analysis.sql new file mode 100644 index 00000000000..144478eb721 --- /dev/null +++ b/tests/queries/0_stateless/03147_datetime64_constant_index_analysis.sql @@ -0,0 +1,12 @@ +DROP TABLE IF EXISTS test; +CREATE TABLE test (d DateTime, PRIMARY KEY (d)); +INSERT INTO test SELECT toDateTime('2024-01-01') + number FROM numbers(1e6); +SET max_rows_to_read = 10000; +SELECT count() FROM test WHERE d <= '2024-01-01 02:03:04'; +SELECT count() FROM test WHERE d <= toDateTime('2024-01-01 02:03:04'); +SELECT count() FROM test WHERE d <= toDateTime64('2024-01-01 02:03:04', 0); +SELECT count() FROM test WHERE d <= toDateTime64('2024-01-01 02:03:04', 3); +SET max_rows_to_read = 100_000; +SELECT count() FROM test WHERE d <= '2024-01-02'; +SELECT count() FROM test WHERE d <= toDate('2024-01-02'); +DROP TABLE test; From 2b8254f987a65d5c21d74fe67b4ee9757970466e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 7 May 2024 06:43:37 +0200 Subject: [PATCH 0874/1018] Update test --- .../02373_datetime64_monotonicity.queries | 57 ------------------- .../02373_datetime64_monotonicity.reference | 12 ++++ .../02373_datetime64_monotonicity.sh | 40 ++++++++++++- 3 files changed, 51 insertions(+), 58 deletions(-) delete mode 100644 tests/queries/0_stateless/02373_datetime64_monotonicity.queries diff --git a/tests/queries/0_stateless/02373_datetime64_monotonicity.queries b/tests/queries/0_stateless/02373_datetime64_monotonicity.queries deleted file mode 100644 index 404e3391205..00000000000 --- a/tests/queries/0_stateless/02373_datetime64_monotonicity.queries +++ /dev/null @@ -1,57 +0,0 @@ -drop table if exists dt64_monot_test; -drop table if exists dt64_monot_test_string; -CREATE TABLE dt64_monot_test(`date_time` DateTime64(3, 'Europe/Berlin'), `id` String) ENGINE = MergeTree PARTITION BY toDate(date_time, 'Europe/Berlin') ORDER BY date_time; -insert into dt64_monot_test select toDateTime64('2020-01-01 00:00:00.000',3)+number , '' from numbers(10); - -SELECT count() FROM dt64_monot_test WHERE toDateTime(date_time) >= toDateTime('2020-01-01 00:00:00') SETTINGS force_index_by_date = 1, force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time, 3) >= '2020-01-01 00:00:01.111' SETTINGS force_index_by_date = 1, force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time, 3) >= '2020-01-01 00:00:00.000' SETTINGS force_index_by_date = 1, force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) >= toDateTime64('2020-01-01 00:00:00.000001', 3) SETTINGS force_index_by_date = 1, force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) >= toDateTime64('2020-01-01 00:00:00.000001', 3, 'Europe/Berlin') SETTINGS force_index_by_date = 1, force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) >= toDateTime64('2020-01-01 00:00:00.000001',6) SETTINGS force_index_by_date = 1; -- { serverError 277} - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) >= toDateTime64('2020-01-01 00:00:00.000001',6, 'Europe/Berlin') SETTINGS force_primary_key = 1; -- { serverError 277} - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) >= toDateTime64('2020-01-01 00:00:00.000001',6) SETTINGS force_primary_key = 1; -- { serverError 277} - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) <= toDateTime64('2020-01-01 00:00:00.000001',3, 'Europe/Berlin') settings force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) <= toDateTime64('2020-01-01 00:00:00.000001',3) settings force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) = toDateTime64('2020-01-01 00:00:00.000000',6); - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) = toDateTime64('2020-01-01 00:00:00.000000',6, 'Europe/Berlin'); - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,6) = toDateTime64('2020-01-01 00:00:00.000000',6) settings force_index_by_date = 1, force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,6) = toDateTime64('2020-01-01 00:00:00.000001',6, 'Europe/Berlin') settings force_index_by_date = 1, force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,6) > toDateTime64('2020-01-01 00:00:00.000001',6, 'Europe/Berlin') settings force_index_by_date = 1, force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,6) >= toDateTime64('2020-01-01 00:00:00.000001',6, 'Europe/Berlin') settings force_index_by_date = 1, force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,6) >= toDateTime64('2020-01-01 00:00:00.000001',6) settings force_index_by_date = 1, force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,0) >= toDateTime64('2020-01-01 00:00:00.000001',0, 'Europe/Berlin') settings force_index_by_date = 1, force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,0) >= toDateTime64('2020-01-01 00:00:00.000001',0) settings force_index_by_date = 1, force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,0) >= '2020-01-01 00:00:00' settings force_index_by_date = 1, force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,0) >= '2020-01-01 00:00:01.1' settings force_index_by_date = 1, force_primary_key = 1; - -create table dt64_monot_test_string(date_time String, x String) Engine=MergeTree order by date_time; -insert into dt64_monot_test_string select '2020-01-01 00:00:00.000000001', '' from numbers(1); -insert into dt64_monot_test_string select '2020-01-01 00:00:00.000', '' from numbers(10); - -SELECT count() FROM dt64_monot_test_string WHERE toDateTime64(date_time,9) = '2020-01-01 00:00:00.000000000'; -SELECT count() FROM dt64_monot_test_string WHERE toDateTime64(date_time,3) = '2020-01-01 00:00:00.000000001'; -SELECT count() FROM dt64_monot_test_string WHERE toDateTime64(date_time,9) = '2020-01-01 00:00:00'; - -drop table dt64_monot_test; -drop table dt64_monot_test_string; diff --git a/tests/queries/0_stateless/02373_datetime64_monotonicity.reference b/tests/queries/0_stateless/02373_datetime64_monotonicity.reference index 935ee685cc9..dd7ddf43e70 100644 --- a/tests/queries/0_stateless/02373_datetime64_monotonicity.reference +++ b/tests/queries/0_stateless/02373_datetime64_monotonicity.reference @@ -4,6 +4,9 @@ Asia/Tehran 0 10 0 +9 +0 +9 10 1 1 @@ -27,6 +30,9 @@ UTC 10 10 10 +9 +10 +9 0 1 1 @@ -50,6 +56,9 @@ Canada/Atlantic 10 10 10 +9 +10 +9 0 1 1 @@ -73,6 +82,9 @@ Europe/Berlin 10 10 10 +9 +9 +9 1 1 1 diff --git a/tests/queries/0_stateless/02373_datetime64_monotonicity.sh b/tests/queries/0_stateless/02373_datetime64_monotonicity.sh index 0e0dc0ec22a..7e852aa24b0 100755 --- a/tests/queries/0_stateless/02373_datetime64_monotonicity.sh +++ b/tests/queries/0_stateless/02373_datetime64_monotonicity.sh @@ -7,6 +7,44 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) for tz in Asia/Tehran UTC Canada/Atlantic Europe/Berlin do echo "$tz" - TZ=$tz $CLICKHOUSE_LOCAL -mn < ${CUR_DIR}/02373_datetime64_monotonicity.queries + TZ=$tz $CLICKHOUSE_LOCAL --multiline --multiquery " + drop table if exists dt64_monotonicity_test; + drop table if exists dt64_monotonicity_test_string; + CREATE TABLE dt64_monotonicity_test (date_time DateTime64(3, 'Europe/Berlin'), id String) ENGINE = MergeTree PARTITION BY toDate(date_time, 'Europe/Berlin') ORDER BY date_time; + insert into dt64_monotonicity_test select toDateTime64('2020-01-01 00:00:00.000', 3) + number, '' from numbers(10); + + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime(date_time) >= toDateTime('2020-01-01 00:00:00') SETTINGS force_index_by_date = 1, force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) >= '2020-01-01 00:00:01.111' SETTINGS force_index_by_date = 1, force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) >= '2020-01-01 00:00:00.000' SETTINGS force_index_by_date = 1, force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) >= toDateTime64('2020-01-01 00:00:00.000001', 3) SETTINGS force_index_by_date = 1, force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) >= toDateTime64('2020-01-01 00:00:00.000001', 3, 'Europe/Berlin') SETTINGS force_index_by_date = 1, force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) >= toDateTime64('2020-01-01 00:00:00.000001', 6) SETTINGS force_index_by_date = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) >= toDateTime64('2020-01-01 00:00:00.000001', 6, 'Europe/Berlin') SETTINGS force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) >= toDateTime64('2020-01-01 00:00:00.000001', 6) SETTINGS force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) <= toDateTime64('2020-01-01 00:00:00.000001', 3, 'Europe/Berlin') settings force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) <= toDateTime64('2020-01-01 00:00:00.000001', 3) settings force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) = toDateTime64('2020-01-01 00:00:00.000000', 6); + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) = toDateTime64('2020-01-01 00:00:00.000000', 6, 'Europe/Berlin'); + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 6) = toDateTime64('2020-01-01 00:00:00.000000', 6) settings force_index_by_date = 1, force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 6) = toDateTime64('2020-01-01 00:00:00.000001', 6, 'Europe/Berlin') settings force_index_by_date = 1, force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 6) > toDateTime64('2020-01-01 00:00:00.000001', 6, 'Europe/Berlin') settings force_index_by_date = 1, force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 6) >= toDateTime64('2020-01-01 00:00:00.000001', 6, 'Europe/Berlin') settings force_index_by_date = 1, force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 6) >= toDateTime64('2020-01-01 00:00:00.000001', 6) settings force_index_by_date = 1, force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 0) >= toDateTime64('2020-01-01 00:00:00.000001', 0, 'Europe/Berlin') settings force_index_by_date = 1, force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 0) >= toDateTime64('2020-01-01 00:00:00.000001', 0) settings force_index_by_date = 1, force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 0) >= '2020-01-01 00:00:00' settings force_index_by_date = 1, force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 0) >= '2020-01-01 00:00:01.1' settings force_index_by_date = 1, force_primary_key = 1; + + create table dt64_monotonicity_test_string(date_time String, x String) Engine=MergeTree order by date_time; + insert into dt64_monotonicity_test_string select '2020-01-01 00:00:00.000000001', '' from numbers(1); + insert into dt64_monotonicity_test_string select '2020-01-01 00:00:00.000', '' from numbers(10); + + SELECT count() FROM dt64_monotonicity_test_string WHERE toDateTime64(date_time,9) = '2020-01-01 00:00:00.000000000'; + SELECT count() FROM dt64_monotonicity_test_string WHERE toDateTime64(date_time,3) = '2020-01-01 00:00:00.000000001'; + SELECT count() FROM dt64_monotonicity_test_string WHERE toDateTime64(date_time,9) = '2020-01-01 00:00:00'; + + drop table dt64_monotonicity_test; + drop table dt64_monotonicity_test_string; + " echo "" done From 796ffb3a0a60bef212aec0e971d8853fa086f636 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 7 May 2024 08:39:33 +0300 Subject: [PATCH 0875/1018] Update 03147_system_columns_access_checks.sh --- tests/queries/0_stateless/03147_system_columns_access_checks.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03147_system_columns_access_checks.sh b/tests/queries/0_stateless/03147_system_columns_access_checks.sh index 70693e0985f..0c8aafe16ec 100755 --- a/tests/queries/0_stateless/03147_system_columns_access_checks.sh +++ b/tests/queries/0_stateless/03147_system_columns_access_checks.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-parallel +# Tags: no-fasttest, no-parallel, long CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 5e5477302a71e8959feb8daab1b9459e16b2e168 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 7 May 2024 09:31:39 +0200 Subject: [PATCH 0876/1018] Update src/Storages/MergeTree/MergeTreeReadPoolBase.cpp Co-authored-by: Alexander Tokmakov --- src/Storages/MergeTree/MergeTreeReadPoolBase.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp index c759a12e151..36673238f3b 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp @@ -130,7 +130,7 @@ MergeTreeReadTaskPtr MergeTreeReadPoolBase::createTask( parent_part_name, {MergeTreeDataPartState::PreActive, MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); if (!parent_part) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Did not find parent part {} for potentially broken projection part {}", + throw Exception(ErrorCodes::LOGICAL_ERROR, "Did not find parent part {} for projection part {}", parent_part_name, data_part->getDataPartStorage().getFullPath()); return parent_part_name; From 5f52cb43b47538d98428dae13ae551c73114ef02 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 7 May 2024 09:33:47 +0200 Subject: [PATCH 0877/1018] Fix coverage. --- docker/test/base/setup_export_logs.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index 8858e12c50e..740fdd2da1f 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -143,7 +143,7 @@ function setup_logs_replication time DateTime COMMENT 'The time of test run', test_name String COMMENT 'The name of the test', coverage Array(UInt64) COMMENT 'An array of addresses of the code (a subset of addresses instrumented for coverage) that were encountered during the test run' - ) ENGINE = Null COMMENT 'Contains information about per-test coverage from the CI, but used only for exporting to the CI cluster' + ) ENGINE = MergeTree ORDER BY test_name COMMENT 'Contains information about per-test coverage from the CI, but used only for exporting to the CI cluster' " # For each system log table: From 20f3bb487812abb793fc0f9c7db3a1142dd5e2e7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 7 May 2024 10:37:46 +0300 Subject: [PATCH 0878/1018] Update 03147_system_columns_access_checks.sh --- tests/queries/0_stateless/03147_system_columns_access_checks.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03147_system_columns_access_checks.sh b/tests/queries/0_stateless/03147_system_columns_access_checks.sh index 0c8aafe16ec..2bd7fb083ea 100755 --- a/tests/queries/0_stateless/03147_system_columns_access_checks.sh +++ b/tests/queries/0_stateless/03147_system_columns_access_checks.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-parallel, long +# Tags: no-fasttest, no-parallel, no-ordinary-database, long CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From da4f6f7b6ce4d7c46f3bd1955352656fd2826f19 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 30 Apr 2024 16:35:57 +0300 Subject: [PATCH 0879/1018] Added recursive CTE documentation --- .../sql-reference/statements/select/with.md | 235 +++++++++++++++++- 1 file changed, 229 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/statements/select/with.md b/docs/en/sql-reference/statements/select/with.md index a59ef463419..84d3c02eca1 100644 --- a/docs/en/sql-reference/statements/select/with.md +++ b/docs/en/sql-reference/statements/select/with.md @@ -5,21 +5,21 @@ sidebar_label: WITH # WITH Clause -ClickHouse supports Common Table Expressions ([CTE](https://en.wikipedia.org/wiki/Hierarchical_and_recursive_queries_in_SQL)) and substitutes the code defined in the `WITH` clause in all places of use for the rest of `SELECT` query. Named subqueries can be included to the current and child query context in places where table objects are allowed. Recursion is prevented by hiding the current level CTEs from the WITH expression. +ClickHouse supports Common Table Expressions ([CTE](https://en.wikipedia.org/wiki/Hierarchical_and_recursive_queries_in_SQL)) and substitutes the code defined in the `WITH` clause in all places of use for the rest of `SELECT` query. Named subqueries can be included to the current and child query context in places where table objects are allowed. Recursion is prevented by hiding the current level CTEs from the WITH expression. Please note that CTEs do not guarantee the same results in all places they are called because the query will be re-executed for each use case. An example of such behavior is below ``` sql -with cte_numbers as +with cte_numbers as ( - select - num - from generateRandom('num UInt64', NULL) + select + num + from generateRandom('num UInt64', NULL) limit 1000000 ) select - count() + count() from cte_numbers where num in (select num from cte_numbers) ``` @@ -87,3 +87,226 @@ LIMIT 10; WITH test1 AS (SELECT i + 1, j + 1 FROM test1) SELECT * FROM test1; ``` + +# Recursive Queries + +The optional RECURSIVE modifier allows for a WITH query to refer to its own output. Example: + +**Example:** Sum integers from 1 throught 100 + +```sql +WITH RECURSIVE test_table AS ( + SELECT 1 AS number +UNION ALL + SELECT number + 1 FROM test_table WHERE number < 100 +) +SELECT sum(number) FROM test_table; +``` + +``` text +┌─sum(number)─┐ +│ 5050 │ +└─────────────┘ +``` + +The general form of a recursive `WITH` query is always a non-recursive term, then `UNION ALL`, then a recursive term, where only the recursive term can contain a reference to the query's own output. Recursive CTE query is executed as follows: + +1. Evaluate the non-recursive term. Place result of non-recursive term query in a temporary working table. +2. As long as the working table is not empty, repeat these steps: + 1. Evaluate the recursive term, substituting the current contents of the working table for the recursive self-reference. Place result of recursive term query in a temporary intermediate table. + 2. Replace the contents of the working table with the contents of the intermediate table, then empty the intermediate table. + +Recursive queries are typically used to work with hierarchical or tree-structured data. For example, we can write a query that performs tree traversal: + +**Example:** Tree traversal + +First let's create tree table: + +```sql +DROP TABLE IF EXISTS tree; +CREATE TABLE tree +( + id UInt64, + parent_id Nullable(UInt64), + data String +) ENGINE = MergeTree ORDER BY id; + +INSERT INTO tree VALUES (0, NULL, 'ROOT'), (1, 0, 'Child_1'), (2, 0, 'Child_2'), (3, 1, 'Child_1_1'); +``` + +We can traverse those tree with such query: + +**Example:** Tree traversal +```sql +WITH RECURSIVE search_tree AS ( + SELECT id, parent_id, data + FROM tree t + WHERE t.id = 0 +UNION ALL + SELECT t.id, t.parent_id, t.data + FROM tree t, search_tree st + WHERE t.parent_id = st.id +) +SELECT * FROM search_tree; +``` + +```text +┌─id─┬─parent_id─┬─data──────┐ +│ 0 │ ᴺᵁᴸᴸ │ ROOT │ +│ 1 │ 0 │ Child_1 │ +│ 2 │ 0 │ Child_2 │ +│ 3 │ 1 │ Child_1_1 │ +└────┴───────────┴───────────┘ +``` + +## Search order + +To create a depth-first order, we compute for each result row an array of rows that we have already visited: + +**Example:** Tree traversal depth-first order +```sql +WITH RECURSIVE search_tree AS ( + SELECT id, parent_id, data, [t.id] AS path + FROM tree t + WHERE t.id = 0 +UNION ALL + SELECT t.id, t.parent_id, t.data, arrayConcat(path, [t.id]) + FROM tree t, search_tree st + WHERE t.parent_id = st.id +) +SELECT * FROM search_tree ORDER BY path; +``` + +```text +┌─id─┬─parent_id─┬─data──────┬─path────┐ +│ 0 │ ᴺᵁᴸᴸ │ ROOT │ [0] │ +│ 1 │ 0 │ Child_1 │ [0,1] │ +│ 3 │ 1 │ Child_1_1 │ [0,1,3] │ +│ 2 │ 0 │ Child_2 │ [0,2] │ +└────┴───────────┴───────────┴─────────┘ +``` + +To create a breadth-first order, standard approach is to add column that tracks the depth of the search: + +**Example:** Tree traversal breadth-first order +```sql +WITH RECURSIVE search_tree AS ( + SELECT id, parent_id, data, [t.id] AS path, toUInt64(0) AS depth + FROM tree t + WHERE t.id = 0 +UNION ALL + SELECT t.id, t.parent_id, t.data, arrayConcat(path, [t.id]), depth + 1 + FROM tree t, search_tree st + WHERE t.parent_id = st.id +) +SELECT * FROM search_tree ORDER BY depth; +``` + +```text +┌─id─┬─link─┬─data──────┬─path────┬─depth─┐ +│ 0 │ ᴺᵁᴸᴸ │ ROOT │ [0] │ 0 │ +│ 1 │ 0 │ Child_1 │ [0,1] │ 1 │ +│ 2 │ 0 │ Child_2 │ [0,2] │ 1 │ +│ 3 │ 1 │ Child_1_1 │ [0,1,3] │ 2 │ +└────┴──────┴───────────┴─────────┴───────┘ +``` + +## Cycle detection + +First let's create graph table: + +```sql +DROP TABLE IF EXISTS graph; +CREATE TABLE graph +( + from UInt64, + to UInt64, + label String +) ENGINE = MergeTree ORDER BY (from, to); + +INSERT INTO graph VALUES (1, 2, '1 -> 2'), (1, 3, '1 -> 3'), (2, 3, '2 -> 3'), (1, 4, '1 -> 4'), (4, 5, '4 -> 5'); +``` + +We can traverse that graph with such query: + +**Example:** Graph traversal without cycle detection +```sql +WITH RECURSIVE search_graph AS ( + SELECT from, to, label FROM graph g + UNION ALL + SELECT g.from, g.to, g.label + FROM graph g, search_graph sg + WHERE g.from = sg.to +) +SELECT DISTINCT * FROM search_graph ORDER BY from; +``` +```text +┌─from─┬─to─┬─label──┐ +│ 1 │ 4 │ 1 -> 4 │ +│ 1 │ 2 │ 1 -> 2 │ +│ 1 │ 3 │ 1 -> 3 │ +│ 2 │ 3 │ 2 -> 3 │ +│ 4 │ 5 │ 4 -> 5 │ +└──────┴────┴────────┘ +``` + +But if we add cycle in that graph, previous query will fail with `Maximum recursive CTE evaluation depth` error: + +```sql +INSERT INTO graph VALUES (5, 1, '5 -> 1'); + +WITH RECURSIVE search_graph AS ( + SELECT from, to, label FROM graph g +UNION ALL + SELECT g.from, g.to, g.label + FROM graph g, search_graph sg + WHERE g.from = sg.to +) +SELECT DISTINCT * FROM search_graph ORDER BY from; +``` + +```text +Code: 306. DB::Exception: Received from localhost:9000. DB::Exception: Maximum recursive CTE evaluation depth (1000) exceeded, during evaluation of search_graph AS (SELECT from, to, label FROM graph AS g UNION ALL SELECT g.from, g.to, g.label FROM graph AS g, search_graph AS sg WHERE g.from = sg.to). Consider raising max_recursive_cte_evaluation_depth setting.: While executing RecursiveCTESource. (TOO_DEEP_RECURSION) +``` + +The standard method for handling cycles is to compute an array of the already visited nodes: + +**Example:** Graph traversal with cycle detection +```sql +WITH RECURSIVE search_graph AS ( + SELECT from, to, label, false AS is_cycle, [tuple(g.from, g.to)] AS path FROM graph g +UNION ALL + SELECT g.from, g.to, g.label, has(path, tuple(g.from, g.to)), arrayConcat(sg.path, [tuple(g.from, g.to)]) + FROM graph g, search_graph sg + WHERE g.from = sg.to AND NOT is_cycle +) +SELECT * FROM search_graph WHERE is_cycle ORDER BY from; +``` + +```text +┌─from─┬─to─┬─label──┬─is_cycle─┬─path──────────────────────┐ +│ 1 │ 4 │ 1 -> 4 │ true │ [(1,4),(4,5),(5,1),(1,4)] │ +│ 4 │ 5 │ 4 -> 5 │ true │ [(4,5),(5,1),(1,4),(4,5)] │ +│ 5 │ 1 │ 5 -> 1 │ true │ [(5,1),(1,4),(4,5),(5,1)] │ +└──────┴────┴────────┴──────────┴───────────────────────────┘ +``` + +## Infinite queries + +It is also possible to use inifinite recursive CTE queries if `LIMIT` is used in outer query: + +**Example:** Infinite recursive CTE query +```sql +WITH RECURSIVE test_table AS ( + SELECT 1 AS number +UNION ALL + SELECT number + 1 FROM test_table +) +SELECT sum(number) FROM (SELECT number FROM test_table LIMIT 100); +``` + +```text +┌─sum(number)─┐ +│ 5050 │ +└─────────────┘ +``` From 0b59c24866a6e61989b907aed0219530d6503b30 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 2 May 2024 18:50:38 +0300 Subject: [PATCH 0880/1018] Fixed style check --- docs/en/sql-reference/statements/select/with.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/select/with.md b/docs/en/sql-reference/statements/select/with.md index 84d3c02eca1..ffde7a3fe54 100644 --- a/docs/en/sql-reference/statements/select/with.md +++ b/docs/en/sql-reference/statements/select/with.md @@ -92,7 +92,7 @@ SELECT * FROM test1; The optional RECURSIVE modifier allows for a WITH query to refer to its own output. Example: -**Example:** Sum integers from 1 throught 100 +**Example:** Sum integers from 1 through 100 ```sql WITH RECURSIVE test_table AS ( @@ -293,7 +293,7 @@ SELECT * FROM search_graph WHERE is_cycle ORDER BY from; ## Infinite queries -It is also possible to use inifinite recursive CTE queries if `LIMIT` is used in outer query: +It is also possible to use infinite recursive CTE queries if `LIMIT` is used in outer query: **Example:** Infinite recursive CTE query ```sql From a61cb622f10586b519d41c43601c8c4ed428dfef Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 7 May 2024 09:39:19 +0000 Subject: [PATCH 0881/1018] fix test flaky Signed-off-by: Duc Canh Le --- tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh index 8acc83fc86c..45e65b18e07 100755 --- a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh +++ b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh @@ -27,9 +27,9 @@ ${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm;" # Testing insert with multiple sinks and fixed block size ${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE rocksdb_worm;" -${CLICKHOUSE_CLIENT} --query "ALTER TABLE rocksdb_worm MODIFY SETTING bulk_insert_block_size = 500000;" -${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers_mt(1000000) SETTINGS max_insert_threads = 2, max_block_size = 100000;" -${CLICKHOUSE_CLIENT} --query "SELECT sum(value) FROM system.rocksdb WHERE database = currentDatabase() AND table = 'rocksdb_worm' AND name = 'no.file.opens';" # should be 2 as max_block_size is set to 500000 +# Must set both max_threads and max_insert_threads to 2 to make sure there is only two sinks +${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers_mt(1000000) SETTINGS max_threads = 2, max_insert_threads = 2, max_block_size = 10000, min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0, insert_deduplication_token = '';" +${CLICKHOUSE_CLIENT} --query "SELECT sum(value) FROM system.rocksdb WHERE database = currentDatabase() AND table = 'rocksdb_worm' AND name = 'no.file.opens';" # should be 2 because default bulk sink size is ~1M rows / SST file ${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm;" # Testing insert with duplicated keys From f50f28d413bf0ba8cd40dccdbb631c2283dc32f1 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 7 May 2024 12:11:01 +0200 Subject: [PATCH 0882/1018] Update 03145_non_loaded_projection_backup.sh --- .../queries/0_stateless/03145_non_loaded_projection_backup.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh b/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh index 721ed784fc2..36ae9843dc4 100755 --- a/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh +++ b/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh @@ -10,6 +10,7 @@ create table tp_1 (x Int32, y Int32, projection p (select x, y order by x)) engi insert into tp_1 select number, number from numbers(3); set mutations_sync = 2; +system stop merges; alter table tp_1 add projection pp (select x, count() group by x); insert into tp_1 select number, number from numbers(4); @@ -46,4 +47,4 @@ check table tp_1 settings check_query_single_value_result = 0;" | grep -o "Found $CLICKHOUSE_CLIENT -nm -q " set send_logs_level='fatal'; check table tp_1" -$CLICKHOUSE_CLIENT -q "drop table tp_1 sync" +$CLICKHOUSE_CLIENT -q "drop table tp_1" From 791278ba47676ef497c95a308eaca91698717f91 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 6 May 2024 21:45:22 +0200 Subject: [PATCH 0883/1018] Add logging after a failure with evaluating Replicated*MergeTree engine arguments. --- ...tractZooKeeperPathFromReplicatedTableDef.h | 3 +- .../MergeTree/registerStorageMergeTree.cpp | 60 +++++++++++-------- 2 files changed, 36 insertions(+), 27 deletions(-) diff --git a/src/Storages/MergeTree/extractZooKeeperPathFromReplicatedTableDef.h b/src/Storages/MergeTree/extractZooKeeperPathFromReplicatedTableDef.h index 1bd58392201..5ef5e1db62e 100644 --- a/src/Storages/MergeTree/extractZooKeeperPathFromReplicatedTableDef.h +++ b/src/Storages/MergeTree/extractZooKeeperPathFromReplicatedTableDef.h @@ -11,8 +11,9 @@ class ASTCreateQuery; class Context; using ContextPtr = std::shared_ptr; -/// Extracts a zookeeper path from a specified CREATE TABLE query. Returns std::nullopt if fails. +/// Extracts a zookeeper path from a specified CREATE TABLE query. /// The function checks the table engine and if it is Replicated*MergeTree then it takes the first argument and expands macros in it. +/// Returns std::nullopt if the specified CREATE query doesn't describe a Replicated table or its arguments can't be evaluated. std::optional extractZooKeeperPathFromReplicatedTableDef(const ASTCreateQuery & create_query, const ContextPtr & context); } diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 9b0200d5a1c..4244ccccfe0 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -296,9 +296,6 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs( else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Replica name must be a string literal{}", verbose_help_message); - if (replica_name.empty()) - throw Exception(ErrorCodes::NO_REPLICA_NAME_GIVEN, "No replica name in config{}", verbose_help_message); - expand_macro(ast_zk_path, ast_replica_name); } else if (is_extended_storage_def @@ -332,38 +329,45 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs( throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected two string literal arguments: zookeeper_path and replica_name"); } -/// Extracts a zookeeper path from a specified CREATE TABLE query. Returns std::nullopt if fails. +/// Extracts a zookeeper path from a specified CREATE TABLE query. std::optional extractZooKeeperPathFromReplicatedTableDef(const ASTCreateQuery & query, const ContextPtr & context) { + if (!query.storage || !query.storage->engine) + return {}; + + const String & engine_name = query.storage->engine->name; + if (!isReplicated(engine_name)) + return {}; + + StorageID table_id{query.getDatabase(), query.getTable(), query.uuid}; + + ASTs engine_args; + if (query.storage->engine->arguments) + engine_args = query.storage->engine->arguments->children; + for (auto & engine_arg : engine_args) + engine_arg = engine_arg->clone(); + + LoadingStrictnessLevel mode = LoadingStrictnessLevel::CREATE; + String zookeeper_path; + String replica_name; + RenamingRestrictions renaming_restrictions; + try { - if (!query.storage || !query.storage->engine) - return {}; - - const String & engine_name = query.storage->engine->name; - if (!isReplicated(engine_name)) - return {}; - - StorageID table_id{query.getDatabase(), query.getTable(), query.uuid}; - ASTs engine_args; - if (query.storage->engine->arguments) - engine_args = query.storage->engine->arguments->children; - for (auto & engine_arg : engine_args) - engine_arg = engine_arg->clone(); - LoadingStrictnessLevel mode = LoadingStrictnessLevel::CREATE; - String zookeeper_path; - String replica_name; - RenamingRestrictions renaming_restrictions; - extractZooKeeperPathAndReplicaNameFromEngineArgs(query, table_id, engine_name, engine_args, mode, context, zookeeper_path, replica_name, renaming_restrictions); - - return zookeeper_path; } - catch (...) + catch (Exception & e) { - return {}; + if (e.code() == ErrorCodes::BAD_ARGUMENTS) + { + tryLogCurrentException(__PRETTY_FUNCTION__, "Couldn't evaluate engine arguments"); + return {}; + } + throw; } + + return zookeeper_path; } static StoragePtr create(const StorageFactory::Arguments & args) @@ -539,6 +543,10 @@ static StoragePtr create(const StorageFactory::Arguments & args) { extractZooKeeperPathAndReplicaNameFromEngineArgs(args.query, args.table_id, args.engine_name, args.engine_args, args.mode, args.getLocalContext(), zookeeper_path, replica_name, renaming_restrictions); + + if (replica_name.empty()) + throw Exception(ErrorCodes::NO_REPLICA_NAME_GIVEN, "No replica name in config{}", verbose_help_message); + arg_cnt = engine_args.size(); /// Update `arg_cnt` here because extractZooKeeperPathAndReplicaNameFromEngineArgs() could add arguments. arg_num = 2; /// zookeeper_path and replica_name together are always two arguments. } From e6926dc65d37658f48540490190398c91ea22273 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 7 May 2024 12:32:34 +0200 Subject: [PATCH 0884/1018] Update 03145_non_loaded_projection_backup.sh --- tests/queries/0_stateless/03145_non_loaded_projection_backup.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh b/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh index 36ae9843dc4..d92b85a984f 100755 --- a/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh +++ b/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh @@ -10,7 +10,6 @@ create table tp_1 (x Int32, y Int32, projection p (select x, y order by x)) engi insert into tp_1 select number, number from numbers(3); set mutations_sync = 2; -system stop merges; alter table tp_1 add projection pp (select x, count() group by x); insert into tp_1 select number, number from numbers(4); From 0a580e84eaae0dff89ca5fd151b8c98d3b8f9adb Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 7 May 2024 12:41:06 +0200 Subject: [PATCH 0885/1018] Update 03145_non_loaded_projection_backup.sh --- tests/queries/0_stateless/03145_non_loaded_projection_backup.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh b/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh index d92b85a984f..6f0e00ce3fc 100755 --- a/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh +++ b/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh @@ -7,6 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -nm -q " drop table if exists tp_1; create table tp_1 (x Int32, y Int32, projection p (select x, y order by x)) engine = MergeTree order by y partition by intDiv(y, 100); +system stop merges tp_1; insert into tp_1 select number, number from numbers(3); set mutations_sync = 2; From 1bae2d9d4ffa6b4757dc2aeccb9eccf89bebc072 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 7 May 2024 12:57:14 +0200 Subject: [PATCH 0886/1018] update comment --- src/Storages/StorageS3.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 8a4e30fed1d..e65d0cb5be4 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -489,7 +489,8 @@ size_t StorageS3Source::DisclosedGlobIterator::estimatedKeysCount() { /// 1000 files were listed, and we cannot make any estimation of _how many more_ there are (because we list bucket lazily); /// If there are more objects in the bucket, limiting the number of streams is the last thing we may want to do - /// as it would lead to serious (up to times) reading performance degradation. + /// as it would lead to serious slow down of the execution, since objects are going + /// to be fetched sequentially rather than in-parallel with up to times. return std::numeric_limits::max(); } else From 463f95e956b62a98c3817e27943c753123ebb2d3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 7 May 2024 13:47:35 +0200 Subject: [PATCH 0887/1018] Extra constraints for stress and fuzzer tests --- .../test/fuzzer/query-fuzzer-tweaks-users.xml | 5 ++++ docker/test/stateless/stress_tests.lib | 30 ++++++++++++++++++- 2 files changed, 34 insertions(+), 1 deletion(-) diff --git a/docker/test/fuzzer/query-fuzzer-tweaks-users.xml b/docker/test/fuzzer/query-fuzzer-tweaks-users.xml index c31d2fd7f39..ad261be1abe 100644 --- a/docker/test/fuzzer/query-fuzzer-tweaks-users.xml +++ b/docker/test/fuzzer/query-fuzzer-tweaks-users.xml @@ -31,6 +31,11 @@ + + + + + diff --git a/docker/test/stateless/stress_tests.lib b/docker/test/stateless/stress_tests.lib index a0865f638a1..6aaddbfe590 100644 --- a/docker/test/stateless/stress_tests.lib +++ b/docker/test/stateless/stress_tests.lib @@ -120,13 +120,41 @@ EOL local max_users_mem max_users_mem=$((total_mem*30/100)) # 30% + # Similar to docker/test/fuzzer/query-fuzzer-tweaks-users.xml echo "Setting max_memory_usage_for_user=$max_users_mem and max_memory_usage for queries to 10G" - cat > /etc/clickhouse-server/users.d/max_memory_usage_for_user.xml < /etc/clickhouse-server/users.d/stress_test_tweaks-users.xml < + 60 10G ${max_users_mem} + + 200 + + + + 60 + + + + 10G + + + + 200 + + + + + + + + + + + + From 0609054e9849b915847cbd93cb3d76786eebe0cc Mon Sep 17 00:00:00 2001 From: unashi Date: Tue, 7 May 2024 21:09:33 +0800 Subject: [PATCH 0888/1018] [update] update a stateless case --- .../02187_async_inserts_all_formats. | 52 +++++++++++++++++++ .../02187_async_inserts_all_formats.reference | 3 ++ 2 files changed, 55 insertions(+) create mode 100644 tests/queries/0_stateless/02187_async_inserts_all_formats. diff --git a/tests/queries/0_stateless/02187_async_inserts_all_formats. b/tests/queries/0_stateless/02187_async_inserts_all_formats. new file mode 100644 index 00000000000..f42a6d39d4f --- /dev/null +++ b/tests/queries/0_stateless/02187_async_inserts_all_formats. @@ -0,0 +1,52 @@ +Arrow +ArrowStream +Avro +BSONEachRow +CSV +CSVWithNames +CSVWithNamesAndTypes +CustomSeparated +CustomSeparatedWithNames +CustomSeparatedWithNamesAndTypes +JSON +JSONColumns +JSONColumnsWithMetadata +JSONCompact +JSONCompactColumns +JSONCompactEachRow +JSONCompactEachRowWithNames +JSONCompactEachRowWithNamesAndTypes +JSONCompactStringsEachRow +JSONCompactStringsEachRowWithNames +JSONCompactStringsEachRowWithNamesAndTypes +JSONEachRow +JSONLines +JSONObjectEachRow +JSONStringsEachRow +MsgPack +NDJSON +Native +ORC +Parquet +Raw +RawWithNames +RawWithNamesAndTypes +RowBinary +RowBinaryWithNames +RowBinaryWithNamesAndTypes +TSKV +TSV +TSVRaw +TSVRawWithNames +TSVRawWithNamesAndTypes +TSVWithNames +TSVWithNamesAndTypes +TabSeparated +TabSeparatedRaw +TabSeparatedRawWithNames +TabSeparatedRawWithNamesAndTypes +TabSeparatedWithNames +TabSeparatedWithNamesAndTypes +Values +LineAsString +OK diff --git a/tests/queries/0_stateless/02187_async_inserts_all_formats.reference b/tests/queries/0_stateless/02187_async_inserts_all_formats.reference index 2de728b4cb4..f42a6d39d4f 100644 --- a/tests/queries/0_stateless/02187_async_inserts_all_formats.reference +++ b/tests/queries/0_stateless/02187_async_inserts_all_formats.reference @@ -28,6 +28,9 @@ NDJSON Native ORC Parquet +Raw +RawWithNames +RawWithNamesAndTypes RowBinary RowBinaryWithNames RowBinaryWithNamesAndTypes From 85f766f27c0cbd49e267334c3d184627b554e853 Mon Sep 17 00:00:00 2001 From: unashi Date: Tue, 7 May 2024 21:11:45 +0800 Subject: [PATCH 0889/1018] [update] update a stateless case --- .../02187_async_inserts_all_formats. | 52 ------------------- 1 file changed, 52 deletions(-) delete mode 100644 tests/queries/0_stateless/02187_async_inserts_all_formats. diff --git a/tests/queries/0_stateless/02187_async_inserts_all_formats. b/tests/queries/0_stateless/02187_async_inserts_all_formats. deleted file mode 100644 index f42a6d39d4f..00000000000 --- a/tests/queries/0_stateless/02187_async_inserts_all_formats. +++ /dev/null @@ -1,52 +0,0 @@ -Arrow -ArrowStream -Avro -BSONEachRow -CSV -CSVWithNames -CSVWithNamesAndTypes -CustomSeparated -CustomSeparatedWithNames -CustomSeparatedWithNamesAndTypes -JSON -JSONColumns -JSONColumnsWithMetadata -JSONCompact -JSONCompactColumns -JSONCompactEachRow -JSONCompactEachRowWithNames -JSONCompactEachRowWithNamesAndTypes -JSONCompactStringsEachRow -JSONCompactStringsEachRowWithNames -JSONCompactStringsEachRowWithNamesAndTypes -JSONEachRow -JSONLines -JSONObjectEachRow -JSONStringsEachRow -MsgPack -NDJSON -Native -ORC -Parquet -Raw -RawWithNames -RawWithNamesAndTypes -RowBinary -RowBinaryWithNames -RowBinaryWithNamesAndTypes -TSKV -TSV -TSVRaw -TSVRawWithNames -TSVRawWithNamesAndTypes -TSVWithNames -TSVWithNamesAndTypes -TabSeparated -TabSeparatedRaw -TabSeparatedRawWithNames -TabSeparatedRawWithNamesAndTypes -TabSeparatedWithNames -TabSeparatedWithNamesAndTypes -Values -LineAsString -OK From 6f2a39b29131578acd10a79486f29f23e323e311 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 7 May 2024 15:32:10 +0200 Subject: [PATCH 0890/1018] Revert "Do in reverse order" This reverts commit 0b0e97917e3e4ab27a17cbf14d9e73163a20adbe. --- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 492e4065502..eb757e1d8c7 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -1007,10 +1007,6 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar filterColumns(columns, read_result.final_filter); } - /// If columns not empty, then apply on-fly alter conversions if any required - if (!prewhere_info || prewhere_info->perform_alter_conversions) - merge_tree_reader->performRequiredConversions(columns); - /// If some columns absent in part, then evaluate default values if (should_evaluate_missing_defaults) { @@ -1022,6 +1018,9 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar merge_tree_reader->evaluateMissingDefaults(additional_columns, columns); } + /// If columns not empty, then apply on-fly alter conversions if any required + if (!prewhere_info || prewhere_info->perform_alter_conversions) + merge_tree_reader->performRequiredConversions(columns); } read_result.columns.reserve(read_result.columns.size() + columns.size()); @@ -1047,14 +1046,14 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar bool should_evaluate_missing_defaults; merge_tree_reader->fillMissingColumns(columns, should_evaluate_missing_defaults, read_result.num_rows); - /// If result not empty, then apply on-fly alter conversions if any required - if (!prewhere_info || prewhere_info->perform_alter_conversions) - merge_tree_reader->performRequiredConversions(columns); - /// If some columns absent in part, then evaluate default values if (should_evaluate_missing_defaults) merge_tree_reader->evaluateMissingDefaults({}, columns); + /// If result not empty, then apply on-fly alter conversions if any required + if (!prewhere_info || prewhere_info->perform_alter_conversions) + merge_tree_reader->performRequiredConversions(columns); + for (size_t i = 0; i < columns.size(); ++i) read_result.columns[i] = std::move(columns[i]); } From 54418fcea24fbe1e713ccf3dfbde2db0ea046ec8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 7 May 2024 16:59:29 +0300 Subject: [PATCH 0891/1018] Update src/Storages/System/StorageSystemColumns.cpp MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- src/Storages/System/StorageSystemColumns.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 15b73fd1884..74b44cc0a2d 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -419,7 +419,7 @@ void ReadFromSystemColumns::initializePipeline(QueryPipelineBuilder & pipeline, /// Add `table` column. MutableColumnPtr table_column_mut = ColumnString::create(); - size_t num_databases = database_column->size(); + const auto num_databases = database_column->size(); IColumn::Offsets offsets(num_databases); for (size_t i = 0; i < num_databases; ++i) From 49284724ef4b34bbc5112bfa63a99625e8d35fe7 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 7 May 2024 16:10:21 +0200 Subject: [PATCH 0892/1018] fix errorCodes in fast tests --- tests/queries/0_stateless/00909_arrayEnumerateUniq.sql | 2 +- tests/queries/0_stateless/01045_array_zip.sql | 2 +- tests/queries/0_stateless/02354_parse_timedelta.sql | 4 ++-- tests/queries/0_stateless/02372_now_in_block.sql | 2 +- tests/queries/0_stateless/02718_array_fold.sql | 8 ++++---- 5 files changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/00909_arrayEnumerateUniq.sql b/tests/queries/0_stateless/00909_arrayEnumerateUniq.sql index 0bdb338e9d2..fe01b2185c2 100644 --- a/tests/queries/0_stateless/00909_arrayEnumerateUniq.sql +++ b/tests/queries/0_stateless/00909_arrayEnumerateUniq.sql @@ -152,7 +152,7 @@ DROP TABLE arrays_test; select '---------BAD'; -SELECT arrayEnumerateUniqRanked(); -- { serverError 42 } +SELECT arrayEnumerateUniqRanked(); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION } SELECT arrayEnumerateUniqRanked([]); SELECT arrayEnumerateUniqRanked(1); -- { serverError 36 } SELECT arrayEnumerateUniqRanked(2,[]); -- { serverError 36 } diff --git a/tests/queries/0_stateless/01045_array_zip.sql b/tests/queries/0_stateless/01045_array_zip.sql index 1a85e6a0874..a2d54c8ae3f 100644 --- a/tests/queries/0_stateless/01045_array_zip.sql +++ b/tests/queries/0_stateless/01045_array_zip.sql @@ -2,7 +2,7 @@ SELECT arrayZip(['a', 'b', 'c'], ['d', 'e', 'f']); SELECT arrayZip(['a', 'b', 'c'], ['d', 'e', 'f'], ['g', 'h', 'i']); -SELECT arrayZip(); -- { serverError 42 } +SELECT arrayZip(); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION } SELECT arrayZip('a', 'b', 'c'); -- { serverError 43 } diff --git a/tests/queries/0_stateless/02354_parse_timedelta.sql b/tests/queries/0_stateless/02354_parse_timedelta.sql index 29f2bf9fdfc..e876de64374 100644 --- a/tests/queries/0_stateless/02354_parse_timedelta.sql +++ b/tests/queries/0_stateless/02354_parse_timedelta.sql @@ -11,8 +11,8 @@ SELECT parseTimeDelta('1s - 1ms : 1μs ; 1ns'); SELECT parseTimeDelta('1.11s1.11ms1.11us1.11ns'); -- invalid expressions -SELECT parseTimeDelta(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} -SELECT parseTimeDelta('1yr', 1); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +SELECT parseTimeDelta(); -- {serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION} +SELECT parseTimeDelta('1yr', 1); -- {serverError TOO_MANY_ARGUMENTS_FOR_FUNCTION} SELECT parseTimeDelta(1); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} SELECT parseTimeDelta(' '); -- {serverError BAD_ARGUMENTS} SELECT parseTimeDelta('-1yr'); -- {serverError BAD_ARGUMENTS} diff --git a/tests/queries/0_stateless/02372_now_in_block.sql b/tests/queries/0_stateless/02372_now_in_block.sql index 7c884c0ba7d..815f74e5845 100644 --- a/tests/queries/0_stateless/02372_now_in_block.sql +++ b/tests/queries/0_stateless/02372_now_in_block.sql @@ -1,4 +1,4 @@ SELECT count() FROM (SELECT DISTINCT nowInBlock(), nowInBlock('Pacific/Pitcairn') FROM system.numbers LIMIT 2); SELECT nowInBlock(1); -- { serverError 43 } SELECT nowInBlock(NULL) IS NULL; -SELECT nowInBlock('UTC', 'UTC'); -- { serverError 42 } +SELECT nowInBlock('UTC', 'UTC'); -- { serverError TOO_MANY_ARGUMENTS_FOR_FUNCTION } diff --git a/tests/queries/0_stateless/02718_array_fold.sql b/tests/queries/0_stateless/02718_array_fold.sql index 7dee33c4705..73a3cebdda1 100644 --- a/tests/queries/0_stateless/02718_array_fold.sql +++ b/tests/queries/0_stateless/02718_array_fold.sql @@ -1,10 +1,10 @@ SELECT '-- Negative tests'; -SELECT arrayFold(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT arrayFold(1); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT arrayFold(1, toUInt64(0)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT arrayFold(); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION } +SELECT arrayFold(1); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION } +SELECT arrayFold(1, toUInt64(0)); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION } SELECT arrayFold(1, emptyArrayUInt64(), toUInt64(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT arrayFold( acc,x -> x, emptyArrayString(), toInt8(0)); -- { serverError TYPE_MISMATCH } -SELECT arrayFold( acc,x -> x, 'not an array', toUInt8(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT arrayFold( acc,x -> x, 'not an array', toUInt8(0)); -- { serverError I02718_array_foldLLEGAL_TYPE_OF_ARGUMENT } SELECT arrayFold( acc,x,y -> x, [0, 1], 'not an array', toUInt8(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT arrayFold( acc,x -> x, [0, 1], [2, 3], toUInt8(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT arrayFold( acc,x,y -> x, [0, 1], [2, 3, 4], toUInt8(0)); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } From cad9c97725e4943730d0dabaa3df2cdf008be948 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 7 May 2024 17:21:25 +0300 Subject: [PATCH 0893/1018] Analyzer setting max_streams_to_max_threads_ratio overflow fix --- .../AggregateFunctionSparkbar.cpp | 4 ++-- src/Planner/PlannerJoinTree.cpp | 10 +++++++++- ...streams_to_max_threads_ratio_overflow.reference | 0 ...g_max_streams_to_max_threads_ratio_overflow.sql | 14 ++++++++++++++ 4 files changed, 25 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/03148_setting_max_streams_to_max_threads_ratio_overflow.reference create mode 100644 tests/queries/0_stateless/03148_setting_max_streams_to_max_threads_ratio_overflow.sql diff --git a/src/AggregateFunctions/AggregateFunctionSparkbar.cpp b/src/AggregateFunctions/AggregateFunctionSparkbar.cpp index b6e538520a8..362ffbe20d2 100644 --- a/src/AggregateFunctions/AggregateFunctionSparkbar.cpp +++ b/src/AggregateFunctions/AggregateFunctionSparkbar.cpp @@ -253,9 +253,9 @@ private: else { Y scaled; - bool has_overfllow = common::mulOverflow(y, levels_num, scaled); + bool has_overflow = common::mulOverflow(y, levels_num, scaled); - if (has_overfllow) + if (has_overflow) y = y / (y_max / levels_num) + 1; else y = scaled / y_max + 1; diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 85cad1dcd69..094cf73dbc6 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -708,7 +708,15 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres /// If necessary, we request more sources than the number of threads - to distribute the work evenly over the threads if (max_streams > 1 && !is_sync_remote) - max_streams = static_cast(max_streams * settings.max_streams_to_max_threads_ratio); + { + if (auto streams_with_ratio = max_streams * settings.max_streams_to_max_threads_ratio; canConvertTo(streams_with_ratio)) + max_streams = static_cast(streams_with_ratio); + else + throw Exception(ErrorCodes::PARAMETER_OUT_OF_BOUND, + "Exceeded limit for `max_streams` with `max_streams_to_max_threads_ratio`. " + "Make sure that `max_streams * max_streams_to_max_threads_ratio` is in some reasonable boundaries, current value: {}", + streams_with_ratio); + } if (table_node) table_expression_query_info.table_expression_modifiers = table_node->getTableExpressionModifiers(); diff --git a/tests/queries/0_stateless/03148_setting_max_streams_to_max_threads_ratio_overflow.reference b/tests/queries/0_stateless/03148_setting_max_streams_to_max_threads_ratio_overflow.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03148_setting_max_streams_to_max_threads_ratio_overflow.sql b/tests/queries/0_stateless/03148_setting_max_streams_to_max_threads_ratio_overflow.sql new file mode 100644 index 00000000000..af326c15bd8 --- /dev/null +++ b/tests/queries/0_stateless/03148_setting_max_streams_to_max_threads_ratio_overflow.sql @@ -0,0 +1,14 @@ +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value String +) ENGINE = MergeTree ORDER BY id; + +INSERT INTO test_table VALUES (0, 'Value_0'); + +SELECT * FROM test_table SETTINGS max_threads = 1025, max_streams_to_max_threads_ratio = -9223372036854775808, allow_experimental_analyzer = 1; -- { serverError PARAMETER_OUT_OF_BOUND } + +SELECT * FROM test_table SETTINGS max_threads = 1025, max_streams_to_max_threads_ratio = -9223372036854775808, allow_experimental_analyzer = 0; -- { serverError PARAMETER_OUT_OF_BOUND } + +DROP TABLE test_table; From c45c1444355b740ed01fe1849f18f17c689d826b Mon Sep 17 00:00:00 2001 From: Yohann Jardin Date: Tue, 7 May 2024 16:39:43 +0200 Subject: [PATCH 0894/1018] fix typo in fast tests --- tests/queries/0_stateless/02718_array_fold.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02718_array_fold.sql b/tests/queries/0_stateless/02718_array_fold.sql index 73a3cebdda1..e59eae87fdf 100644 --- a/tests/queries/0_stateless/02718_array_fold.sql +++ b/tests/queries/0_stateless/02718_array_fold.sql @@ -4,7 +4,7 @@ SELECT arrayFold(1); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION } SELECT arrayFold(1, toUInt64(0)); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION } SELECT arrayFold(1, emptyArrayUInt64(), toUInt64(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT arrayFold( acc,x -> x, emptyArrayString(), toInt8(0)); -- { serverError TYPE_MISMATCH } -SELECT arrayFold( acc,x -> x, 'not an array', toUInt8(0)); -- { serverError I02718_array_foldLLEGAL_TYPE_OF_ARGUMENT } +SELECT arrayFold( acc,x -> x, 'not an array', toUInt8(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT arrayFold( acc,x,y -> x, [0, 1], 'not an array', toUInt8(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT arrayFold( acc,x -> x, [0, 1], [2, 3], toUInt8(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT arrayFold( acc,x,y -> x, [0, 1], [2, 3, 4], toUInt8(0)); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } From 5117422c7bebd04e3511abff3b02fd26561cda13 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 7 May 2024 14:48:50 +0000 Subject: [PATCH 0895/1018] init --- src/Core/Settings.h | 1 + src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + .../Formats/Impl/PrettyBlockOutputFormat.cpp | 10 +++++----- .../Impl/PrettyCompactBlockOutputFormat.cpp | 4 ++-- .../Impl/PrettySpaceBlockOutputFormat.cpp | 4 ++-- .../03132_pretty_format_break_line.reference | 19 +++++++++++++++++++ .../03132_pretty_format_break_line.sql | 7 +++++++ 8 files changed, 38 insertions(+), 9 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b4313d9af56..d640a556fb6 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1113,6 +1113,7 @@ class IColumn; \ M(String, format_json_object_each_row_column_for_object_name, "", "The name of column that will be used as object names in JSONObjectEachRow format. Column type should be String", 0) \ \ + M(Bool, output_format_pretty_preserve_border_for_multiline_string, true, "Applies better rendering for multiline strings.", 0) \ M(UInt64, output_format_pretty_max_rows, 10000, "Rows limit for Pretty formats.", 0) \ M(UInt64, output_format_pretty_max_column_pad_width, 250, "Maximum width to pad all values in a column in Pretty formats.", 0) \ M(UInt64, output_format_pretty_max_value_width, 10000, "Maximum width of value to display in Pretty formats. If greater - it will be cut.", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index b7e9899da46..cca4c30f5e0 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -173,6 +173,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.pretty.max_value_width_apply_for_single_value = settings.output_format_pretty_max_value_width_apply_for_single_value; format_settings.pretty.highlight_digit_groups = settings.output_format_pretty_highlight_digit_groups; format_settings.pretty.output_format_pretty_row_numbers = settings.output_format_pretty_row_numbers; + format_settings.pretty.preserve_border_for_multiline_string = settings.output_format_pretty_preserve_border_for_multiline_string; format_settings.pretty.output_format_pretty_single_large_number_tip_threshold = settings.output_format_pretty_single_large_number_tip_threshold; format_settings.protobuf.input_flatten_google_wrappers = settings.input_format_protobuf_flatten_google_wrappers; format_settings.protobuf.output_nullables_with_google_wrappers = settings.output_format_protobuf_nullables_with_google_wrappers; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index da225a39ec9..e320ea8e6b6 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -282,6 +282,7 @@ struct FormatSettings SettingFieldUInt64Auto color{"auto"}; bool output_format_pretty_row_numbers = false; + bool preserve_border_for_multiline_string = true; UInt64 output_format_pretty_single_large_number_tip_threshold = 1'000'000; enum class Charset diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index 178d0b912e1..e23622bc2e5 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -38,7 +38,7 @@ void PrettyBlockOutputFormat::calculateWidths( max_padded_widths.resize_fill(num_columns); name_widths.resize(num_columns); - const bool need_cut_to_width = format_settings.pretty.max_value_width_apply_for_single_value || num_rows != 1 || num_columns != 1 || total_rows != 0; + const bool need_cut_to_width = format_settings.pretty.preserve_border_for_multiline_string && (format_settings.pretty.max_value_width_apply_for_single_value || num_rows != 1 || num_columns != 1 || total_rows != 0); /// Calculate widths of all values. String serialized_value; @@ -333,7 +333,7 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind WriteBufferFromString out_serialize(serialized_value, AppendModeTag()); serializations[j]->serializeText(*columns[j], i, out_serialize, format_settings); } - if (cut_to_width) + if (cut_to_width && format_settings.pretty.preserve_border_for_multiline_string) splitValueAtBreakLine(serialized_value, transferred_row[j], cur_width); has_transferred_row |= !transferred_row[j].empty() && cur_width <= cut_to_width; @@ -345,7 +345,7 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind writeReadableNumberTip(chunk); writeCString("\n", out); - if (has_transferred_row) + if (has_transferred_row && format_settings.pretty.preserve_border_for_multiline_string) writeTransferredRow(max_widths, header, transferred_row, cut_to_width, false); } @@ -453,7 +453,7 @@ void PrettyBlockOutputFormat::writeValueWithPadding( value_width = format_settings.pretty.max_value_width; has_break_line = false; } - else if (!has_break_line) + else if (!has_break_line || !format_settings.pretty.preserve_border_for_multiline_string) value += ' '; auto write_padding = [&]() @@ -478,7 +478,7 @@ void PrettyBlockOutputFormat::writeValueWithPadding( write_padding(); } - if (has_break_line) + if (has_break_line && format_settings.pretty.preserve_border_for_multiline_string) writeString("…", out); } diff --git a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp index 345b6c84ecd..1ccb6d713d7 100644 --- a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp @@ -182,7 +182,7 @@ void PrettyCompactBlockOutputFormat::writeRow( WriteBufferFromString out_serialize(serialized_value, AppendModeTag()); serializations[j]->serializeText(*columns[j], row_num, out_serialize, format_settings); } - if (cut_to_width) + if (cut_to_width && format_settings.pretty.preserve_border_for_multiline_string) splitValueAtBreakLine(serialized_value, transferred_row[j], cur_width); has_transferred_row |= !transferred_row[j].empty() && cur_width <= cut_to_width; @@ -194,7 +194,7 @@ void PrettyCompactBlockOutputFormat::writeRow( writeReadableNumberTip(chunk); writeCString("\n", out); - if (has_transferred_row) + if (has_transferred_row && format_settings.pretty.preserve_border_for_multiline_string) writeTransferredRow(max_widths, header, transferred_row, cut_to_width, false); } diff --git a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp index 6940c20e25b..0ed8c4398e4 100644 --- a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp @@ -100,7 +100,7 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port WriteBufferFromString out_serialize(serialized_value, AppendModeTag()); serializations[column]->serializeText(*columns[column], row, out_serialize, format_settings); } - if (cut_to_width) + if (cut_to_width && format_settings.pretty.preserve_border_for_multiline_string) splitValueAtBreakLine(serialized_value, transferred_row[column], cur_width); has_transferred_row |= !transferred_row[column].empty() && cur_width <= cut_to_width; @@ -111,7 +111,7 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port writeReadableNumberTip(chunk); writeChar('\n', out); - if (has_transferred_row) + if (has_transferred_row && format_settings.pretty.preserve_border_for_multiline_string) writeTransferredRow(max_widths, header, transferred_row, cut_to_width, true); } diff --git a/tests/queries/0_stateless/03132_pretty_format_break_line.reference b/tests/queries/0_stateless/03132_pretty_format_break_line.reference index a5282f89327..b7b59df24fc 100644 --- a/tests/queries/0_stateless/03132_pretty_format_break_line.reference +++ b/tests/queries/0_stateless/03132_pretty_format_break_line.reference @@ -86,3 +86,22 @@ 2. │ 1 │ hello world │ hellow …│ │ │ │…мир │ └────┴─────────────┴─────────────┘ +┏━━━━┳━━━━━━━━━━━━━┳━━━━━━━━━━━━━┓ +┃ id ┃ value ┃ value1 ┃ +┡━━━━╇━━━━━━━━━━━━━╇━━━━━━━━━━━━━┩ +│ 0 │ привет +world │ hello world │ +├────┼─────────────┼─────────────┤ +│ 1 │ hello world │ hellow +мир │ +└────┴─────────────┴─────────────┘ + ┏━━━━┳━━━━━━━━━━━━━┳━━━━━━━━━━━━━┓ + ┃ id ┃ value ┃ value1 ┃ + ┡━━━━╇━━━━━━━━━━━━━╇━━━━━━━━━━━━━┩ +1. │ 0 │ привет +world │ hello world │ + ├────┼─────────────┼─────────────┤ +2. │ 1 │ hello world │ hellow +мир │ + └────┴─────────────┴─────────────┘ + \ No newline at end of file diff --git a/tests/queries/0_stateless/03132_pretty_format_break_line.sql b/tests/queries/0_stateless/03132_pretty_format_break_line.sql index ecf967c1836..5f002d8df67 100644 --- a/tests/queries/0_stateless/03132_pretty_format_break_line.sql +++ b/tests/queries/0_stateless/03132_pretty_format_break_line.sql @@ -1,5 +1,7 @@ DROP TABLE IF EXISTS t_break_line; +SET output_format_pretty_preserve_border_for_multiline_string=1; + CREATE TABLE t_break_line (id UInt64, value String, value1 String) ENGINE=MergeTree ORDER BY id; INSERT INTO t_break_line VALUES(0, 'hello\nworld', 'hello world'); @@ -30,4 +32,9 @@ INSERT INTO t_break_line VALUES(1, 'hello world', 'hellow\nмир'); SELECT * FROM t_break_line ORDER BY id FORMAT PrettyMonoBlock SETTINGS output_format_pretty_row_numbers = 0; SELECT * FROM t_break_line ORDER BY id FORMAT PrettyMonoBlock; +SET output_format_pretty_preserve_border_for_multiline_string=0; + +SELECT * FROM t_break_line ORDER BY id FORMAT PrettyMonoBlock SETTINGS output_format_pretty_row_numbers = 0; +SELECT * FROM t_break_line ORDER BY id FORMAT PrettyMonoBlock; + DROP TABLE t_break_line; \ No newline at end of file From ac485b612de0e7e79188fdb07cd63a3500f92132 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 7 May 2024 17:11:37 +0200 Subject: [PATCH 0896/1018] add setting to SettingsChangesHistory --- src/Core/SettingsChangesHistory.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index cd1cd341c29..9a3ed14dd7c 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -91,6 +91,7 @@ static std::map sett {"cross_join_min_rows_to_compress", 0, 10000000, "A new setting."}, {"cross_join_min_bytes_to_compress", 0, 1_GiB, "A new setting."}, {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, + {"output_format_pretty_preserve_border_for_multiline_string", 1, 1, "Applies better rendering for multiline strings."}, }}, {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, {"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"}, From 07472b3e95b8c0beceb8efc177872bb049faf6c6 Mon Sep 17 00:00:00 2001 From: Constantine Peresypkin Date: Wed, 10 Apr 2024 19:54:29 -0400 Subject: [PATCH 0897/1018] Add setting to force NULL for omitted fields Fixes #60884 --- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.h | 1 + src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + .../Impl/BSONEachRowRowInputFormat.cpp | 9 +++- .../Impl/JSONColumnsBlockInputFormatBase.cpp | 3 ++ .../Impl/JSONEachRowRowInputFormat.cpp | 10 ++++- .../Formats/Impl/TSKVRowInputFormat.cpp | 12 ++++- .../RowInputFormatWithNamesAndTypes.cpp | 20 +++++++++ .../03004_force_null_for_omitted.reference | 44 +++++++++++++++++++ .../03004_force_null_for_omitted.sql | 36 +++++++++++++++ 11 files changed, 135 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/03004_force_null_for_omitted.reference create mode 100644 tests/queries/0_stateless/03004_force_null_for_omitted.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b4313d9af56..f80bf1e4e3e 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1006,6 +1006,7 @@ class IColumn; M(Bool, input_format_tsv_empty_as_default, false, "Treat empty fields in TSV input as default values.", 0) \ M(Bool, input_format_tsv_enum_as_number, false, "Treat inserted enum values in TSV formats as enum indices.", 0) \ M(Bool, input_format_null_as_default, true, "Initialize null fields with default values if the data type of this field is not nullable and it is supported by the input format", 0) \ + M(Bool, input_format_force_null_for_omitted_fields, false, "Force initialize omitted fields with null values", 0) \ M(Bool, input_format_arrow_case_insensitive_column_matching, false, "Ignore case when matching Arrow columns with CH columns.", 0) \ M(Int64, input_format_orc_row_batch_size, 100'000, "Batch size when reading ORC stripes.", 0) \ M(Bool, input_format_orc_case_insensitive_column_matching, false, "Ignore case when matching ORC columns with CH columns.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index cd1cd341c29..5ea99aa0192 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -91,6 +91,7 @@ static std::map sett {"cross_join_min_rows_to_compress", 0, 10000000, "A new setting."}, {"cross_join_min_bytes_to_compress", 0, 1_GiB, "A new setting."}, {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, + {"input_format_force_null_for_omitted_fields", false, false, "Disable type-defaults for omitted fields when needed"}, }}, {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, {"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"}, diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index b7e9899da46..3199445864d 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -146,6 +146,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.json.throw_on_bad_escape_sequence = settings.input_format_json_throw_on_bad_escape_sequence; format_settings.json.ignore_unnecessary_fields = settings.input_format_json_ignore_unnecessary_fields; format_settings.null_as_default = settings.input_format_null_as_default; + format_settings.force_null_for_omitted_fields = settings.input_format_force_null_for_omitted_fields; format_settings.decimal_trailing_zeros = settings.output_format_decimal_trailing_zeros; format_settings.parquet.row_group_rows = settings.output_format_parquet_row_group_size; format_settings.parquet.row_group_bytes = settings.output_format_parquet_row_group_size_bytes; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index da225a39ec9..83b5c534297 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -32,6 +32,7 @@ struct FormatSettings bool write_statistics = true; bool import_nested_json = false; bool null_as_default = true; + bool force_null_for_omitted_fields = false; bool decimal_trailing_zeros = false; bool defaults_for_omitted_fields = true; bool is_writing_to_terminal = false; diff --git a/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp index 340bcc8aae5..6a3475a1830 100644 --- a/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp @@ -39,6 +39,7 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; extern const int TOO_LARGE_STRING_SIZE; extern const int UNKNOWN_TYPE; + extern const int TYPE_MISMATCH; } namespace @@ -820,7 +821,13 @@ bool BSONEachRowRowInputFormat::readRow(MutableColumns & columns, RowReadExtensi /// Fill non-visited columns with the default values. for (size_t i = 0; i < num_columns; ++i) if (!seen_columns[i]) - header.getByPosition(i).type->insertDefaultInto(*columns[i]); + { + const auto & type = header.getByPosition(i).type; + if (format_settings.force_null_for_omitted_fields && !isNullableOrLowCardinalityNullable(type)) + throw Exception(ErrorCodes::TYPE_MISMATCH, "Cannot insert NULL value into a column of type '{}' at index {}", type->getName(), i); + else + type->insertDefaultInto(*columns[i]); + } if (format_settings.defaults_for_omitted_fields) ext.read_columns = read_columns; diff --git a/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.cpp b/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.cpp index faa4f36bbb0..e61e55efc8e 100644 --- a/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.cpp +++ b/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.cpp @@ -13,6 +13,7 @@ namespace ErrorCodes { extern const int INCORRECT_DATA; extern const int EMPTY_DATA_PASSED; + extern const int TYPE_MISMATCH; } @@ -194,6 +195,8 @@ Chunk JSONColumnsBlockInputFormatBase::read() { if (!seen_columns[i]) { + if (format_settings.force_null_for_omitted_fields && !isNullableOrLowCardinalityNullable(fields[i].type)) + throw Exception(ErrorCodes::TYPE_MISMATCH, "Cannot insert NULL value into a column `{}` of type '{}'", fields[i].name, fields[i].type->getName()); columns[i]->insertManyDefaults(rows); if (format_settings.defaults_for_omitted_fields) block_missing_values.setBits(i, rows); diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index a78d8d016cd..8855a1bc28d 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -18,6 +18,7 @@ namespace ErrorCodes extern const int INCORRECT_DATA; extern const int CANNOT_READ_ALL_DATA; extern const int LOGICAL_ERROR; + extern const int TYPE_MISMATCH; } namespace @@ -233,7 +234,14 @@ bool JSONEachRowRowInputFormat::readRow(MutableColumns & columns, RowReadExtensi /// Fill non-visited columns with the default values. for (size_t i = 0; i < num_columns; ++i) if (!seen_columns[i]) - header.getByPosition(i).type->insertDefaultInto(*columns[i]); + { + const auto & type = header.getByPosition(i).type; + if (format_settings.force_null_for_omitted_fields && !isNullableOrLowCardinalityNullable(type)) + throw Exception(ErrorCodes::TYPE_MISMATCH, "Cannot insert NULL value into a column `{}` of type '{}'", columnName(i), type->getName()); + else + type->insertDefaultInto(*columns[i]); + } + /// Return info about defaults set. /// If defaults_for_omitted_fields is set to 0, we should just leave already inserted defaults. diff --git a/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp b/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp index 29bc0012dc0..5382527fcdc 100644 --- a/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp @@ -15,6 +15,7 @@ namespace ErrorCodes extern const int CANNOT_PARSE_ESCAPE_SEQUENCE; extern const int CANNOT_READ_ALL_DATA; extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED; + extern const int TYPE_MISMATCH; } @@ -190,7 +191,16 @@ bool TSKVRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ex /// Fill in the not met columns with default values. for (size_t i = 0; i < num_columns; ++i) if (!seen_columns[i]) - header.getByPosition(i).type->insertDefaultInto(*columns[i]); + { + const auto & type = header.getByPosition(i).type; + if (format_settings.force_null_for_omitted_fields && !isNullableOrLowCardinalityNullable(type)) + throw Exception( + ErrorCodes::TYPE_MISMATCH, + "Cannot insert NULL value into a column `{}` of type '{}'", + header.getByPosition(i).name, + type->getName()); + type->insertDefaultInto(*columns[i]); + } /// return info about defaults set if (format_settings.defaults_for_omitted_fields) diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp index 2ad6a825c8f..ae30d741c2f 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp @@ -18,6 +18,7 @@ namespace ErrorCodes { extern const int INCORRECT_DATA; extern const int LOGICAL_ERROR; + extern const int TYPE_MISMATCH; } namespace @@ -124,6 +125,17 @@ void RowInputFormatWithNamesAndTypes::readPrefix() } } } + + if (format_settings.force_null_for_omitted_fields) + { + for (auto index : column_mapping->not_presented_columns) + if (!isNullableOrLowCardinalityNullable(data_types[index])) + throw Exception( + ErrorCodes::TYPE_MISMATCH, + "Cannot insert NULL value into a column type '{}' at index {}", + data_types[index]->getName(), + index); + } } void RowInputFormatWithNamesAndTypes::tryDetectHeader(std::vector & column_names_out, std::vector & type_names_out) @@ -217,7 +229,15 @@ bool RowInputFormatWithNamesAndTypes::readRow(MutableColumns & columns, RowReadE { const auto & rem_column_index = column_mapping->column_indexes_for_input_fields[file_column]; if (rem_column_index) + { + if (format_settings.force_null_for_omitted_fields && !isNullableOrLowCardinalityNullable(data_types[*rem_column_index])) + throw Exception( + ErrorCodes::TYPE_MISMATCH, + "Cannot insert NULL value into a column type '{}' at index {}", + data_types[*rem_column_index]->getName(), + *rem_column_index); columns[*rem_column_index]->insertDefault(); + } ++file_column; } break; diff --git a/tests/queries/0_stateless/03004_force_null_for_omitted.reference b/tests/queries/0_stateless/03004_force_null_for_omitted.reference new file mode 100644 index 00000000000..a4c928aae8c --- /dev/null +++ b/tests/queries/0_stateless/03004_force_null_for_omitted.reference @@ -0,0 +1,44 @@ +0 0 +0 0 +2 0 +0 0 +4 0 +0 \N +0 \N +2 \N +0 \N +4 \N +0 \N +0 \N +2 \N +0 \N +4 \N +0 \N +0 \N +2 \N +0 \N +4 \N +0 \N +0 \N +2 \N +0 \N +4 \N +0 +0 \N +1 \N +1 \N +1 \N +1 \N +1 0 +1 \N +1 \N +1 2 +3 0 +1 0 +1 \N +1 \N +1 2 +3 0 +1 0 +1 \N +1 \N diff --git a/tests/queries/0_stateless/03004_force_null_for_omitted.sql b/tests/queries/0_stateless/03004_force_null_for_omitted.sql new file mode 100644 index 00000000000..43ba2568acb --- /dev/null +++ b/tests/queries/0_stateless/03004_force_null_for_omitted.sql @@ -0,0 +1,36 @@ +set allow_suspicious_low_cardinality_types = 1; +insert into function file(concat(currentDatabase(), '.03004_data.bsonEachRow'), auto, 'null Nullable(UInt32)') select number % 2 ? NULL : number from numbers(5) settings engine_file_truncate_on_insert=1; +select * from file(concat(currentDatabase(), '.03004_data.bsonEachRow'), auto, 'null UInt32, foo UInt32'); +select * from file(concat(currentDatabase(), '.03004_data.bsonEachRow'), auto, 'null UInt32, foo UInt32') settings input_format_force_null_for_omitted_fields = 1; -- { serverError TYPE_MISMATCH } +select * from file(concat(currentDatabase(), '.03004_data.bsonEachRow'), auto, 'null UInt32, foo Nullable(UInt32)'); +select * from file(concat(currentDatabase(), '.03004_data.bsonEachRow'), auto, 'null UInt32, foo Nullable(UInt32)') settings input_format_force_null_for_omitted_fields = 1; +select * from file(concat(currentDatabase(), '.03004_data.bsonEachRow'), auto, 'null UInt32, foo LowCardinality(Nullable(UInt32))'); +select * from file(concat(currentDatabase(), '.03004_data.bsonEachRow'), auto, 'null UInt32, foo LowCardinality(Nullable(UInt32))') settings input_format_force_null_for_omitted_fields = 1; + +select * from format(JSONEachRow, 'foo UInt32', '{}'); +select * from format(JSONEachRow, 'foo UInt32', '{}') settings input_format_force_null_for_omitted_fields = 1; -- { serverError TYPE_MISMATCH } +select * from format(JSONEachRow, 'foo UInt32, bar Nullable(UInt32)', '{}'); +select * from format(JSONEachRow, 'foo UInt32, bar Nullable(UInt32)', '{\"foo\":1}'); +select * from format(JSONEachRow, 'foo UInt32, bar Nullable(UInt32)', '{}') settings input_format_force_null_for_omitted_fields = 1; -- { serverError TYPE_MISMATCH } +select * from format(JSONEachRow, 'foo UInt32, bar Nullable(UInt32)', '{\"foo\":1}') settings input_format_force_null_for_omitted_fields = 1; +select * from format(JSONEachRow, 'foo UInt32, bar LowCardinality(Nullable(UInt32))', '{\"foo\":1}'); +select * from format(JSONEachRow, 'foo UInt32, bar LowCardinality(Nullable(UInt32))', '{\"foo\":1}') settings input_format_force_null_for_omitted_fields = 1; + +select * from format(CSVWithNamesAndTypes, 'foo UInt32, bar UInt32', 'foo\nUInt32\n1'); +select * from format(CSVWithNamesAndTypes, 'foo UInt32, bar UInt32', 'foo\nUInt32\n1') settings input_format_force_null_for_omitted_fields = 1; -- { serverError TYPE_MISMATCH } +select * from format(CSVWithNamesAndTypes, 'foo UInt32, bar Nullable(UInt32)', 'foo\nUInt32\n1') settings input_format_force_null_for_omitted_fields = 1; +select * from format(CSVWithNamesAndTypes, 'foo UInt32, bar LowCardinality(Nullable(UInt32))', 'foo\nUInt32\n1') settings input_format_force_null_for_omitted_fields = 1; +select * from format(CSVWithNamesAndTypes, 'foo UInt32, bar UInt32', 'foo,bar\nUInt32,UInt32\n1,2\n3\n') settings input_format_csv_allow_variable_number_of_columns = 1; +select * from format(CSVWithNamesAndTypes, 'foo UInt32, bar UInt32', 'foo,bar\nUInt32,UInt32\n1,2\n3\n') settings input_format_csv_allow_variable_number_of_columns = 1, input_format_force_null_for_omitted_fields = 1; -- { serverError TYPE_MISMATCH } + +select * from format(TSVWithNamesAndTypes, 'foo UInt32, bar UInt32', 'foo\nUInt32\n1'); +select * from format(TSVWithNamesAndTypes, 'foo UInt32, bar UInt32', 'foo\nUInt32\n1') settings input_format_force_null_for_omitted_fields = 1; -- { serverError TYPE_MISMATCH } +select * from format(TSVWithNamesAndTypes, 'foo UInt32, bar Nullable(UInt32)', 'foo\nUInt32\n1') settings input_format_force_null_for_omitted_fields = 1; +select * from format(TSVWithNamesAndTypes, 'foo UInt32, bar LowCardinality(Nullable(UInt32))', 'foo\nUInt32\n1') settings input_format_force_null_for_omitted_fields = 1; +select * from format(TSVWithNamesAndTypes, 'foo UInt32, bar UInt32', 'foo\tbar\nUInt32\tUInt32\n1\t2\n3\n') settings input_format_tsv_allow_variable_number_of_columns = 1; +select * from format(TSVWithNamesAndTypes, 'foo UInt32, bar UInt32', 'foo\tbar\nUInt32\tUInt32\n1\t2\n3\n') settings input_format_tsv_allow_variable_number_of_columns = 1, input_format_force_null_for_omitted_fields = 1; -- { serverError TYPE_MISMATCH } + +select * from format(TSKV, 'foo UInt32, bar UInt32', 'foo=1\n'); +select * from format(TSKV, 'foo UInt32, bar UInt32', 'foo=1\n') settings input_format_force_null_for_omitted_fields = 1; -- { serverError TYPE_MISMATCH } +select * from format(TSKV, 'foo UInt32, bar Nullable(UInt32)', 'foo=1\n') settings input_format_force_null_for_omitted_fields = 1; +select * from format(TSKV, 'foo UInt32, bar LowCardinality(Nullable(UInt32))', 'foo=1\n') settings input_format_force_null_for_omitted_fields = 1; From b2377c3fefe8951158de201ea399485f6805f955 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 7 May 2024 15:31:35 +0000 Subject: [PATCH 0898/1018] Fix mysql dictionary source --- src/Dictionaries/ExternalQueryBuilder.cpp | 2 +- .../test_dictionaries_mysql/test.py | 38 ++++++++++++++++++- 2 files changed, 37 insertions(+), 3 deletions(-) diff --git a/src/Dictionaries/ExternalQueryBuilder.cpp b/src/Dictionaries/ExternalQueryBuilder.cpp index 792c4e3e907..a31301cd0f3 100644 --- a/src/Dictionaries/ExternalQueryBuilder.cpp +++ b/src/Dictionaries/ExternalQueryBuilder.cpp @@ -401,7 +401,7 @@ std::string ExternalQueryBuilder::composeLoadKeysQuery( { writeString("SELECT * FROM (", out); writeString(query, out); - writeString(") WHERE ", out); + writeString(") AS subquery WHERE ", out); composeKeysCondition(key_columns, requested_rows, method, partition_key_prefix, out); writeString(";", out); diff --git a/tests/integration/test_dictionaries_mysql/test.py b/tests/integration/test_dictionaries_mysql/test.py index 360456b2046..332f4ca11bb 100644 --- a/tests/integration/test_dictionaries_mysql/test.py +++ b/tests/integration/test_dictionaries_mysql/test.py @@ -76,7 +76,7 @@ def test_mysql_dictionaries_custom_query_full_load(started_cluster): query = instance.query query( - """ + f""" CREATE DICTIONARY test_dictionary_custom_query ( id UInt64, @@ -95,12 +95,46 @@ def test_mysql_dictionaries_custom_query_full_load(started_cluster): """ ) - result = query("SELECT id, value_1, value_2 FROM test_dictionary_custom_query") + result = query( + "SELECT dictGetString('test_dictionary_custom_query', 'value_1', toUInt64(1))" + ) + assert result == "Value_1\n" + result = query("SELECT id, value_1, value_2 FROM test_dictionary_custom_query") assert result == "1\tValue_1\tValue_2\n" query("DROP DICTIONARY test_dictionary_custom_query;") + query( + f""" + CREATE DICTIONARY test_cache_dictionary_custom_query + ( + id1 UInt64, + id2 UInt64, + value_concat String + ) + PRIMARY KEY id1, id2 + LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10)) + SOURCE(MYSQL( + HOST 'mysql80' + PORT 3306 + USER 'root' + PASSWORD 'clickhouse' + QUERY 'SELECT id AS id1, id + 1 AS id2, CONCAT_WS(" ", "The", value_1) AS value_concat FROM test.test_table_1')) + LIFETIME(0) + """ + ) + + result = query( + "SELECT dictGetString('test_cache_dictionary_custom_query', 'value_concat', (1, 2))" + ) + assert result == "The Value_1\n" + + result = query("SELECT id1, value_concat FROM test_cache_dictionary_custom_query") + assert result == "1\tThe Value_1\n" + + query("DROP DICTIONARY test_cache_dictionary_custom_query;") + execute_mysql_query(mysql_connection, "DROP TABLE test.test_table_1;") execute_mysql_query(mysql_connection, "DROP TABLE test.test_table_2;") From 8e072f6b8d567ea82aa0c44bad6f26228b7bb96f Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 7 May 2024 18:58:51 +0200 Subject: [PATCH 0899/1018] Newline at the end of .reference --- .../queries/0_stateless/03132_pretty_format_break_line.reference | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/03132_pretty_format_break_line.reference b/tests/queries/0_stateless/03132_pretty_format_break_line.reference index b7b59df24fc..06b17ce4e12 100644 --- a/tests/queries/0_stateless/03132_pretty_format_break_line.reference +++ b/tests/queries/0_stateless/03132_pretty_format_break_line.reference @@ -104,4 +104,3 @@ world │ hello world │ 2. │ 1 │ hello world │ hellow мир │ └────┴─────────────┴─────────────┘ - \ No newline at end of file From 836cf150b5b4a9625aee0d440a0d64a966b4c4e0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 7 May 2024 17:39:04 +0200 Subject: [PATCH 0900/1018] Fix --- src/Disks/StoragePolicy.cpp | 13 ++++++---- .../test_disk_over_web_server/test.py | 24 +++++++++++++++++++ 2 files changed, 32 insertions(+), 5 deletions(-) diff --git a/src/Disks/StoragePolicy.cpp b/src/Disks/StoragePolicy.cpp index 390afb368f8..ccdc34d5d06 100644 --- a/src/Disks/StoragePolicy.cpp +++ b/src/Disks/StoragePolicy.cpp @@ -462,15 +462,18 @@ StoragePolicySelectorPtr StoragePolicySelector::updateFromConfig(const Poco::Uti /// First pass, check. for (const auto & [name, policy] : policies) { - if (name.starts_with(TMP_STORAGE_POLICY_PREFIX)) - continue; + if (!name.starts_with(TMP_STORAGE_POLICY_PREFIX)) + { + if (!result->policies.contains(name)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Storage policy {} is missing in new configuration", backQuote(name)); - if (!result->policies.contains(name)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Storage policy {} is missing in new configuration", backQuote(name)); + policy->checkCompatibleWith(result->policies[name]); + } - policy->checkCompatibleWith(result->policies[name]); for (const auto & disk : policy->getDisks()) + { disks_before_reload.insert(disk->getName()); + } } /// Second pass, load. diff --git a/tests/integration/test_disk_over_web_server/test.py b/tests/integration/test_disk_over_web_server/test.py index 8ddc1ff3c31..15b26c6b09e 100644 --- a/tests/integration/test_disk_over_web_server/test.py +++ b/tests/integration/test_disk_over_web_server/test.py @@ -40,6 +40,12 @@ def cluster(): image="clickhouse/clickhouse-server", tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, ) + cluster.add_instance( + "node5", + main_configs=["configs/storage_conf.xml"], + with_nginx=True, + allow_analyzer=False, + ) cluster.start() @@ -390,3 +396,21 @@ def test_page_cache(cluster): node.query("DROP TABLE test{} SYNC".format(i)) print(f"Ok {i}") + + +def test_config_reload(cluster): + node1 = cluster.instances["node5"] + table_name = "config_reload" + + global uuids + node1.query( + f""" + DROP TABLE IF EXISTS {table_name}; + CREATE TABLE {table_name} UUID '{uuids[0]}' + (id Int32) ENGINE = MergeTree() ORDER BY id + SETTINGS disk = disk(type=web, endpoint='http://nginx:80/test1/'); + """ + ) + + node1.query("SYSTEM RELOAD CONFIG") + node1.query(f"DROP TABLE {table_name} SYNC") From 9ecbff2573adc3f335ba60c0a2e487a89339a852 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 7 May 2024 19:03:17 +0200 Subject: [PATCH 0901/1018] Update 03145_non_loaded_projection_backup.sh --- .../0_stateless/03145_non_loaded_projection_backup.sh | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh b/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh index 6f0e00ce3fc..b542c9fff9a 100755 --- a/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh +++ b/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh @@ -36,6 +36,7 @@ backup table tp_1 to Disk('backups', '$backup_id'); " | grep -o "BACKUP_CREATED" $CLICKHOUSE_CLIENT -nm -q " +set send_logs_level='fatal'; drop table tp_1; restore table tp_1 from Disk('backups', '$backup_id'); " | grep -o "RESTORED" @@ -47,4 +48,6 @@ check table tp_1 settings check_query_single_value_result = 0;" | grep -o "Found $CLICKHOUSE_CLIENT -nm -q " set send_logs_level='fatal'; check table tp_1" -$CLICKHOUSE_CLIENT -q "drop table tp_1" +$CLICKHOUSE_CLIENT -nm -q " +set send_logs_level='fatal'; +drop table tp_1" From ed59a5c248e613aeab40f98dfedd2e9cd5c88dee Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 7 May 2024 19:17:00 +0200 Subject: [PATCH 0902/1018] Update PrettyBlockOutputFormat.cpp --- src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index e23622bc2e5..d84ffe800e7 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -345,7 +345,7 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind writeReadableNumberTip(chunk); writeCString("\n", out); - if (has_transferred_row && format_settings.pretty.preserve_border_for_multiline_string) + if (has_transferred_row) writeTransferredRow(max_widths, header, transferred_row, cut_to_width, false); } From cdcdb4cf71420aced463b01869da25be8015b75c Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 7 May 2024 19:35:09 +0200 Subject: [PATCH 0903/1018] remove unnecessary checks --- src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp | 4 ++-- .../Formats/Impl/PrettyCompactBlockOutputFormat.cpp | 2 +- src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index d84ffe800e7..41c7bfa316b 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -453,7 +453,7 @@ void PrettyBlockOutputFormat::writeValueWithPadding( value_width = format_settings.pretty.max_value_width; has_break_line = false; } - else if (!has_break_line || !format_settings.pretty.preserve_border_for_multiline_string) + else if (!has_break_line) value += ' '; auto write_padding = [&]() @@ -478,7 +478,7 @@ void PrettyBlockOutputFormat::writeValueWithPadding( write_padding(); } - if (has_break_line && format_settings.pretty.preserve_border_for_multiline_string) + if (has_break_line) writeString("…", out); } diff --git a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp index 1ccb6d713d7..ce22a3b2864 100644 --- a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp @@ -194,7 +194,7 @@ void PrettyCompactBlockOutputFormat::writeRow( writeReadableNumberTip(chunk); writeCString("\n", out); - if (has_transferred_row && format_settings.pretty.preserve_border_for_multiline_string) + if (has_transferred_row) writeTransferredRow(max_widths, header, transferred_row, cut_to_width, false); } diff --git a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp index 0ed8c4398e4..d311f005173 100644 --- a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp @@ -111,7 +111,7 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port writeReadableNumberTip(chunk); writeChar('\n', out); - if (has_transferred_row && format_settings.pretty.preserve_border_for_multiline_string) + if (has_transferred_row) writeTransferredRow(max_widths, header, transferred_row, cut_to_width, true); } From a2e9b6f4c61de96a12edee926c757c3d776d01a7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 7 May 2024 17:42:43 +0000 Subject: [PATCH 0904/1018] Another attempt. --- src/Planner/PlannerExpressionAnalysis.cpp | 49 ++++++++++++++++++++--- 1 file changed, 43 insertions(+), 6 deletions(-) diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index d7fa270a643..6e194b2c03e 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -51,6 +51,41 @@ FilterAnalysisResult analyzeFilter(const QueryTreeNodePtr & filter_expression_no return result; } +bool isDeterministicConstant(const ConstantNode & root) +{ + const auto & source_expression = root.getSourceExpression(); + if (!source_expression) + return true; + + std::stack nodes; + nodes.push(source_expression.get()); + while (!nodes.empty()) + { + const auto * node = nodes.top(); + nodes.pop(); + + const auto * constant_node = node->as(); + const auto * function_node = node->as(); + if (constant_node) + { + if (!isDeterministicConstant(*constant_node)) + return false; + } + else if (function_node) + { + if (!function_node->getFunctionOrThrow()->isDeterministic()) + return false; + + for (const auto & child : function_node->getArguments()) + nodes.push(child.get()); + } + else + return false; + } + + return true; +} + /** Construct aggregation analysis result if query tree has GROUP BY or aggregates. * Actions before aggregation are added into actions chain, if result is not null optional. */ @@ -86,6 +121,8 @@ std::optional analyzeAggregation(const QueryTreeNodeP (query_node.isGroupByWithGroupingSets() || query_node.isGroupByWithRollup() || query_node.isGroupByWithCube()); bool is_secondary_query = planner_context->getQueryContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY; + bool is_distributed_query = planner_context->getQueryContext()->isDistributed(); + bool check_deterministic_constants = is_secondary_query || is_distributed_query; if (query_node.hasGroupBy()) { @@ -99,10 +136,10 @@ std::optional analyzeAggregation(const QueryTreeNodeP for (auto & grouping_set_key_node : grouping_set_keys_list_node_typed.getNodes()) { - auto is_constant_key = grouping_set_key_node->as() != nullptr; - group_by_with_constant_keys |= is_constant_key; + const auto * constant_key = grouping_set_key_node->as(); + group_by_with_constant_keys |= (constant_key != nullptr); - if (!is_secondary_query && is_constant_key && !aggregates_descriptions.empty()) + if (constant_key && !aggregates_descriptions.empty() && (!check_deterministic_constants || isDeterministicConstant(*constant_key))) continue; auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions, grouping_set_key_node); @@ -151,10 +188,10 @@ std::optional analyzeAggregation(const QueryTreeNodeP { for (auto & group_by_key_node : query_node.getGroupBy().getNodes()) { - auto is_constant_key = group_by_key_node->as() != nullptr; - group_by_with_constant_keys |= is_constant_key; + const auto * constant_key = group_by_key_node->as(); + group_by_with_constant_keys |= (constant_key != nullptr); - if (!is_secondary_query && is_constant_key && !aggregates_descriptions.empty()) + if (constant_key && !aggregates_descriptions.empty() && (!check_deterministic_constants || isDeterministicConstant(*constant_key))) continue; auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions, group_by_key_node); From 511146c99c7d0c92802052643ae71e6f3f4c6dad Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 7 May 2024 19:51:47 +0200 Subject: [PATCH 0905/1018] Update CHANGELOG.md --- CHANGELOG.md | 1 - 1 file changed, 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 955e2f5b72f..f40c42c4462 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -33,7 +33,6 @@ * A mode for `topK`/`topkWeighed` support mode, which return count of values and its error. [#54508](https://github.com/ClickHouse/ClickHouse/pull/54508) ([UnamedRus](https://github.com/UnamedRus)). * Added function `toMillisecond` which returns the millisecond component for values of type`DateTime` or `DateTime64`. [#60281](https://github.com/ClickHouse/ClickHouse/pull/60281) ([Shaun Struwig](https://github.com/Blargian)). * Allow configuring HTTP redirect handlers for clickhouse-server. For example, you can make `/` redirect to the Play UI. [#60390](https://github.com/ClickHouse/ClickHouse/pull/60390) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Allow Raw as a synonym for TSVRaw. [#63394](https://github.com/ClickHouse/ClickHouse/pull/63394) ([Unalian](https://github.com/Unalian)) #### Performance Improvement * Optimized function `dotProduct` to omit unnecessary and expensive memory copies. [#60928](https://github.com/ClickHouse/ClickHouse/pull/60928) ([Robert Schulze](https://github.com/rschu1ze)). From f52dfd98aa0ff7d1c037da02fdf2cf402e7ad3a6 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 7 May 2024 16:31:21 +0000 Subject: [PATCH 0906/1018] add test for 49307 --- .../03148_mutations_virtual_columns.reference | 1 + .../03148_mutations_virtual_columns.sql | 16 ++++++++++++++++ 2 files changed, 17 insertions(+) create mode 100644 tests/queries/0_stateless/03148_mutations_virtual_columns.reference create mode 100644 tests/queries/0_stateless/03148_mutations_virtual_columns.sql diff --git a/tests/queries/0_stateless/03148_mutations_virtual_columns.reference b/tests/queries/0_stateless/03148_mutations_virtual_columns.reference new file mode 100644 index 00000000000..7c5e8041147 --- /dev/null +++ b/tests/queries/0_stateless/03148_mutations_virtual_columns.reference @@ -0,0 +1 @@ +2 all_2_2_0 diff --git a/tests/queries/0_stateless/03148_mutations_virtual_columns.sql b/tests/queries/0_stateless/03148_mutations_virtual_columns.sql new file mode 100644 index 00000000000..045869b224a --- /dev/null +++ b/tests/queries/0_stateless/03148_mutations_virtual_columns.sql @@ -0,0 +1,16 @@ +DROP TABLE IF EXISTS t_mut_virtuals; + +CREATE TABLE t_mut_virtuals (id UInt64, s String) ENGINE = MergeTree ORDER BY id; + +INSERT INTO t_mut_virtuals VALUES (1, 'a'); +INSERT INTO t_mut_virtuals VALUES (2, 'b'); + +SET insert_keeper_fault_injection_probability = 0; +SET mutations_sync = 2; + +ALTER TABLE t_mut_virtuals UPDATE s = _part WHERE 1; +ALTER TABLE t_mut_virtuals DELETE WHERE _part LIKE 'all_1_1_0%'; + +SELECT * FROM t_mut_virtuals ORDER BY id; + +DROP TABLE t_mut_virtuals; From 95b76bf6a47f0e23d41ce33c2223cee93066ad3e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 7 May 2024 20:58:19 +0200 Subject: [PATCH 0907/1018] Remove leftovers of GCC support in cmake rules Signed-off-by: Azat Khuzhin --- CMakeLists.txt | 119 +++++++++------------ cmake/linux/default_libs.cmake | 16 ++- cmake/sanitize.cmake | 8 +- cmake/tools.cmake | 85 +++++---------- cmake/warnings.cmake | 66 ++++++------ contrib/capnproto-cmake/CMakeLists.txt | 4 +- contrib/openssl-cmake/CMakeLists.txt | 10 +- contrib/sentry-native-cmake/CMakeLists.txt | 2 +- 8 files changed, 125 insertions(+), 185 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index be804a14765..abbc48ab23a 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -135,23 +135,21 @@ endif () include (cmake/check_flags.cmake) include (cmake/add_warning.cmake) -if (COMPILER_CLANG) - # generate ranges for fast "addr2line" search - if (NOT CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE") - # NOTE: that clang has a bug because of it does not emit .debug_aranges - # with ThinLTO, so custom ld.lld wrapper is shipped in docker images. - set(COMPILER_FLAGS "${COMPILER_FLAGS} -gdwarf-aranges") - endif () - - # See https://blog.llvm.org/posts/2021-04-05-constructor-homing-for-debug-info/ - if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG" OR CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO") - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Xclang -fuse-ctor-homing") - set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Xclang -fuse-ctor-homing") - endif() - - no_warning(enum-constexpr-conversion) # breaks Protobuf in clang-16 +# generate ranges for fast "addr2line" search +if (NOT CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE") + # NOTE: that clang has a bug because of it does not emit .debug_aranges + # with ThinLTO, so custom ld.lld wrapper is shipped in docker images. + set(COMPILER_FLAGS "${COMPILER_FLAGS} -gdwarf-aranges") endif () +# See https://blog.llvm.org/posts/2021-04-05-constructor-homing-for-debug-info/ +if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG" OR CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO") + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Xclang -fuse-ctor-homing") + set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Xclang -fuse-ctor-homing") +endif() + +no_warning(enum-constexpr-conversion) # breaks Protobuf in clang-16 + option(ENABLE_TESTS "Provide unit_test_dbms target with Google.Test unit tests" ON) option(ENABLE_EXAMPLES "Build all example programs in 'examples' subdirectories" OFF) option(ENABLE_BENCHMARKS "Build all benchmark programs in 'benchmarks' subdirectories" OFF) @@ -284,16 +282,12 @@ endif () option (ENABLE_BUILD_PROFILING "Enable profiling of build time" OFF) if (ENABLE_BUILD_PROFILING) - if (COMPILER_CLANG) - set (COMPILER_FLAGS "${COMPILER_FLAGS} -ftime-trace") + set (COMPILER_FLAGS "${COMPILER_FLAGS} -ftime-trace") - if (LINKER_NAME MATCHES "lld") - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--time-trace") - set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -Wl,--time-trace") - endif () - else () - message (${RECONFIGURE_MESSAGE_LEVEL} "Build profiling is only available with CLang") - endif () + if (LINKER_NAME MATCHES "lld") + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--time-trace") + set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -Wl,--time-trace") + endif () endif () set (CMAKE_CXX_STANDARD 23) @@ -304,22 +298,20 @@ set (CMAKE_C_STANDARD 11) set (CMAKE_C_EXTENSIONS ON) # required by most contribs written in C set (CMAKE_C_STANDARD_REQUIRED ON) -if (COMPILER_CLANG) - # Enable C++14 sized global deallocation functions. It should be enabled by setting -std=c++14 but I'm not sure. - # See https://reviews.llvm.org/D112921 - set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fsized-deallocation") +# Enable C++14 sized global deallocation functions. It should be enabled by setting -std=c++14 but I'm not sure. +# See https://reviews.llvm.org/D112921 +set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fsized-deallocation") - # falign-functions=32 prevents from random performance regressions with the code change. Thus, providing more stable - # benchmarks. - set(COMPILER_FLAGS "${COMPILER_FLAGS} -falign-functions=32") +# falign-functions=32 prevents from random performance regressions with the code change. Thus, providing more stable +# benchmarks. +set(COMPILER_FLAGS "${COMPILER_FLAGS} -falign-functions=32") - if (ARCH_AMD64) - # align branches within a 32-Byte boundary to avoid the potential performance loss when code layout change, - # which makes benchmark results more stable. - set(BRANCHES_WITHIN_32B_BOUNDARIES "-mbranches-within-32B-boundaries") - set(COMPILER_FLAGS "${COMPILER_FLAGS} ${BRANCHES_WITHIN_32B_BOUNDARIES}") - endif() -endif () +if (ARCH_AMD64) + # align branches within a 32-Byte boundary to avoid the potential performance loss when code layout change, + # which makes benchmark results more stable. + set(BRANCHES_WITHIN_32B_BOUNDARIES "-mbranches-within-32B-boundaries") + set(COMPILER_FLAGS "${COMPILER_FLAGS} ${BRANCHES_WITHIN_32B_BOUNDARIES}") +endif() # Disable floating-point expression contraction in order to get consistent floating point calculation results across platforms set (COMPILER_FLAGS "${COMPILER_FLAGS} -ffp-contract=off") @@ -348,39 +340,34 @@ set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} ${COMPILER_FLAGS} $ set (CMAKE_ASM_FLAGS_RELWITHDEBINFO "${CMAKE_ASM_FLAGS_RELWITHDEBINFO} -O3 ${DEBUG_INFO_FLAGS} ${CMAKE_ASM_FLAGS_ADD}") set (CMAKE_ASM_FLAGS_DEBUG "${CMAKE_ASM_FLAGS_DEBUG} -O0 ${DEBUG_INFO_FLAGS} ${CMAKE_ASM_FLAGS_ADD}") -if (COMPILER_CLANG) - if (OS_DARWIN) - set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -stdlib=libc++") - set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,-U,_inside_main") - endif() +if (OS_DARWIN) + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -stdlib=libc++") + set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,-U,_inside_main") +endif() - # Display absolute paths in error messages. Otherwise KDevelop fails to navigate to correct file and opens a new file instead. - set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fdiagnostics-absolute-paths") - set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fdiagnostics-absolute-paths") +# Display absolute paths in error messages. Otherwise KDevelop fails to navigate to correct file and opens a new file instead. +set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fdiagnostics-absolute-paths") +set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fdiagnostics-absolute-paths") - if (NOT ENABLE_TESTS AND NOT SANITIZE AND NOT SANITIZE_COVERAGE AND OS_LINUX) - # https://clang.llvm.org/docs/ThinLTO.html - # Applies to clang and linux only. - # Disabled when building with tests or sanitizers. - option(ENABLE_THINLTO "Clang-specific link time optimization" ON) - endif() +if (NOT ENABLE_TESTS AND NOT SANITIZE AND NOT SANITIZE_COVERAGE AND OS_LINUX) + # https://clang.llvm.org/docs/ThinLTO.html + # Applies to clang and linux only. + # Disabled when building with tests or sanitizers. + option(ENABLE_THINLTO "Clang-specific link time optimization" ON) +endif() - set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fstrict-vtable-pointers") - - # We cannot afford to use LTO when compiling unit tests, and it's not enough - # to only supply -fno-lto at the final linking stage. So we disable it - # completely. - if (ENABLE_THINLTO AND NOT ENABLE_TESTS AND NOT SANITIZE) - # Link time optimization - set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -flto=thin -fwhole-program-vtables") - set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -flto=thin -fwhole-program-vtables") - set (CMAKE_EXE_LINKER_FLAGS_RELWITHDEBINFO "${CMAKE_EXE_LINKER_FLAGS_RELWITHDEBINFO} -flto=thin -fwhole-program-vtables") - elseif (ENABLE_THINLTO) - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot enable ThinLTO") - endif () +set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fstrict-vtable-pointers") +# We cannot afford to use LTO when compiling unit tests, and it's not enough +# to only supply -fno-lto at the final linking stage. So we disable it +# completely. +if (ENABLE_THINLTO AND NOT ENABLE_TESTS AND NOT SANITIZE) + # Link time optimization + set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -flto=thin -fwhole-program-vtables") + set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -flto=thin -fwhole-program-vtables") + set (CMAKE_EXE_LINKER_FLAGS_RELWITHDEBINFO "${CMAKE_EXE_LINKER_FLAGS_RELWITHDEBINFO} -flto=thin -fwhole-program-vtables") elseif (ENABLE_THINLTO) - message (${RECONFIGURE_MESSAGE_LEVEL} "ThinLTO is only available with Clang") + message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot enable ThinLTO") endif () # Turns on all external libs like s3, kafka, ODBC, ... diff --git a/cmake/linux/default_libs.cmake b/cmake/linux/default_libs.cmake index e5ca8e296fc..4a06243243e 100644 --- a/cmake/linux/default_libs.cmake +++ b/cmake/linux/default_libs.cmake @@ -5,17 +5,15 @@ set (DEFAULT_LIBS "-nodefaultlibs") # We need builtins from Clang's RT even without libcxx - for ubsan+int128. # See https://bugs.llvm.org/show_bug.cgi?id=16404 -if (COMPILER_CLANG) - execute_process (COMMAND ${CMAKE_CXX_COMPILER} --target=${CMAKE_CXX_COMPILER_TARGET} --print-libgcc-file-name --rtlib=compiler-rt OUTPUT_VARIABLE BUILTINS_LIBRARY OUTPUT_STRIP_TRAILING_WHITESPACE) +execute_process (COMMAND ${CMAKE_CXX_COMPILER} --target=${CMAKE_CXX_COMPILER_TARGET} --print-libgcc-file-name --rtlib=compiler-rt OUTPUT_VARIABLE BUILTINS_LIBRARY OUTPUT_STRIP_TRAILING_WHITESPACE) - # Apparently, in clang-19, the UBSan support library for C++ was moved out into ubsan_standalone_cxx.a, so we have to include both. - if (SANITIZE STREQUAL undefined) - string(REPLACE "builtins.a" "ubsan_standalone_cxx.a" EXTRA_BUILTINS_LIBRARY "${BUILTINS_LIBRARY}") - endif () +# Apparently, in clang-19, the UBSan support library for C++ was moved out into ubsan_standalone_cxx.a, so we have to include both. +if (SANITIZE STREQUAL undefined) + string(REPLACE "builtins.a" "ubsan_standalone_cxx.a" EXTRA_BUILTINS_LIBRARY "${BUILTINS_LIBRARY}") +endif () - if (NOT EXISTS "${BUILTINS_LIBRARY}") - set (BUILTINS_LIBRARY "-lgcc") - endif () +if (NOT EXISTS "${BUILTINS_LIBRARY}") + set (BUILTINS_LIBRARY "-lgcc") endif () if (OS_ANDROID) diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index a3523203912..08716c1196b 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -26,9 +26,7 @@ if (SANITIZE) elseif (SANITIZE STREQUAL "thread") set (TSAN_FLAGS "-fsanitize=thread") - if (COMPILER_CLANG) - set (TSAN_FLAGS "${TSAN_FLAGS} -fsanitize-ignorelist=${PROJECT_SOURCE_DIR}/tests/tsan_ignorelist.txt") - endif() + set (TSAN_FLAGS "${TSAN_FLAGS} -fsanitize-ignorelist=${PROJECT_SOURCE_DIR}/tests/tsan_ignorelist.txt") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} ${TSAN_FLAGS}") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} ${TSAN_FLAGS}") @@ -44,9 +42,7 @@ if (SANITIZE) # that's why we often receive reports about UIO. The simplest way to avoid this is just set this flag here. set(UBSAN_FLAGS "${UBSAN_FLAGS} -fno-sanitize=unsigned-integer-overflow") endif() - if (COMPILER_CLANG) - set (UBSAN_FLAGS "${UBSAN_FLAGS} -fsanitize-ignorelist=${PROJECT_SOURCE_DIR}/tests/ubsan_ignorelist.txt") - endif() + set (UBSAN_FLAGS "${UBSAN_FLAGS} -fsanitize-ignorelist=${PROJECT_SOURCE_DIR}/tests/ubsan_ignorelist.txt") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} ${UBSAN_FLAGS}") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} ${UBSAN_FLAGS}") diff --git a/cmake/tools.cmake b/cmake/tools.cmake index 1ba3007b0f3..024505411a3 100644 --- a/cmake/tools.cmake +++ b/cmake/tools.cmake @@ -1,10 +1,6 @@ # Compiler -if (CMAKE_CXX_COMPILER_ID MATCHES "AppleClang") - set (COMPILER_CLANG 1) # Safe to treat AppleClang as a regular Clang, in general. -elseif (CMAKE_CXX_COMPILER_ID MATCHES "Clang") - set (COMPILER_CLANG 1) -else () +if (NOT CMAKE_CXX_COMPILER_ID MATCHES "Clang") message (FATAL_ERROR "Compiler ${CMAKE_CXX_COMPILER_ID} is not supported") endif () @@ -17,30 +13,26 @@ set (CLANG_MINIMUM_VERSION 16) set (XCODE_MINIMUM_VERSION 12.0) set (APPLE_CLANG_MINIMUM_VERSION 12.0.0) -if (COMPILER_CLANG) - if (CMAKE_CXX_COMPILER_ID MATCHES "AppleClang") - # (Experimental!) Specify "-DALLOW_APPLECLANG=ON" when running CMake configuration step, if you want to experiment with using it. - if (NOT ALLOW_APPLECLANG AND NOT DEFINED ENV{ALLOW_APPLECLANG}) - message (FATAL_ERROR "Compilation with AppleClang is unsupported. Please use vanilla Clang, e.g. from Homebrew.") - endif () +if (CMAKE_CXX_COMPILER_ID MATCHES "AppleClang") + # (Experimental!) Specify "-DALLOW_APPLECLANG=ON" when running CMake configuration step, if you want to experiment with using it. + if (NOT ALLOW_APPLECLANG AND NOT DEFINED ENV{ALLOW_APPLECLANG}) + message (FATAL_ERROR "Compilation with AppleClang is unsupported. Please use vanilla Clang, e.g. from Homebrew.") + endif () - # For a mapping between XCode / AppleClang / vanilla Clang versions, see https://en.wikipedia.org/wiki/Xcode - if (CMAKE_CXX_COMPILER_VERSION VERSION_LESS ${APPLE_CLANG_MINIMUM_VERSION}) - message (FATAL_ERROR "Compilation with AppleClang version ${CMAKE_CXX_COMPILER_VERSION} is unsupported, the minimum required version is ${APPLE_CLANG_MINIMUM_VERSION} (Xcode ${XCODE_MINIMUM_VERSION}).") - endif () - else () - if (CMAKE_CXX_COMPILER_VERSION VERSION_LESS ${CLANG_MINIMUM_VERSION}) - message (FATAL_ERROR "Compilation with Clang version ${CMAKE_CXX_COMPILER_VERSION} is unsupported, the minimum required version is ${CLANG_MINIMUM_VERSION}.") - endif () + # For a mapping between XCode / AppleClang / vanilla Clang versions, see https://en.wikipedia.org/wiki/Xcode + if (CMAKE_CXX_COMPILER_VERSION VERSION_LESS ${APPLE_CLANG_MINIMUM_VERSION}) + message (FATAL_ERROR "Compilation with AppleClang version ${CMAKE_CXX_COMPILER_VERSION} is unsupported, the minimum required version is ${APPLE_CLANG_MINIMUM_VERSION} (Xcode ${XCODE_MINIMUM_VERSION}).") + endif () +else () + if (CMAKE_CXX_COMPILER_VERSION VERSION_LESS ${CLANG_MINIMUM_VERSION}) + message (FATAL_ERROR "Compilation with Clang version ${CMAKE_CXX_COMPILER_VERSION} is unsupported, the minimum required version is ${CLANG_MINIMUM_VERSION}.") endif () endif () -# Linker - string (REGEX MATCHALL "[0-9]+" COMPILER_VERSION_LIST ${CMAKE_CXX_COMPILER_VERSION}) list (GET COMPILER_VERSION_LIST 0 COMPILER_VERSION_MAJOR) -# Example values: `lld-10` +# Linker option (LINKER_NAME "Linker name or full path") if (LINKER_NAME MATCHES "gold") @@ -48,19 +40,15 @@ if (LINKER_NAME MATCHES "gold") endif () if (NOT LINKER_NAME) - if (COMPILER_CLANG) - if (OS_LINUX AND NOT ARCH_S390X) - find_program (LLD_PATH NAMES "ld.lld-${COMPILER_VERSION_MAJOR}" "ld.lld") - elseif (OS_DARWIN) - find_program (LLD_PATH NAMES "ld") - endif () + if (OS_LINUX AND NOT ARCH_S390X) + find_program (LLD_PATH NAMES "ld.lld-${COMPILER_VERSION_MAJOR}" "ld.lld") + elseif (OS_DARWIN) + find_program (LLD_PATH NAMES "ld") endif () if (LLD_PATH) if (OS_LINUX OR OS_DARWIN) - if (COMPILER_CLANG) - # Clang driver simply allows full linker path. - set (LINKER_NAME ${LLD_PATH}) - endif () + # Clang driver simply allows full linker path. + set (LINKER_NAME ${LLD_PATH}) endif () endif() endif() @@ -82,47 +70,28 @@ else () endif () # Archiver - -if (COMPILER_CLANG) - find_program (LLVM_AR_PATH NAMES "llvm-ar-${COMPILER_VERSION_MAJOR}" "llvm-ar") -endif () - +find_program (LLVM_AR_PATH NAMES "llvm-ar-${COMPILER_VERSION_MAJOR}" "llvm-ar") if (LLVM_AR_PATH) set (CMAKE_AR "${LLVM_AR_PATH}") endif () - message(STATUS "Using archiver: ${CMAKE_AR}") # Ranlib - -if (COMPILER_CLANG) - find_program (LLVM_RANLIB_PATH NAMES "llvm-ranlib-${COMPILER_VERSION_MAJOR}" "llvm-ranlib") -endif () - +find_program (LLVM_RANLIB_PATH NAMES "llvm-ranlib-${COMPILER_VERSION_MAJOR}" "llvm-ranlib") if (LLVM_RANLIB_PATH) set (CMAKE_RANLIB "${LLVM_RANLIB_PATH}") endif () - message(STATUS "Using ranlib: ${CMAKE_RANLIB}") # Install Name Tool - -if (COMPILER_CLANG) - find_program (LLVM_INSTALL_NAME_TOOL_PATH NAMES "llvm-install-name-tool-${COMPILER_VERSION_MAJOR}" "llvm-install-name-tool") -endif () - +find_program (LLVM_INSTALL_NAME_TOOL_PATH NAMES "llvm-install-name-tool-${COMPILER_VERSION_MAJOR}" "llvm-install-name-tool") if (LLVM_INSTALL_NAME_TOOL_PATH) set (CMAKE_INSTALL_NAME_TOOL "${LLVM_INSTALL_NAME_TOOL_PATH}") endif () - message(STATUS "Using install-name-tool: ${CMAKE_INSTALL_NAME_TOOL}") # Objcopy - -if (COMPILER_CLANG) - find_program (OBJCOPY_PATH NAMES "llvm-objcopy-${COMPILER_VERSION_MAJOR}" "llvm-objcopy" "objcopy") -endif () - +find_program (OBJCOPY_PATH NAMES "llvm-objcopy-${COMPILER_VERSION_MAJOR}" "llvm-objcopy" "objcopy") if (OBJCOPY_PATH) message (STATUS "Using objcopy: ${OBJCOPY_PATH}") else () @@ -130,11 +99,7 @@ else () endif () # Strip - -if (COMPILER_CLANG) - find_program (STRIP_PATH NAMES "llvm-strip-${COMPILER_VERSION_MAJOR}" "llvm-strip" "strip") -endif () - +find_program (STRIP_PATH NAMES "llvm-strip-${COMPILER_VERSION_MAJOR}" "llvm-strip" "strip") if (STRIP_PATH) message (STATUS "Using strip: ${STRIP_PATH}") else () diff --git a/cmake/warnings.cmake b/cmake/warnings.cmake index 455e4f09939..807d92d9077 100644 --- a/cmake/warnings.cmake +++ b/cmake/warnings.cmake @@ -15,37 +15,35 @@ if ((NOT CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG") AND (NOT SANITIZE) AND (NOT CMAKE add_warning(frame-larger-than=65536) endif () -if (COMPILER_CLANG) - # Add some warnings that are not available even with -Wall -Wextra -Wpedantic. - # We want to get everything out of the compiler for code quality. - add_warning(everything) - add_warning(pedantic) - no_warning(zero-length-array) - no_warning(c++98-compat-pedantic) - no_warning(c++98-compat) - no_warning(c++20-compat) # Use constinit in C++20 without warnings - no_warning(sign-conversion) - no_warning(implicit-int-conversion) - no_warning(implicit-int-float-conversion) - no_warning(ctad-maybe-unsupported) # clang 9+, linux-only - no_warning(disabled-macro-expansion) - no_warning(documentation-unknown-command) - no_warning(double-promotion) - no_warning(exit-time-destructors) - no_warning(float-equal) - no_warning(global-constructors) - no_warning(missing-prototypes) - no_warning(missing-variable-declarations) - no_warning(padded) - no_warning(switch-enum) - no_warning(undefined-func-template) - no_warning(unused-template) - no_warning(vla) - no_warning(weak-template-vtables) - no_warning(weak-vtables) - no_warning(thread-safety-negative) # experimental flag, too many false positives - no_warning(enum-constexpr-conversion) # breaks magic-enum library in clang-16 - no_warning(unsafe-buffer-usage) # too aggressive - no_warning(switch-default) # conflicts with "defaults in a switch covering all enum values" - # TODO Enable conversion, sign-conversion, double-promotion warnings. -endif () +# Add some warnings that are not available even with -Wall -Wextra -Wpedantic. +# We want to get everything out of the compiler for code quality. +add_warning(everything) +add_warning(pedantic) +no_warning(zero-length-array) +no_warning(c++98-compat-pedantic) +no_warning(c++98-compat) +no_warning(c++20-compat) # Use constinit in C++20 without warnings +no_warning(sign-conversion) +no_warning(implicit-int-conversion) +no_warning(implicit-int-float-conversion) +no_warning(ctad-maybe-unsupported) # clang 9+, linux-only +no_warning(disabled-macro-expansion) +no_warning(documentation-unknown-command) +no_warning(double-promotion) +no_warning(exit-time-destructors) +no_warning(float-equal) +no_warning(global-constructors) +no_warning(missing-prototypes) +no_warning(missing-variable-declarations) +no_warning(padded) +no_warning(switch-enum) +no_warning(undefined-func-template) +no_warning(unused-template) +no_warning(vla) +no_warning(weak-template-vtables) +no_warning(weak-vtables) +no_warning(thread-safety-negative) # experimental flag, too many false positives +no_warning(enum-constexpr-conversion) # breaks magic-enum library in clang-16 +no_warning(unsafe-buffer-usage) # too aggressive +no_warning(switch-default) # conflicts with "defaults in a switch covering all enum values" +# TODO Enable conversion, sign-conversion, double-promotion warnings. diff --git a/contrib/capnproto-cmake/CMakeLists.txt b/contrib/capnproto-cmake/CMakeLists.txt index e76268592ee..c07e9e6925b 100644 --- a/contrib/capnproto-cmake/CMakeLists.txt +++ b/contrib/capnproto-cmake/CMakeLists.txt @@ -81,9 +81,7 @@ set (CAPNPC_SRCS add_library(_capnpc ${CAPNPC_SRCS}) target_link_libraries(_capnpc PUBLIC _capnp) -if (COMPILER_CLANG) - set (CAPNP_PRIVATE_CXX_FLAGS -fno-char8_t) -endif () +set (CAPNP_PRIVATE_CXX_FLAGS -fno-char8_t) target_compile_options(_kj PRIVATE ${CAPNP_PRIVATE_CXX_FLAGS}) target_compile_options(_capnp PRIVATE ${CAPNP_PRIVATE_CXX_FLAGS}) diff --git a/contrib/openssl-cmake/CMakeLists.txt b/contrib/openssl-cmake/CMakeLists.txt index 021c88bcb04..72846143b9e 100644 --- a/contrib/openssl-cmake/CMakeLists.txt +++ b/contrib/openssl-cmake/CMakeLists.txt @@ -91,12 +91,10 @@ set(LIB_SOVERSION ${VERSION_MAJOR}) enable_language(ASM) -if(COMPILER_CLANG) - add_definitions(-Wno-unused-command-line-argument) - # Note that s390x build uses mold linker - if(NOT ARCH_S390X) - set(CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=lld") # only relevant for -DENABLE_OPENSSL_DYNAMIC=1 - endif() +add_definitions(-Wno-unused-command-line-argument) +# Note that s390x build uses mold linker +if(NOT ARCH_S390X) + set(CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=lld") # only relevant for -DENABLE_OPENSSL_DYNAMIC=1 endif() if(ARCH_AMD64) diff --git a/contrib/sentry-native-cmake/CMakeLists.txt b/contrib/sentry-native-cmake/CMakeLists.txt index 6364e75db28..6e4c8c36081 100644 --- a/contrib/sentry-native-cmake/CMakeLists.txt +++ b/contrib/sentry-native-cmake/CMakeLists.txt @@ -1,4 +1,4 @@ -if (NOT OS_FREEBSD AND NOT (OS_DARWIN AND COMPILER_CLANG)) +if (NOT OS_FREEBSD AND NOT OS_DARWIN) option (ENABLE_SENTRY "Enable Sentry" ${ENABLE_LIBRARIES}) else() option (ENABLE_SENTRY "Enable Sentry" OFF) From f45e032715dcc04d3ef78d84750c0f5f835b5008 Mon Sep 17 00:00:00 2001 From: Yohann Jardin Date: Tue, 7 May 2024 21:11:04 +0200 Subject: [PATCH 0908/1018] fix errorCodes in stateless tests --- tests/queries/0_stateless/00727_concat.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00727_concat.sql b/tests/queries/0_stateless/00727_concat.sql index f5048dcaaae..01792545b5a 100644 --- a/tests/queries/0_stateless/00727_concat.sql +++ b/tests/queries/0_stateless/00727_concat.sql @@ -93,4 +93,4 @@ SELECT concat(materialize(NULL :: Nullable(UInt64))); SELECT CONCAT('Testing the ', 'alias'); -SELECT concat(); -- { serverError 42 } +SELECT concat(); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION } From 577dccd47ff70af55140b673a436354d289c1344 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 7 May 2024 21:09:19 +0200 Subject: [PATCH 0909/1018] Fix ProfileEventTimeIncrement code Signed-off-by: Azat Khuzhin --- src/Common/ElapsedTimeProfileEventIncrement.h | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/src/Common/ElapsedTimeProfileEventIncrement.h b/src/Common/ElapsedTimeProfileEventIncrement.h index 731295a4cfd..aa944beeaa9 100644 --- a/src/Common/ElapsedTimeProfileEventIncrement.h +++ b/src/Common/ElapsedTimeProfileEventIncrement.h @@ -17,19 +17,18 @@ enum Time template