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 001/624] [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 002/624] 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 003/624] 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 004/624] 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 005/624] 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 006/624] 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 007/624] 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 008/624] 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 009/624] 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 010/624] 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 011/624] 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 012/624] 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 013/624] 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 014/624] 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 015/624] 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 016/624] 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 017/624] 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 018/624] 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 019/624] 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 020/624] 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 021/624] 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 022/624] 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 023/624] 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 024/624] 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 025/624] 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 026/624] 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 027/624] 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 4d57aaebb1057d717e8df3830131ace76e5e2fe6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 22 Feb 2024 16:56:33 +0100 Subject: [PATCH 028/624] 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 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 029/624] 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 030/624] 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 031/624] 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 032/624] 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 033/624] 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 034/624] 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 035/624] 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 036/624] 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 037/624] 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 038/624] 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 039/624] 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 040/624] 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 041/624] 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 042/624] 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 043/624] 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 044/624] 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 045/624] 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 046/624] 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 047/624] 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 048/624] 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 6a41dbca0bbaae258ee034e271293479a793df9b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 14 Mar 2024 13:08:01 +0100 Subject: [PATCH 049/624] 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 050/624] 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 051/624] 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 052/624] 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 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 053/624] 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 054/624] 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 02ff01f2468b36479bd40abe23138bb28a4071b6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 19 Mar 2024 16:48:42 +0100 Subject: [PATCH 055/624] 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 d03ae0655abaef54117f6455063009aa7aed790b Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 20 Mar 2024 16:39:37 +0800 Subject: [PATCH 056/624] finish opt utf8 version of substring/substringindex/reverse --- src/Common/UTF8Helpers.cpp | 75 +++++++++++++++++++++++++++++++- src/Common/UTF8Helpers.h | 5 ++- src/Functions/reverse.cpp | 38 +--------------- src/Functions/reverse.h | 42 ++++++++++++++++++ src/Functions/reverseUTF8.cpp | 12 ++++- src/Functions/substring.cpp | 18 +++++++- src/Functions/substringIndex.cpp | 20 +++++++-- 7 files changed, 164 insertions(+), 46 deletions(-) create mode 100644 src/Functions/reverse.h diff --git a/src/Common/UTF8Helpers.cpp b/src/Common/UTF8Helpers.cpp index 0af31726f40..78d645994e5 100644 --- a/src/Common/UTF8Helpers.cpp +++ b/src/Common/UTF8Helpers.cpp @@ -1,9 +1,13 @@ -#include #include +#include +#include #include #include +#if USE_MULTITARGET_CODE +#include +#endif namespace DB { @@ -205,5 +209,74 @@ size_t computeBytesBeforeWidth(const UInt8 * data, size_t size, size_t prefix, s return computeWidthImpl(data, size, prefix, limit); } + +DECLARE_DEFAULT_CODE( +bool isAllASCII(const UInt8 * data, size_t size) +{ + UInt8 mask = 0; + for (size_t i = 0; i < size; ++i) + mask |= data[i]; + + return !(mask & 0x80); +}) + +DECLARE_SSE42_SPECIFIC_CODE( +/// Copy from https://github.com/lemire/fastvalidate-utf-8/blob/master/include/simdasciicheck.h +bool isAllASCII(const UInt8 * data, size_t size) { + size_t i = 0; + __m128i masks = _mm_setzero_si128(); + if (size >= 16) + { + for (; i <= size - 16; i += 16) + { + __m128i bytes = _mm_loadu_si128(reinterpret_cast(data + i)); + masks = _mm_or_si128(masks, bytes); + } + } + int mask = _mm_movemask_epi8(masks); + + UInt8 tail_mask = 0; + for (; i < size; i++) + tail_mask |= data[i]; + + mask |= (tail_mask & 0x80); + return !mask; +}) + +DECLARE_AVX2_SPECIFIC_CODE( +bool isAllASCII(const UInt8 * data, size_t size) +{ + size_t i = 0; + __m256i masks = _mm256_setzero_si256(); + if (size >= 32) + { + for (; i <= size - 32; i += 32) + { + __m256i bytes = _mm256_loadu_si256(reinterpret_cast(data + i)); + masks = _mm256_or_si256(masks, bytes); + } + } + int mask = _mm256_movemask_epi8(masks); + + UInt8 tail_mask = 0; + for (; i < size; i++) + tail_mask |= data[i]; + + mask |= (tail_mask & 0x80); + return !mask; +}) + +bool isAllASCII(const UInt8* data, size_t size) +{ +#if USE_MULTITARGET_CODE + if (isArchSupported(TargetArch::AVX2)) + return TargetSpecific::AVX2::isAllASCII(data, size); + if (isArchSupported(TargetArch::SSE42)) + return TargetSpecific::SSE42::isAllASCII(data, size); +#endif + return TargetSpecific::Default::isAllASCII(data, size); +} + + } } diff --git a/src/Common/UTF8Helpers.h b/src/Common/UTF8Helpers.h index a4dd88921b7..933b62c7b63 100644 --- a/src/Common/UTF8Helpers.h +++ b/src/Common/UTF8Helpers.h @@ -136,7 +136,10 @@ size_t computeWidth(const UInt8 * data, size_t size, size_t prefix = 0) noexcept */ size_t computeBytesBeforeWidth(const UInt8 * data, size_t size, size_t prefix, size_t limit) noexcept; -} +/// If all the characters in the string are ASCII, return true. +bool isAllASCII(const UInt8* data, size_t size); + +} } diff --git a/src/Functions/reverse.cpp b/src/Functions/reverse.cpp index 32b998523c7..39608b77997 100644 --- a/src/Functions/reverse.cpp +++ b/src/Functions/reverse.cpp @@ -1,10 +1,10 @@ #include -#include #include #include #include #include #include +#include "reverse.h" namespace DB @@ -17,42 +17,6 @@ namespace ErrorCodes namespace { - -/** Reverse the string as a sequence of bytes. - */ -struct ReverseImpl -{ - static void vector(const ColumnString::Chars & data, - const ColumnString::Offsets & offsets, - ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) - { - res_data.resize(data.size()); - res_offsets.assign(offsets); - size_t size = offsets.size(); - - ColumnString::Offset prev_offset = 0; - for (size_t i = 0; i < size; ++i) - { - for (size_t j = prev_offset; j < offsets[i] - 1; ++j) - res_data[j] = data[offsets[i] + prev_offset - 2 - j]; - res_data[offsets[i] - 1] = 0; - prev_offset = offsets[i]; - } - } - - static void vectorFixed(const ColumnString::Chars & data, size_t n, ColumnString::Chars & res_data) - { - res_data.resize(data.size()); - size_t size = data.size() / n; - - for (size_t i = 0; i < size; ++i) - for (size_t j = i * n; j < (i + 1) * n; ++j) - res_data[j] = data[(i * 2 + 1) * n - j - 1]; - } -}; - - class FunctionReverse : public IFunction { public: diff --git a/src/Functions/reverse.h b/src/Functions/reverse.h new file mode 100644 index 00000000000..128a897c041 --- /dev/null +++ b/src/Functions/reverse.h @@ -0,0 +1,42 @@ +#pragma once + +#include + +namespace DB +{ + +/** Reverse the string as a sequence of bytes. + */ +struct ReverseImpl +{ + static void vector(const ColumnString::Chars & data, + const ColumnString::Offsets & offsets, + ColumnString::Chars & res_data, + ColumnString::Offsets & res_offsets) + { + res_data.resize(data.size()); + res_offsets.assign(offsets); + size_t size = offsets.size(); + + ColumnString::Offset prev_offset = 0; + for (size_t i = 0; i < size; ++i) + { + for (size_t j = prev_offset; j < offsets[i] - 1; ++j) + res_data[j] = data[offsets[i] + prev_offset - 2 - j]; + res_data[offsets[i] - 1] = 0; + prev_offset = offsets[i]; + } + } + + static void vectorFixed(const ColumnString::Chars & data, size_t n, ColumnString::Chars & res_data) + { + res_data.resize(data.size()); + size_t size = data.size() / n; + + for (size_t i = 0; i < size; ++i) + for (size_t j = i * n; j < (i + 1) * n; ++j) + res_data[j] = data[(i * 2 + 1) * n - j - 1]; + } +}; + +} diff --git a/src/Functions/reverseUTF8.cpp b/src/Functions/reverseUTF8.cpp index 8a76af05d86..4ea861919a1 100644 --- a/src/Functions/reverseUTF8.cpp +++ b/src/Functions/reverseUTF8.cpp @@ -1,7 +1,9 @@ -#include #include +#include #include #include +#include +#include "reverse.h" namespace DB @@ -25,10 +27,18 @@ struct ReverseUTF8Impl ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { + bool all_ascii = UTF8::isAllASCII(data.data(), data.size()); + if (all_ascii) + { + ReverseImpl::vector(data, offsets, res_data, res_offsets); + return; + } + res_data.resize(data.size()); res_offsets.assign(offsets); size_t size = offsets.size(); + ColumnString::Offset prev_offset = 0; for (size_t i = 0; i < size; ++i) { diff --git a/src/Functions/substring.cpp b/src/Functions/substring.cpp index e809914f5f0..759d41e2ab8 100644 --- a/src/Functions/substring.cpp +++ b/src/Functions/substring.cpp @@ -148,9 +148,23 @@ public: if constexpr (is_utf8) { if (const ColumnString * col = checkAndGetColumn(column_string.get())) - return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, UTF8StringSource(*col), input_rows_count); + { + bool all_ascii = UTF8::isAllASCII(col->getChars().data(), col->getChars().size()); + if (all_ascii) + return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, StringSource(*col), input_rows_count); + else + return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, UTF8StringSource(*col), input_rows_count); + } + if (const ColumnConst * col_const = checkAndGetColumnConst(column_string.get())) - return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, ConstSource(*col_const), input_rows_count); + { + StringRef str_ref = col_const->getDataAt(0); + bool all_ascii = UTF8::isAllASCII(reinterpret_cast(str_ref.data), str_ref.size); + if (all_ascii) + return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, ConstSource(*col_const), input_rows_count); + else + return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, ConstSource(*col_const), input_rows_count); + } throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", arguments[0].column->getName(), getName()); } else diff --git a/src/Functions/substringIndex.cpp b/src/Functions/substringIndex.cpp index 5f3f054b624..462941b90ec 100644 --- a/src/Functions/substringIndex.cpp +++ b/src/Functions/substringIndex.cpp @@ -129,8 +129,10 @@ namespace res_data.reserve(str_column->getChars().size() / 2); res_offsets.reserve(rows); + bool all_ascii = UTF8::isAllASCII(str_column->getChars().data(), str_column->getChars().size()) + && UTF8::isAllASCII(reinterpret_cast(delim.data()), delim.size()); std::unique_ptr searcher - = !is_utf8 ? nullptr : std::make_unique(delim.data(), delim.size()); + = !is_utf8 || all_ascii ? nullptr : std::make_unique(delim.data(), delim.size()); for (size_t i = 0; i < rows; ++i) { @@ -140,6 +142,8 @@ namespace StringRef res_ref; if constexpr (!is_utf8) res_ref = substringIndex(str_ref, delim[0], count); + else if (all_ascii) + res_ref = substringIndex(str_ref, delim[0], count); else res_ref = substringIndexUTF8(searcher.get(), str_ref, delim, count); @@ -158,8 +162,10 @@ namespace res_data.reserve(str_column->getChars().size() / 2); res_offsets.reserve(rows); + bool all_ascii = UTF8::isAllASCII(str_column->getChars().data(), str_column->getChars().size()) + && UTF8::isAllASCII(reinterpret_cast(delim.data()), delim.size()); std::unique_ptr searcher - = !is_utf8 ? nullptr : std::make_unique(delim.data(), delim.size()); + = !is_utf8 || all_ascii ? nullptr : std::make_unique(delim.data(), delim.size()); for (size_t i = 0; i < rows; ++i) { @@ -168,6 +174,8 @@ namespace StringRef res_ref; if constexpr (!is_utf8) res_ref = substringIndex(str_ref, delim[0], count); + else if (all_ascii) + res_ref = substringIndex(str_ref, delim[0], count); else res_ref = substringIndexUTF8(searcher.get(), str_ref, delim, count); @@ -186,8 +194,10 @@ namespace res_data.reserve(str.size() * rows / 2); res_offsets.reserve(rows); + bool all_ascii = UTF8::isAllASCII(reinterpret_cast(str.data()), str.size()) + && UTF8::isAllASCII(reinterpret_cast(delim.data()), delim.size()); std::unique_ptr searcher - = !is_utf8 ? nullptr : std::make_unique(delim.data(), delim.size()); + = !is_utf8 || all_ascii ? nullptr : std::make_unique(delim.data(), delim.size()); StringRef str_ref{str.data(), str.size()}; for (size_t i = 0; i < rows; ++i) @@ -197,6 +207,8 @@ namespace StringRef res_ref; if constexpr (!is_utf8) res_ref = substringIndex(str_ref, delim[0], count); + else if (all_ascii) + res_ref = substringIndex(str_ref, delim[0], count); else res_ref = substringIndexUTF8(searcher.get(), str_ref, delim, count); @@ -208,7 +220,7 @@ namespace { size_t res_offset = res_data.size(); res_data.resize(res_offset + res_ref.size + 1); - memcpy(&res_data[res_offset], res_ref.data, res_ref.size); + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], res_ref.data, res_ref.size); res_offset += res_ref.size; res_data[res_offset] = 0; ++res_offset; From 75d4cebf7c35ea744816161322f99753bdd65a8c Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 20 Mar 2024 17:20:46 +0800 Subject: [PATCH 057/624] opt lowerUTF8/upperUTF8 --- src/Functions/LowerUpperImpl.h | 4 ++-- src/Functions/LowerUpperUTF8Impl.h | 13 +++++++++++-- 2 files changed, 13 insertions(+), 4 deletions(-) diff --git a/src/Functions/LowerUpperImpl.h b/src/Functions/LowerUpperImpl.h index f093e00f7ab..72b3ce1ca34 100644 --- a/src/Functions/LowerUpperImpl.h +++ b/src/Functions/LowerUpperImpl.h @@ -13,14 +13,14 @@ struct LowerUpperImpl ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { - res_data.resize(data.size()); + res_data.resize_exact(data.size()); res_offsets.assign(offsets); array(data.data(), data.data() + data.size(), res_data.data()); } static void vectorFixed(const ColumnString::Chars & data, size_t /*n*/, ColumnString::Chars & res_data) { - res_data.resize(data.size()); + res_data.resize_exact(data.size()); array(data.data(), data.data() + data.size(), res_data.data()); } diff --git a/src/Functions/LowerUpperUTF8Impl.h b/src/Functions/LowerUpperUTF8Impl.h index 460f75f9bde..f2a1351c048 100644 --- a/src/Functions/LowerUpperUTF8Impl.h +++ b/src/Functions/LowerUpperUTF8Impl.h @@ -1,8 +1,9 @@ #pragma once #include +#include +#include #include #include -#include #ifdef __SSE2__ #include @@ -92,7 +93,15 @@ struct LowerUpperUTF8Impl { if (data.empty()) return; - res_data.resize(data.size()); + + bool all_ascii = UTF8::isAllASCII(data.data(), data.size()); + if (all_ascii) + { + LowerUpperImpl::vector(data, offsets, res_data, res_offsets); + return; + } + + res_data.resize_exact(data.size()); res_offsets.assign(offsets); array(data.data(), data.data() + data.size(), offsets, res_data.data()); } From a4466496488d357f6f0049c5906c23893336cad8 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 20 Mar 2024 18:48:51 +0800 Subject: [PATCH 058/624] opt pad string for utf8 --- src/Functions/GatherUtils/Sinks.h | 16 +++++----- src/Functions/GatherUtils/Sources.h | 9 ++++-- src/Functions/padString.cpp | 49 +++++++++++++++++++---------- src/Functions/reverse.h | 4 +-- 4 files changed, 49 insertions(+), 29 deletions(-) diff --git a/src/Functions/GatherUtils/Sinks.h b/src/Functions/GatherUtils/Sinks.h index a8054da1159..2aa7c147136 100644 --- a/src/Functions/GatherUtils/Sinks.h +++ b/src/Functions/GatherUtils/Sinks.h @@ -48,7 +48,7 @@ struct NumericArraySink : public ArraySinkImpl> NumericArraySink(IColumn & elements_, ColumnArray::Offsets & offsets_, size_t column_size) : elements(assert_cast(elements_).getData()), offsets(offsets_) { - offsets.resize(column_size); + offsets.resize_exact(column_size); } void next() @@ -69,7 +69,7 @@ struct NumericArraySink : public ArraySinkImpl> void reserve(size_t num_elements) { - elements.reserve(num_elements); + elements.reserve_exact(num_elements); } }; @@ -85,7 +85,7 @@ struct StringSink StringSink(ColumnString & col, size_t column_size) : elements(col.getChars()), offsets(col.getOffsets()) { - offsets.resize(column_size); + offsets.resize_exact(column_size); } void ALWAYS_INLINE next() @@ -108,7 +108,7 @@ struct StringSink void reserve(size_t num_elements) { - elements.reserve(num_elements); + elements.reserve_exact(num_elements); } }; @@ -125,7 +125,7 @@ struct FixedStringSink FixedStringSink(ColumnFixedString & col, size_t column_size) : elements(col.getChars()), string_size(col.getN()), total_rows(column_size) { - elements.resize(column_size * string_size); + elements.resize_exact(column_size * string_size); } void next() @@ -146,7 +146,7 @@ struct FixedStringSink void reserve(size_t num_elements) { - elements.reserve(num_elements); + elements.reserve_exact(num_elements); } }; @@ -165,7 +165,7 @@ struct GenericArraySink : public ArraySinkImpl GenericArraySink(IColumn & elements_, ColumnArray::Offsets & offsets_, size_t column_size) : elements(elements_), offsets(offsets_) { - offsets.resize(column_size); + offsets.resize_exact(column_size); } void next() @@ -210,7 +210,7 @@ struct NullableArraySink : public ArraySink void reserve(size_t num_elements) { ArraySink::reserve(num_elements); - null_map.reserve(num_elements); + null_map.reserve_exact(num_elements); } }; diff --git a/src/Functions/GatherUtils/Sources.h b/src/Functions/GatherUtils/Sources.h index 222f9f19168..41e38e6fa79 100644 --- a/src/Functions/GatherUtils/Sources.h +++ b/src/Functions/GatherUtils/Sources.h @@ -319,6 +319,8 @@ struct StringSource return {&elements[prev_offset], length + elem_size > offset ? std::min(elem_size, length + elem_size - offset) : 0}; return {&elements[prev_offset + elem_size - offset], std::min(length, offset)}; } + + const ColumnString::Chars & getElements() const { return elements; } }; /// Treats Enum values as Strings, modeled after StringSource @@ -513,11 +515,12 @@ struct FixedStringSource const UInt8 * pos; const UInt8 * end; size_t string_size; + const typename ColumnString::Chars & elements; + size_t row_num = 0; size_t column_size = 0; - explicit FixedStringSource(const ColumnFixedString & col) - : string_size(col.getN()) + explicit FixedStringSource(const ColumnFixedString & col) : string_size(col.getN()), elements(col.getChars()) { const auto & chars = col.getChars(); pos = chars.data(); @@ -588,6 +591,8 @@ struct FixedStringSource return {pos, length + string_size > offset ? std::min(string_size, length + string_size - offset) : 0}; return {pos + string_size - offset, std::min(length, offset)}; } + + const ColumnString::Chars & getElements() const { return elements; } }; diff --git a/src/Functions/padString.cpp b/src/Functions/padString.cpp index b26a4ec3d6a..7a424bb1198 100644 --- a/src/Functions/padString.cpp +++ b/src/Functions/padString.cpp @@ -211,19 +211,18 @@ namespace pad_string = column_pad_const->getValue(); } - PaddingChars padding_chars{pad_string}; auto col_res = ColumnString::create(); StringSink res_sink{*col_res, input_rows_count}; if (const ColumnString * col = checkAndGetColumn(column_string.get())) - executeForSource(StringSource{*col}, column_length, padding_chars, res_sink); + executeForSource(StringSource{*col}, column_length, pad_string, res_sink); else if (const ColumnFixedString * col_fixed = checkAndGetColumn(column_string.get())) - executeForSource(FixedStringSource{*col_fixed}, column_length, padding_chars, res_sink); + executeForSource(FixedStringSource{*col_fixed}, column_length, pad_string, res_sink); else if (const ColumnConst * col_const = checkAndGetColumnConst(column_string.get())) - executeForSource(ConstSource{*col_const}, column_length, padding_chars, res_sink); + executeForSource(ConstSource{*col_const}, column_length, pad_string, res_sink); else if (const ColumnConst * col_const_fixed = checkAndGetColumnConst(column_string.get())) - executeForSource(ConstSource{*col_const_fixed}, column_length, padding_chars, res_sink); + executeForSource(ConstSource{*col_const_fixed}, column_length, pad_string, res_sink); else throw Exception( ErrorCodes::ILLEGAL_COLUMN, @@ -236,23 +235,39 @@ namespace private: template - void executeForSource( - SourceStrings && strings, - const ColumnPtr & column_length, - const PaddingChars & padding_chars, - StringSink & res_sink) const + void executeForSource(SourceStrings && strings, const ColumnPtr & column_length, const String & pad_string, StringSink & res_sink) const { - if (const auto * col_const = checkAndGetColumn(column_length.get())) - executeForSourceAndLength(std::forward(strings), ConstSource{*col_const}, padding_chars, res_sink); + const auto & chars = strings.getElements(); + bool all_ascii = UTF8::isAllASCII(reinterpret_cast(pad_string.data()), pad_string.size()) + && UTF8::isAllASCII(chars.data(), chars.size()); + + if (all_ascii) + { + PaddingChars padding_chars{pad_string}; + if (const auto * col_const = checkAndGetColumn(column_length.get())) + executeForSourceAndLength( + std::forward(strings), ConstSource{*col_const}, padding_chars, res_sink); + else + executeForSourceAndLength( + std::forward(strings), GenericValueSource{*column_length}, padding_chars, res_sink); + } else - executeForSourceAndLength(std::forward(strings), GenericValueSource{*column_length}, padding_chars, res_sink); + { + PaddingChars padding_chars{pad_string}; + if (const auto * col_const = checkAndGetColumn(column_length.get())) + executeForSourceAndLength( + std::forward(strings), ConstSource{*col_const}, padding_chars, res_sink); + else + executeForSourceAndLength( + std::forward(strings), GenericValueSource{*column_length}, padding_chars, res_sink); + } } - template + template void executeForSourceAndLength( SourceStrings && strings, SourceLengths && lengths, - const PaddingChars & padding_chars, + const PaddingChars & padding_chars, StringSink & res_sink) const { bool is_const_new_length = lengths.isConst(); @@ -264,7 +279,7 @@ namespace for (; !res_sink.isEnd(); res_sink.next(), strings.next(), lengths.next()) { auto str = strings.getWhole(); - ssize_t current_length = getLengthOfSlice(str); + ssize_t current_length = getLengthOfSlice(str); if (!res_sink.rowNum() || !is_const_new_length) { @@ -294,7 +309,7 @@ namespace } else if (new_length < current_length) { - str = removeSuffixFromSlice(str, current_length - new_length); + str = removeSuffixFromSlice(str, current_length - new_length); writeSlice(str, res_sink); } else if (new_length > current_length) diff --git a/src/Functions/reverse.h b/src/Functions/reverse.h index 128a897c041..5f999af4297 100644 --- a/src/Functions/reverse.h +++ b/src/Functions/reverse.h @@ -14,7 +14,7 @@ struct ReverseImpl ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { - res_data.resize(data.size()); + res_data.resize_exact(data.size()); res_offsets.assign(offsets); size_t size = offsets.size(); @@ -30,7 +30,7 @@ struct ReverseImpl static void vectorFixed(const ColumnString::Chars & data, size_t n, ColumnString::Chars & res_data) { - res_data.resize(data.size()); + res_data.resize_exact(data.size()); size_t size = data.size() / n; for (size_t i = 0; i < size; ++i) From 49422debafbf773c887c60f2affb2e038edea911 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 20 Mar 2024 18:59:22 +0800 Subject: [PATCH 059/624] add perf tests --- src/Common/UTF8Helpers.cpp | 3 ++- tests/performance/ascii.xml | 9 +++++++++ 2 files changed, 11 insertions(+), 1 deletion(-) create mode 100644 tests/performance/ascii.xml diff --git a/src/Common/UTF8Helpers.cpp b/src/Common/UTF8Helpers.cpp index 78d645994e5..6e1c3f5397b 100644 --- a/src/Common/UTF8Helpers.cpp +++ b/src/Common/UTF8Helpers.cpp @@ -222,7 +222,8 @@ bool isAllASCII(const UInt8 * data, size_t size) DECLARE_SSE42_SPECIFIC_CODE( /// Copy from https://github.com/lemire/fastvalidate-utf-8/blob/master/include/simdasciicheck.h -bool isAllASCII(const UInt8 * data, size_t size) { +bool isAllASCII(const UInt8 * data, size_t size) +{ size_t i = 0; __m128i masks = _mm_setzero_si128(); if (size >= 16) diff --git a/tests/performance/ascii.xml b/tests/performance/ascii.xml new file mode 100644 index 00000000000..89c6dd57e07 --- /dev/null +++ b/tests/performance/ascii.xml @@ -0,0 +1,9 @@ + + select substringUTF8(materialize('hello world'), 2, 5) from numbers(10000000) + select substringIndexUTF8(materialize('www.clickhouse.com'), '.', 2) from numbers(10000000) + select reverseUTF8(materialize('hello world')) from numbers(10000000) + select lowerUTF8(materialize('hello world')) from numbers(10000000) + select upperUTF8(materialize('hello world')) from numbers(10000000) + select leftPadUTF8(materialize('hello '), 10, ',') from numbers(10000000) + select rightPadUTF8(materialize('hello '), 10, ',') from numbers(10000000) + From 83645350fb9f3db299061f939659595cefe771ee Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 20 Mar 2024 22:08:15 +0800 Subject: [PATCH 060/624] fix failed uts --- src/Functions/padString.cpp | 2 +- src/Functions/substring.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/padString.cpp b/src/Functions/padString.cpp index 7a424bb1198..119baaec723 100644 --- a/src/Functions/padString.cpp +++ b/src/Functions/padString.cpp @@ -241,7 +241,7 @@ namespace bool all_ascii = UTF8::isAllASCII(reinterpret_cast(pad_string.data()), pad_string.size()) && UTF8::isAllASCII(chars.data(), chars.size()); - if (all_ascii) + if (!is_utf8 || all_ascii) { PaddingChars padding_chars{pad_string}; if (const auto * col_const = checkAndGetColumn(column_length.get())) diff --git a/src/Functions/substring.cpp b/src/Functions/substring.cpp index 759d41e2ab8..122f83d758b 100644 --- a/src/Functions/substring.cpp +++ b/src/Functions/substring.cpp @@ -161,9 +161,9 @@ public: StringRef str_ref = col_const->getDataAt(0); bool all_ascii = UTF8::isAllASCII(reinterpret_cast(str_ref.data), str_ref.size); if (all_ascii) - return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, ConstSource(*col_const), input_rows_count); - else return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, ConstSource(*col_const), input_rows_count); + else + return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, ConstSource(*col_const), input_rows_count); } throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", arguments[0].column->getName(), getName()); } From c350991005253183c83fac4d380b368b343f2763 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 21 Mar 2024 10:29:35 +0800 Subject: [PATCH 061/624] add more tests --- tests/performance/ascii.xml | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/tests/performance/ascii.xml b/tests/performance/ascii.xml index 89c6dd57e07..83440437d2c 100644 --- a/tests/performance/ascii.xml +++ b/tests/performance/ascii.xml @@ -1,9 +1,22 @@ select substringUTF8(materialize('hello world'), 2, 5) from numbers(10000000) + select substringUTF8(materialize('hello 世界'), 2, 5) from numbers(10000000) + select substringIndexUTF8(materialize('www.clickhouse.com'), '.', 2) from numbers(10000000) + select substringIndexUTF8(materialize('官网www.clickhouse.com'), '.', 2) from numbers(10000000) + select reverseUTF8(materialize('hello world')) from numbers(10000000) + select reverseUTF8(materialize('hello 世界')) from numbers(10000000) + select lowerUTF8(materialize('hello world')) from numbers(10000000) + select lowerUTF8(materialize('hello 世界')) from numbers(10000000) + select upperUTF8(materialize('hello world')) from numbers(10000000) + select upperUTF8(materialize('hello 世界')) from numbers(10000000) + select leftPadUTF8(materialize('hello '), 10, ',') from numbers(10000000) + select leftPadUTF8(materialize('hello '), 10, '世界') from numbers(10000000) + select rightPadUTF8(materialize('hello '), 10, ',') from numbers(10000000) + select rightPadUTF8(materialize('hello '), 10, '世界') from numbers(10000000) From 490a8bc7e48d2d87e67ba8d713a01d0fc78a901d Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 21 Mar 2024 10:31:45 +0800 Subject: [PATCH 062/624] apply resize_extact for trim --- src/Functions/trim.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/trim.cpp b/src/Functions/trim.cpp index dd51c606ff7..8b3d2870260 100644 --- a/src/Functions/trim.cpp +++ b/src/Functions/trim.cpp @@ -46,8 +46,8 @@ public: ColumnString::Offsets & res_offsets) { size_t size = offsets.size(); - res_offsets.resize(size); - res_data.reserve(data.size()); + res_offsets.resize_exact(size); + res_data.reserve_exact(data.size()); size_t prev_offset = 0; size_t res_offset = 0; @@ -59,7 +59,7 @@ public: { execute(reinterpret_cast(&data[prev_offset]), offsets[i] - prev_offset - 1, start, length); - res_data.resize(res_data.size() + length + 1); + res_data.resize_exact(res_data.size() + length + 1); memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], start, length); res_offset += length + 1; res_data[res_offset - 1] = '\0'; From 0beb74b931bc6e7f946d1d16d6a205025e89faa7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 23 Mar 2024 22:12:17 +0100 Subject: [PATCH 063/624] 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 064/624] 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 065/624] 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 066/624] 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 972af33400952abce28451f0b248ae3d0fc4579c Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 26 Mar 2024 12:30:16 +0000 Subject: [PATCH 067/624] 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 bbb14955a6b44fc4c4c4bcb76105c54ab70cbe67 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 26 Mar 2024 12:48:01 +0000 Subject: [PATCH 068/624] 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 9e719868a0b4141d4ee918f4116719f8227d4757 Mon Sep 17 00:00:00 2001 From: Maxim Alexeev Date: Wed, 27 Mar 2024 23:28:13 +0300 Subject: [PATCH 069/624] 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 070/624] 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 fd925770b2de0ea067541a156c796552a94f3d11 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 28 Mar 2024 17:00:54 +0100 Subject: [PATCH 071/624] 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 98c2048d30d0a6e324875c98a4f138e7ee92734a Mon Sep 17 00:00:00 2001 From: zvonand Date: Sun, 31 Mar 2024 22:12:03 +0200 Subject: [PATCH 072/624] 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 073/624] 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 074/624] 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 075/624] 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 5f190e6774aa1a38e6aae740b97f51882355cd25 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 2 Apr 2024 11:12:34 +0000 Subject: [PATCH 076/624] Use intrusive lists for `ResourceRequest` instead of deque --- src/Common/Scheduler/Nodes/FifoQueue.h | 39 +++++++++++++++----------- src/Common/Scheduler/ResourceRequest.h | 4 ++- 2 files changed, 26 insertions(+), 17 deletions(-) diff --git a/src/Common/Scheduler/Nodes/FifoQueue.h b/src/Common/Scheduler/Nodes/FifoQueue.h index 45ed32343ff..9ec997c06d2 100644 --- a/src/Common/Scheduler/Nodes/FifoQueue.h +++ b/src/Common/Scheduler/Nodes/FifoQueue.h @@ -6,7 +6,8 @@ #include -#include +#include + #include @@ -15,6 +16,7 @@ namespace DB namespace ErrorCodes { + extern const int LOGICAL_ERROR; extern const int INVALID_SCHEDULER_NODE; } @@ -42,7 +44,7 @@ public: std::lock_guard lock(mutex); queue_cost += request->cost; bool was_empty = requests.empty(); - requests.push_back(request); + requests.push_back(*request); if (was_empty) scheduleActivation(); } @@ -52,7 +54,7 @@ public: std::lock_guard lock(mutex); if (requests.empty()) return {nullptr, false}; - ResourceRequest * result = requests.front(); + ResourceRequest * result = &requests.front(); requests.pop_front(); if (requests.empty()) busy_periods++; @@ -65,19 +67,24 @@ public: bool cancelRequest(ResourceRequest * request) override { std::lock_guard lock(mutex); - // TODO(serxa): reimplement queue as intrusive list of ResourceRequest to make this O(1) instead of O(N) - for (auto i = requests.begin(), e = requests.end(); i != e; ++i) + if (request->is_linked()) { - if (*i == request) - { - requests.erase(i); - if (requests.empty()) - busy_periods++; - queue_cost -= request->cost; - canceled_requests++; - canceled_cost += request->cost; - return true; - } + // It's impossible to check that `request` is indeed inserted to this queue and not another queue. + // It's up to caller to make sure this is the case. Otherwise, list sizes will be corrupted. + // Not tracking list sizes is not an option, because another problem appears: removing from list w/o locking. + // Another possible solution - keep track if request `is_cancelable` guarded by `mutex` + // Simple check for list size corruption + if (requests.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "trying to cancel request (linked into another queue) from empty queue: {}", getPath()); + + requests.erase(requests.iterator_to(*request)); + + if (requests.empty()) + busy_periods++; + queue_cost -= request->cost; + canceled_requests++; + canceled_cost += request->cost; + return true; } return false; } @@ -124,7 +131,7 @@ public: private: std::mutex mutex; Int64 queue_cost = 0; - std::deque requests; // TODO(serxa): reimplement it using intrusive list to avoid allocations/deallocations and O(N) during cancel + boost::intrusive::list requests; }; } diff --git a/src/Common/Scheduler/ResourceRequest.h b/src/Common/Scheduler/ResourceRequest.h index f3153ad382c..d64f624cec5 100644 --- a/src/Common/Scheduler/ResourceRequest.h +++ b/src/Common/Scheduler/ResourceRequest.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include @@ -41,7 +42,7 @@ constexpr ResourceCost ResourceCostMax = std::numeric_limits::max(); * Returning true means successful cancel and therefore steps (4) and (5) are not going to happen * and step (6) MUST be omitted. */ -class ResourceRequest +class ResourceRequest : public boost::intrusive::list_base_hook<> { public: /// Cost of request execution; should be filled before request enqueueing. @@ -62,6 +63,7 @@ public: { cost = cost_; constraint = nullptr; + // Note that list_base_hook should be reset independently (by intrusive list) } virtual ~ResourceRequest() = default; From 7232bf45768f56c768ac03ed4b34c085bc6f060a Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 2 Apr 2024 16:12:11 +0200 Subject: [PATCH 077/624] 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 74ce999645f5598b6d79c24510fb7b28fb2b5f17 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 2 Apr 2024 19:47:26 +0200 Subject: [PATCH 078/624] 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 079/624] 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 7fd8d84fb67b6587b8f3ef5d4194dca04df9f4cb Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 3 Apr 2024 12:25:29 +0200 Subject: [PATCH 080/624] 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 081/624] 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 082/624] 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 083/624] 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 084/624] 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 085/624] 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 086/624] 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 087/624] 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 088/624] 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 089/624] 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 090/624] 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 307f69380efbd7af9a3e897df04cc9307558ad91 Mon Sep 17 00:00:00 2001 From: skyoct Date: Sun, 7 Apr 2024 09:02:43 +0000 Subject: [PATCH 091/624] support clamp function --- .../functions/conditional-functions.md | 31 +++++++++ src/Functions/clamp.cpp | 68 +++++++++++++++++++ .../queries/0_stateless/03036_clamp.reference | 4 ++ tests/queries/0_stateless/03036_clamp.sql | 4 ++ 4 files changed, 107 insertions(+) create mode 100644 src/Functions/clamp.cpp create mode 100644 tests/queries/0_stateless/03036_clamp.reference create mode 100644 tests/queries/0_stateless/03036_clamp.sql diff --git a/docs/en/sql-reference/functions/conditional-functions.md b/docs/en/sql-reference/functions/conditional-functions.md index eb4e98961f1..c89de4d9958 100644 --- a/docs/en/sql-reference/functions/conditional-functions.md +++ b/docs/en/sql-reference/functions/conditional-functions.md @@ -234,3 +234,34 @@ SELECT least(toDateTime32(now() + toIntervalDay(1)), toDateTime64(now(), 3)) :::note The type returned is a DateTime64 as the DataTime32 must be promoted to 64 bit for the comparison. ::: + +## clamp + +Constrain the return value between A and B. + +**Syntax** + +``` sql +if(x, min, max) +``` + +**Arguments** + +- `x` – Input value. +- `min` – Limit the lower bound. +- `max` – Limit the upper bound. + +**Returned values** + +If the value is less than the minimum value, return the minimum value; if it is greater than the maximum value, return the maximum value; otherwise, return the current value. + +Examples: + +```sql +SELECT least(1, 2, 3) result, toTypeName(result) type; +``` +```response +┌─result─┬─type────┐ +│ 2 │ Float64 │ +└────────┴─────────┘ +``` \ No newline at end of file diff --git a/src/Functions/clamp.cpp b/src/Functions/clamp.cpp new file mode 100644 index 00000000000..38be5a28f28 --- /dev/null +++ b/src/Functions/clamp.cpp @@ -0,0 +1,68 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + + +class FunctionClamp : public IFunction +{ + +public: + static constexpr auto name = "clamp"; + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 0; } + bool isVariadic() const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + DataTypePtr getReturnTypeImpl(const DataTypes & types) const override + { + if (types.size() != 3) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} must be 3 arguments", getName()); + + return getLeastSupertype(types); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + + size_t arg_size = arguments.size(); + Columns converted_columns(arg_size); + for (size_t arg = 0; arg < arg_size; ++arg) + converted_columns[arg] = castColumn(arguments[arg], result_type)->convertToFullColumnIfConst(); + + auto result_column = result_type->createColumn(); + for (size_t row_num = 0; row_num < input_rows_count; ++row_num) + { + size_t best_arg = 0; + + if (converted_columns[1]->compareAt(row_num, row_num, *converted_columns[best_arg], 1) > 0) + best_arg = 1; + else if (converted_columns[2]->compareAt(row_num, row_num, *converted_columns[best_arg], 1) < 0) + best_arg = 2; + + result_column->insertFrom(*converted_columns[best_arg], row_num); + } + + return result_column; + } + +}; + +REGISTER_FUNCTION(Clamp) +{ + factory.registerFunction(); +} +} diff --git a/tests/queries/0_stateless/03036_clamp.reference b/tests/queries/0_stateless/03036_clamp.reference new file mode 100644 index 00000000000..bd0d34dabea --- /dev/null +++ b/tests/queries/0_stateless/03036_clamp.reference @@ -0,0 +1,4 @@ +10 +20 +15 +b diff --git a/tests/queries/0_stateless/03036_clamp.sql b/tests/queries/0_stateless/03036_clamp.sql new file mode 100644 index 00000000000..8250325661a --- /dev/null +++ b/tests/queries/0_stateless/03036_clamp.sql @@ -0,0 +1,4 @@ +SELECT clamp(1, 10, 20); +SELECT clamp(30, 10, 20); +SELECT clamp(15, 10, 20); +SELECT clamp('a', 'b', 'c'); \ No newline at end of file From 8faa1487549e1dae0ca27d0550d7c90dd3102668 Mon Sep 17 00:00:00 2001 From: skyoct Date: Mon, 8 Apr 2024 06:44:11 +0000 Subject: [PATCH 092/624] batter --- docs/en/sql-reference/functions/conditional-functions.md | 4 ++-- src/Functions/clamp.cpp | 1 - tests/queries/0_stateless/03036_clamp.reference | 4 ++++ tests/queries/0_stateless/03036_clamp.sql | 6 +++++- 4 files changed, 11 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/conditional-functions.md b/docs/en/sql-reference/functions/conditional-functions.md index c89de4d9958..f19386af9f1 100644 --- a/docs/en/sql-reference/functions/conditional-functions.md +++ b/docs/en/sql-reference/functions/conditional-functions.md @@ -242,7 +242,7 @@ Constrain the return value between A and B. **Syntax** ``` sql -if(x, min, max) +clamp(x, min, max) ``` **Arguments** @@ -258,7 +258,7 @@ If the value is less than the minimum value, return the minimum value; if it is Examples: ```sql -SELECT least(1, 2, 3) result, toTypeName(result) type; +SELECT clamp(1, 2, 3) result, toTypeName(result) type; ``` ```response ┌─result─┬─type────┐ diff --git a/src/Functions/clamp.cpp b/src/Functions/clamp.cpp index 38be5a28f28..0210d135235 100644 --- a/src/Functions/clamp.cpp +++ b/src/Functions/clamp.cpp @@ -37,7 +37,6 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { - size_t arg_size = arguments.size(); Columns converted_columns(arg_size); for (size_t arg = 0; arg < arg_size; ++arg) diff --git a/tests/queries/0_stateless/03036_clamp.reference b/tests/queries/0_stateless/03036_clamp.reference index bd0d34dabea..bea85d8ccdd 100644 --- a/tests/queries/0_stateless/03036_clamp.reference +++ b/tests/queries/0_stateless/03036_clamp.reference @@ -2,3 +2,7 @@ 20 15 b +0 +['hello'] +-1 +234 diff --git a/tests/queries/0_stateless/03036_clamp.sql b/tests/queries/0_stateless/03036_clamp.sql index 8250325661a..8d4f2112830 100644 --- a/tests/queries/0_stateless/03036_clamp.sql +++ b/tests/queries/0_stateless/03036_clamp.sql @@ -1,4 +1,8 @@ SELECT clamp(1, 10, 20); SELECT clamp(30, 10, 20); SELECT clamp(15, 10, 20); -SELECT clamp('a', 'b', 'c'); \ No newline at end of file +SELECT clamp('a', 'b', 'c'); +SELECT clamp(today(), yesterday() - 10, yesterday() + 10) - today() +SELECT clamp([], ['hello'], ['world']); +SELECT clamp(-1., -1000., 18446744073709551615.); +SELECT clamp(toNullable(123), 234, 456); From a6d088bed8ff87c1901077b7c8960034445aa9d4 Mon Sep 17 00:00:00 2001 From: skyoct Date: Mon, 8 Apr 2024 10:35:53 +0000 Subject: [PATCH 093/624] batter --- src/Functions/clamp.cpp | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/src/Functions/clamp.cpp b/src/Functions/clamp.cpp index 0210d135235..dc0ddffbd82 100644 --- a/src/Functions/clamp.cpp +++ b/src/Functions/clamp.cpp @@ -11,6 +11,7 @@ namespace DB namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; } @@ -30,7 +31,7 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & types) const override { if (types.size() != 3) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} must be 3 arguments", getName()); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires 3 arguments", getName()); return getLeastSupertype(types); } @@ -41,18 +42,20 @@ public: Columns converted_columns(arg_size); for (size_t arg = 0; arg < arg_size; ++arg) converted_columns[arg] = castColumn(arguments[arg], result_type)->convertToFullColumnIfConst(); + if (converted_columns[1]->compareAt(0, 0, *converted_columns[2], 1) > 0) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Function {} the minimum value cannot be greater than the maximum value", getName()); auto result_column = result_type->createColumn(); for (size_t row_num = 0; row_num < input_rows_count; ++row_num) { - size_t best_arg = 0; - - if (converted_columns[1]->compareAt(row_num, row_num, *converted_columns[best_arg], 1) > 0) - best_arg = 1; - else if (converted_columns[2]->compareAt(row_num, row_num, *converted_columns[best_arg], 1) < 0) - best_arg = 2; + size_t best_arg = 0; + + if (converted_columns[1]->compareAt(row_num, row_num, *converted_columns[best_arg], 1) > 0) + best_arg = 1; + else if (converted_columns[2]->compareAt(row_num, row_num, *converted_columns[best_arg], 1) < 0) + best_arg = 2; - result_column->insertFrom(*converted_columns[best_arg], row_num); + result_column->insertFrom(*converted_columns[best_arg], row_num); } return result_column; 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 094/624] 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 095/624] 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 096/624] 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 0d6426d781538b899acfcfbff736f16422097466 Mon Sep 17 00:00:00 2001 From: skyoct Date: Mon, 8 Apr 2024 13:34:54 +0000 Subject: [PATCH 097/624] batter --- tests/queries/0_stateless/03036_clamp.reference | 2 ++ tests/queries/0_stateless/03036_clamp.sql | 2 ++ 2 files changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/03036_clamp.reference b/tests/queries/0_stateless/03036_clamp.reference index bea85d8ccdd..5c8de57794d 100644 --- a/tests/queries/0_stateless/03036_clamp.reference +++ b/tests/queries/0_stateless/03036_clamp.reference @@ -6,3 +6,5 @@ b ['hello'] -1 234 +null +null \ No newline at end of file diff --git a/tests/queries/0_stateless/03036_clamp.sql b/tests/queries/0_stateless/03036_clamp.sql index 8d4f2112830..e075096f8ef 100644 --- a/tests/queries/0_stateless/03036_clamp.sql +++ b/tests/queries/0_stateless/03036_clamp.sql @@ -6,3 +6,5 @@ SELECT clamp(today(), yesterday() - 10, yesterday() + 10) - today() SELECT clamp([], ['hello'], ['world']); SELECT clamp(-1., -1000., 18446744073709551615.); SELECT clamp(toNullable(123), 234, 456); +select clamp(1, null, 5); +select clamp(1, 6, null); \ No newline at end of file From 5d36d1aa4efad0843d4a9389003c8389f7b2e48a Mon Sep 17 00:00:00 2001 From: skyoct Date: Mon, 8 Apr 2024 13:53:33 +0000 Subject: [PATCH 098/624] style --- src/Functions/clamp.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/clamp.cpp b/src/Functions/clamp.cpp index dc0ddffbd82..0593c6c6b16 100644 --- a/src/Functions/clamp.cpp +++ b/src/Functions/clamp.cpp @@ -49,7 +49,6 @@ public: for (size_t row_num = 0; row_num < input_rows_count; ++row_num) { size_t best_arg = 0; - if (converted_columns[1]->compareAt(row_num, row_num, *converted_columns[best_arg], 1) > 0) best_arg = 1; else if (converted_columns[2]->compareAt(row_num, row_num, *converted_columns[best_arg], 1) < 0) From 88548eb19eb07ea29c8d603d77a43039844a351c Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 8 Apr 2024 18:59:04 +0000 Subject: [PATCH 099/624] 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 21ab0e4ea27b389ab697cb5f045a9600bd0fa7c2 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 8 Apr 2024 19:01:00 +0000 Subject: [PATCH 100/624] 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 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 101/624] 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 8accf395fb9cb3f2b896151ab09a902c674b9809 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 8 Apr 2024 20:37:06 +0000 Subject: [PATCH 102/624] 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 08f700118f266234ad3eba15ea95f91a85d81db8 Mon Sep 17 00:00:00 2001 From: HowePa <2873679104@qq.com> Date: Tue, 9 Apr 2024 13:24:37 +0800 Subject: [PATCH 103/624] [doc] update npy format data types --- docs/en/interfaces/formats.md | 33 ++++++++++++++++----------------- 1 file changed, 16 insertions(+), 17 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 03cf345349e..cc3ea467ab1 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -2465,23 +2465,22 @@ Result: ## Npy {#data-format-npy} -This function is designed to load a NumPy array from a .npy file into ClickHouse. The NumPy file format is a binary format used for efficiently storing arrays of numerical data. During import, ClickHouse treats top level dimension as an array of rows with single column. Supported Npy data types and their corresponding type in ClickHouse: -| Npy type | ClickHouse type | -|:--------:|:---------------:| -| b1 | UInt8 | -| i1 | Int8 | -| i2 | Int16 | -| i4 | Int32 | -| i8 | Int64 | -| u1 | UInt8 | -| u2 | UInt16 | -| u4 | UInt32 | -| u8 | UInt64 | -| f2 | Float32 | -| f4 | Float32 | -| f8 | Float64 | -| S | String | -| U | String | +This function is designed to load a NumPy array from a .npy file into ClickHouse. The NumPy file format is a binary format used for efficiently storing arrays of numerical data. During import, ClickHouse treats top level dimension as an array of rows with single column. Supported Npy data types and their corresponding type in ClickHouse: + +| Npy data type (`INSERT`) | ClickHouse data type | Npy data type (`SELECT`) | +|--------------------------|-----------------------------------------------------------------|--------------------------| +| `i1` | [Int8](/docs/en/sql-reference/data-types/int-uint.md) | `i1` | +| `i2` | [Int16](/docs/en/sql-reference/data-types/int-uint.md) | `i2` | +| `i4` | [Int32](/docs/en/sql-reference/data-types/int-uint.md) | `i4` | +| `i8` | [Int64](/docs/en/sql-reference/data-types/int-uint.md) | `i8` | +| `u1`, `b1` | [UInt8](/docs/en/sql-reference/data-types/int-uint.md) | `u1` | +| `u2` | [UInt16](/docs/en/sql-reference/data-types/int-uint.md) | `u2` | +| `u4` | [UInt32](/docs/en/sql-reference/data-types/int-uint.md) | `u4` | +| `u8` | [UInt64](/docs/en/sql-reference/data-types/int-uint.md) | `u8` | +| `f2`, `f4` | [Float32](/docs/en/sql-reference/data-types/float.md) | `f4` | +| `f8` | [Float64](/docs/en/sql-reference/data-types/float.md) | `f8` | +| `S`, `U` | [String](/docs/en/sql-reference/data-types/string.md) | `S` | +| | [FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | `S` | **Example of saving an array in .npy format using Python** From c0174fa17efc766bb49db0be67b6b5f7383429fc Mon Sep 17 00:00:00 2001 From: HowePa <2873679104@qq.com> Date: Tue, 9 Apr 2024 14:30:14 +0800 Subject: [PATCH 104/624] [feature] add npy output format --- docs/en/interfaces/formats.md | 8 + src/Formats/registerFormats.cpp | 6 +- .../Formats/Impl/NpyOutputFormat.cpp | 223 ++++++++++++++++++ src/Processors/Formats/Impl/NpyOutputFormat.h | 66 ++++++ .../02895_npy_output_format.reference | 60 +++++ .../0_stateless/02895_npy_output_format.sh | 110 +++++++++ 6 files changed, 471 insertions(+), 2 deletions(-) create mode 100644 src/Processors/Formats/Impl/NpyOutputFormat.cpp create mode 100644 src/Processors/Formats/Impl/NpyOutputFormat.h create mode 100644 tests/queries/0_stateless/02895_npy_output_format.reference create mode 100755 tests/queries/0_stateless/02895_npy_output_format.sh diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index cc3ea467ab1..f4b082c57ab 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -2507,6 +2507,14 @@ Result: └───────────────┘ ``` +**Selecting Data** + +You can select data from a ClickHouse table and save them into some file in the Npy format by the following command: + +```bash +$ clickhouse-client --query="SELECT {column} FROM {some_table} FORMAT Npy" > {filename.npy} +``` + ## LineAsString {#lineasstring} In this format, every line of input data is interpreted as a single string value. This format can only be parsed for table with a single field of type [String](/docs/en/sql-reference/data-types/string.md). The remaining columns must be set to [DEFAULT](/docs/en/sql-reference/statements/create/table.md/#default) or [MATERIALIZED](/docs/en/sql-reference/statements/create/table.md/#materialized), or omitted. diff --git a/src/Formats/registerFormats.cpp b/src/Formats/registerFormats.cpp index cc9cf380693..2070956883c 100644 --- a/src/Formats/registerFormats.cpp +++ b/src/Formats/registerFormats.cpp @@ -76,6 +76,8 @@ void registerInputFormatCustomSeparated(FormatFactory & factory); void registerOutputFormatCustomSeparated(FormatFactory & factory); void registerInputFormatCapnProto(FormatFactory & factory); void registerOutputFormatCapnProto(FormatFactory & factory); +void registerInputFormatNpy(FormatFactory & factory); +void registerOutputFormatNpy(FormatFactory & factory); /// Output only (presentational) formats. @@ -103,7 +105,6 @@ void registerInputFormatMySQLDump(FormatFactory & factory); void registerInputFormatParquetMetadata(FormatFactory & factory); void registerInputFormatDWARF(FormatFactory & factory); void registerInputFormatOne(FormatFactory & factory); -void registerInputFormatNpy(FormatFactory & factory); #if USE_HIVE void registerInputFormatHiveText(FormatFactory & factory); @@ -221,6 +222,8 @@ void registerFormats() registerOutputFormatAvro(factory); registerInputFormatArrow(factory); registerOutputFormatArrow(factory); + registerInputFormatNpy(factory); + registerOutputFormatNpy(factory); registerOutputFormatPretty(factory); registerOutputFormatPrettyCompact(factory); @@ -251,7 +254,6 @@ void registerFormats() registerInputFormatParquetMetadata(factory); registerInputFormatDWARF(factory); registerInputFormatOne(factory); - registerInputFormatNpy(factory); registerNonTrivialPrefixAndSuffixCheckerJSONEachRow(factory); registerNonTrivialPrefixAndSuffixCheckerJSONAsString(factory); diff --git a/src/Processors/Formats/Impl/NpyOutputFormat.cpp b/src/Processors/Formats/Impl/NpyOutputFormat.cpp new file mode 100644 index 00000000000..4a4af67b07e --- /dev/null +++ b/src/Processors/Formats/Impl/NpyOutputFormat.cpp @@ -0,0 +1,223 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int TOO_MANY_COLUMNS; + extern const int BAD_ARGUMENTS; + extern const int ILLEGAL_COLUMN; +} + +namespace +{ + +template +void writeNumpyNumbers(const ColumnPtr & column, WriteBuffer & buf) +{ + const auto * number_column = assert_cast(column.get()); + for (size_t i = 0; i < number_column->size(); ++i) + writeBinaryLittleEndian(ValueType(number_column->getElement(i)), buf); +} + +template +void writeNumpyStrings(const ColumnPtr & column, size_t length, WriteBuffer & buf) +{ + const auto * string_column = assert_cast(column.get()); + for (size_t i = 0; i < string_column->size(); ++i) + buf.write(string_column->getDataAt(i).data, length); +} + +} + +String NpyOutputFormat::NumpyDataType::str() +{ + std::ostringstream dtype; + dtype << endianness << type << std::to_string(size); + return dtype.str(); +} + +NpyOutputFormat::NpyOutputFormat(WriteBuffer & out_, const Block & header_) : IOutputFormat(header_, out_) +{ + const auto & header = getPort(PortKind::Main).getHeader(); + auto data_types = header.getDataTypes(); + if (data_types.size() > 1) + throw Exception(ErrorCodes::TOO_MANY_COLUMNS, "Expected single column for Npy output format, got {}", data_types.size()); + data_type = data_types[0]; +} + +void NpyOutputFormat::initialize(const ColumnPtr & column) +{ + auto type = data_type; + ColumnPtr nested_column = column; + while (type->getTypeId() == TypeIndex::Array) + { + const auto * array_column = assert_cast(nested_column.get()); + numpy_shape.push_back(array_column->getOffsets()[0]); + type = assert_cast(type.get())->getNestedType(); + nested_column = array_column->getDataPtr(); + } + + switch (type->getTypeId()) + { + case TypeIndex::Int8: numpy_data_type = NumpyDataType('<', 'i', sizeof(Int8)); break; + case TypeIndex::Int16: numpy_data_type = NumpyDataType('<', 'i', sizeof(Int16)); break; + case TypeIndex::Int32: numpy_data_type = NumpyDataType('<', 'i', sizeof(Int32)); break; + case TypeIndex::Int64: numpy_data_type = NumpyDataType('<', 'i', sizeof(Int64)); break; + case TypeIndex::UInt8: numpy_data_type = NumpyDataType('<', 'u', sizeof(UInt8)); break; + case TypeIndex::UInt16: numpy_data_type = NumpyDataType('<', 'u', sizeof(UInt16)); break; + case TypeIndex::UInt32: numpy_data_type = NumpyDataType('<', 'u', sizeof(UInt32)); break; + case TypeIndex::UInt64: numpy_data_type = NumpyDataType('<', 'u', sizeof(UInt64)); break; + case TypeIndex::Float32: numpy_data_type = NumpyDataType('<', 'f', sizeof(Float32)); break; + case TypeIndex::Float64: numpy_data_type = NumpyDataType('<', 'f', sizeof(Float64)); break; + case TypeIndex::FixedString: numpy_data_type = NumpyDataType('|', 'S', assert_cast(type.get())->getN()); break; + case TypeIndex::String: numpy_data_type = NumpyDataType('|', 'S', 0); break; + default: + has_exception = true; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Type {} is not supported for Npy output format", type->getName()); + } + nested_data_type = type; +} + +void NpyOutputFormat::consume(Chunk chunk) +{ + if (!has_exception) + { + num_rows += chunk.getNumRows(); + auto column = chunk.getColumns()[0]; + + if (!is_initialized) + { + initialize(column); + is_initialized = true; + } + + /// check shape + auto type = data_type; + ColumnPtr nested_column = column; + int dim = 0; + while (type->getTypeId() == TypeIndex::Array) + { + const auto * array_column = assert_cast(nested_column.get()); + const auto & array_offset = array_column->getOffsets(); + for (size_t i = 1; i < array_offset.size(); ++i) + { + if (array_offset[i] - array_offset[i - 1] != numpy_shape[dim]) + { + has_exception = true; + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "ClickHouse doesn't support object types, cannot format ragged nested sequences (which is a list of arrays with different shapes)"); + } + } + type = assert_cast(type.get())->getNestedType(); + nested_column = array_column->getDataPtr(); + dim++; + } + + /// for type String, get maximum string length + if (type->getTypeId() == TypeIndex::String) + { + const auto & string_offsets = assert_cast(nested_column.get())->getOffsets(); + for (size_t i = 0; i < string_offsets.size(); ++i) + { + size_t string_length = static_cast(string_offsets[i] - 1 - string_offsets[i - 1]); + numpy_data_type.size = numpy_data_type.size > string_length ? numpy_data_type.size : string_length; + } + } + + columns.push_back(nested_column); + } +} + +void NpyOutputFormat::finalizeImpl() +{ + if (!has_exception) + { + writeHeader(); + writeColumns(); + } +} + +void NpyOutputFormat::writeHeader() +{ + std::ostringstream static_header; + static_header << MAGIC_STRING << MAJOR_VERSION << MINOR_VERSION; + String static_header_str = static_header.str(); + + std::ostringstream shape; + shape << '(' << std::to_string(num_rows) << ','; + for (auto dim : numpy_shape) + shape << std::to_string(dim) << ','; + shape << ')'; + + std::ostringstream dict; + dict << "{'descr':'" << numpy_data_type.str() << "','fortran_order':False,'shape':" << shape.str() << ",}"; + String dict_str = dict.str(); + String padding_str = "\n"; + + /// completes the length of the header, which is divisible by 64. + size_t dict_length = dict_str.length() + 1; + size_t header_length = static_header_str.length() + sizeof(UInt32) + dict_length; + if (header_length % 64) + { + header_length = ((header_length / 64) + 1) * 64; + dict_length = header_length - static_header_str.length() - sizeof(UInt32); + padding_str = std::string(dict_length - dict_str.length(), '\x20'); + padding_str.back() = '\n'; + } + + out.write(static_header_str.data(), static_header_str.length()); + writeBinaryLittleEndian(assert_cast(dict_length), out); + out.write(dict_str.data(), dict_str.length()); + out.write(padding_str.data(), padding_str.length()); +} + +void NpyOutputFormat::writeColumns() +{ + for (auto column : columns) + { + switch (nested_data_type->getTypeId()) + { + case TypeIndex::Int8: writeNumpyNumbers(column, out); break; + case TypeIndex::Int16: writeNumpyNumbers(column, out); break; + case TypeIndex::Int32: writeNumpyNumbers(column, out); break; + case TypeIndex::Int64: writeNumpyNumbers(column, out); break; + case TypeIndex::UInt8: writeNumpyNumbers(column, out); break; + case TypeIndex::UInt16: writeNumpyNumbers(column, out); break; + case TypeIndex::UInt32: writeNumpyNumbers(column, out); break; + case TypeIndex::UInt64: writeNumpyNumbers(column, out); break; + case TypeIndex::Float32: writeNumpyNumbers(column, out); break; + case TypeIndex::Float64: writeNumpyNumbers(column, out); break; + case TypeIndex::FixedString: writeNumpyStrings(column, numpy_data_type.size, out); break; + case TypeIndex::String: writeNumpyStrings(column, numpy_data_type.size, out); break; + default: break; + } + } +} + +void registerOutputFormatNpy(FormatFactory & factory) +{ + factory.registerOutputFormat("Npy",[]( + WriteBuffer & buf, + const Block & sample, + const FormatSettings &) + { + return std::make_shared(buf, sample); + }); + factory.markFormatHasNoAppendSupport("Npy"); +} + +} diff --git a/src/Processors/Formats/Impl/NpyOutputFormat.h b/src/Processors/Formats/Impl/NpyOutputFormat.h new file mode 100644 index 00000000000..083aa928b7c --- /dev/null +++ b/src/Processors/Formats/Impl/NpyOutputFormat.h @@ -0,0 +1,66 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include + + +namespace DB +{ + +/** Stream for output data in Npy format. + * https://numpy.org/doc/stable/reference/generated/numpy.lib.format.html + */ +class NpyOutputFormat : public IOutputFormat +{ +public: + NpyOutputFormat(WriteBuffer & out_, const Block & header_); + + String getName() const override { return "NpyOutputFormat"; } + + String getContentType() const override { return "application/octet-stream"; } + +private: + struct NumpyDataType + { + char endianness; + char type; + size_t size; + + NumpyDataType() = default; + NumpyDataType(char endianness_, char type_, size_t size_) + : endianness(endianness_), type(type_), size(size_) {} + String str(); + }; + + void initialize(const ColumnPtr & column); + void consume(Chunk) override; + void finalizeImpl() override; + void writeHeader(); + void writeColumns(); + + bool is_initialized = false; + bool has_exception = false; + + DataTypePtr data_type; + DataTypePtr nested_data_type; + NumpyDataType numpy_data_type; + UInt64 num_rows = 0; + std::vector numpy_shape; + Columns columns; + + /// static header (version 3.0) + constexpr static auto MAGIC_STRING = "\x93NUMPY"; + constexpr static auto MAJOR_VERSION = '\x03'; + constexpr static auto MINOR_VERSION = '\x00'; +}; + +} diff --git a/tests/queries/0_stateless/02895_npy_output_format.reference b/tests/queries/0_stateless/02895_npy_output_format.reference new file mode 100644 index 00000000000..b599f1dceea --- /dev/null +++ b/tests/queries/0_stateless/02895_npy_output_format.reference @@ -0,0 +1,60 @@ +-1 +-1 +-1 +-1 +-1 +-1 +-1 +-1 +-1 +-1 +-1 +-1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +0.1 +0.1 +0.1 +0.01 +0.01 +0.01 +npy +npy +npy +npy +npy +npy +array Int8 +array Int16 +array Int32 +array Int64 +array UInt8 +array UInt16 +array UInt32 +array UInt64 +array Float32 +array Float64 +array String +array String +[[[1],[2]],[[3],[4]]] +[[[1],[2]],[[3],[4]]] +[[[1],[2]],[[3],[4]]] +[[0.1],[0.2]] +[[0.1],[0.2]] +[[0.1],[0.2]] +[['abb','bbc'],['ccc','dddd']] +[['abb','bbc'],['ccc','dddd']] +[['abb','bbc'],['ccc','dddd']] +array Array(Array(Array(Int8))) +array Array(Array(Float64)) +array Array(Array(String)) diff --git a/tests/queries/0_stateless/02895_npy_output_format.sh b/tests/queries/0_stateless/02895_npy_output_format.sh new file mode 100755 index 00000000000..e5226e88a8d --- /dev/null +++ b/tests/queries/0_stateless/02895_npy_output_format.sh @@ -0,0 +1,110 @@ +#!/usr/bin/env bash +# Tags: no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +user_files_path=$($CLICKHOUSE_CLIENT_BINARY -q "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +mkdir -p ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ +rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/* +chmod 777 ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ + +${CLICKHOUSE_CLIENT} -q "DROP DATABASE IF EXISTS npy_output_02895;" +${CLICKHOUSE_CLIENT} -q "CREATE DATABASE IF NOT EXISTS npy_output_02895;" + +### test common type +${CLICKHOUSE_CLIENT} -q "CREATE TABLE IF NOT EXISTS npy_output_02895.common +( + i1 Int8, + i2 Int16, + i4 Int32, + i8 Int64, + u1 UInt8, + u2 UInt16, + u4 UInt32, + u8 UInt64, + f4 Float32, + f8 Float64, + fs FixedString(10), + s String, + unknow Int128 +) Engine = MergeTree ORDER BY i1;" + +${CLICKHOUSE_CLIENT} -q "INSERT INTO npy_output_02895.common VALUES (-1,-1,-1,-1,1,1,1,1,0.1,0.01,'npy','npy',1), (-1,-1,-1,-1,1,1,1,1,0.1,0.01,'npy','npy',1), (-1,-1,-1,-1,1,1,1,1,0.1,0.01,'npy','npy',1);" + +${CLICKHOUSE_CLIENT} -n -q "SELECT * FROM npy_output_02895.common FORMAT Npy; -- { clientError TOO_MANY_COLUMNS }" +${CLICKHOUSE_CLIENT} -n -q "SELECT unknow FROM npy_output_02895.common FORMAT Npy; -- { clientError BAD_ARGUMENTS }" + +${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int8.npy') SELECT i1 FROM npy_output_02895.common;" +${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int16.npy') SELECT i2 FROM npy_output_02895.common;" +${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int32.npy') SELECT i4 FROM npy_output_02895.common;" +${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int64.npy') SELECT i8 FROM npy_output_02895.common;" +${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint8.npy') SELECT u1 FROM npy_output_02895.common;" +${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint16.npy') SELECT u2 FROM npy_output_02895.common;" +${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint32.npy') SELECT u4 FROM npy_output_02895.common;" +${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint64.npy') SELECT u8 FROM npy_output_02895.common;" +${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_float32.npy') SELECT f4 FROM npy_output_02895.common;" +${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_float64.npy') SELECT f8 FROM npy_output_02895.common;" +${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_fixedstring.npy') SELECT fs FROM npy_output_02895.common;" +${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_string.npy') SELECT s FROM npy_output_02895.common;" + +${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int8.npy');" +${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int16.npy');" +${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int32.npy');" +${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int64.npy');" +${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint8.npy');" +${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint16.npy');" +${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint32.npy');" +${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint64.npy');" +${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_float32.npy');" +${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_float64.npy');" +${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_fixedstring.npy');" +${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_string.npy');" + +${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int8.npy');" +${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int16.npy');" +${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int32.npy');" +${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int64.npy');" +${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint8.npy');" +${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint16.npy');" +${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint32.npy');" +${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint64.npy');" +${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_float32.npy');" +${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_float64.npy');" +${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_fixedstring.npy');" +${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_string.npy');" + +### test nested type +${CLICKHOUSE_CLIENT} -q "CREATE TABLE IF NOT EXISTS npy_output_02895.nested +( + i4 Array(Array(Array(Int8))), + f8 Array(Array(Float64)), + s Array(Array(String)), + unknow Array(Int128), + ragged_1 Array(Array(Int32)), + ragged_2 Array(Array(Int32)) +) Engine = MergeTree ORDER BY i4;" + +${CLICKHOUSE_CLIENT} -q "INSERT INTO npy_output_02895.nested VALUES ([[[1], [2]], [[3], [4]]], [[0.1], [0.2]], [['a', 'bb'], ['ccc', 'dddd']], [1, 2], [[1, 2], [3, 4]], [[1, 2], [3]]), ([[[1], [2]], [[3], [4]]], [[0.1], [0.2]], [['a', 'bb'], ['ccc', 'dddd']], [1, 2], [[1, 2, 3], [4]], [[1, 2], [3]]), ([[[1], [2]], [[3], [4]]], [[0.1], [0.2]], [['a', 'bb'], ['ccc', 'dddd']], [1, 2], [[1], [2, 3, 4]], [[1, 2], [3]]);" + +${CLICKHOUSE_CLIENT} -n -q "SELECT * FROM npy_output_02895.nested FORMAT Npy; -- { clientError TOO_MANY_COLUMNS }" +${CLICKHOUSE_CLIENT} -n -q "SELECT unknow FROM npy_output_02895.nested FORMAT Npy; -- { clientError BAD_ARGUMENTS }" +${CLICKHOUSE_CLIENT} -n -q "SELECT ragged_1 FROM npy_output_02895.nested FORMAT Npy; -- { clientError ILLEGAL_COLUMN }" +${CLICKHOUSE_CLIENT} -n -q "SELECT ragged_2 FROM npy_output_02895.nested FORMAT Npy; -- { clientError ILLEGAL_COLUMN }" + +${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_int32.npy') SELECT i4 FROM npy_output_02895.nested;" +${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_float64.npy') SELECT f8 FROM npy_output_02895.nested;" +${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_string.npy') SELECT s FROM npy_output_02895.nested;" + +${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_int32.npy');" +${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_float64.npy');" +${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_string.npy');" + +${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_int32.npy');" +${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_float64.npy');" +${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_string.npy');" + +${CLICKHOUSE_CLIENT} -q "DROP DATABASE IF EXISTS npy_output_02895;" + +rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?} From c3a1ec30b1c2432700c72863103415a1b295b5fa Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 9 Apr 2024 11:02:10 +0200 Subject: [PATCH 105/624] 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 fe66d1b67a357155d35ca364a06c43cd44770056 Mon Sep 17 00:00:00 2001 From: HowePa <2873679104@qq.com> Date: Tue, 9 Apr 2024 20:23:45 +0800 Subject: [PATCH 106/624] fix style --- .../Formats/Impl/NpyOutputFormat.cpp | 45 ++++++++++--------- src/Processors/Formats/Impl/NpyOutputFormat.h | 5 +-- 2 files changed, 25 insertions(+), 25 deletions(-) diff --git a/src/Processors/Formats/Impl/NpyOutputFormat.cpp b/src/Processors/Formats/Impl/NpyOutputFormat.cpp index 4a4af67b07e..d04d095e84f 100644 --- a/src/Processors/Formats/Impl/NpyOutputFormat.cpp +++ b/src/Processors/Formats/Impl/NpyOutputFormat.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -46,8 +47,11 @@ void writeNumpyStrings(const ColumnPtr & column, size_t length, WriteBuffer & bu String NpyOutputFormat::NumpyDataType::str() { - std::ostringstream dtype; - dtype << endianness << type << std::to_string(size); + WriteBufferFromOwnString dtype; + writeChar(endianness, dtype); + writeChar(type, dtype); + writeIntText(size, dtype); + return dtype.str(); } @@ -153,36 +157,33 @@ void NpyOutputFormat::finalizeImpl() void NpyOutputFormat::writeHeader() { - std::ostringstream static_header; - static_header << MAGIC_STRING << MAJOR_VERSION << MINOR_VERSION; - String static_header_str = static_header.str(); - - std::ostringstream shape; - shape << '(' << std::to_string(num_rows) << ','; + WriteBufferFromOwnString shape; + writeIntText(num_rows, shape); + writeChar(',', shape); for (auto dim : numpy_shape) - shape << std::to_string(dim) << ','; - shape << ')'; + { + writeIntText(dim, shape); + writeChar(',', shape); + } - std::ostringstream dict; - dict << "{'descr':'" << numpy_data_type.str() << "','fortran_order':False,'shape':" << shape.str() << ",}"; - String dict_str = dict.str(); - String padding_str = "\n"; + String dict = "{'descr':'" + numpy_data_type.str() + "','fortran_order':False,'shape':(" + shape.str() + "),}"; + String padding = "\n"; /// completes the length of the header, which is divisible by 64. - size_t dict_length = dict_str.length() + 1; - size_t header_length = static_header_str.length() + sizeof(UInt32) + dict_length; + size_t dict_length = dict.length() + 1; + size_t header_length = STATIC_HEADER_LENGTH + sizeof(UInt32) + dict_length; if (header_length % 64) { header_length = ((header_length / 64) + 1) * 64; - dict_length = header_length - static_header_str.length() - sizeof(UInt32); - padding_str = std::string(dict_length - dict_str.length(), '\x20'); - padding_str.back() = '\n'; + dict_length = header_length - STATIC_HEADER_LENGTH - sizeof(UInt32); + padding = std::string(dict_length - dict.length(), '\x20'); + padding.back() = '\n'; } - out.write(static_header_str.data(), static_header_str.length()); + out.write(STATIC_HEADER, STATIC_HEADER_LENGTH); writeBinaryLittleEndian(assert_cast(dict_length), out); - out.write(dict_str.data(), dict_str.length()); - out.write(padding_str.data(), padding_str.length()); + out.write(dict.data(), dict.length()); + out.write(padding.data(), padding.length()); } void NpyOutputFormat::writeColumns() diff --git a/src/Processors/Formats/Impl/NpyOutputFormat.h b/src/Processors/Formats/Impl/NpyOutputFormat.h index 083aa928b7c..a483420d2d6 100644 --- a/src/Processors/Formats/Impl/NpyOutputFormat.h +++ b/src/Processors/Formats/Impl/NpyOutputFormat.h @@ -58,9 +58,8 @@ private: Columns columns; /// static header (version 3.0) - constexpr static auto MAGIC_STRING = "\x93NUMPY"; - constexpr static auto MAJOR_VERSION = '\x03'; - constexpr static auto MINOR_VERSION = '\x00'; + constexpr static auto STATIC_HEADER = "\x93NUMPY\x03\x00"; + constexpr static size_t STATIC_HEADER_LENGTH = 8; }; } From 3c58e5873b24e0aec20a4c5b97c3ab6bb849c47e Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Tue, 9 Apr 2024 19:06:14 +0000 Subject: [PATCH 107/624] 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 108/624] 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 109/624] 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 110/624] 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 d9de697305522a1c267debaf13792ab900b16a83 Mon Sep 17 00:00:00 2001 From: HowePa <2873679104@qq.com> Date: Wed, 10 Apr 2024 17:48:23 +0800 Subject: [PATCH 111/624] refactor test --- .../0_stateless/02895_npy_output_format.sh | 165 +++++++++--------- 1 file changed, 82 insertions(+), 83 deletions(-) diff --git a/tests/queries/0_stateless/02895_npy_output_format.sh b/tests/queries/0_stateless/02895_npy_output_format.sh index e5226e88a8d..27274f6a925 100755 --- a/tests/queries/0_stateless/02895_npy_output_format.sh +++ b/tests/queries/0_stateless/02895_npy_output_format.sh @@ -10,101 +10,100 @@ mkdir -p ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/* chmod 777 ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ -${CLICKHOUSE_CLIENT} -q "DROP DATABASE IF EXISTS npy_output_02895;" -${CLICKHOUSE_CLIENT} -q "CREATE DATABASE IF NOT EXISTS npy_output_02895;" +${CLICKHOUSE_CLIENT} -n -q " + DROP DATABASE IF EXISTS npy_output_02895; + CREATE DATABASE IF NOT EXISTS npy_output_02895; -### test common type -${CLICKHOUSE_CLIENT} -q "CREATE TABLE IF NOT EXISTS npy_output_02895.common -( - i1 Int8, - i2 Int16, - i4 Int32, - i8 Int64, - u1 UInt8, - u2 UInt16, - u4 UInt32, - u8 UInt64, - f4 Float32, - f8 Float64, - fs FixedString(10), - s String, - unknow Int128 -) Engine = MergeTree ORDER BY i1;" + CREATE TABLE IF NOT EXISTS npy_output_02895.common + ( + i1 Int8, + i2 Int16, + i4 Int32, + i8 Int64, + u1 UInt8, + u2 UInt16, + u4 UInt32, + u8 UInt64, + f4 Float32, + f8 Float64, + fs FixedString(10), + s String, + unknow Int128 + ) Engine = MergeTree ORDER BY i1; -${CLICKHOUSE_CLIENT} -q "INSERT INTO npy_output_02895.common VALUES (-1,-1,-1,-1,1,1,1,1,0.1,0.01,'npy','npy',1), (-1,-1,-1,-1,1,1,1,1,0.1,0.01,'npy','npy',1), (-1,-1,-1,-1,1,1,1,1,0.1,0.01,'npy','npy',1);" + INSERT INTO npy_output_02895.common VALUES (-1,-1,-1,-1,1,1,1,1,0.1,0.01,'npy','npy',1), (-1,-1,-1,-1,1,1,1,1,0.1,0.01,'npy','npy',1), (-1,-1,-1,-1,1,1,1,1,0.1,0.01,'npy','npy',1); -${CLICKHOUSE_CLIENT} -n -q "SELECT * FROM npy_output_02895.common FORMAT Npy; -- { clientError TOO_MANY_COLUMNS }" -${CLICKHOUSE_CLIENT} -n -q "SELECT unknow FROM npy_output_02895.common FORMAT Npy; -- { clientError BAD_ARGUMENTS }" + SELECT * FROM npy_output_02895.common FORMAT Npy; -- { clientError TOO_MANY_COLUMNS } + SELECT unknow FROM npy_output_02895.common FORMAT Npy; -- { clientError BAD_ARGUMENTS } -${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int8.npy') SELECT i1 FROM npy_output_02895.common;" -${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int16.npy') SELECT i2 FROM npy_output_02895.common;" -${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int32.npy') SELECT i4 FROM npy_output_02895.common;" -${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int64.npy') SELECT i8 FROM npy_output_02895.common;" -${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint8.npy') SELECT u1 FROM npy_output_02895.common;" -${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint16.npy') SELECT u2 FROM npy_output_02895.common;" -${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint32.npy') SELECT u4 FROM npy_output_02895.common;" -${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint64.npy') SELECT u8 FROM npy_output_02895.common;" -${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_float32.npy') SELECT f4 FROM npy_output_02895.common;" -${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_float64.npy') SELECT f8 FROM npy_output_02895.common;" -${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_fixedstring.npy') SELECT fs FROM npy_output_02895.common;" -${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_string.npy') SELECT s FROM npy_output_02895.common;" + INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int8.npy') SELECT i1 FROM npy_output_02895.common; + INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int16.npy') SELECT i2 FROM npy_output_02895.common; + INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int32.npy') SELECT i4 FROM npy_output_02895.common; + INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int64.npy') SELECT i8 FROM npy_output_02895.common; + INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint8.npy') SELECT u1 FROM npy_output_02895.common; + INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint16.npy') SELECT u2 FROM npy_output_02895.common; + INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint32.npy') SELECT u4 FROM npy_output_02895.common; + INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint64.npy') SELECT u8 FROM npy_output_02895.common; + INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_float32.npy') SELECT f4 FROM npy_output_02895.common; + INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_float64.npy') SELECT f8 FROM npy_output_02895.common; + INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_fixedstring.npy') SELECT fs FROM npy_output_02895.common; + INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_string.npy') SELECT s FROM npy_output_02895.common; -${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int8.npy');" -${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int16.npy');" -${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int32.npy');" -${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int64.npy');" -${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint8.npy');" -${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint16.npy');" -${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint32.npy');" -${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint64.npy');" -${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_float32.npy');" -${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_float64.npy');" -${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_fixedstring.npy');" -${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_string.npy');" + SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int8.npy'); + SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int16.npy'); + SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int32.npy'); + SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int64.npy'); + SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint8.npy'); + SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint16.npy'); + SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint32.npy'); + SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint64.npy'); + SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_float32.npy'); + SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_float64.npy'); + SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_fixedstring.npy'); + SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_string.npy'); -${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int8.npy');" -${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int16.npy');" -${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int32.npy');" -${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int64.npy');" -${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint8.npy');" -${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint16.npy');" -${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint32.npy');" -${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint64.npy');" -${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_float32.npy');" -${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_float64.npy');" -${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_fixedstring.npy');" -${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_string.npy');" + DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int8.npy'); + DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int16.npy'); + DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int32.npy'); + DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_int64.npy'); + DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint8.npy'); + DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint16.npy'); + DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint32.npy'); + DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_uint64.npy'); + DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_float32.npy'); + DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_float64.npy'); + DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_fixedstring.npy'); + DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_common_string.npy'); -### test nested type -${CLICKHOUSE_CLIENT} -q "CREATE TABLE IF NOT EXISTS npy_output_02895.nested -( - i4 Array(Array(Array(Int8))), - f8 Array(Array(Float64)), - s Array(Array(String)), - unknow Array(Int128), - ragged_1 Array(Array(Int32)), - ragged_2 Array(Array(Int32)) -) Engine = MergeTree ORDER BY i4;" + CREATE TABLE IF NOT EXISTS npy_output_02895.nested + ( + i4 Array(Array(Array(Int8))), + f8 Array(Array(Float64)), + s Array(Array(String)), + unknow Array(Int128), + ragged_1 Array(Array(Int32)), + ragged_2 Array(Array(Int32)) + ) Engine = MergeTree ORDER BY i4; -${CLICKHOUSE_CLIENT} -q "INSERT INTO npy_output_02895.nested VALUES ([[[1], [2]], [[3], [4]]], [[0.1], [0.2]], [['a', 'bb'], ['ccc', 'dddd']], [1, 2], [[1, 2], [3, 4]], [[1, 2], [3]]), ([[[1], [2]], [[3], [4]]], [[0.1], [0.2]], [['a', 'bb'], ['ccc', 'dddd']], [1, 2], [[1, 2, 3], [4]], [[1, 2], [3]]), ([[[1], [2]], [[3], [4]]], [[0.1], [0.2]], [['a', 'bb'], ['ccc', 'dddd']], [1, 2], [[1], [2, 3, 4]], [[1, 2], [3]]);" + INSERT INTO npy_output_02895.nested VALUES ([[[1], [2]], [[3], [4]]], [[0.1], [0.2]], [['a', 'bb'], ['ccc', 'dddd']], [1, 2], [[1, 2], [3, 4]], [[1, 2], [3]]), ([[[1], [2]], [[3], [4]]], [[0.1], [0.2]], [['a', 'bb'], ['ccc', 'dddd']], [1, 2], [[1, 2, 3], [4]], [[1, 2], [3]]), ([[[1], [2]], [[3], [4]]], [[0.1], [0.2]], [['a', 'bb'], ['ccc', 'dddd']], [1, 2], [[1], [2, 3, 4]], [[1, 2], [3]]); -${CLICKHOUSE_CLIENT} -n -q "SELECT * FROM npy_output_02895.nested FORMAT Npy; -- { clientError TOO_MANY_COLUMNS }" -${CLICKHOUSE_CLIENT} -n -q "SELECT unknow FROM npy_output_02895.nested FORMAT Npy; -- { clientError BAD_ARGUMENTS }" -${CLICKHOUSE_CLIENT} -n -q "SELECT ragged_1 FROM npy_output_02895.nested FORMAT Npy; -- { clientError ILLEGAL_COLUMN }" -${CLICKHOUSE_CLIENT} -n -q "SELECT ragged_2 FROM npy_output_02895.nested FORMAT Npy; -- { clientError ILLEGAL_COLUMN }" + SELECT * FROM npy_output_02895.nested FORMAT Npy; -- { clientError TOO_MANY_COLUMNS } + SELECT unknow FROM npy_output_02895.nested FORMAT Npy; -- { clientError BAD_ARGUMENTS } + SELECT ragged_1 FROM npy_output_02895.nested FORMAT Npy; -- { clientError ILLEGAL_COLUMN } + SELECT ragged_2 FROM npy_output_02895.nested FORMAT Npy; -- { clientError ILLEGAL_COLUMN } -${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_int32.npy') SELECT i4 FROM npy_output_02895.nested;" -${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_float64.npy') SELECT f8 FROM npy_output_02895.nested;" -${CLICKHOUSE_CLIENT} -q "INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_string.npy') SELECT s FROM npy_output_02895.nested;" + INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_int32.npy') SELECT i4 FROM npy_output_02895.nested; + INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_float64.npy') SELECT f8 FROM npy_output_02895.nested; + INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_string.npy') SELECT s FROM npy_output_02895.nested; -${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_int32.npy');" -${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_float64.npy');" -${CLICKHOUSE_CLIENT} -q "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_string.npy');" + SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_int32.npy'); + SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_float64.npy'); + SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_string.npy'); -${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_int32.npy');" -${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_float64.npy');" -${CLICKHOUSE_CLIENT} -q "DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_string.npy');" + DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_int32.npy'); + DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_float64.npy'); + DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_string.npy'); -${CLICKHOUSE_CLIENT} -q "DROP DATABASE IF EXISTS npy_output_02895;" + DROP DATABASE IF EXISTS npy_output_02895;" rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?} From b6b7c3f80f6461e8198e9d70f6b4742f62d6435d Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 10 Apr 2024 13:41:21 +0200 Subject: [PATCH 112/624] 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 113/624] 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 9975b6a0f5c313a2552a8169f5be555dd2f7f4ad Mon Sep 17 00:00:00 2001 From: HowePa <2873679104@qq.com> Date: Wed, 10 Apr 2024 20:18:30 +0800 Subject: [PATCH 114/624] fix build --- src/Processors/Formats/Impl/NpyOutputFormat.cpp | 4 ++-- src/Processors/Formats/Impl/NpyOutputFormat.h | 3 +-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Processors/Formats/Impl/NpyOutputFormat.cpp b/src/Processors/Formats/Impl/NpyOutputFormat.cpp index d04d095e84f..f43deb816e0 100644 --- a/src/Processors/Formats/Impl/NpyOutputFormat.cpp +++ b/src/Processors/Formats/Impl/NpyOutputFormat.cpp @@ -45,7 +45,7 @@ void writeNumpyStrings(const ColumnPtr & column, size_t length, WriteBuffer & bu } -String NpyOutputFormat::NumpyDataType::str() +String NpyOutputFormat::NumpyDataType::str() const { WriteBufferFromOwnString dtype; writeChar(endianness, dtype); @@ -188,7 +188,7 @@ void NpyOutputFormat::writeHeader() void NpyOutputFormat::writeColumns() { - for (auto column : columns) + for (const auto & column : columns) { switch (nested_data_type->getTypeId()) { diff --git a/src/Processors/Formats/Impl/NpyOutputFormat.h b/src/Processors/Formats/Impl/NpyOutputFormat.h index a483420d2d6..f1d0216a8c3 100644 --- a/src/Processors/Formats/Impl/NpyOutputFormat.h +++ b/src/Processors/Formats/Impl/NpyOutputFormat.h @@ -9,7 +9,6 @@ #include #include -#include #include @@ -38,7 +37,7 @@ private: NumpyDataType() = default; NumpyDataType(char endianness_, char type_, size_t size_) : endianness(endianness_), type(type_), size(size_) {} - String str(); + String str() const; }; void initialize(const ColumnPtr & column); From 9085816fcb25c34ffc4ce664f3f10b4c3aca160d Mon Sep 17 00:00:00 2001 From: HowePa <2873679104@qq.com> Date: Wed, 10 Apr 2024 23:06:51 +0800 Subject: [PATCH 115/624] remove from async test --- .../queries/0_stateless/02187_async_inserts_all_formats.python | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02187_async_inserts_all_formats.python b/tests/queries/0_stateless/02187_async_inserts_all_formats.python index fa555c78f8b..943e32d4cf2 100644 --- a/tests/queries/0_stateless/02187_async_inserts_all_formats.python +++ b/tests/queries/0_stateless/02187_async_inserts_all_formats.python @@ -38,7 +38,7 @@ def run_test(data_format, gen_data_template, settings): formats = ( client.query( "SELECT name FROM system.formats WHERE is_input AND is_output \ - AND name NOT IN ('CapnProto', 'RawBLOB', 'Template', 'ProtobufSingle', 'LineAsString', 'Protobuf', 'ProtobufList') ORDER BY name" + AND name NOT IN ('CapnProto', 'RawBLOB', 'Template', 'ProtobufSingle', 'LineAsString', 'Protobuf', 'ProtobufList', 'Npy') ORDER BY name" ) .strip() .split("\n") From 8344167d8bb038bce36da5bb473d6298757be09d Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 10 Apr 2024 19:17:21 +0200 Subject: [PATCH 116/624] 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 117/624] 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 118/624] 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 619d1b42e14be65e77bd0f1980e2b16696f74070 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 10 Apr 2024 18:35:20 +0000 Subject: [PATCH 119/624] 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 120/624] 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 b464d16d5fbb67797677d63d05d8bd9802821a7c Mon Sep 17 00:00:00 2001 From: HowePa <2873679104@qq.com> Date: Thu, 11 Apr 2024 11:34:56 +0800 Subject: [PATCH 121/624] refactor and fix type conversion --- .../Formats/Impl/NpyOutputFormat.cpp | 139 ++++++++++-------- src/Processors/Formats/Impl/NpyOutputFormat.h | 10 +- 2 files changed, 87 insertions(+), 62 deletions(-) diff --git a/src/Processors/Formats/Impl/NpyOutputFormat.cpp b/src/Processors/Formats/Impl/NpyOutputFormat.cpp index f43deb816e0..d54fc7e68f2 100644 --- a/src/Processors/Formats/Impl/NpyOutputFormat.cpp +++ b/src/Processors/Formats/Impl/NpyOutputFormat.cpp @@ -55,6 +55,20 @@ String NpyOutputFormat::NumpyDataType::str() const return dtype.str(); } +String NpyOutputFormat::shapeStr() const +{ + WriteBufferFromOwnString shape; + writeIntText(num_rows, shape); + writeChar(',', shape); + for (UInt64 dim : numpy_shape) + { + writeIntText(dim, shape); + writeChar(',', shape); + } + + return shape.str(); +} + NpyOutputFormat::NpyOutputFormat(WriteBuffer & out_, const Block & header_) : IOutputFormat(header_, out_) { const auto & header = getPort(PortKind::Main).getHeader(); @@ -62,20 +76,13 @@ NpyOutputFormat::NpyOutputFormat(WriteBuffer & out_, const Block & header_) : IO if (data_types.size() > 1) throw Exception(ErrorCodes::TOO_MANY_COLUMNS, "Expected single column for Npy output format, got {}", data_types.size()); data_type = data_types[0]; + + if (!getNumpyDataType(data_type)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Type {} is not supported for Npy output format", nested_data_type->getName()); } -void NpyOutputFormat::initialize(const ColumnPtr & column) +bool NpyOutputFormat::getNumpyDataType(const DataTypePtr & type) { - auto type = data_type; - ColumnPtr nested_column = column; - while (type->getTypeId() == TypeIndex::Array) - { - const auto * array_column = assert_cast(nested_column.get()); - numpy_shape.push_back(array_column->getOffsets()[0]); - type = assert_cast(type.get())->getNestedType(); - nested_column = array_column->getDataPtr(); - } - switch (type->getTypeId()) { case TypeIndex::Int8: numpy_data_type = NumpyDataType('<', 'i', sizeof(Int8)); break; @@ -90,65 +97,86 @@ void NpyOutputFormat::initialize(const ColumnPtr & column) case TypeIndex::Float64: numpy_data_type = NumpyDataType('<', 'f', sizeof(Float64)); break; case TypeIndex::FixedString: numpy_data_type = NumpyDataType('|', 'S', assert_cast(type.get())->getN()); break; case TypeIndex::String: numpy_data_type = NumpyDataType('|', 'S', 0); break; - default: - has_exception = true; - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Type {} is not supported for Npy output format", type->getName()); + case TypeIndex::Array: return getNumpyDataType(assert_cast(type.get())->getNestedType()); + default: nested_data_type = type; return false; } + nested_data_type = type; + return true; } void NpyOutputFormat::consume(Chunk chunk) { - if (!has_exception) + if (!invalid_shape) { num_rows += chunk.getNumRows(); - auto column = chunk.getColumns()[0]; + const auto & column = chunk.getColumns()[0]; if (!is_initialized) { - initialize(column); + initShape(column); is_initialized = true; } - /// check shape - auto type = data_type; - ColumnPtr nested_column = column; - int dim = 0; - while (type->getTypeId() == TypeIndex::Array) + if (!checkShape(column)) { - const auto * array_column = assert_cast(nested_column.get()); - const auto & array_offset = array_column->getOffsets(); - for (size_t i = 1; i < array_offset.size(); ++i) - { - if (array_offset[i] - array_offset[i - 1] != numpy_shape[dim]) - { - has_exception = true; - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "ClickHouse doesn't support object types, cannot format ragged nested sequences (which is a list of arrays with different shapes)"); - } - } - type = assert_cast(type.get())->getNestedType(); - nested_column = array_column->getDataPtr(); - dim++; + invalid_shape = true; + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "ClickHouse doesn't support object types, cannot format ragged nested sequences (which is a list of arrays with different shapes)"); } - - /// for type String, get maximum string length - if (type->getTypeId() == TypeIndex::String) - { - const auto & string_offsets = assert_cast(nested_column.get())->getOffsets(); - for (size_t i = 0; i < string_offsets.size(); ++i) - { - size_t string_length = static_cast(string_offsets[i] - 1 - string_offsets[i - 1]); - numpy_data_type.size = numpy_data_type.size > string_length ? numpy_data_type.size : string_length; - } - } - - columns.push_back(nested_column); } } +void NpyOutputFormat::initShape(const ColumnPtr & column) +{ + auto type = data_type; + ColumnPtr nested_column = column; + while (type->getTypeId() == TypeIndex::Array) + { + const auto * array_column = assert_cast(nested_column.get()); + + numpy_shape.push_back(array_column->getOffsets()[0]); + + type = assert_cast(type.get())->getNestedType(); + nested_column = array_column->getDataPtr(); + } +} + +bool NpyOutputFormat::checkShape(const ColumnPtr & column) +{ + auto type = data_type; + ColumnPtr nested_column = column; + int dim = 0; + while (type->getTypeId() == TypeIndex::Array) + { + const auto * array_column = assert_cast(nested_column.get()); + const auto & array_offset = array_column->getOffsets(); + + for (size_t i = 1; i < array_offset.size(); ++i) + if (array_offset[i] - array_offset[i - 1] != numpy_shape[dim]) + return false; + + type = assert_cast(type.get())->getNestedType(); + nested_column = array_column->getDataPtr(); + dim += 1; + } + + if (type->getTypeId() == TypeIndex::String) + { + const auto & string_offsets = assert_cast(nested_column.get())->getOffsets(); + for (size_t i = 0; i < string_offsets.size(); ++i) + { + size_t string_length = static_cast(string_offsets[i] - 1 - string_offsets[i - 1]); + numpy_data_type.size = numpy_data_type.size > string_length ? numpy_data_type.size : string_length; + } + } + + columns.push_back(nested_column); + return true; +} + void NpyOutputFormat::finalizeImpl() { - if (!has_exception) + if (!invalid_shape) { writeHeader(); writeColumns(); @@ -157,16 +185,7 @@ void NpyOutputFormat::finalizeImpl() void NpyOutputFormat::writeHeader() { - WriteBufferFromOwnString shape; - writeIntText(num_rows, shape); - writeChar(',', shape); - for (auto dim : numpy_shape) - { - writeIntText(dim, shape); - writeChar(',', shape); - } - - String dict = "{'descr':'" + numpy_data_type.str() + "','fortran_order':False,'shape':(" + shape.str() + "),}"; + String dict = "{'descr':'" + numpy_data_type.str() + "','fortran_order':False,'shape':(" + shapeStr() + "),}"; String padding = "\n"; /// completes the length of the header, which is divisible by 64. @@ -181,7 +200,7 @@ void NpyOutputFormat::writeHeader() } out.write(STATIC_HEADER, STATIC_HEADER_LENGTH); - writeBinaryLittleEndian(assert_cast(dict_length), out); + writeBinaryLittleEndian(static_cast(dict_length), out); out.write(dict.data(), dict.length()); out.write(padding.data(), padding.length()); } diff --git a/src/Processors/Formats/Impl/NpyOutputFormat.h b/src/Processors/Formats/Impl/NpyOutputFormat.h index f1d0216a8c3..83fad657b2e 100644 --- a/src/Processors/Formats/Impl/NpyOutputFormat.h +++ b/src/Processors/Formats/Impl/NpyOutputFormat.h @@ -40,14 +40,20 @@ private: String str() const; }; - void initialize(const ColumnPtr & column); + String shapeStr() const; + + bool getNumpyDataType(const DataTypePtr & type); + void consume(Chunk) override; + void initShape(const ColumnPtr & column); + bool checkShape(const ColumnPtr & column); + void finalizeImpl() override; void writeHeader(); void writeColumns(); bool is_initialized = false; - bool has_exception = false; + bool invalid_shape = false; DataTypePtr data_type; DataTypePtr nested_data_type; From 7806f64226e7bdba5240088ee8fd8213d0a4586e Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 16 Apr 2024 14:31:54 +0100 Subject: [PATCH 122/624] 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 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 123/624] 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 ad442ce024acebe8c617388817622b42534e50b3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 17 Apr 2024 17:47:51 +0100 Subject: [PATCH 124/624] 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 125/624] 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 506517e42b0073daac4510a107f3d6eba4147bb4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 18 Apr 2024 11:13:59 +0100 Subject: [PATCH 126/624] 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 11eab640c855938bd317348bdc4922753ecbc849 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 18 Apr 2024 13:14:41 +0100 Subject: [PATCH 127/624] Ping CI From 613dd1a6c040c1399a9027652d73321c71fbeeb7 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 18 Apr 2024 18:26:07 +0200 Subject: [PATCH 128/624] 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 129/624] 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 ff0376fde24c58ebd65ec0fa200db1036ebb2fcf Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 19 Apr 2024 12:30:13 +0200 Subject: [PATCH 130/624] 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 2f6db2ac47d55a1d740463173d2613d895419256 Mon Sep 17 00:00:00 2001 From: skyoct Date: Sun, 21 Apr 2024 12:01:31 +0000 Subject: [PATCH 131/624] fix: batter --- src/Functions/clamp.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Functions/clamp.cpp b/src/Functions/clamp.cpp index 0593c6c6b16..3438377afbf 100644 --- a/src/Functions/clamp.cpp +++ b/src/Functions/clamp.cpp @@ -42,12 +42,13 @@ public: Columns converted_columns(arg_size); for (size_t arg = 0; arg < arg_size; ++arg) converted_columns[arg] = castColumn(arguments[arg], result_type)->convertToFullColumnIfConst(); - if (converted_columns[1]->compareAt(0, 0, *converted_columns[2], 1) > 0) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Function {} the minimum value cannot be greater than the maximum value", getName()); auto result_column = result_type->createColumn(); for (size_t row_num = 0; row_num < input_rows_count; ++row_num) { + if (converted_columns[1]->compareAt(row_num, row_num, *converted_columns[2], 1) > 0) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Function {} the minimum value cannot be greater than the maximum value", getName()); + size_t best_arg = 0; if (converted_columns[1]->compareAt(row_num, row_num, *converted_columns[best_arg], 1) > 0) best_arg = 1; From ab49ab9172dba3c26351b9ab1e417545696a86e8 Mon Sep 17 00:00:00 2001 From: skyoct Date: Sun, 21 Apr 2024 12:03:15 +0000 Subject: [PATCH 132/624] test --- tests/queries/0_stateless/03036_clamp.reference | 8 ++++++-- tests/queries/0_stateless/03036_clamp.sql | 9 +++++++-- 2 files changed, 13 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/03036_clamp.reference b/tests/queries/0_stateless/03036_clamp.reference index 5c8de57794d..b866caf2261 100644 --- a/tests/queries/0_stateless/03036_clamp.reference +++ b/tests/queries/0_stateless/03036_clamp.reference @@ -6,5 +6,9 @@ b ['hello'] -1 234 -null -null \ No newline at end of file +\N +\N +5 +0 +1 +2 diff --git a/tests/queries/0_stateless/03036_clamp.sql b/tests/queries/0_stateless/03036_clamp.sql index e075096f8ef..d225be63f46 100644 --- a/tests/queries/0_stateless/03036_clamp.sql +++ b/tests/queries/0_stateless/03036_clamp.sql @@ -2,9 +2,14 @@ SELECT clamp(1, 10, 20); SELECT clamp(30, 10, 20); SELECT clamp(15, 10, 20); SELECT clamp('a', 'b', 'c'); -SELECT clamp(today(), yesterday() - 10, yesterday() + 10) - today() +SELECT clamp(today(), yesterday() - 10, yesterday() + 10) - today(); SELECT clamp([], ['hello'], ['world']); SELECT clamp(-1., -1000., 18446744073709551615.); SELECT clamp(toNullable(123), 234, 456); select clamp(1, null, 5); -select clamp(1, 6, null); \ No newline at end of file +select clamp(1, 6, null); +select clamp(1, 5, nan); +select clamp(toInt64(number), toInt64(number-1), toInt64(number+1)) from numbers(3); +select clamp(number, number-1, number+1) from numbers(3); -- { serverError 386 } +select clamp(1, 3, 2); -- { serverError 36 } +select clamp(1, data[1], data[2])from (select arrayJoin([[1, 2], [2,3], [3,2], [4, 4]]) as data); -- { serverError 36 } 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 133/624] 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 1f207de7a10dfc9c9b0ca76f8d62d55da6e04c61 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 24 Apr 2024 16:40:27 +0000 Subject: [PATCH 134/624] Add another case with GROUP BY USE NULLS --- ...3023_group_by_use_nulls_analyzer_crashes.reference | 10 ++++++++++ .../03023_group_by_use_nulls_analyzer_crashes.sql | 11 +++++++++++ 2 files changed, 21 insertions(+) diff --git a/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.reference b/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.reference index 17a17484a0c..e2682487229 100644 --- a/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.reference +++ b/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.reference @@ -66,3 +66,13 @@ a a a a a a +0 0 +0 \N +1 2 +1 \N +2 4 +2 \N +\N 0 +\N 2 +\N 4 +\N \N diff --git a/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.sql b/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.sql index 68710137542..f2d4ac8acee 100644 --- a/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.sql +++ b/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.sql @@ -21,3 +21,14 @@ SELECT tuple(number + 1) AS x FROM numbers(10) GROUP BY number + 1, toString(x) SELECT tuple(tuple(number)) AS x FROM numbers(10) WHERE toString(toUUID(tuple(number), NULL), x) GROUP BY number, (toString(x), number) WITH CUBE SETTINGS group_by_use_nulls = 1 FORMAT Null; SELECT materialize('a'), 'a' AS key GROUP BY key WITH CUBE WITH TOTALS SETTINGS group_by_use_nulls = 1; + +EXPLAIN QUERY TREE +SELECT a, b +FROM numbers(3) +GROUP BY number as a, (number + number) as b WITH CUBE +ORDER BY a, b format Null; + +SELECT a, b +FROM numbers(3) +GROUP BY number as a, (number + number) as b WITH CUBE +ORDER BY a, b; From abf5be05b86254fe50d747ce57760aea12219c76 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 24 Apr 2024 18:56:03 +0000 Subject: [PATCH 135/624] Set table for parallel replicas with analyzer --- src/Planner/PlannerJoinTree.cpp | 2 +- src/Planner/findParallelReplicasQuery.cpp | 6 +++++- .../test_parallel_replicas_failover_timeout/test.py | 2 -- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index ba4f9718107..cedf5c471de 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -492,7 +492,7 @@ FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage, throw DB::Exception( ErrorCodes::BAD_ARGUMENTS, "Parallel replicas processing with custom_key has been requested " - "(setting 'max_parallel_replcias'), but the table does not have custom_key defined for it " + "(setting 'max_parallel_replicas'), but the table does not have custom_key defined for it " " or it's invalid (setting 'parallel_replicas_custom_key')"); LOG_TRACE(getLogger("Planner"), "Processing query on a replica using custom_key '{}'", settings.parallel_replicas_custom_key.value); diff --git a/src/Planner/findParallelReplicasQuery.cpp b/src/Planner/findParallelReplicasQuery.cpp index fbe81185239..664dc403000 100644 --- a/src/Planner/findParallelReplicasQuery.cpp +++ b/src/Planner/findParallelReplicasQuery.cpp @@ -412,10 +412,14 @@ JoinTreeQueryPlan buildQueryPlanForParallelReplicas( Block header = InterpreterSelectQueryAnalyzer::getSampleBlock( modified_query_tree, context, SelectQueryOptions(processed_stage).analyze()); + const TableNode * table_node = findTableForParallelReplicas(modified_query_tree.get()); + if (!table_node) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't determine table for parallel replicas"); + QueryPlan query_plan; ClusterProxy::executeQueryWithParallelReplicas( query_plan, - StorageID::createEmpty(), + table_node->getStorageID(), header, processed_stage, modified_query_ast, diff --git a/tests/integration/test_parallel_replicas_failover_timeout/test.py b/tests/integration/test_parallel_replicas_failover_timeout/test.py index 6031f8cbe4a..cf54577b490 100644 --- a/tests/integration/test_parallel_replicas_failover_timeout/test.py +++ b/tests/integration/test_parallel_replicas_failover_timeout/test.py @@ -70,11 +70,9 @@ def test_skip_unresponsive_replicas(start_cluster): 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 33b9e589c6c6a8590bc69fc8614fe66934b6901b Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 24 Apr 2024 21:13:07 +0000 Subject: [PATCH 136/624] Add test_skip_replicas_without_table --- src/Client/Connection.cpp | 9 ++-- .../__init__.py | 0 .../configs/remote_servers.xml | 0 .../test.py | 44 ++++++++++++++++--- 4 files changed, 43 insertions(+), 10 deletions(-) rename tests/integration/{test_parallel_replicas_failover_timeout => test_parallel_replicas_failover}/__init__.py (100%) rename tests/integration/{test_parallel_replicas_failover_timeout => test_parallel_replicas_failover}/configs/remote_servers.xml (100%) rename tests/integration/{test_parallel_replicas_failover_timeout => test_parallel_replicas_failover}/test.py (52%) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index e167bd83a5b..30f16d26443 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -613,16 +613,17 @@ TablesStatusResponse Connection::getTablesStatus(const ConnectionTimeouts & time if (!connected) connect(timeouts); + 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); + }); + TimeoutSetter timeout_setter(*socket, timeouts.sync_request_timeout, true); writeVarUInt(Protocol::Client::TablesStatusRequest, *out); request.write(*out, server_revision); out->next(); - fiu_do_on(FailPoints::receive_timeout_on_table_status_response, { - 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/tests/integration/test_parallel_replicas_failover_timeout/__init__.py b/tests/integration/test_parallel_replicas_failover/__init__.py similarity index 100% rename from tests/integration/test_parallel_replicas_failover_timeout/__init__.py rename to tests/integration/test_parallel_replicas_failover/__init__.py diff --git a/tests/integration/test_parallel_replicas_failover_timeout/configs/remote_servers.xml b/tests/integration/test_parallel_replicas_failover/configs/remote_servers.xml similarity index 100% rename from tests/integration/test_parallel_replicas_failover_timeout/configs/remote_servers.xml rename to tests/integration/test_parallel_replicas_failover/configs/remote_servers.xml diff --git a/tests/integration/test_parallel_replicas_failover_timeout/test.py b/tests/integration/test_parallel_replicas_failover/test.py similarity index 52% rename from tests/integration/test_parallel_replicas_failover_timeout/test.py rename to tests/integration/test_parallel_replicas_failover/test.py index cf54577b490..4bb817e8971 100644 --- a/tests/integration/test_parallel_replicas_failover_timeout/test.py +++ b/tests/integration/test_parallel_replicas_failover/test.py @@ -24,8 +24,9 @@ def start_cluster(): cluster.shutdown() -def create_tables(cluster, table_name): +def create_tables(cluster, table_name, skip_last_replica): node1.query(f"DROP TABLE IF EXISTS {table_name} SYNC") + node2.query(f"DROP TABLE IF EXISTS {table_name} SYNC") node3.query(f"DROP TABLE IF EXISTS {table_name} SYNC") node1.query( @@ -34,9 +35,10 @@ def create_tables(cluster, table_name): 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)" - ) + if not skip_last_replica: + 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( @@ -52,13 +54,43 @@ def create_tables(cluster, table_name): 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}") + if not skip_last_replica: + node3.query(f"SYSTEM SYNC REPLICA {table_name}") + + +def test_skip_replicas_without_table(start_cluster): + cluster_name = "test_1_shard_3_replicas" + table_name = "tt" + create_tables(cluster_name, table_name, skip_last_replica=True) + + expected_result = "" + for i in range(4): + expected_result += f"{i}\t1000\n" + + log_comment = "5230b069-9574-407d-9b80-891b5a175f41" + assert ( + node1.query( + f"SELECT key, count() FROM {table_name} GROUP BY key ORDER BY key", + settings={ + "allow_experimental_parallel_reading_from_replicas": 2, + "max_parallel_replicas": 3, + "cluster_for_parallel_replicas": cluster_name, + "log_comment": log_comment, + }, + ) + == expected_result + ) + + node1.query("SYSTEM FLUSH LOGS") + assert ( node1.query(f"SELECT ProfileEvents['DistributedConnectionMissingTable'] FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '{log_comment}' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0") == "1\n" ) + + assert ( node1.query(f"SELECT ProfileEvents['ParallelReplicasUnavailableCount'] FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '{log_comment}' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0") == "1\n" ) def test_skip_unresponsive_replicas(start_cluster): cluster_name = "test_1_shard_3_replicas" table_name = "tt" - create_tables(cluster_name, table_name) + create_tables(cluster_name, table_name, skip_last_replica=False) expected_result = "" for i in range(4): From cf7dd8b86a6a945bd186b123d0899f7ccf41696b Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 24 Apr 2024 21:23:57 +0000 Subject: [PATCH 137/624] Automatic style fix --- .../test_parallel_replicas_failover/test.py | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_parallel_replicas_failover/test.py b/tests/integration/test_parallel_replicas_failover/test.py index 4bb817e8971..fa9e1488b20 100644 --- a/tests/integration/test_parallel_replicas_failover/test.py +++ b/tests/integration/test_parallel_replicas_failover/test.py @@ -82,9 +82,19 @@ def test_skip_replicas_without_table(start_cluster): ) node1.query("SYSTEM FLUSH LOGS") - assert ( node1.query(f"SELECT ProfileEvents['DistributedConnectionMissingTable'] FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '{log_comment}' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0") == "1\n" ) + assert ( + node1.query( + f"SELECT ProfileEvents['DistributedConnectionMissingTable'] FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '{log_comment}' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0" + ) + == "1\n" + ) - assert ( node1.query(f"SELECT ProfileEvents['ParallelReplicasUnavailableCount'] FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '{log_comment}' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0") == "1\n" ) + assert ( + node1.query( + f"SELECT ProfileEvents['ParallelReplicasUnavailableCount'] FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '{log_comment}' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0" + ) + == "1\n" + ) def test_skip_unresponsive_replicas(start_cluster): From c9a31599c08f7281acff09b86aa68d7da345efdc Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Wed, 24 Apr 2024 22:14:47 +0000 Subject: [PATCH 138/624] fix single-threading failsafe when number of files cannot be estimated --- src/Storages/StorageS3.cpp | 25 +++++++++++++++++++++++-- src/Storages/StorageS3.h | 1 + 2 files changed, 24 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index acef213c1f4..daab457e46b 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -229,6 +229,14 @@ public: return buffer.size(); } + bool hasMore() + { + if (!buffer.size()) + return !(expanded_keys_iter == expanded_keys.end() && is_finished_for_key); + else + return true; + } + ~Impl() { list_objects_pool.wait(); @@ -481,6 +489,11 @@ size_t StorageS3Source::DisclosedGlobIterator::estimatedKeysCount() return pimpl->objectsCount(); } +bool StorageS3Source::DisclosedGlobIterator::hasMore() +{ + return pimpl->hasMore(); +} + class StorageS3Source::KeysIterator::Impl { public: @@ -1243,8 +1256,16 @@ void ReadFromStorageS3Step::initializePipeline(QueryPipelineBuilder & pipeline, if (estimated_keys_count > 1) num_streams = std::min(num_streams, estimated_keys_count); else - /// 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 auto glob_iter = std::dynamic_pointer_cast(iterator_wrapper); + if (!(glob_iter && glob_iter->hasMore())) + { + /// Disclosed glob iterator can underestimate the amount of keys in some cases. We will keep one stream for this particular case. + num_streams = 1; + } + /// Otherwise, 1000 files were already listed, but none of them is actually what we are looking for. + /// We cannot estimate _how many_ there are left, but if there are more files to list, it's faster to do it in many streams. + } 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)); diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 19cbfaa6f08..8d21f1d8e8e 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -83,6 +83,7 @@ public: KeyWithInfoPtr next(size_t idx = 0) override; /// NOLINT size_t estimatedKeysCount() override; + bool hasMore(); private: class Impl; From ae17941e63e1f66520ef13616ff0370e83996a4e Mon Sep 17 00:00:00 2001 From: HowePa <2873679104@qq.com> Date: Thu, 25 Apr 2024 15:51:06 +0800 Subject: [PATCH 139/624] add docs --- 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 f4b082c57ab..80ca0109f0b 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -75,7 +75,7 @@ The supported formats are: | [ArrowStream](#data-format-arrow-stream) | ✔ | ✔ | | [ORC](#data-format-orc) | ✔ | ✔ | | [One](#data-format-one) | ✔ | ✗ | -| [Npy](#data-format-npy) | ✔ | ✗ | +| [Npy](#data-format-npy) | ✔ | ✔ | | [RowBinary](#rowbinary) | ✔ | ✔ | | [RowBinaryWithNames](#rowbinarywithnamesandtypes) | ✔ | ✔ | | [RowBinaryWithNamesAndTypes](#rowbinarywithnamesandtypes) | ✔ | ✔ | From 5e8bc4402ab4df42d228c0474ee01fbb83c97a71 Mon Sep 17 00:00:00 2001 From: HowePa <2873679104@qq.com> Date: Thu, 25 Apr 2024 15:52:30 +0800 Subject: [PATCH 140/624] unified NumpyDataTypes --- src/Formats/NumpyDataTypes.h | 50 ++++++++-- .../Formats/Impl/NpyOutputFormat.cpp | 91 ++++++++++++------- src/Processors/Formats/Impl/NpyOutputFormat.h | 15 +-- 3 files changed, 99 insertions(+), 57 deletions(-) diff --git a/src/Formats/NumpyDataTypes.h b/src/Formats/NumpyDataTypes.h index 712797515c9..5cf2ebf5b40 100644 --- a/src/Formats/NumpyDataTypes.h +++ b/src/Formats/NumpyDataTypes.h @@ -1,10 +1,12 @@ #pragma once #include #include +#include namespace ErrorCodes { extern const int BAD_ARGUMENTS; + extern const int NOT_IMPLEMENTED; } enum class NumpyDataTypeIndex @@ -29,9 +31,9 @@ class NumpyDataType public: enum Endianness { - LITTLE, - BIG, - NONE, + LITTLE = '<', + BIG = '>', + NONE = '|', }; NumpyDataTypeIndex type_index; @@ -41,15 +43,18 @@ public: Endianness getEndianness() const { return endianness; } virtual NumpyDataTypeIndex getTypeIndex() const = 0; + virtual size_t getSize() const { throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Function getSize() is not implemented"); } + virtual void setSize(size_t) { throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Function setSize() is not implemented"); } + virtual String str() const { throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Function str() is not implemented"); } -private: +protected: Endianness endianness; }; class NumpyDataTypeInt : public NumpyDataType { public: - NumpyDataTypeInt(Endianness endianness, size_t size_, bool is_signed_) : NumpyDataType(endianness), size(size_), is_signed(is_signed_) + NumpyDataTypeInt(Endianness endianness_, size_t size_, bool is_signed_) : NumpyDataType(endianness_), size(size_), is_signed(is_signed_) { switch (size) { @@ -67,6 +72,14 @@ public: return type_index; } bool isSigned() const { return is_signed; } + String str() const override + { + DB::WriteBufferFromOwnString buf; + writeChar(static_cast(endianness), buf); + writeChar(is_signed ? 'i' : 'u', buf); + writeIntText(size, buf); + return buf.str(); + } private: size_t size; @@ -76,7 +89,7 @@ private: class NumpyDataTypeFloat : public NumpyDataType { public: - NumpyDataTypeFloat(Endianness endianness, size_t size_) : NumpyDataType(endianness), size(size_) + NumpyDataTypeFloat(Endianness endianness_, size_t size_) : NumpyDataType(endianness_), size(size_) { switch (size) { @@ -92,6 +105,14 @@ public: { return type_index; } + String str() const override + { + DB::WriteBufferFromOwnString buf; + writeChar(static_cast(endianness), buf); + writeChar('f', buf); + writeIntText(size, buf); + return buf.str(); + } private: size_t size; }; @@ -99,13 +120,22 @@ private: class NumpyDataTypeString : public NumpyDataType { public: - NumpyDataTypeString(Endianness endianness, size_t size_) : NumpyDataType(endianness), size(size_) + NumpyDataTypeString(Endianness endianness_, size_t size_) : NumpyDataType(endianness_), size(size_) { type_index = NumpyDataTypeIndex::String; } NumpyDataTypeIndex getTypeIndex() const override { return type_index; } - size_t getSize() const { return size; } + size_t getSize() const override { return size; } + void setSize(size_t size_) override { size = size_; } + String str() const override + { + DB::WriteBufferFromOwnString buf; + writeChar(static_cast(endianness), buf); + writeChar('S', buf); + writeIntText(size, buf); + return buf.str(); + } private: size_t size; }; @@ -113,13 +143,13 @@ private: class NumpyDataTypeUnicode : public NumpyDataType { public: - NumpyDataTypeUnicode(Endianness endianness, size_t size_) : NumpyDataType(endianness), size(size_) + NumpyDataTypeUnicode(Endianness endianness_, size_t size_) : NumpyDataType(endianness_), size(size_) { type_index = NumpyDataTypeIndex::Unicode; } NumpyDataTypeIndex getTypeIndex() const override { return type_index; } - size_t getSize() const { return size * 4; } + size_t getSize() const override { return size * 4; } private: size_t size; }; diff --git a/src/Processors/Formats/Impl/NpyOutputFormat.cpp b/src/Processors/Formats/Impl/NpyOutputFormat.cpp index d54fc7e68f2..64272307e9d 100644 --- a/src/Processors/Formats/Impl/NpyOutputFormat.cpp +++ b/src/Processors/Formats/Impl/NpyOutputFormat.cpp @@ -45,16 +45,6 @@ void writeNumpyStrings(const ColumnPtr & column, size_t length, WriteBuffer & bu } -String NpyOutputFormat::NumpyDataType::str() const -{ - WriteBufferFromOwnString dtype; - writeChar(endianness, dtype); - writeChar(type, dtype); - writeIntText(size, dtype); - - return dtype.str(); -} - String NpyOutputFormat::shapeStr() const { WriteBufferFromOwnString shape; @@ -85,20 +75,48 @@ bool NpyOutputFormat::getNumpyDataType(const DataTypePtr & type) { switch (type->getTypeId()) { - case TypeIndex::Int8: numpy_data_type = NumpyDataType('<', 'i', sizeof(Int8)); break; - case TypeIndex::Int16: numpy_data_type = NumpyDataType('<', 'i', sizeof(Int16)); break; - case TypeIndex::Int32: numpy_data_type = NumpyDataType('<', 'i', sizeof(Int32)); break; - case TypeIndex::Int64: numpy_data_type = NumpyDataType('<', 'i', sizeof(Int64)); break; - case TypeIndex::UInt8: numpy_data_type = NumpyDataType('<', 'u', sizeof(UInt8)); break; - case TypeIndex::UInt16: numpy_data_type = NumpyDataType('<', 'u', sizeof(UInt16)); break; - case TypeIndex::UInt32: numpy_data_type = NumpyDataType('<', 'u', sizeof(UInt32)); break; - case TypeIndex::UInt64: numpy_data_type = NumpyDataType('<', 'u', sizeof(UInt64)); break; - case TypeIndex::Float32: numpy_data_type = NumpyDataType('<', 'f', sizeof(Float32)); break; - case TypeIndex::Float64: numpy_data_type = NumpyDataType('<', 'f', sizeof(Float64)); break; - case TypeIndex::FixedString: numpy_data_type = NumpyDataType('|', 'S', assert_cast(type.get())->getN()); break; - case TypeIndex::String: numpy_data_type = NumpyDataType('|', 'S', 0); break; - case TypeIndex::Array: return getNumpyDataType(assert_cast(type.get())->getNestedType()); - default: nested_data_type = type; return false; + case TypeIndex::Int8: + numpy_data_type = std::make_shared(NumpyDataType::Endianness::LITTLE, sizeof(Int8), true); + break; + case TypeIndex::Int16: + numpy_data_type = std::make_shared(NumpyDataType::Endianness::LITTLE, sizeof(Int16), true); + break; + case TypeIndex::Int32: + numpy_data_type = std::make_shared(NumpyDataType::Endianness::LITTLE, sizeof(Int32), true); + break; + case TypeIndex::Int64: + numpy_data_type = std::make_shared(NumpyDataType::Endianness::LITTLE, sizeof(Int64), true); + break; + case TypeIndex::UInt8: + numpy_data_type = std::make_shared(NumpyDataType::Endianness::LITTLE, sizeof(UInt8), false); + break; + case TypeIndex::UInt16: + numpy_data_type = std::make_shared(NumpyDataType::Endianness::LITTLE, sizeof(UInt16), false); + break; + case TypeIndex::UInt32: + numpy_data_type = std::make_shared(NumpyDataType::Endianness::LITTLE, sizeof(UInt32), false); + break; + case TypeIndex::UInt64: + numpy_data_type = std::make_shared(NumpyDataType::Endianness::LITTLE, sizeof(UInt64), false); + break; + case TypeIndex::Float32: + numpy_data_type = std::make_shared(NumpyDataType::Endianness::LITTLE, sizeof(Float32)); + break; + case TypeIndex::Float64: + numpy_data_type = std::make_shared(NumpyDataType::Endianness::LITTLE, sizeof(Float64)); + break; + case TypeIndex::FixedString: + numpy_data_type = std::make_shared( + NumpyDataType::Endianness::NONE, assert_cast(type.get())->getN()); + break; + case TypeIndex::String: + numpy_data_type = std::make_shared(NumpyDataType::Endianness::NONE, 0); + break; + case TypeIndex::Array: + return getNumpyDataType(assert_cast(type.get())->getNestedType()); + default: + nested_data_type = type; + return false; } nested_data_type = type; @@ -117,6 +135,9 @@ void NpyOutputFormat::consume(Chunk chunk) initShape(column); is_initialized = true; } + // ColumnPtr checkShape, if nullptr? + // updateSizeIfTypeString + // columns.push_back() if (!checkShape(column)) { @@ -130,13 +151,9 @@ void NpyOutputFormat::initShape(const ColumnPtr & column) { auto type = data_type; ColumnPtr nested_column = column; - while (type->getTypeId() == TypeIndex::Array) + while (const auto * array_column = typeid_cast(nested_column.get())) { - const auto * array_column = assert_cast(nested_column.get()); - numpy_shape.push_back(array_column->getOffsets()[0]); - - type = assert_cast(type.get())->getNestedType(); nested_column = array_column->getDataPtr(); } } @@ -166,7 +183,8 @@ bool NpyOutputFormat::checkShape(const ColumnPtr & column) for (size_t i = 0; i < string_offsets.size(); ++i) { size_t string_length = static_cast(string_offsets[i] - 1 - string_offsets[i - 1]); - numpy_data_type.size = numpy_data_type.size > string_length ? numpy_data_type.size : string_length; + if (numpy_data_type->getSize() < string_length) + numpy_data_type->setSize(string_length); } } @@ -185,7 +203,7 @@ void NpyOutputFormat::finalizeImpl() void NpyOutputFormat::writeHeader() { - String dict = "{'descr':'" + numpy_data_type.str() + "','fortran_order':False,'shape':(" + shapeStr() + "),}"; + String dict = "{'descr':'" + numpy_data_type->str() + "','fortran_order':False,'shape':(" + shapeStr() + "),}"; String padding = "\n"; /// completes the length of the header, which is divisible by 64. @@ -221,9 +239,14 @@ void NpyOutputFormat::writeColumns() case TypeIndex::UInt64: writeNumpyNumbers(column, out); break; case TypeIndex::Float32: writeNumpyNumbers(column, out); break; case TypeIndex::Float64: writeNumpyNumbers(column, out); break; - case TypeIndex::FixedString: writeNumpyStrings(column, numpy_data_type.size, out); break; - case TypeIndex::String: writeNumpyStrings(column, numpy_data_type.size, out); break; - default: break; + case TypeIndex::FixedString: + writeNumpyStrings(column, numpy_data_type->getSize(), out); + break; + case TypeIndex::String: + writeNumpyStrings(column, numpy_data_type->getSize(), out); + break; + default: + break; } } } diff --git a/src/Processors/Formats/Impl/NpyOutputFormat.h b/src/Processors/Formats/Impl/NpyOutputFormat.h index 83fad657b2e..6859cf10e69 100644 --- a/src/Processors/Formats/Impl/NpyOutputFormat.h +++ b/src/Processors/Formats/Impl/NpyOutputFormat.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -28,18 +29,6 @@ public: String getContentType() const override { return "application/octet-stream"; } private: - struct NumpyDataType - { - char endianness; - char type; - size_t size; - - NumpyDataType() = default; - NumpyDataType(char endianness_, char type_, size_t size_) - : endianness(endianness_), type(type_), size(size_) {} - String str() const; - }; - String shapeStr() const; bool getNumpyDataType(const DataTypePtr & type); @@ -57,7 +46,7 @@ private: DataTypePtr data_type; DataTypePtr nested_data_type; - NumpyDataType numpy_data_type; + std::shared_ptr numpy_data_type; UInt64 num_rows = 0; std::vector numpy_shape; Columns columns; From 686ea6af9c3512c7b07345cabc785bb975311162 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Thu, 25 Apr 2024 09:06:49 +0000 Subject: [PATCH 141/624] fix style and logic of estimation --- src/Storages/StorageS3.cpp | 20 +++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 6ba41d21766..bdfd2b8b453 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -231,7 +231,7 @@ public: bool hasMore() { - if (!buffer.size()) + if (buffer.empty()) return !(expanded_keys_iter == expanded_keys.end() && is_finished_for_key); else return true; @@ -1286,19 +1286,21 @@ void ReadFromStorageS3Step::initializePipeline(QueryPipelineBuilder & pipeline, createIterator(nullptr); size_t estimated_keys_count = iterator_wrapper->estimatedKeysCount(); - if (estimated_keys_count > 1) - num_streams = std::min(num_streams, estimated_keys_count); - else + const auto glob_iter = std::dynamic_pointer_cast(iterator_wrapper); + + if (!(glob_iter && glob_iter->hasMore())) { - const auto glob_iter = std::dynamic_pointer_cast(iterator_wrapper); - if (!(glob_iter && glob_iter->hasMore())) + if (estimated_keys_count > 1) + num_streams = std::min(num_streams, estimated_keys_count); + else { - /// Disclosed glob iterator can underestimate the amount of keys in some cases. We will keep one stream for this particular case. + /// The amount of keys (zero) was probably underestimated. We will keep one stream for this particular case. num_streams = 1; } - /// Otherwise, 1000 files were already listed, but none of them is actually what we are looking for. - /// We cannot estimate _how many_ there are left, but if there are more files to list, it's faster to do it in many streams. } + /// OTHERWISE, 1000 files were listed, but we cannot make any estimation of _how many_ 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. 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)); From d3d7d3575db1f0d76f4a910c1bc30faf911cf839 Mon Sep 17 00:00:00 2001 From: HowePa <2873679104@qq.com> Date: Thu, 25 Apr 2024 17:16:11 +0800 Subject: [PATCH 142/624] fix empty array & refactor --- .../Formats/Impl/NpyOutputFormat.cpp | 45 +++++++++--------- src/Processors/Formats/Impl/NpyOutputFormat.h | 3 +- .../02895_npy_output_format.reference | Bin 626 -> 1010 bytes .../0_stateless/02895_npy_output_format.sh | 6 ++- 4 files changed, 28 insertions(+), 26 deletions(-) diff --git a/src/Processors/Formats/Impl/NpyOutputFormat.cpp b/src/Processors/Formats/Impl/NpyOutputFormat.cpp index 64272307e9d..f031b776ff7 100644 --- a/src/Processors/Formats/Impl/NpyOutputFormat.cpp +++ b/src/Processors/Formats/Impl/NpyOutputFormat.cpp @@ -135,51 +135,53 @@ void NpyOutputFormat::consume(Chunk chunk) initShape(column); is_initialized = true; } - // ColumnPtr checkShape, if nullptr? - // updateSizeIfTypeString - // columns.push_back() - if (!checkShape(column)) - { - invalid_shape = true; - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "ClickHouse doesn't support object types, cannot format ragged nested sequences (which is a list of arrays with different shapes)"); - } + ColumnPtr nested_column = column; + checkShape(nested_column); + updateSizeIfTypeString(nested_column); + columns.push_back(nested_column); } } void NpyOutputFormat::initShape(const ColumnPtr & column) { - auto type = data_type; ColumnPtr nested_column = column; while (const auto * array_column = typeid_cast(nested_column.get())) { - numpy_shape.push_back(array_column->getOffsets()[0]); + auto dim = array_column->getOffsets()[0]; + invalid_shape = dim == 0; + numpy_shape.push_back(dim); nested_column = array_column->getDataPtr(); } + + if (invalid_shape) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Shape ({}) is an invalid shape, as dimension size cannot be 0", shapeStr()); } -bool NpyOutputFormat::checkShape(const ColumnPtr & column) +void NpyOutputFormat::checkShape(ColumnPtr & column) { - auto type = data_type; - ColumnPtr nested_column = column; int dim = 0; - while (type->getTypeId() == TypeIndex::Array) + while (const auto * array_column = typeid_cast(column.get())) { - const auto * array_column = assert_cast(nested_column.get()); const auto & array_offset = array_column->getOffsets(); for (size_t i = 1; i < array_offset.size(); ++i) if (array_offset[i] - array_offset[i - 1] != numpy_shape[dim]) - return false; + { + invalid_shape = true; + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "ClickHouse doesn't support object types, cannot format ragged nested sequences (which is a list of arrays with different shapes)"); + } - type = assert_cast(type.get())->getNestedType(); - nested_column = array_column->getDataPtr(); + column = array_column->getDataPtr(); dim += 1; } +} - if (type->getTypeId() == TypeIndex::String) +void NpyOutputFormat::updateSizeIfTypeString(const ColumnPtr & column) +{ + if (nested_data_type->getTypeId() == TypeIndex::String) { - const auto & string_offsets = assert_cast(nested_column.get())->getOffsets(); + const auto & string_offsets = assert_cast(column.get())->getOffsets(); for (size_t i = 0; i < string_offsets.size(); ++i) { size_t string_length = static_cast(string_offsets[i] - 1 - string_offsets[i - 1]); @@ -187,9 +189,6 @@ bool NpyOutputFormat::checkShape(const ColumnPtr & column) numpy_data_type->setSize(string_length); } } - - columns.push_back(nested_column); - return true; } void NpyOutputFormat::finalizeImpl() diff --git a/src/Processors/Formats/Impl/NpyOutputFormat.h b/src/Processors/Formats/Impl/NpyOutputFormat.h index 6859cf10e69..5dd6552ac0c 100644 --- a/src/Processors/Formats/Impl/NpyOutputFormat.h +++ b/src/Processors/Formats/Impl/NpyOutputFormat.h @@ -35,7 +35,8 @@ private: void consume(Chunk) override; void initShape(const ColumnPtr & column); - bool checkShape(const ColumnPtr & column); + void checkShape(ColumnPtr & column); + void updateSizeIfTypeString(const ColumnPtr & column); void finalizeImpl() override; void writeHeader(); diff --git a/tests/queries/0_stateless/02895_npy_output_format.reference b/tests/queries/0_stateless/02895_npy_output_format.reference index b599f1dceeaabbd60cb66d02ab02a5d1a30c97f2..590c0581ac112679dea10b0c431dd1b69fae4ab6 100644 GIT binary patch literal 1010 zcmdNj*ax)~q=R6{@_sHT8KP>pfR$xlR61QtS765RbshDz{GyVg#Ju?YqLfsis9R!AajK4baYkZ6s=Aeiv5pZCYUmJMli1i;F49DS$rz~7Ko4jLP$xc~dSX%%Fk+LE ylGTAa)RU8wfxHwTPzP(JL>iJA96?z`1DOORbxlo8 Date: Thu, 25 Apr 2024 17:17:35 +0800 Subject: [PATCH 143/624] unified array travel Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- src/Processors/Formats/Impl/NpyOutputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/NpyOutputFormat.cpp b/src/Processors/Formats/Impl/NpyOutputFormat.cpp index d54fc7e68f2..64bc916da51 100644 --- a/src/Processors/Formats/Impl/NpyOutputFormat.cpp +++ b/src/Processors/Formats/Impl/NpyOutputFormat.cpp @@ -151,7 +151,7 @@ bool NpyOutputFormat::checkShape(const ColumnPtr & column) const auto * array_column = assert_cast(nested_column.get()); const auto & array_offset = array_column->getOffsets(); - for (size_t i = 1; i < array_offset.size(); ++i) + for (size_t i = 0; i < array_offset.size(); ++i) if (array_offset[i] - array_offset[i - 1] != numpy_shape[dim]) return false; From d85f6ae35d64cd8f40a5cfb05aa53d60af48955e Mon Sep 17 00:00:00 2001 From: HowePa <2873679104@qq.com> Date: Thu, 25 Apr 2024 17:31:18 +0800 Subject: [PATCH 144/624] reset test --- .../Formats/Impl/NpyOutputFormat.cpp | 2 +- .../02895_npy_output_format.reference | Bin 1010 -> 626 bytes 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/NpyOutputFormat.cpp b/src/Processors/Formats/Impl/NpyOutputFormat.cpp index f67c91540e4..b3d5042aa79 100644 --- a/src/Processors/Formats/Impl/NpyOutputFormat.cpp +++ b/src/Processors/Formats/Impl/NpyOutputFormat.cpp @@ -155,7 +155,7 @@ void NpyOutputFormat::initShape(const ColumnPtr & column) } if (invalid_shape) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Shape ({}) is an invalid shape, as dimension size cannot be 0", shapeStr()); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Shape ({}) is invalid, as dimension size cannot be 0", shapeStr()); } void NpyOutputFormat::checkShape(ColumnPtr & column) diff --git a/tests/queries/0_stateless/02895_npy_output_format.reference b/tests/queries/0_stateless/02895_npy_output_format.reference index 590c0581ac112679dea10b0c431dd1b69fae4ab6..b599f1dceeaabbd60cb66d02ab02a5d1a30c97f2 100644 GIT binary patch delta 16 Ycmeyw{)uISGvnm*ax)~q=R6{@_sHT8KP>pfR$xlR61QtS765RbshDz{GyVg#Ju?YqLfsis9R!AajK4baYkZ6s=Aeiv5pZCYUmJMli1i;F49DS$rz~7Ko4jLP$xc~dSX%%Fk+LE ylGTAa)RU8wfxHwTPzP(JL>iJA96?z`1DOORbxlo8 Date: Wed, 24 Apr 2024 11:04:56 +0200 Subject: [PATCH 145/624] dx: Enhance error message when non-deterministic function is used with Replicated source --- src/Interpreters/MutationsInterpreter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 35fd549559b..667a3e2e7a6 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -1313,7 +1313,7 @@ void MutationsInterpreter::validate() if (nondeterministic_func_data.nondeterministic_function_name) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "ALTER UPDATE/ALTER DELETE statements must use only deterministic functions. " + "The source storage is replicated so ALTER UPDATE/ALTER DELETE statements must use only deterministic functions. " "Function '{}' is non-deterministic", *nondeterministic_func_data.nondeterministic_function_name); } } From b13c7d004c6f533a1931eb8ac5c529ca82914cd9 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Thu, 25 Apr 2024 14:51:44 +0000 Subject: [PATCH 146/624] fix tidy --- src/Storages/StorageS3.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index bdfd2b8b453..cb5734cfe0c 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -278,7 +278,6 @@ private: filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); fillInternalBufferAssumeLocked(); - return; } KeyWithInfoPtr nextAssumeLocked() From 34dd0a229f04f3b7f8b3181ced3be6430c0f1d2c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 25 Apr 2024 15:46:29 +0000 Subject: [PATCH 147/624] Another one case of aliases with group_by_use_null --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 87 ++++++++++++------- ...up_by_use_nulls_analyzer_crashes.reference | 10 +++ ...23_group_by_use_nulls_analyzer_crashes.sql | 11 +++ 3 files changed, 76 insertions(+), 32 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index aedf860f5be..13ce3d7f0e2 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -474,7 +474,7 @@ struct TableExpressionData class ExpressionsStack { public: - void pushNode(const QueryTreeNodePtr & node) + void push(const QueryTreeNodePtr & node) { if (node->hasAlias()) { @@ -491,7 +491,7 @@ public: expressions.emplace_back(node); } - void popNode() + void pop() { const auto & top_expression = expressions.back(); const auto & top_expression_alias = top_expression->getAlias(); @@ -729,6 +729,8 @@ struct IdentifierResolveScope join_use_nulls = context->getSettingsRef().join_use_nulls; else if (parent_scope) join_use_nulls = parent_scope->join_use_nulls; + + alias_name_to_expression_node = &alias_name_to_expression_node_before_group_by; } QueryTreeNodePtr scope_node; @@ -744,7 +746,10 @@ struct IdentifierResolveScope std::unordered_map expression_argument_name_to_node; /// Alias name to query expression node - std::unordered_map alias_name_to_expression_node; + std::unordered_map alias_name_to_expression_node_before_group_by; + std::unordered_map alias_name_to_expression_node_after_group_by; + + std::unordered_map * alias_name_to_expression_node = nullptr; /// Alias name to lambda node std::unordered_map alias_name_to_lambda_node; @@ -877,6 +882,22 @@ struct IdentifierResolveScope return it->second; } + void pushExpressionNode(const QueryTreeNodePtr & node) + { + bool had_aggregate_function = expressions_in_resolve_process_stack.hasAggregateFunction(); + expressions_in_resolve_process_stack.push(node); + if (group_by_use_nulls && had_aggregate_function != expressions_in_resolve_process_stack.hasAggregateFunction()) + alias_name_to_expression_node = &alias_name_to_expression_node_before_group_by; + } + + void popExpressionNode() + { + bool had_aggregate_function = expressions_in_resolve_process_stack.hasAggregateFunction(); + expressions_in_resolve_process_stack.pop(); + if (group_by_use_nulls && had_aggregate_function != expressions_in_resolve_process_stack.hasAggregateFunction()) + alias_name_to_expression_node = &alias_name_to_expression_node_after_group_by; + } + /// Dump identifier resolve scope [[maybe_unused]] void dump(WriteBuffer & buffer) const { @@ -893,8 +914,8 @@ struct IdentifierResolveScope for (const auto & [alias_name, node] : expression_argument_name_to_node) buffer << "Alias name " << alias_name << " node " << node->formatASTForErrorMessage() << '\n'; - buffer << "Alias name to expression node table size " << alias_name_to_expression_node.size() << '\n'; - for (const auto & [alias_name, node] : alias_name_to_expression_node) + buffer << "Alias name to expression node table size " << alias_name_to_expression_node->size() << '\n'; + for (const auto & [alias_name, node] : *alias_name_to_expression_node) buffer << "Alias name " << alias_name << " expression node " << node->dumpTree() << '\n'; buffer << "Alias name to function node table size " << alias_name_to_lambda_node.size() << '\n'; @@ -1022,7 +1043,7 @@ private: if (is_lambda_node) { - if (scope.alias_name_to_expression_node.contains(alias)) + if (scope.alias_name_to_expression_node->contains(alias)) scope.nodes_with_duplicated_aliases.insert(node); auto [_, inserted] = scope.alias_name_to_lambda_node.insert(std::make_pair(alias, node)); @@ -1035,7 +1056,7 @@ private: if (scope.alias_name_to_lambda_node.contains(alias)) scope.nodes_with_duplicated_aliases.insert(node); - auto [_, inserted] = scope.alias_name_to_expression_node.insert(std::make_pair(alias, node)); + auto [_, inserted] = scope.alias_name_to_expression_node->insert(std::make_pair(alias, node)); if (!inserted) scope.nodes_with_duplicated_aliases.insert(node); @@ -1837,7 +1858,7 @@ void QueryAnalyzer::collectScopeValidIdentifiersForTypoCorrection( if (allow_expression_identifiers) { - for (const auto & [name, expression] : scope.alias_name_to_expression_node) + for (const auto & [name, expression] : *scope.alias_name_to_expression_node) { assert(expression); auto expression_identifier = Identifier(name); @@ -1867,7 +1888,7 @@ void QueryAnalyzer::collectScopeValidIdentifiersForTypoCorrection( { if (allow_function_identifiers) { - for (const auto & [name, _] : scope.alias_name_to_expression_node) + for (const auto & [name, _] : *scope.alias_name_to_expression_node) valid_identifiers_result.insert(Identifier(name)); } @@ -2767,7 +2788,7 @@ bool QueryAnalyzer::tryBindIdentifierToAliases(const IdentifierLookup & identifi auto get_alias_name_to_node_map = [&]() -> const std::unordered_map & { if (identifier_lookup.isExpressionLookup()) - return scope.alias_name_to_expression_node; + return *scope.alias_name_to_expression_node; else if (identifier_lookup.isFunctionLookup()) return scope.alias_name_to_lambda_node; @@ -2829,7 +2850,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier auto get_alias_name_to_node_map = [&]() -> std::unordered_map & { if (identifier_lookup.isExpressionLookup()) - return scope.alias_name_to_expression_node; + return *scope.alias_name_to_expression_node; else if (identifier_lookup.isFunctionLookup()) return scope.alias_name_to_lambda_node; @@ -2867,7 +2888,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier /// Resolve expression if necessary if (node_type == QueryTreeNodeType::IDENTIFIER) { - scope.expressions_in_resolve_process_stack.pushNode(it->second); + scope.pushExpressionNode(it->second); auto & alias_identifier_node = it->second->as(); auto identifier = alias_identifier_node.getIdentifier(); @@ -2898,9 +2919,9 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier if (identifier_lookup.isExpressionLookup()) scope.alias_name_to_lambda_node.erase(identifier_bind_part); else if (identifier_lookup.isFunctionLookup()) - scope.alias_name_to_expression_node.erase(identifier_bind_part); + scope.alias_name_to_expression_node->erase(identifier_bind_part); - scope.expressions_in_resolve_process_stack.popNode(); + scope.popExpressionNode(); } else if (node_type == QueryTreeNodeType::FUNCTION) { @@ -5195,7 +5216,7 @@ ProjectionNames QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_nod auto & lambda_argument_node_typed = lambda_argument_node->as(); const auto & lambda_argument_name = lambda_argument_node_typed.getIdentifier().getFullName(); - bool has_expression_node = scope.alias_name_to_expression_node.contains(lambda_argument_name); + bool has_expression_node = scope.alias_name_to_expression_node->contains(lambda_argument_name); bool has_alias_node = scope.alias_name_to_lambda_node.contains(lambda_argument_name); if (has_expression_node || has_alias_node) @@ -6200,8 +6221,8 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id * * To resolve b we need to resolve a. */ - auto it = scope.alias_name_to_expression_node.find(node_alias); - if (it != scope.alias_name_to_expression_node.end()) + auto it = scope.alias_name_to_expression_node->find(node_alias); + if (it != scope.alias_name_to_expression_node->end()) node = it->second; if (allow_lambda_expression) @@ -6212,7 +6233,7 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id } } - scope.expressions_in_resolve_process_stack.pushNode(node); + scope.pushExpressionNode(node); auto node_type = node->getNodeType(); @@ -6241,7 +6262,7 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id resolved_identifier_node = tryResolveIdentifier({unresolved_identifier, IdentifierLookupContext::FUNCTION}, scope).resolved_identifier; if (resolved_identifier_node && !node_alias.empty()) - scope.alias_name_to_expression_node.erase(node_alias); + scope.alias_name_to_expression_node->erase(node_alias); } if (!resolved_identifier_node && allow_table_expression) @@ -6472,8 +6493,8 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id */ if (!node_alias.empty() && use_alias_table && !scope.group_by_use_nulls) { - auto it = scope.alias_name_to_expression_node.find(node_alias); - if (it != scope.alias_name_to_expression_node.end()) + auto it = scope.alias_name_to_expression_node->find(node_alias); + if (it != scope.alias_name_to_expression_node->end()) it->second = node; if (allow_lambda_expression) @@ -6486,7 +6507,7 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id resolved_expressions.emplace(node, result_projection_names); - scope.expressions_in_resolve_process_stack.popNode(); + scope.popExpressionNode(); bool expression_was_root = scope.expressions_in_resolve_process_stack.empty(); if (expression_was_root) scope.non_cached_identifier_lookups_during_expression_resolve.clear(); @@ -6830,11 +6851,11 @@ void QueryAnalyzer::initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_nod */ resolve_settings.allow_to_resolve_subquery_during_identifier_resolution = false; - scope.expressions_in_resolve_process_stack.pushNode(current_join_tree_node); + scope.pushExpressionNode(current_join_tree_node); auto table_identifier_resolve_result = tryResolveIdentifier(table_identifier_lookup, scope, resolve_settings); - scope.expressions_in_resolve_process_stack.popNode(); + scope.popExpressionNode(); bool expression_was_root = scope.expressions_in_resolve_process_stack.empty(); if (expression_was_root) scope.non_cached_identifier_lookups_during_expression_resolve.clear(); @@ -7418,7 +7439,7 @@ void QueryAnalyzer::resolveArrayJoin(QueryTreeNodePtr & array_join_node, Identif for (auto & array_join_expression : array_join_nodes) { auto array_join_expression_alias = array_join_expression->getAlias(); - if (!array_join_expression_alias.empty() && scope.alias_name_to_expression_node.contains(array_join_expression_alias)) + if (!array_join_expression_alias.empty() && scope.alias_name_to_expression_node->contains(array_join_expression_alias)) throw Exception(ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS, "ARRAY JOIN expression {} with duplicate alias {}. In scope {}", array_join_expression->formatASTForErrorMessage(), @@ -7512,8 +7533,8 @@ void QueryAnalyzer::resolveArrayJoin(QueryTreeNodePtr & array_join_node, Identif array_join_nodes = std::move(array_join_column_expressions); for (auto & array_join_column_expression : array_join_nodes) { - auto it = scope.alias_name_to_expression_node.find(array_join_column_expression->getAlias()); - if (it != scope.alias_name_to_expression_node.end()) + auto it = scope.alias_name_to_expression_node->find(array_join_column_expression->getAlias()); + if (it != scope.alias_name_to_expression_node->end()) { auto & array_join_column_expression_typed = array_join_column_expression->as(); auto array_join_column = std::make_shared(array_join_column_expression_typed.getColumn(), @@ -8037,8 +8058,10 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier /// Clone is needed cause aliases share subtrees. /// If not clone, the same (shared) subtree could be resolved again with different (Nullable) type /// See 03023_group_by_use_nulls_analyzer_crashes - for (auto & [_, node] : scope.alias_name_to_expression_node) - node = node->clone(); + for (auto & [key, node] : scope.alias_name_to_expression_node_before_group_by) + scope.alias_name_to_expression_node_after_group_by[key] = node->clone(); + + scope.alias_name_to_expression_node = &scope.alias_name_to_expression_node_after_group_by; } if (query_node_typed.hasHaving()) @@ -8115,8 +8138,8 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier bool has_node_in_alias_table = false; - auto it = scope.alias_name_to_expression_node.find(node_alias); - if (it != scope.alias_name_to_expression_node.end()) + auto it = scope.alias_name_to_expression_node->find(node_alias); + if (it != scope.alias_name_to_expression_node->end()) { has_node_in_alias_table = true; @@ -8175,7 +8198,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier /// Remove aliases from expression and lambda nodes - for (auto & [_, node] : scope.alias_name_to_expression_node) + for (auto & [_, node] : *scope.alias_name_to_expression_node) node->removeAlias(); for (auto & [_, node] : scope.alias_name_to_lambda_node) diff --git a/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.reference b/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.reference index e2682487229..0eb9d94e85a 100644 --- a/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.reference +++ b/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.reference @@ -76,3 +76,13 @@ a a \N 2 \N 4 \N \N +0 0 nan +2 4 nan +1 2 nan +2 \N nan +0 \N nan +1 \N nan +\N 2 nan +\N 0 nan +\N 4 nan +\N \N nan diff --git a/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.sql b/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.sql index f2d4ac8acee..7311ce54e39 100644 --- a/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.sql +++ b/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.sql @@ -32,3 +32,14 @@ SELECT a, b FROM numbers(3) GROUP BY number as a, (number + number) as b WITH CUBE ORDER BY a, b; + +SELECT + a, + b, + cramersVBiasCorrected(a, b) +FROM numbers(3) +GROUP BY + number AS a, + number + number AS b + WITH CUBE +SETTINGS group_by_use_nulls = 1; From 64dba76afe7d5fd423617b7370e5ccacadcd60e2 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 25 Apr 2024 16:54:49 +0000 Subject: [PATCH 148/624] Analyzer: prefer column name to alias from array join --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 15 +++++++++++++++ ...nalyzer_array_join_prefer_column.reference | 2 ++ ...3130_analyzer_array_join_prefer_column.sql | 19 +++++++++++++++++++ 3 files changed, 36 insertions(+) create mode 100644 tests/queries/0_stateless/03130_analyzer_array_join_prefer_column.reference create mode 100644 tests/queries/0_stateless/03130_analyzer_array_join_prefer_column.sql diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index a3c60c76132..958246df89d 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -4096,6 +4096,21 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLook { bool prefer_column_name_to_alias = scope.context->getSettingsRef().prefer_column_name_to_alias; + if (identifier_lookup.isExpressionLookup()) + { + /* For aliases from ARRAY JOIN we prefer column from join tree: + * SELECT id FROM ( SELECT ... ) AS subquery ARRAY JOIN [0] AS id INNER JOIN second_table USING (id) + * In the example, identifier `id` should be resolved into one from USING (id) column. + */ + auto alias_it = scope.alias_name_to_expression_node.find(identifier_lookup.identifier.front()); + if (alias_it != scope.alias_name_to_expression_node.end() && alias_it->second->getNodeType() == QueryTreeNodeType::COLUMN) + { + const auto & column_node = alias_it->second->as(); + if (column_node.getColumnSource()->getNodeType() == QueryTreeNodeType::ARRAY_JOIN) + prefer_column_name_to_alias = true; + } + } + if (unlikely(prefer_column_name_to_alias)) { if (identifier_resolve_settings.allow_to_check_join_tree) diff --git a/tests/queries/0_stateless/03130_analyzer_array_join_prefer_column.reference b/tests/queries/0_stateless/03130_analyzer_array_join_prefer_column.reference new file mode 100644 index 00000000000..c64254b157c --- /dev/null +++ b/tests/queries/0_stateless/03130_analyzer_array_join_prefer_column.reference @@ -0,0 +1,2 @@ +0 UInt64 +0 UInt64 diff --git a/tests/queries/0_stateless/03130_analyzer_array_join_prefer_column.sql b/tests/queries/0_stateless/03130_analyzer_array_join_prefer_column.sql new file mode 100644 index 00000000000..1f7bc9f9df1 --- /dev/null +++ b/tests/queries/0_stateless/03130_analyzer_array_join_prefer_column.sql @@ -0,0 +1,19 @@ +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table (`id` UInt64, `value` String, `value_array` Array(UInt64)) ENGINE = MergeTree() ORDER BY id; +INSERT INTO test_table VALUES (0, 'aaa', [0]), (1, 'bbb', [1]), (2, 'ccc', [2]); + + +SELECT materialize(id), toTypeName(id) +FROM ( SELECT 'aaa' ) AS subquery +ARRAY JOIN [0] AS id +INNER JOIN test_table +USING (id) +; + +SELECT materialize(id), toTypeName(id) +FROM ( SELECT 'aaa' ) AS subquery +ARRAY JOIN [0] AS id +INNER JOIN test_table +USING (id) +SETTINGS prefer_column_name_to_alias = 1 +; From e09530ab755964b6da12718279ef345bf2800d43 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 8 Dec 2023 16:51:35 +0100 Subject: [PATCH 149/624] Fix making backup when multiple shards are used. --- src/Backups/BackupCoordinationLocal.cpp | 24 ++++---- src/Backups/BackupCoordinationLocal.h | 12 ++-- src/Backups/BackupCoordinationRemote.cpp | 48 ++++++++-------- src/Backups/BackupCoordinationRemote.h | 12 ++-- .../BackupCoordinationReplicatedTables.cpp | 24 ++++---- .../BackupCoordinationReplicatedTables.h | 14 ++--- src/Backups/BackupEntriesCollector.cpp | 10 ++-- src/Backups/BackupEntriesCollector.h | 2 +- src/Backups/BackupUtils.cpp | 2 +- src/Backups/DDLAdjustingForBackupVisitor.cpp | 10 +--- src/Backups/DDLAdjustingForBackupVisitor.h | 5 +- src/Backups/IBackupCoordination.h | 12 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 56 +++++-------------- src/Storages/StorageReplicatedMergeTree.h | 7 +-- 14 files changed, 100 insertions(+), 138 deletions(-) diff --git a/src/Backups/BackupCoordinationLocal.cpp b/src/Backups/BackupCoordinationLocal.cpp index 9964de2ad6e..efdc18cc29c 100644 --- a/src/Backups/BackupCoordinationLocal.cpp +++ b/src/Backups/BackupCoordinationLocal.cpp @@ -33,42 +33,42 @@ Strings BackupCoordinationLocal::waitForStage(const String &, std::chrono::milli return {}; } -void BackupCoordinationLocal::addReplicatedPartNames(const String & table_shared_id, const String & table_name_for_logs, const String & replica_name, const std::vector & part_names_and_checksums) +void BackupCoordinationLocal::addReplicatedPartNames(const String & table_zk_path, const String & table_name_for_logs, const String & replica_name, const std::vector & part_names_and_checksums) { std::lock_guard lock{replicated_tables_mutex}; - replicated_tables.addPartNames({table_shared_id, table_name_for_logs, replica_name, part_names_and_checksums}); + replicated_tables.addPartNames({table_zk_path, table_name_for_logs, replica_name, part_names_and_checksums}); } -Strings BackupCoordinationLocal::getReplicatedPartNames(const String & table_shared_id, const String & replica_name) const +Strings BackupCoordinationLocal::getReplicatedPartNames(const String & table_zk_path, const String & replica_name) const { std::lock_guard lock{replicated_tables_mutex}; - return replicated_tables.getPartNames(table_shared_id, replica_name); + return replicated_tables.getPartNames(table_zk_path, replica_name); } -void BackupCoordinationLocal::addReplicatedMutations(const String & table_shared_id, const String & table_name_for_logs, const String & replica_name, const std::vector & mutations) +void BackupCoordinationLocal::addReplicatedMutations(const String & table_zk_path, const String & table_name_for_logs, const String & replica_name, const std::vector & mutations) { std::lock_guard lock{replicated_tables_mutex}; - replicated_tables.addMutations({table_shared_id, table_name_for_logs, replica_name, mutations}); + replicated_tables.addMutations({table_zk_path, table_name_for_logs, replica_name, mutations}); } -std::vector BackupCoordinationLocal::getReplicatedMutations(const String & table_shared_id, const String & replica_name) const +std::vector BackupCoordinationLocal::getReplicatedMutations(const String & table_zk_path, const String & replica_name) const { std::lock_guard lock{replicated_tables_mutex}; - return replicated_tables.getMutations(table_shared_id, replica_name); + return replicated_tables.getMutations(table_zk_path, replica_name); } -void BackupCoordinationLocal::addReplicatedDataPath(const String & table_shared_id, const String & data_path) +void BackupCoordinationLocal::addReplicatedDataPath(const String & table_zk_path, const String & data_path) { std::lock_guard lock{replicated_tables_mutex}; - replicated_tables.addDataPath({table_shared_id, data_path}); + replicated_tables.addDataPath({table_zk_path, data_path}); } -Strings BackupCoordinationLocal::getReplicatedDataPaths(const String & table_shared_id) const +Strings BackupCoordinationLocal::getReplicatedDataPaths(const String & table_zk_path) const { std::lock_guard lock{replicated_tables_mutex}; - return replicated_tables.getDataPaths(table_shared_id); + return replicated_tables.getDataPaths(table_zk_path); } diff --git a/src/Backups/BackupCoordinationLocal.h b/src/Backups/BackupCoordinationLocal.h index e0aa5dc67a4..a7f15c79649 100644 --- a/src/Backups/BackupCoordinationLocal.h +++ b/src/Backups/BackupCoordinationLocal.h @@ -29,16 +29,16 @@ public: Strings waitForStage(const String & stage_to_wait) override; Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) override; - void addReplicatedPartNames(const String & table_shared_id, const String & table_name_for_logs, const String & replica_name, + void addReplicatedPartNames(const String & table_zk_path, const String & table_name_for_logs, const String & replica_name, const std::vector & part_names_and_checksums) override; - Strings getReplicatedPartNames(const String & table_shared_id, const String & replica_name) const override; + Strings getReplicatedPartNames(const String & table_zk_path, const String & replica_name) const override; - void addReplicatedMutations(const String & table_shared_id, const String & table_name_for_logs, const String & replica_name, + void addReplicatedMutations(const String & table_zk_path, const String & table_name_for_logs, const String & replica_name, const std::vector & mutations) override; - std::vector getReplicatedMutations(const String & table_shared_id, const String & replica_name) const override; + std::vector getReplicatedMutations(const String & table_zk_path, const String & replica_name) const override; - void addReplicatedDataPath(const String & table_shared_id, const String & data_path) override; - Strings getReplicatedDataPaths(const String & table_shared_id) const override; + void addReplicatedDataPath(const String & table_zk_path, const String & data_path) override; + Strings getReplicatedDataPaths(const String & table_zk_path) const override; void addReplicatedAccessFilePath(const String & access_zk_path, AccessEntityType access_entity_type, const String & file_path) override; Strings getReplicatedAccessFilePaths(const String & access_zk_path, AccessEntityType access_entity_type) const override; diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index 455f45a7a77..f353062f628 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -358,7 +358,7 @@ String BackupCoordinationRemote::deserializeFromMultipleZooKeeperNodes(const Str void BackupCoordinationRemote::addReplicatedPartNames( - const String & table_shared_id, + const String & table_zk_path, const String & table_name_for_logs, const String & replica_name, const std::vector & part_names_and_checksums) @@ -374,22 +374,22 @@ void BackupCoordinationRemote::addReplicatedPartNames( [&, &zk = holder.faulty_zookeeper]() { with_retries.renewZooKeeper(zk); - String path = zookeeper_path + "/repl_part_names/" + escapeForFileName(table_shared_id); + String path = zookeeper_path + "/repl_part_names/" + escapeForFileName(table_zk_path); zk->createIfNotExists(path, ""); path += "/" + escapeForFileName(replica_name); zk->createIfNotExists(path, ReplicatedPartNames::serialize(part_names_and_checksums, table_name_for_logs)); }); } -Strings BackupCoordinationRemote::getReplicatedPartNames(const String & table_shared_id, const String & replica_name) const +Strings BackupCoordinationRemote::getReplicatedPartNames(const String & table_zk_path, const String & replica_name) const { std::lock_guard lock{replicated_tables_mutex}; prepareReplicatedTables(); - return replicated_tables->getPartNames(table_shared_id, replica_name); + return replicated_tables->getPartNames(table_zk_path, replica_name); } void BackupCoordinationRemote::addReplicatedMutations( - const String & table_shared_id, + const String & table_zk_path, const String & table_name_for_logs, const String & replica_name, const std::vector & mutations) @@ -405,23 +405,23 @@ void BackupCoordinationRemote::addReplicatedMutations( [&, &zk = holder.faulty_zookeeper]() { with_retries.renewZooKeeper(zk); - String path = zookeeper_path + "/repl_mutations/" + escapeForFileName(table_shared_id); + String path = zookeeper_path + "/repl_mutations/" + escapeForFileName(table_zk_path); zk->createIfNotExists(path, ""); path += "/" + escapeForFileName(replica_name); zk->createIfNotExists(path, ReplicatedMutations::serialize(mutations, table_name_for_logs)); }); } -std::vector BackupCoordinationRemote::getReplicatedMutations(const String & table_shared_id, const String & replica_name) const +std::vector BackupCoordinationRemote::getReplicatedMutations(const String & table_zk_path, const String & replica_name) const { std::lock_guard lock{replicated_tables_mutex}; prepareReplicatedTables(); - return replicated_tables->getMutations(table_shared_id, replica_name); + return replicated_tables->getMutations(table_zk_path, replica_name); } void BackupCoordinationRemote::addReplicatedDataPath( - const String & table_shared_id, const String & data_path) + const String & table_zk_path, const String & data_path) { { std::lock_guard lock{replicated_tables_mutex}; @@ -434,18 +434,18 @@ void BackupCoordinationRemote::addReplicatedDataPath( [&, &zk = holder.faulty_zookeeper]() { with_retries.renewZooKeeper(zk); - String path = zookeeper_path + "/repl_data_paths/" + escapeForFileName(table_shared_id); + String path = zookeeper_path + "/repl_data_paths/" + escapeForFileName(table_zk_path); zk->createIfNotExists(path, ""); path += "/" + escapeForFileName(data_path); zk->createIfNotExists(path, ""); }); } -Strings BackupCoordinationRemote::getReplicatedDataPaths(const String & table_shared_id) const +Strings BackupCoordinationRemote::getReplicatedDataPaths(const String & table_zk_path) const { std::lock_guard lock{replicated_tables_mutex}; prepareReplicatedTables(); - return replicated_tables->getDataPaths(table_shared_id); + return replicated_tables->getDataPaths(table_zk_path); } @@ -464,16 +464,16 @@ void BackupCoordinationRemote::prepareReplicatedTables() const with_retries.renewZooKeeper(zk); String path = zookeeper_path + "/repl_part_names"; - for (const String & escaped_table_shared_id : zk->getChildren(path)) + for (const String & escaped_table_zk_path : zk->getChildren(path)) { - String table_shared_id = unescapeForFileName(escaped_table_shared_id); - String path2 = path + "/" + escaped_table_shared_id; + String table_zk_path = unescapeForFileName(escaped_table_zk_path); + String path2 = path + "/" + escaped_table_zk_path; for (const String & escaped_replica_name : zk->getChildren(path2)) { String replica_name = unescapeForFileName(escaped_replica_name); auto part_names = ReplicatedPartNames::deserialize(zk->get(path2 + "/" + escaped_replica_name)); part_names_for_replicated_tables.push_back( - {table_shared_id, part_names.table_name_for_logs, replica_name, part_names.part_names_and_checksums}); + {table_zk_path, part_names.table_name_for_logs, replica_name, part_names.part_names_and_checksums}); } } }); @@ -489,16 +489,16 @@ void BackupCoordinationRemote::prepareReplicatedTables() const with_retries.renewZooKeeper(zk); String path = zookeeper_path + "/repl_mutations"; - for (const String & escaped_table_shared_id : zk->getChildren(path)) + for (const String & escaped_table_zk_path : zk->getChildren(path)) { - String table_shared_id = unescapeForFileName(escaped_table_shared_id); - String path2 = path + "/" + escaped_table_shared_id; + String table_zk_path = unescapeForFileName(escaped_table_zk_path); + String path2 = path + "/" + escaped_table_zk_path; for (const String & escaped_replica_name : zk->getChildren(path2)) { String replica_name = unescapeForFileName(escaped_replica_name); auto mutations = ReplicatedMutations::deserialize(zk->get(path2 + "/" + escaped_replica_name)); mutations_for_replicated_tables.push_back( - {table_shared_id, mutations.table_name_for_logs, replica_name, mutations.mutations}); + {table_zk_path, mutations.table_name_for_logs, replica_name, mutations.mutations}); } } }); @@ -514,14 +514,14 @@ void BackupCoordinationRemote::prepareReplicatedTables() const with_retries.renewZooKeeper(zk); String path = zookeeper_path + "/repl_data_paths"; - for (const String & escaped_table_shared_id : zk->getChildren(path)) + for (const String & escaped_table_zk_path : zk->getChildren(path)) { - String table_shared_id = unescapeForFileName(escaped_table_shared_id); - String path2 = path + "/" + escaped_table_shared_id; + String table_zk_path = unescapeForFileName(escaped_table_zk_path); + String path2 = path + "/" + escaped_table_zk_path; for (const String & escaped_data_path : zk->getChildren(path2)) { String data_path = unescapeForFileName(escaped_data_path); - data_paths_for_replicated_tables.push_back({table_shared_id, data_path}); + data_paths_for_replicated_tables.push_back({table_zk_path, data_path}); } } }); diff --git a/src/Backups/BackupCoordinationRemote.h b/src/Backups/BackupCoordinationRemote.h index ce891699bd2..7a56b1a4eb8 100644 --- a/src/Backups/BackupCoordinationRemote.h +++ b/src/Backups/BackupCoordinationRemote.h @@ -41,23 +41,23 @@ public: Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) override; void addReplicatedPartNames( - const String & table_shared_id, + const String & table_zk_path, const String & table_name_for_logs, const String & replica_name, const std::vector & part_names_and_checksums) override; - Strings getReplicatedPartNames(const String & table_shared_id, const String & replica_name) const override; + Strings getReplicatedPartNames(const String & table_zk_path, const String & replica_name) const override; void addReplicatedMutations( - const String & table_shared_id, + const String & table_zk_path, const String & table_name_for_logs, const String & replica_name, const std::vector & mutations) override; - std::vector getReplicatedMutations(const String & table_shared_id, const String & replica_name) const override; + std::vector getReplicatedMutations(const String & table_zk_path, const String & replica_name) const override; - void addReplicatedDataPath(const String & table_shared_id, const String & data_path) override; - Strings getReplicatedDataPaths(const String & table_shared_id) const override; + void addReplicatedDataPath(const String & table_zk_path, const String & data_path) override; + Strings getReplicatedDataPaths(const String & table_zk_path) const override; void addReplicatedAccessFilePath(const String & access_zk_path, AccessEntityType access_entity_type, const String & file_path) override; Strings getReplicatedAccessFilePaths(const String & access_zk_path, AccessEntityType access_entity_type) const override; diff --git a/src/Backups/BackupCoordinationReplicatedTables.cpp b/src/Backups/BackupCoordinationReplicatedTables.cpp index 1cbb88acb82..a435667f79a 100644 --- a/src/Backups/BackupCoordinationReplicatedTables.cpp +++ b/src/Backups/BackupCoordinationReplicatedTables.cpp @@ -151,7 +151,7 @@ BackupCoordinationReplicatedTables::~BackupCoordinationReplicatedTables() = defa void BackupCoordinationReplicatedTables::addPartNames(PartNamesForTableReplica && part_names) { - const auto & table_shared_id = part_names.table_shared_id; + const auto & table_zk_path = part_names.table_zk_path; const auto & table_name_for_logs = part_names.table_name_for_logs; const auto & replica_name = part_names.replica_name; const auto & part_names_and_checksums = part_names.part_names_and_checksums; @@ -159,7 +159,7 @@ void BackupCoordinationReplicatedTables::addPartNames(PartNamesForTableReplica & if (prepared) throw Exception(ErrorCodes::LOGICAL_ERROR, "addPartNames() must not be called after preparing"); - auto & table_info = table_infos[table_shared_id]; + auto & table_info = table_infos[table_zk_path]; table_info.table_name_for_logs = table_name_for_logs; if (!table_info.covered_parts_finder) @@ -200,11 +200,11 @@ void BackupCoordinationReplicatedTables::addPartNames(PartNamesForTableReplica & } } -Strings BackupCoordinationReplicatedTables::getPartNames(const String & table_shared_id, const String & replica_name) const +Strings BackupCoordinationReplicatedTables::getPartNames(const String & table_zk_path, const String & replica_name) const { prepare(); - auto it = table_infos.find(table_shared_id); + auto it = table_infos.find(table_zk_path); if (it == table_infos.end()) return {}; @@ -218,7 +218,7 @@ Strings BackupCoordinationReplicatedTables::getPartNames(const String & table_sh void BackupCoordinationReplicatedTables::addMutations(MutationsForTableReplica && mutations_for_table_replica) { - const auto & table_shared_id = mutations_for_table_replica.table_shared_id; + const auto & table_zk_path = mutations_for_table_replica.table_zk_path; const auto & table_name_for_logs = mutations_for_table_replica.table_name_for_logs; const auto & replica_name = mutations_for_table_replica.replica_name; const auto & mutations = mutations_for_table_replica.mutations; @@ -226,7 +226,7 @@ void BackupCoordinationReplicatedTables::addMutations(MutationsForTableReplica & if (prepared) throw Exception(ErrorCodes::LOGICAL_ERROR, "addMutations() must not be called after preparing"); - auto & table_info = table_infos[table_shared_id]; + auto & table_info = table_infos[table_zk_path]; table_info.table_name_for_logs = table_name_for_logs; for (const auto & [mutation_id, mutation_entry] : mutations) table_info.mutations.emplace(mutation_id, mutation_entry); @@ -236,11 +236,11 @@ void BackupCoordinationReplicatedTables::addMutations(MutationsForTableReplica & } std::vector -BackupCoordinationReplicatedTables::getMutations(const String & table_shared_id, const String & replica_name) const +BackupCoordinationReplicatedTables::getMutations(const String & table_zk_path, const String & replica_name) const { prepare(); - auto it = table_infos.find(table_shared_id); + auto it = table_infos.find(table_zk_path); if (it == table_infos.end()) return {}; @@ -257,16 +257,16 @@ BackupCoordinationReplicatedTables::getMutations(const String & table_shared_id, void BackupCoordinationReplicatedTables::addDataPath(DataPathForTableReplica && data_path_for_table_replica) { - const auto & table_shared_id = data_path_for_table_replica.table_shared_id; + const auto & table_zk_path = data_path_for_table_replica.table_zk_path; const auto & data_path = data_path_for_table_replica.data_path; - auto & table_info = table_infos[table_shared_id]; + auto & table_info = table_infos[table_zk_path]; table_info.data_paths.emplace(data_path); } -Strings BackupCoordinationReplicatedTables::getDataPaths(const String & table_shared_id) const +Strings BackupCoordinationReplicatedTables::getDataPaths(const String & table_zk_path) const { - auto it = table_infos.find(table_shared_id); + auto it = table_infos.find(table_zk_path); if (it == table_infos.end()) return {}; diff --git a/src/Backups/BackupCoordinationReplicatedTables.h b/src/Backups/BackupCoordinationReplicatedTables.h index 74f21eb9c7c..50ab56aef75 100644 --- a/src/Backups/BackupCoordinationReplicatedTables.h +++ b/src/Backups/BackupCoordinationReplicatedTables.h @@ -40,7 +40,7 @@ public: struct PartNamesForTableReplica { - String table_shared_id; + String table_zk_path; String table_name_for_logs; String replica_name; std::vector part_names_and_checksums; @@ -55,13 +55,13 @@ public: /// Returns the names of the parts which a specified replica of a replicated table should put to the backup. /// This is the same list as it was added by call of the function addPartNames() but without duplications and without /// parts covered by another parts. - Strings getPartNames(const String & table_shared_id, const String & replica_name) const; + Strings getPartNames(const String & table_zk_path, const String & replica_name) const; using MutationInfo = IBackupCoordination::MutationInfo; struct MutationsForTableReplica { - String table_shared_id; + String table_zk_path; String table_name_for_logs; String replica_name; std::vector mutations; @@ -71,11 +71,11 @@ public: void addMutations(MutationsForTableReplica && mutations_for_table_replica); /// Returns all mutations of a replicated table which are not finished for some data parts added by addReplicatedPartNames(). - std::vector getMutations(const String & table_shared_id, const String & replica_name) const; + std::vector getMutations(const String & table_zk_path, const String & replica_name) const; struct DataPathForTableReplica { - String table_shared_id; + String table_zk_path; String data_path; }; @@ -85,7 +85,7 @@ public: void addDataPath(DataPathForTableReplica && data_path_for_table_replica); /// Returns all the data paths in backup added for a replicated table (see also addReplicatedDataPath()). - Strings getDataPaths(const String & table_shared_id) const; + Strings getDataPaths(const String & table_zk_path) const; private: void prepare() const; @@ -110,7 +110,7 @@ private: std::unordered_set data_paths; }; - std::map table_infos; /// Should be ordered because we need this map to be in the same order on every replica. + std::map table_infos; /// Should be ordered because we need this map to be in the same order on every replica. mutable bool prepared = false; }; diff --git a/src/Backups/BackupEntriesCollector.cpp b/src/Backups/BackupEntriesCollector.cpp index cc014c279cc..136e3c49321 100644 --- a/src/Backups/BackupEntriesCollector.cpp +++ b/src/Backups/BackupEntriesCollector.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -758,7 +759,7 @@ void BackupEntriesCollector::makeBackupEntriesForDatabasesDefs() checkIsQueryCancelled(); ASTPtr new_create_query = database_info.create_database_query; - adjustCreateQueryForBackup(new_create_query, context->getGlobalContext(), nullptr); + adjustCreateQueryForBackup(new_create_query, context->getGlobalContext()); renameDatabaseAndTableNameInCreateQuery(new_create_query, renaming_map, context->getGlobalContext()); const String & metadata_path_in_backup = database_info.metadata_path_in_backup; @@ -775,7 +776,8 @@ void BackupEntriesCollector::makeBackupEntriesForTablesDefs() checkIsQueryCancelled(); ASTPtr new_create_query = table_info.create_table_query; - adjustCreateQueryForBackup(new_create_query, context->getGlobalContext(), &table_info.replicated_table_shared_id); + table_info.replicated_table_zk_path = tryExtractZkPathFromCreateQuery(*new_create_query, context->getGlobalContext()); + adjustCreateQueryForBackup(new_create_query, context->getGlobalContext()); renameDatabaseAndTableNameInCreateQuery(new_create_query, renaming_map, context->getGlobalContext()); const String & metadata_path_in_backup = table_info.metadata_path_in_backup; @@ -814,8 +816,8 @@ void BackupEntriesCollector::makeBackupEntriesForTableData(const QualifiedTableN /// If this table is replicated in this case we call IBackupCoordination::addReplicatedDataPath() which will cause /// other replicas to fill the storage's data in the backup. /// If this table is not replicated we'll do nothing leaving the storage's data empty in the backup. - if (table_info.replicated_table_shared_id) - backup_coordination->addReplicatedDataPath(*table_info.replicated_table_shared_id, data_path_in_backup); + if (table_info.replicated_table_zk_path) + backup_coordination->addReplicatedDataPath(*table_info.replicated_table_zk_path, data_path_in_backup); return; } diff --git a/src/Backups/BackupEntriesCollector.h b/src/Backups/BackupEntriesCollector.h index 01e8d594334..c7bce077a2d 100644 --- a/src/Backups/BackupEntriesCollector.h +++ b/src/Backups/BackupEntriesCollector.h @@ -164,7 +164,7 @@ private: ASTPtr create_table_query; String metadata_path_in_backup; std::filesystem::path data_path_in_backup; - std::optional replicated_table_shared_id; + std::optional replicated_table_zk_path; std::optional partitions; }; diff --git a/src/Backups/BackupUtils.cpp b/src/Backups/BackupUtils.cpp index fb448fb64ad..fa8ed5855dd 100644 --- a/src/Backups/BackupUtils.cpp +++ b/src/Backups/BackupUtils.cpp @@ -103,7 +103,7 @@ bool compareRestoredTableDef(const IAST & restored_table_create_query, const IAS auto adjust_before_comparison = [&](const IAST & query) -> ASTPtr { auto new_query = query.clone(); - adjustCreateQueryForBackup(new_query, global_context, nullptr); + adjustCreateQueryForBackup(new_query, global_context); ASTCreateQuery & create = typeid_cast(*new_query); create.setUUID({}); create.if_not_exists = false; diff --git a/src/Backups/DDLAdjustingForBackupVisitor.cpp b/src/Backups/DDLAdjustingForBackupVisitor.cpp index 5ea91094b75..7e5ce91629b 100644 --- a/src/Backups/DDLAdjustingForBackupVisitor.cpp +++ b/src/Backups/DDLAdjustingForBackupVisitor.cpp @@ -27,9 +27,6 @@ namespace { /// Precondition: engine_name.starts_with("Replicated") && engine_name.ends_with("MergeTree") - if (data.replicated_table_shared_id) - *data.replicated_table_shared_id = StorageReplicatedMergeTree::tryGetTableSharedIDFromCreateQuery(*data.create_query, data.global_context); - /// Before storing the metadata in a backup we have to find a zookeeper path in its definition and turn the table's UUID in there /// back into "{uuid}", and also we probably can remove the zookeeper path and replica name if they're default. /// So we're kind of reverting what we had done to the table's definition in registerStorageMergeTree.cpp before we created this table. @@ -98,12 +95,9 @@ void DDLAdjustingForBackupVisitor::visit(ASTPtr ast, const Data & data) visitCreateQuery(*create, data); } -void adjustCreateQueryForBackup(ASTPtr ast, const ContextPtr & global_context, std::optional * replicated_table_shared_id) +void adjustCreateQueryForBackup(ASTPtr ast, const ContextPtr & global_context) { - if (replicated_table_shared_id) - *replicated_table_shared_id = {}; - - DDLAdjustingForBackupVisitor::Data data{ast, global_context, replicated_table_shared_id}; + DDLAdjustingForBackupVisitor::Data data{ast, global_context}; DDLAdjustingForBackupVisitor::Visitor{data}.visit(ast); } diff --git a/src/Backups/DDLAdjustingForBackupVisitor.h b/src/Backups/DDLAdjustingForBackupVisitor.h index 63353dcc000..f0508434e02 100644 --- a/src/Backups/DDLAdjustingForBackupVisitor.h +++ b/src/Backups/DDLAdjustingForBackupVisitor.h @@ -12,9 +12,7 @@ class Context; using ContextPtr = std::shared_ptr; /// Changes a create query to a form which is appropriate or suitable for saving in a backup. -/// Also extracts a replicated table's shared ID from the create query if this is a create query for a replicated table. -/// `replicated_table_shared_id` can be null if you don't need that. -void adjustCreateQueryForBackup(ASTPtr ast, const ContextPtr & global_context, std::optional * replicated_table_shared_id); +void adjustCreateQueryForBackup(ASTPtr ast, const ContextPtr & global_context); /// Visits ASTCreateQuery and changes it to a form which is appropriate or suitable for saving in a backup. class DDLAdjustingForBackupVisitor @@ -24,7 +22,6 @@ public: { ASTPtr create_query; ContextPtr global_context; - std::optional * replicated_table_shared_id = nullptr; }; using Visitor = InDepthNodeVisitor; diff --git a/src/Backups/IBackupCoordination.h b/src/Backups/IBackupCoordination.h index f80b5dee883..4a9f8a23855 100644 --- a/src/Backups/IBackupCoordination.h +++ b/src/Backups/IBackupCoordination.h @@ -36,13 +36,13 @@ public: /// Multiple replicas of the replicated table call this function and then the added part names can be returned by call of the function /// getReplicatedPartNames(). /// Checksums are used only to control that parts under the same names on different replicas are the same. - virtual void addReplicatedPartNames(const String & table_shared_id, const String & table_name_for_logs, const String & replica_name, + virtual void addReplicatedPartNames(const String & table_zk_path, const String & table_name_for_logs, const String & replica_name, const std::vector & part_names_and_checksums) = 0; /// Returns the names of the parts which a specified replica of a replicated table should put to the backup. /// This is the same list as it was added by call of the function addReplicatedPartNames() but without duplications and without /// parts covered by another parts. - virtual Strings getReplicatedPartNames(const String & table_shared_id, const String & replica_name) const = 0; + virtual Strings getReplicatedPartNames(const String & table_zk_path, const String & replica_name) const = 0; struct MutationInfo { @@ -51,10 +51,10 @@ public: }; /// Adds information about mutations of a replicated table. - virtual void addReplicatedMutations(const String & table_shared_id, const String & table_name_for_logs, const String & replica_name, const std::vector & mutations) = 0; + virtual void addReplicatedMutations(const String & table_zk_path, const String & table_name_for_logs, const String & replica_name, const std::vector & mutations) = 0; /// Returns all mutations of a replicated table which are not finished for some data parts added by addReplicatedPartNames(). - virtual std::vector getReplicatedMutations(const String & table_shared_id, const String & replica_name) const = 0; + virtual std::vector getReplicatedMutations(const String & table_zk_path, const String & replica_name) const = 0; /// Adds information about KeeperMap tables virtual void addKeeperMapTable(const String & table_zookeeper_root_path, const String & table_id, const String & data_path_in_backup) = 0; @@ -65,10 +65,10 @@ public: /// Adds a data path in backup for a replicated table. /// Multiple replicas of the replicated table call this function and then all the added paths can be returned by call of the function /// getReplicatedDataPaths(). - virtual void addReplicatedDataPath(const String & table_shared_id, const String & data_path) = 0; + virtual void addReplicatedDataPath(const String & table_zk_path, const String & data_path) = 0; /// Returns all the data paths in backup added for a replicated table (see also addReplicatedDataPath()). - virtual Strings getReplicatedDataPaths(const String & table_shared_id) const = 0; + virtual Strings getReplicatedDataPaths(const String & table_zk_path) const = 0; /// Adds a path to access.txt file keeping access entities of a ReplicatedAccessStorage. virtual void addReplicatedAccessFilePath(const String & access_zk_path, AccessEntityType access_entity_type, const String & file_path) = 0; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c425035dfba..58d1846915f 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -187,7 +187,6 @@ namespace ErrorCodes extern const int NOT_INITIALIZED; extern const int TOO_LARGE_DISTRIBUTED_DEPTH; extern const int TABLE_IS_DROPPED; - extern const int CANNOT_BACKUP_TABLE; extern const int SUPPORT_IS_DISABLED; extern const int FAULT_INJECTED; extern const int CANNOT_FORGET_PARTITION; @@ -310,8 +309,9 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( true, /// require_part_metadata mode, [this] (const std::string & name) { enqueuePartForCheck(name); }) - , zookeeper_name(zkutil::extractZooKeeperName(zookeeper_path_)) - , zookeeper_path(zkutil::extractZooKeeperPath(zookeeper_path_, /* check_starts_with_slash */ mode <= LoadingStrictnessLevel::CREATE, log.load())) + , full_zookeeper_path(zookeeper_path_) + , zookeeper_name(zkutil::extractZooKeeperName(full_zookeeper_path)) + , zookeeper_path(zkutil::extractZooKeeperPath(full_zookeeper_path, /* check_starts_with_slash */ mode <= LoadingStrictnessLevel::CREATE, log.load())) , replica_name(replica_name_) , replica_path(fs::path(zookeeper_path) / "replicas" / replica_name_) , reader(*this) @@ -9242,24 +9242,6 @@ void StorageReplicatedMergeTree::createTableSharedID() const } -std::optional StorageReplicatedMergeTree::tryGetTableSharedIDFromCreateQuery(const IAST & create_query, const ContextPtr & global_context) -{ - auto zk_path = tryExtractZkPathFromCreateQuery(create_query, global_context); - if (!zk_path) - return {}; - - String zk_name = zkutil::extractZooKeeperName(*zk_path); - zk_path = zkutil::extractZooKeeperPath(*zk_path, false, nullptr); - zkutil::ZooKeeperPtr zookeeper = (zk_name == getDefaultZooKeeperName()) ? global_context->getZooKeeper() : global_context->getAuxiliaryZooKeeper(zk_name); - - String id; - if (!zookeeper->tryGet(fs::path(*zk_path) / "table_shared_id", id)) - return {}; - - return id; -} - - zkutil::EphemeralNodeHolderPtr StorageReplicatedMergeTree::lockSharedDataTemporary(const String & part_name, const String & part_id, const DiskPtr & disk) const { auto settings = getSettings(); @@ -10419,21 +10401,10 @@ void StorageReplicatedMergeTree::adjustCreateQueryForBackup(ASTPtr & create_quer auto metadata_diff = ReplicatedMergeTreeTableMetadata(*this, current_metadata).checkAndFindDiff(metadata_from_entry, current_metadata->getColumns(), getContext()); auto adjusted_metadata = metadata_diff.getNewMetadata(columns_from_entry, getContext(), *current_metadata); applyMetadataChangesToCreateQuery(create_query, adjusted_metadata); - - /// Check that tryGetTableSharedIDFromCreateQuery() works for this storage. - auto actual_table_shared_id = getTableSharedID(); - auto expected_table_shared_id = tryGetTableSharedIDFromCreateQuery(*create_query, getContext()); - if (actual_table_shared_id != expected_table_shared_id) - { - throw Exception(ErrorCodes::CANNOT_BACKUP_TABLE, "Table {} has its shared ID different from one from the create query: " - "actual shared id = {}, expected shared id = {}, create query = {}", - getStorageID().getNameForLogs(), actual_table_shared_id, expected_table_shared_id.value_or("nullopt"), - create_query); - } } catch (...) { - /// We can continue making a backup with non-adjusted name. + /// We can continue making a backup with non-adjusted query. tryLogCurrentException(log, "Failed to adjust the create query of this table for backup"); } } @@ -10459,8 +10430,8 @@ void StorageReplicatedMergeTree::backupData( auto parts_backup_entries = backupParts(data_parts, /* data_path_in_backup */ "", backup_settings, read_settings, local_context); auto coordination = backup_entries_collector.getBackupCoordination(); - String shared_id = getTableSharedID(); - coordination->addReplicatedDataPath(shared_id, data_path_in_backup); + + coordination->addReplicatedDataPath(full_zookeeper_path, data_path_in_backup); using PartNameAndChecksum = IBackupCoordination::PartNameAndChecksum; std::vector part_names_with_hashes; @@ -10469,7 +10440,7 @@ void StorageReplicatedMergeTree::backupData( part_names_with_hashes.emplace_back(PartNameAndChecksum{part_backup_entries.part_name, part_backup_entries.part_checksum}); /// Send our list of part names to the coordination (to compare with other replicas). - coordination->addReplicatedPartNames(shared_id, getStorageID().getFullTableName(), getReplicaName(), part_names_with_hashes); + coordination->addReplicatedPartNames(full_zookeeper_path, getStorageID().getFullTableName(), getReplicaName(), part_names_with_hashes); /// Send a list of mutations to the coordination too (we need to find the mutations which are not finished for added part names). { @@ -10511,25 +10482,25 @@ void StorageReplicatedMergeTree::backupData( } if (!mutation_infos.empty()) - coordination->addReplicatedMutations(shared_id, getStorageID().getFullTableName(), getReplicaName(), mutation_infos); + coordination->addReplicatedMutations(full_zookeeper_path, getStorageID().getFullTableName(), getReplicaName(), mutation_infos); } } /// This task will be executed after all replicas have collected their parts and the coordination is ready to /// give us the final list of parts to add to the BackupEntriesCollector. - auto post_collecting_task = [shared_id, + auto post_collecting_task = [my_full_zookeeper_path = full_zookeeper_path, my_replica_name = getReplicaName(), coordination, my_parts_backup_entries = std::move(parts_backup_entries), &backup_entries_collector]() { - Strings data_paths = coordination->getReplicatedDataPaths(shared_id); + Strings data_paths = coordination->getReplicatedDataPaths(my_full_zookeeper_path); std::vector data_paths_fs; data_paths_fs.reserve(data_paths.size()); for (const auto & data_path : data_paths) data_paths_fs.push_back(data_path); - Strings part_names = coordination->getReplicatedPartNames(shared_id, my_replica_name); + Strings part_names = coordination->getReplicatedPartNames(my_full_zookeeper_path, my_replica_name); std::unordered_set part_names_set{part_names.begin(), part_names.end()}; for (const auto & part_backup_entries : my_parts_backup_entries) @@ -10542,7 +10513,7 @@ void StorageReplicatedMergeTree::backupData( } } - auto mutation_infos = coordination->getReplicatedMutations(shared_id, my_replica_name); + auto mutation_infos = coordination->getReplicatedMutations(my_full_zookeeper_path, my_replica_name); for (const auto & mutation_info : mutation_infos) { auto backup_entry = ReplicatedMergeTreeMutationEntry::parse(mutation_info.entry, mutation_info.id).backup(); @@ -10556,8 +10527,7 @@ void StorageReplicatedMergeTree::backupData( void StorageReplicatedMergeTree::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) { - String full_zk_path = getZooKeeperName() + getZooKeeperPath(); - if (!restorer.getRestoreCoordination()->acquireInsertingDataIntoReplicatedTable(full_zk_path)) + if (!restorer.getRestoreCoordination()->acquireInsertingDataIntoReplicatedTable(full_zookeeper_path)) { /// Other replica is already restoring the data of this table. /// We'll get them later due to replication, it's not necessary to read it from the backup. diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index c472c11e7f8..7f33c82e5c2 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -330,17 +330,14 @@ public: // Return default or custom zookeeper name for table const String & getZooKeeperName() const { return zookeeper_name; } - const String & getZooKeeperPath() const { return zookeeper_path; } + const String & getFullZooKeeperPath() const { return full_zookeeper_path; } // Return table id, common for different replicas String getTableSharedID() const override; std::map getUnfinishedMutationCommands() const override; - /// Returns the same as getTableSharedID(), but extracts it from a create query. - static std::optional tryGetTableSharedIDFromCreateQuery(const IAST & create_query, const ContextPtr & global_context); - static const String & getDefaultZooKeeperName() { return default_zookeeper_name; } /// Check if there are new broken disks and enqueue part recovery tasks. @@ -420,9 +417,11 @@ private: bool is_readonly_metric_set = false; + const String full_zookeeper_path; static const String default_zookeeper_name; const String zookeeper_name; const String zookeeper_path; + const String replica_name; const String replica_path; From 6a459fe581a9fd538a6cb51234098505b1cc4501 Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Thu, 25 Apr 2024 13:44:03 -0700 Subject: [PATCH 150/624] [Docs] Add details for wait_for_async_insert --- docs/en/operations/settings/settings.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 869a0be2574..86dc3eb667a 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1923,7 +1923,9 @@ Default value: `16`. ### wait_for_async_insert {#wait-for-async-insert} -Enables or disables waiting for processing of asynchronous insertion. If enabled, server will return `OK` only after the data is inserted. Otherwise, it will return `OK` even if the data wasn't inserted. +Enables or disables waiting for processing of asynchronous insertion. If enabled, server will return `OK` only after the data is inserted. Otherwise, it will return `OK` even if the data has not yet been inserted. + +If you are using asynchronous inserts, we also recommend enabling [`async_insert`](#async-insert). Possible values: From 49740451f7bcd81e660492d34aecc9f922c3c74e Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Wed, 24 Apr 2024 18:25:02 +0800 Subject: [PATCH 151/624] reduce memory usage during merging external sorted file --- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.h | 1 + src/Interpreters/MergeJoin.cpp | 2 +- src/Processors/QueryPlan/SortingStep.cpp | 2 ++ src/Processors/QueryPlan/SortingStep.h | 1 + src/Processors/Transforms/MergeSortingTransform.cpp | 9 ++++++++- src/Processors/Transforms/MergeSortingTransform.h | 2 ++ 7 files changed, 16 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 375bdb1c516..1b0e29fcce7 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -409,6 +409,7 @@ class IColumn; M(UInt64, max_rows_to_sort, 0, "If more than the specified amount of records have to be processed for ORDER BY operation, the behavior will be determined by the 'sort_overflow_mode' which by default is - throw an exception", 0) \ M(UInt64, max_bytes_to_sort, 0, "If more than the specified amount of (uncompressed) bytes have to be processed for ORDER BY operation, the behavior will be determined by the 'sort_overflow_mode' which by default is - throw an exception", 0) \ M(OverflowMode, sort_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ + M(UInt64, prefer_external_sort_block_bytes, 0, "Prefer maximum block bytes for external sort, reduce the memory usage during merging.", 0) \ M(UInt64, max_bytes_before_external_sort, 0, "If memory usage during ORDER BY operation is exceeding this threshold in bytes, activate the 'external sorting' mode (spill data to disk). Recommended value is half of available system memory.", 0) \ M(UInt64, max_bytes_before_remerge_sort, 1000000000, "In case of ORDER BY with LIMIT, when memory usage is higher than specified threshold, perform additional steps of merging blocks before final merge to keep just top LIMIT rows.", 0) \ M(Float, remerge_sort_lowered_memory_bytes_ratio, 2., "If memory usage after remerge does not reduced by this ratio, remerge will be disabled.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index d3b5de06e70..737d620063f 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -85,6 +85,7 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { + {"24.5", {{"prefer_external_sort_block_bytes", 0, 0, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}}, {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, {"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"}, diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index d5fb0208d45..24d403a782a 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -604,7 +604,7 @@ void MergeJoin::mergeInMemoryRightBlocks() /// TODO: there should be no split keys by blocks for RIGHT|FULL JOIN builder.addTransform(std::make_shared( - builder.getHeader(), right_sort_description, max_rows_in_right_block, 0, false, 0, 0, 0, nullptr, 0)); + builder.getHeader(), right_sort_description, 0, max_rows_in_right_block, 0, false, 0, 0, 0, nullptr, 0)); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); PullingPipelineExecutor executor(pipeline); diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp index d0491cb4b82..8f40e523b42 100644 --- a/src/Processors/QueryPlan/SortingStep.cpp +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -37,6 +37,7 @@ SortingStep::Settings::Settings(const Context & context) max_bytes_before_external_sort = settings.max_bytes_before_external_sort; tmp_data = context.getTempDataOnDisk(); min_free_disk_space = settings.min_free_disk_space_for_temporary_data; + max_block_bytes = settings.prefer_external_sort_block_bytes; } SortingStep::Settings::Settings(size_t max_block_size_) @@ -284,6 +285,7 @@ void SortingStep::mergeSorting( header, result_sort_desc, sort_settings.max_block_size, + sort_settings.max_block_bytes, limit_, increase_sort_description_compile_attempts_current, sort_settings.max_bytes_before_remerge / pipeline.getNumStreams(), diff --git a/src/Processors/QueryPlan/SortingStep.h b/src/Processors/QueryPlan/SortingStep.h index 52f48f66a32..393fdab2d3a 100644 --- a/src/Processors/QueryPlan/SortingStep.h +++ b/src/Processors/QueryPlan/SortingStep.h @@ -27,6 +27,7 @@ public: size_t max_bytes_before_external_sort = 0; TemporaryDataOnDiskScopePtr tmp_data = nullptr; size_t min_free_disk_space = 0; + size_t max_block_bytes = 0; explicit Settings(const Context & context); explicit Settings(size_t max_block_size_); diff --git a/src/Processors/Transforms/MergeSortingTransform.cpp b/src/Processors/Transforms/MergeSortingTransform.cpp index 64d84ea4b00..d6cd22b1115 100644 --- a/src/Processors/Transforms/MergeSortingTransform.cpp +++ b/src/Processors/Transforms/MergeSortingTransform.cpp @@ -80,6 +80,7 @@ MergeSortingTransform::MergeSortingTransform( const Block & header, const SortDescription & description_, size_t max_merged_block_size_, + size_t max_block_bytes_, UInt64 limit_, bool increase_sort_description_compile_attempts, size_t max_bytes_before_remerge_, @@ -93,6 +94,7 @@ MergeSortingTransform::MergeSortingTransform( , max_bytes_before_external_sort(max_bytes_before_external_sort_) , tmp_data(std::move(tmp_data_)) , min_free_disk_space(min_free_disk_space_) + , max_block_bytes(max_block_bytes_) { } @@ -169,7 +171,12 @@ void MergeSortingTransform::consume(Chunk chunk) /// If there's less free disk space than reserve_size, an exception will be thrown size_t reserve_size = sum_bytes_in_blocks + min_free_disk_space; auto & tmp_stream = tmp_data->createStream(header_without_constants, reserve_size); - + size_t max_merged_block_size = this->max_merged_block_size; + if (max_merged_block_size > 0) + { + auto avg_row_bytes = sum_bytes_in_blocks / sum_rows_in_blocks; + max_merged_block_size = std::min(max_merged_block_size, max_block_bytes / avg_row_bytes); + } merge_sorter = std::make_unique(header_without_constants, std::move(chunks), description, max_merged_block_size, limit); auto current_processor = std::make_shared(header_without_constants, tmp_stream, log); diff --git a/src/Processors/Transforms/MergeSortingTransform.h b/src/Processors/Transforms/MergeSortingTransform.h index 4478d5a07e8..a39dd66caa0 100644 --- a/src/Processors/Transforms/MergeSortingTransform.h +++ b/src/Processors/Transforms/MergeSortingTransform.h @@ -23,6 +23,7 @@ public: const Block & header, const SortDescription & description_, size_t max_merged_block_size_, + size_t max_block_bytes, UInt64 limit_, bool increase_sort_description_compile_attempts, size_t max_bytes_before_remerge_, @@ -46,6 +47,7 @@ private: size_t max_bytes_before_external_sort; TemporaryDataOnDiskPtr tmp_data; size_t min_free_disk_space; + size_t max_block_bytes; size_t sum_rows_in_blocks = 0; size_t sum_bytes_in_blocks = 0; From f06cb0d42fc06c08333665220cf8aad4dcd7cda0 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Fri, 26 Apr 2024 11:08:56 +0800 Subject: [PATCH 152/624] add minimum block size limitation --- src/Processors/Transforms/MergeSortingTransform.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Processors/Transforms/MergeSortingTransform.cpp b/src/Processors/Transforms/MergeSortingTransform.cpp index d6cd22b1115..b3ac91dee93 100644 --- a/src/Processors/Transforms/MergeSortingTransform.cpp +++ b/src/Processors/Transforms/MergeSortingTransform.cpp @@ -175,7 +175,10 @@ void MergeSortingTransform::consume(Chunk chunk) if (max_merged_block_size > 0) { auto avg_row_bytes = sum_bytes_in_blocks / sum_rows_in_blocks; - max_merged_block_size = std::min(max_merged_block_size, max_block_bytes / avg_row_bytes); + /// The memory usage of the block does not exceed max block bytes, and the number of rows in the block is not less than 128. + /// If the max_merged_block_size is less than 128, then the new block size is not less than max_merged_block_size + auto min_block_size = std::min(128UL, max_merged_block_size); + max_merged_block_size = std::max(std::min(max_merged_block_size, max_block_bytes / avg_row_bytes), min_block_size); } merge_sorter = std::make_unique(header_without_constants, std::move(chunks), description, max_merged_block_size, limit); auto current_processor = std::make_shared(header_without_constants, tmp_stream, log); From 3696d05de90b474389ed0b7c22422231cfe9472c Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Fri, 26 Apr 2024 14:08:14 +0800 Subject: [PATCH 153/624] change default value --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 2 +- src/Processors/Transforms/MergeSortingTransform.cpp | 9 +++++---- 3 files changed, 7 insertions(+), 6 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 1b0e29fcce7..fecd3c6bdc5 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -409,7 +409,7 @@ class IColumn; M(UInt64, max_rows_to_sort, 0, "If more than the specified amount of records have to be processed for ORDER BY operation, the behavior will be determined by the 'sort_overflow_mode' which by default is - throw an exception", 0) \ M(UInt64, max_bytes_to_sort, 0, "If more than the specified amount of (uncompressed) bytes have to be processed for ORDER BY operation, the behavior will be determined by the 'sort_overflow_mode' which by default is - throw an exception", 0) \ M(OverflowMode, sort_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ - M(UInt64, prefer_external_sort_block_bytes, 0, "Prefer maximum block bytes for external sort, reduce the memory usage during merging.", 0) \ + M(UInt64, prefer_external_sort_block_bytes, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging.", 0) \ M(UInt64, max_bytes_before_external_sort, 0, "If memory usage during ORDER BY operation is exceeding this threshold in bytes, activate the 'external sorting' mode (spill data to disk). Recommended value is half of available system memory.", 0) \ M(UInt64, max_bytes_before_remerge_sort, 1000000000, "In case of ORDER BY with LIMIT, when memory usage is higher than specified threshold, perform additional steps of merging blocks before final merge to keep just top LIMIT rows.", 0) \ M(Float, remerge_sort_lowered_memory_bytes_ratio, 2., "If memory usage after remerge does not reduced by this ratio, remerge will be disabled.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 737d620063f..5d63a07fe58 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -85,7 +85,6 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { - {"24.5", {{"prefer_external_sort_block_bytes", 0, 0, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}}, {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, {"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"}, @@ -94,6 +93,7 @@ static std::map sett {"input_format_hive_text_allow_variable_number_of_columns", false, true, "Ignore extra columns in Hive Text input (if file has more columns than expected) and treat missing fields in Hive Text input as default values."}, {"first_day_of_week", "Monday", "Monday", "Added a setting for the first day of the week for date/time functions"}, {"temporary_data_in_cache_reserve_space_wait_lock_timeout_milliseconds", (10 * 60 * 1000), (10 * 60 * 1000), "Wait time to lock cache for sapce reservation in temporary data in filesystem cache"}, + {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."} }}, {"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/Processors/Transforms/MergeSortingTransform.cpp b/src/Processors/Transforms/MergeSortingTransform.cpp index b3ac91dee93..304b79e6c59 100644 --- a/src/Processors/Transforms/MergeSortingTransform.cpp +++ b/src/Processors/Transforms/MergeSortingTransform.cpp @@ -175,10 +175,11 @@ void MergeSortingTransform::consume(Chunk chunk) if (max_merged_block_size > 0) { auto avg_row_bytes = sum_bytes_in_blocks / sum_rows_in_blocks; - /// The memory usage of the block does not exceed max block bytes, and the number of rows in the block is not less than 128. - /// If the max_merged_block_size is less than 128, then the new block size is not less than max_merged_block_size - auto min_block_size = std::min(128UL, max_merged_block_size); - max_merged_block_size = std::max(std::min(max_merged_block_size, max_block_bytes / avg_row_bytes), min_block_size); + /// max_merged_block_size >= 128 + max_merged_block_size = std::max(std::min(max_merged_block_size, max_block_bytes / avg_row_bytes), 128UL); + /// when max_block_size < 128, use max_block_size. + /// max_block_size still works. + max_merged_block_size = std::min(this->max_merged_block_size, max_merged_block_size); } merge_sorter = std::make_unique(header_without_constants, std::move(chunks), description, max_merged_block_size, limit); auto current_processor = std::make_shared(header_without_constants, tmp_stream, log); From 6e579312633f2c0abb8784f122bfc75559a5d05a Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 25 Apr 2024 19:24:36 +0200 Subject: [PATCH 154/624] Get rid of code duplication in extractZkPathFromCreateQuery(). --- src/Backups/BackupEntriesCollector.cpp | 4 +- .../extractZkPathFromCreateQuery.cpp | 61 --- .../MergeTree/extractZkPathFromCreateQuery.h | 19 - ...tractZooKeeperPathFromReplicatedTableDef.h | 18 + .../MergeTree/registerStorageMergeTree.cpp | 401 +++++++++++------- src/Storages/StorageReplicatedMergeTree.cpp | 1 - 6 files changed, 272 insertions(+), 232 deletions(-) delete mode 100644 src/Storages/MergeTree/extractZkPathFromCreateQuery.cpp delete mode 100644 src/Storages/MergeTree/extractZkPathFromCreateQuery.h create mode 100644 src/Storages/MergeTree/extractZooKeeperPathFromReplicatedTableDef.h diff --git a/src/Backups/BackupEntriesCollector.cpp b/src/Backups/BackupEntriesCollector.cpp index 136e3c49321..d91cf47c4d3 100644 --- a/src/Backups/BackupEntriesCollector.cpp +++ b/src/Backups/BackupEntriesCollector.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include #include #include @@ -776,7 +776,7 @@ void BackupEntriesCollector::makeBackupEntriesForTablesDefs() checkIsQueryCancelled(); ASTPtr new_create_query = table_info.create_table_query; - table_info.replicated_table_zk_path = tryExtractZkPathFromCreateQuery(*new_create_query, context->getGlobalContext()); + table_info.replicated_table_zk_path = extractZooKeeperPathFromReplicatedTableDef(new_create_query->as(), context); adjustCreateQueryForBackup(new_create_query, context->getGlobalContext()); renameDatabaseAndTableNameInCreateQuery(new_create_query, renaming_map, context->getGlobalContext()); diff --git a/src/Storages/MergeTree/extractZkPathFromCreateQuery.cpp b/src/Storages/MergeTree/extractZkPathFromCreateQuery.cpp deleted file mode 100644 index 8ea732b0243..00000000000 --- a/src/Storages/MergeTree/extractZkPathFromCreateQuery.cpp +++ /dev/null @@ -1,61 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -std::optional tryExtractZkPathFromCreateQuery(const IAST & create_query, const ContextPtr & global_context) -{ - const auto * create = create_query.as(); - if (!create || !create->storage || !create->storage->engine) - return {}; - - /// Check if the table engine is one of the ReplicatedMergeTree family. - const auto & ast_engine = *create->storage->engine; - if (!ast_engine.name.starts_with("Replicated") || !ast_engine.name.ends_with("MergeTree")) - return {}; - - /// Get the first argument. - const auto * ast_arguments = typeid_cast(ast_engine.arguments.get()); - if (!ast_arguments || ast_arguments->children.empty()) - return {}; - - auto * ast_zk_path = typeid_cast(ast_arguments->children[0].get()); - if (!ast_zk_path || (ast_zk_path->value.getType() != Field::Types::String)) - return {}; - - String zk_path = ast_zk_path->value.safeGet(); - - /// Expand macros. - Macros::MacroExpansionInfo info; - info.table_id.table_name = create->getTable(); - info.table_id.database_name = create->getDatabase(); - info.table_id.uuid = create->uuid; - auto database = DatabaseCatalog::instance().tryGetDatabase(info.table_id.database_name); - if (database && database->getEngineName() == "Replicated") - { - info.shard = getReplicatedDatabaseShardName(database); - info.replica = getReplicatedDatabaseReplicaName(database); - } - - try - { - zk_path = global_context->getMacros()->expand(zk_path, info); - } - catch (...) - { - return {}; /// Couldn't expand macros. - } - - return zk_path; -} - -} diff --git a/src/Storages/MergeTree/extractZkPathFromCreateQuery.h b/src/Storages/MergeTree/extractZkPathFromCreateQuery.h deleted file mode 100644 index e22f76d2cd5..00000000000 --- a/src/Storages/MergeTree/extractZkPathFromCreateQuery.h +++ /dev/null @@ -1,19 +0,0 @@ -#pragma once - -#include -#include -#include - - -namespace DB -{ -class IAST; -class Context; -using ContextPtr = std::shared_ptr; - -/// Extracts a zookeeper path from a specified CREATE TABLE query. Returns std::nullopt if fails. -/// The function takes the first argument of the ReplicatedMergeTree table engine and expands macros in it. -/// It works like a part of what the create() function in registerStorageMergeTree.cpp does but in a simpler manner. -std::optional tryExtractZkPathFromCreateQuery(const IAST & create_query, const ContextPtr & global_context); - -} diff --git a/src/Storages/MergeTree/extractZooKeeperPathFromReplicatedTableDef.h b/src/Storages/MergeTree/extractZooKeeperPathFromReplicatedTableDef.h new file mode 100644 index 00000000000..1bd58392201 --- /dev/null +++ b/src/Storages/MergeTree/extractZooKeeperPathFromReplicatedTableDef.h @@ -0,0 +1,18 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ +class ASTCreateQuery; +class Context; +using ContextPtr = std::shared_ptr; + +/// Extracts a zookeeper path from a specified CREATE TABLE query. Returns std::nullopt if fails. +/// The function checks the table engine and if it is Replicated*MergeTree then it takes the first argument and expands macros in it. +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 d552a4b6fa5..9b0200d5a1c 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -122,6 +123,248 @@ static void verifySortingKey(const KeyDescription & sorting_key) } } +/// Returns whether a new syntax is used to define a table engine, i.e. MergeTree() PRIMARY KEY ... PARTITION BY ... SETTINGS ... +/// instead of MergeTree(MergeTree(date, [sample_key], primary_key). +static bool isExtendedStorageDef(const ASTCreateQuery & query) +{ + if (query.storage && query.storage->isExtendedStorageDefinition()) + return true; + + if (query.columns_list && + ((query.columns_list->indices && !query.columns_list->indices->children.empty()) || + (query.columns_list->projections && !query.columns_list->projections->children.empty()))) + { + return true; + } + + return false; +} + +/// Evaluates expressions in engine arguments. +/// In new syntax an argument can be literal or identifier or array/tuple of identifiers. +static void evaluateEngineArgs(ASTs & engine_args, const ContextPtr & context) +{ + size_t arg_idx = 0; + try + { + for (; arg_idx < engine_args.size(); ++arg_idx) + { + auto & arg = engine_args[arg_idx]; + auto * arg_func = arg->as(); + if (!arg_func) + continue; + + /// If we got ASTFunction, let's evaluate it and replace with ASTLiteral. + /// Do not try evaluate array or tuple, because it's array or tuple of column identifiers. + if (arg_func->name == "array" || arg_func->name == "tuple") + continue; + Field value = evaluateConstantExpression(arg, context).first; + arg = std::make_shared(value); + } + } + catch (Exception & e) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot evaluate engine argument {}: {} {}", + arg_idx, e.message(), verbose_help_message); + } +} + +/// Returns whether this is a Replicated table engine? +static bool isReplicated(const String & engine_name) +{ + return engine_name.starts_with("Replicated") && engine_name.ends_with("MergeTree"); +} + +/// Returns the part of the name of a table engine between "Replicated" (if any) and "MergeTree". +static std::string_view getNamePart(const String & engine_name) +{ + std::string_view name_part = engine_name; + if (name_part.starts_with("Replicated")) + name_part.remove_prefix(strlen("Replicated")); + + if (name_part.ends_with("MergeTree")) + name_part.remove_suffix(strlen("MergeTree")); + + return name_part; +} + +/// Extracts zookeeper path and replica name from the table engine's arguments. +/// The function can modify those arguments (that's why they're passed separately in `engine_args`) and also determines RenamingRestrictions. +/// The function assumes the table engine is Replicated. +static void extractZooKeeperPathAndReplicaNameFromEngineArgs( + const ASTCreateQuery & query, + const StorageID & table_id, + const String & engine_name, + ASTs & engine_args, + LoadingStrictnessLevel mode, + const ContextPtr & context, + String & zookeeper_path, + String & replica_name, + RenamingRestrictions & renaming_restrictions) +{ + chassert(isReplicated(engine_name)); + + zookeeper_path = ""; + replica_name = ""; + renaming_restrictions = RenamingRestrictions::ALLOW_ANY; + + bool is_extended_storage_def = isExtendedStorageDef(query); + + if (is_extended_storage_def) + { + /// Allow expressions in engine arguments. + /// In new syntax argument can be literal or identifier or array/tuple of identifiers. + evaluateEngineArgs(engine_args, context); + } + + bool is_on_cluster = context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY; + bool is_replicated_database = context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY && + DatabaseCatalog::instance().getDatabase(table_id.database_name)->getEngineName() == "Replicated"; + + /// Allow implicit {uuid} macros only for zookeeper_path in ON CLUSTER queries + /// and if UUID was explicitly passed in CREATE TABLE (like for ATTACH) + bool allow_uuid_macro = is_on_cluster || is_replicated_database || query.attach || query.has_uuid; + + auto expand_macro = [&] (ASTLiteral * ast_zk_path, ASTLiteral * ast_replica_name) + { + /// Unfold {database} and {table} macro on table creation, so table can be renamed. + if (mode < LoadingStrictnessLevel::ATTACH) + { + Macros::MacroExpansionInfo info; + /// NOTE: it's not recursive + info.expand_special_macros_only = true; + info.table_id = table_id; + /// Avoid unfolding {uuid} macro on this step. + /// We did unfold it in previous versions to make moving table from Atomic to Ordinary database work correctly, + /// but now it's not allowed (and it was the only reason to unfold {uuid} macro). + info.table_id.uuid = UUIDHelpers::Nil; + zookeeper_path = context->getMacros()->expand(zookeeper_path, info); + + info.level = 0; + replica_name = context->getMacros()->expand(replica_name, info); + } + + ast_zk_path->value = zookeeper_path; + ast_replica_name->value = replica_name; + + /// Expand other macros (such as {shard} and {replica}). We do not expand them on previous step + /// to make possible copying metadata files between replicas. + Macros::MacroExpansionInfo info; + info.table_id = table_id; + if (is_replicated_database) + { + auto database = DatabaseCatalog::instance().getDatabase(table_id.database_name); + info.shard = getReplicatedDatabaseShardName(database); + info.replica = getReplicatedDatabaseReplicaName(database); + } + if (!allow_uuid_macro) + info.table_id.uuid = UUIDHelpers::Nil; + zookeeper_path = context->getMacros()->expand(zookeeper_path, info); + + info.level = 0; + info.table_id.uuid = UUIDHelpers::Nil; + replica_name = context->getMacros()->expand(replica_name, info); + + /// We do not allow renaming table with these macros in metadata, because zookeeper_path will be broken after RENAME TABLE. + /// NOTE: it may happen if table was created by older version of ClickHouse (< 20.10) and macros was not unfolded on table creation + /// or if one of these macros is recursively expanded from some other macro. + /// Also do not allow to move table from Atomic to Ordinary database if there's {uuid} macro + if (info.expanded_database || info.expanded_table) + renaming_restrictions = RenamingRestrictions::DO_NOT_ALLOW; + else if (info.expanded_uuid) + renaming_restrictions = RenamingRestrictions::ALLOW_PRESERVING_UUID; + }; + + size_t arg_num = 0; + size_t arg_cnt = engine_args.size(); + + bool has_arguments = (arg_num + 2 <= arg_cnt); + bool has_valid_arguments = has_arguments && engine_args[arg_num]->as() && engine_args[arg_num + 1]->as(); + + if (has_valid_arguments) + { + /// Get path and name from engine arguments + auto * ast_zk_path = engine_args[arg_num]->as(); + if (ast_zk_path && ast_zk_path->value.getType() == Field::Types::String) + zookeeper_path = ast_zk_path->value.safeGet(); + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path in ZooKeeper must be a string literal{}", verbose_help_message); + + auto * ast_replica_name = engine_args[arg_num + 1]->as(); + if (ast_replica_name && ast_replica_name->value.getType() == Field::Types::String) + replica_name = ast_replica_name->value.safeGet(); + 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 + && (arg_cnt == 0 + || !engine_args[arg_num]->as() + || (arg_cnt == 1 && (getNamePart(engine_name) == "Graphite")))) + { + /// Try use default values if arguments are not specified. + /// Note: {uuid} macro works for ON CLUSTER queries when database engine is Atomic. + const auto & server_settings = context->getServerSettings(); + zookeeper_path = server_settings.default_replica_path; + /// TODO maybe use hostname if {replica} is not defined? + replica_name = server_settings.default_replica_name; + + /// Modify query, so default values will be written to metadata + assert(arg_num == 0); + ASTs old_args; + std::swap(engine_args, old_args); + auto path_arg = std::make_shared(zookeeper_path); + auto name_arg = std::make_shared(replica_name); + auto * ast_zk_path = path_arg.get(); + auto * ast_replica_name = name_arg.get(); + + expand_macro(ast_zk_path, ast_replica_name); + + engine_args.emplace_back(std::move(path_arg)); + engine_args.emplace_back(std::move(name_arg)); + std::move(std::begin(old_args), std::end(old_args), std::back_inserter(engine_args)); + } + else + 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. +std::optional extractZooKeeperPathFromReplicatedTableDef(const ASTCreateQuery & query, const ContextPtr & context) +{ + 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 (...) + { + return {}; + } +} static StoragePtr create(const StorageFactory::Arguments & args) { @@ -156,17 +399,12 @@ static StoragePtr create(const StorageFactory::Arguments & args) * - Additional MergeTreeSettings in the SETTINGS clause; */ - bool is_extended_storage_def = args.storage_def->isExtendedStorageDefinition() - || (args.query.columns_list->indices && !args.query.columns_list->indices->children.empty()) - || (args.query.columns_list->projections && !args.query.columns_list->projections->children.empty()); + bool is_extended_storage_def = isExtendedStorageDef(args.query); const Settings & local_settings = args.getLocalContext()->getSettingsRef(); - String name_part = args.engine_name.substr(0, args.engine_name.size() - strlen("MergeTree")); - - bool replicated = startsWith(name_part, "Replicated"); - if (replicated) - name_part = name_part.substr(strlen("Replicated")); + bool replicated = isReplicated(args.engine_name); + std::string_view name_part = getNamePart(args.engine_name); MergeTreeData::MergingParams merging_params; merging_params.mode = MergeTreeData::MergingParams::Ordinary; @@ -283,29 +521,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) { /// Allow expressions in engine arguments. /// In new syntax argument can be literal or identifier or array/tuple of identifiers. - size_t arg_idx = 0; - try - { - for (; arg_idx < engine_args.size(); ++arg_idx) - { - auto & arg = engine_args[arg_idx]; - auto * arg_func = arg->as(); - if (!arg_func) - continue; - - /// If we got ASTFunction, let's evaluate it and replace with ASTLiteral. - /// Do not try evaluate array or tuple, because it's array or tuple of column identifiers. - if (arg_func->name == "array" || arg_func->name == "tuple") - continue; - Field value = evaluateConstantExpression(arg, args.getLocalContext()).first; - arg = std::make_shared(value); - } - } - catch (Exception & e) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot evaluate engine argument {}: {} {}", - arg_idx, e.message(), verbose_help_message); - } + evaluateEngineArgs(engine_args, args.getLocalContext()); } else if (args.mode <= LoadingStrictnessLevel::CREATE && !local_settings.allow_deprecated_syntax_for_merge_tree) { @@ -314,130 +530,17 @@ static StoragePtr create(const StorageFactory::Arguments & args) "See also `allow_deprecated_syntax_for_merge_tree` setting."); } - /// For Replicated. + /// Extract zookeeper path and replica name from engine arguments. String zookeeper_path; String replica_name; RenamingRestrictions renaming_restrictions = RenamingRestrictions::ALLOW_ANY; - bool is_on_cluster = args.getLocalContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY; - bool is_replicated_database = args.getLocalContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY && - DatabaseCatalog::instance().getDatabase(args.table_id.database_name)->getEngineName() == "Replicated"; - - /// Allow implicit {uuid} macros only for zookeeper_path in ON CLUSTER queries - /// and if UUID was explicitly passed in CREATE TABLE (like for ATTACH) - bool allow_uuid_macro = is_on_cluster || is_replicated_database || args.query.attach || args.query.has_uuid; - - auto expand_macro = [&] (ASTLiteral * ast_zk_path, ASTLiteral * ast_replica_name) - { - /// Unfold {database} and {table} macro on table creation, so table can be renamed. - if (args.mode < LoadingStrictnessLevel::ATTACH) - { - Macros::MacroExpansionInfo info; - /// NOTE: it's not recursive - info.expand_special_macros_only = true; - info.table_id = args.table_id; - /// Avoid unfolding {uuid} macro on this step. - /// We did unfold it in previous versions to make moving table from Atomic to Ordinary database work correctly, - /// but now it's not allowed (and it was the only reason to unfold {uuid} macro). - info.table_id.uuid = UUIDHelpers::Nil; - zookeeper_path = context->getMacros()->expand(zookeeper_path, info); - - info.level = 0; - replica_name = context->getMacros()->expand(replica_name, info); - } - - ast_zk_path->value = zookeeper_path; - ast_replica_name->value = replica_name; - - /// Expand other macros (such as {shard} and {replica}). We do not expand them on previous step - /// to make possible copying metadata files between replicas. - Macros::MacroExpansionInfo info; - info.table_id = args.table_id; - if (is_replicated_database) - { - auto database = DatabaseCatalog::instance().getDatabase(args.table_id.database_name); - info.shard = getReplicatedDatabaseShardName(database); - info.replica = getReplicatedDatabaseReplicaName(database); - } - if (!allow_uuid_macro) - info.table_id.uuid = UUIDHelpers::Nil; - zookeeper_path = context->getMacros()->expand(zookeeper_path, info); - - info.level = 0; - info.table_id.uuid = UUIDHelpers::Nil; - replica_name = context->getMacros()->expand(replica_name, info); - - /// We do not allow renaming table with these macros in metadata, because zookeeper_path will be broken after RENAME TABLE. - /// NOTE: it may happen if table was created by older version of ClickHouse (< 20.10) and macros was not unfolded on table creation - /// or if one of these macros is recursively expanded from some other macro. - /// Also do not allow to move table from Atomic to Ordinary database if there's {uuid} macro - if (info.expanded_database || info.expanded_table) - renaming_restrictions = RenamingRestrictions::DO_NOT_ALLOW; - else if (info.expanded_uuid) - renaming_restrictions = RenamingRestrictions::ALLOW_PRESERVING_UUID; - }; - if (replicated) { - bool has_arguments = arg_num + 2 <= arg_cnt; - bool has_valid_arguments = has_arguments && engine_args[arg_num]->as() && engine_args[arg_num + 1]->as(); - - ASTLiteral * ast_zk_path; - ASTLiteral * ast_replica_name; - - if (has_valid_arguments) - { - /// Get path and name from engine arguments - ast_zk_path = engine_args[arg_num]->as(); - if (ast_zk_path && ast_zk_path->value.getType() == Field::Types::String) - zookeeper_path = ast_zk_path->value.safeGet(); - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path in ZooKeeper must be a string literal{}", verbose_help_message); - ++arg_num; - - ast_replica_name = engine_args[arg_num]->as(); - if (ast_replica_name && ast_replica_name->value.getType() == Field::Types::String) - replica_name = ast_replica_name->value.safeGet(); - 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); - ++arg_num; - - expand_macro(ast_zk_path, ast_replica_name); - } - else if (is_extended_storage_def - && (arg_cnt == 0 - || !engine_args[arg_num]->as() - || (arg_cnt == 1 && merging_params.mode == MergeTreeData::MergingParams::Graphite))) - { - /// Try use default values if arguments are not specified. - /// Note: {uuid} macro works for ON CLUSTER queries when database engine is Atomic. - const auto & server_settings = args.getContext()->getServerSettings(); - zookeeper_path = server_settings.default_replica_path; - /// TODO maybe use hostname if {replica} is not defined? - replica_name = server_settings.default_replica_name; - - /// Modify query, so default values will be written to metadata - assert(arg_num == 0); - ASTs old_args; - std::swap(engine_args, old_args); - auto path_arg = std::make_shared(zookeeper_path); - auto name_arg = std::make_shared(replica_name); - ast_zk_path = path_arg.get(); - ast_replica_name = name_arg.get(); - - expand_macro(ast_zk_path, ast_replica_name); - - engine_args.emplace_back(std::move(path_arg)); - engine_args.emplace_back(std::move(name_arg)); - std::move(std::begin(old_args), std::end(old_args), std::back_inserter(engine_args)); - arg_num = 2; - arg_cnt += 2; - } - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected two string literal arguments: zookeeper_path and replica_name"); + extractZooKeeperPathAndReplicaNameFromEngineArgs(args.query, args.table_id, args.engine_name, args.engine_args, args.mode, + args.getLocalContext(), zookeeper_path, replica_name, renaming_restrictions); + 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. } /// This merging param maybe used as part of sorting key diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 58d1846915f..0639b172d31 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -31,7 +31,6 @@ #include #include #include -#include #include #include #include From faae8a4f2b683eed530b74f92ab58d1a76b5d001 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 24 Apr 2024 20:37:06 +0200 Subject: [PATCH 155/624] Add tests for backup on cluster with 2 shards and 2 replicas. --- .../configs/cluster_2x2.xml | 26 +++ .../test_backup_restore_on_cluster/test.py | 1 - .../test_two_shards_two_replicas.py | 153 ++++++++++++++++++ 3 files changed, 179 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_backup_restore_on_cluster/configs/cluster_2x2.xml create mode 100644 tests/integration/test_backup_restore_on_cluster/test_two_shards_two_replicas.py diff --git a/tests/integration/test_backup_restore_on_cluster/configs/cluster_2x2.xml b/tests/integration/test_backup_restore_on_cluster/configs/cluster_2x2.xml new file mode 100644 index 00000000000..97e60fbbed7 --- /dev/null +++ b/tests/integration/test_backup_restore_on_cluster/configs/cluster_2x2.xml @@ -0,0 +1,26 @@ + + + + + + node_1_1 + 9000 + + + node_1_2 + 9000 + + + + + node_2_1 + 9000 + + + node_2_2 + 9000 + + + + + diff --git a/tests/integration/test_backup_restore_on_cluster/test.py b/tests/integration/test_backup_restore_on_cluster/test.py index d1520444df1..700ed6f15f5 100644 --- a/tests/integration/test_backup_restore_on_cluster/test.py +++ b/tests/integration/test_backup_restore_on_cluster/test.py @@ -41,7 +41,6 @@ node2 = cluster.add_instance( stay_alive=True, # Necessary for the "test_stop_other_host_while_backup" test ) - node3 = cluster.add_instance( "node3", main_configs=main_configs, diff --git a/tests/integration/test_backup_restore_on_cluster/test_two_shards_two_replicas.py b/tests/integration/test_backup_restore_on_cluster/test_two_shards_two_replicas.py new file mode 100644 index 00000000000..c0e318c8bb7 --- /dev/null +++ b/tests/integration/test_backup_restore_on_cluster/test_two_shards_two_replicas.py @@ -0,0 +1,153 @@ +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + + +cluster = ClickHouseCluster(__file__) + +main_configs = [ + "configs/backups_disk.xml", + "configs/cluster_2x2.xml", + "configs/lesser_timeouts.xml", # Default timeouts are quite big (a few minutes), the tests don't need them to be that big. +] + +user_configs = [ + "configs/zookeeper_retries.xml", +] + +node_1_1 = cluster.add_instance( + "node_1_1", + main_configs=main_configs, + user_configs=user_configs, + external_dirs=["/backups/"], + macros={"replica": "1", "shard": "1"}, + with_zookeeper=True, +) + +node_1_2 = cluster.add_instance( + "node_1_2", + main_configs=main_configs, + user_configs=user_configs, + external_dirs=["/backups/"], + macros={"replica": "2", "shard": "1"}, + with_zookeeper=True, +) + +node_2_1 = cluster.add_instance( + "node_2_1", + main_configs=main_configs, + user_configs=user_configs, + external_dirs=["/backups/"], + macros={"replica": "1", "shard": "2"}, + with_zookeeper=True, +) + +node_2_2 = cluster.add_instance( + "node_2_2", + main_configs=main_configs, + user_configs=user_configs, + external_dirs=["/backups/"], + macros={"replica": "2", "shard": "2"}, + with_zookeeper=True, +) + + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def drop_after_test(): + try: + yield + finally: + node_1_1.query("DROP TABLE IF EXISTS tbl ON CLUSTER 'cluster_2x2' SYNC") + node_1_1.query("DROP TABLE IF EXISTS table_a ON CLUSTER 'cluster_2x2' SYNC") + node_1_1.query("DROP TABLE IF EXISTS table_b ON CLUSTER 'cluster_2x2' SYNC") + + +backup_id_counter = 0 + + +def new_backup_name(): + global backup_id_counter + backup_id_counter += 1 + return f"Disk('backups', '{backup_id_counter}')" + + +def test_replicated_table(): + node_1_1.query( + "CREATE TABLE tbl ON CLUSTER 'cluster_2x2' (" + "x Int64" + ") ENGINE=ReplicatedMergeTree('/clickhouse/tables/tbl/{shard}', '{replica}')" + "ORDER BY x" + ) + + node_1_1.query("INSERT INTO tbl VALUES (100), (200)") + node_2_1.query("INSERT INTO tbl VALUES (300), (400)") + + backup_name = new_backup_name() + + node_1_1.query(f"BACKUP TABLE tbl ON CLUSTER 'cluster_2x2' TO {backup_name}") + + node_1_1.query(f"DROP TABLE tbl ON CLUSTER 'cluster_2x2' SYNC") + + node_1_1.query(f"RESTORE ALL ON CLUSTER 'cluster_2x2' FROM {backup_name}") + + node_1_1.query("SYSTEM SYNC REPLICA ON CLUSTER 'cluster_2x2' tbl") + + assert node_1_1.query("SELECT * FROM tbl ORDER BY x") == TSV([[100], [200]]) + assert node_1_2.query("SELECT * FROM tbl ORDER BY x") == TSV([[100], [200]]) + assert node_2_1.query("SELECT * FROM tbl ORDER BY x") == TSV([[300], [400]]) + assert node_2_2.query("SELECT * FROM tbl ORDER BY x") == TSV([[300], [400]]) + + +def test_two_tables_with_uuid_in_zk_path(): + node_1_1.query( + "CREATE TABLE table_a ON CLUSTER 'cluster_2x2' (" + "x Int64" + ") ENGINE=ReplicatedMergeTree('/clickhouse/tables/{uuid}/{shard}', '{replica}')" + "ORDER BY x" + ) + + node_1_1.query( + "CREATE TABLE table_b ON CLUSTER 'cluster_2x2' (" + "x Int64" + ") ENGINE=ReplicatedMergeTree('/clickhouse/tables/{uuid}/{shard}', '{replica}')" + "ORDER BY x" + ) + + node_1_1.query("INSERT INTO table_a VALUES (100), (200)") + node_2_1.query("INSERT INTO table_a VALUES (300), (400)") + + node_1_2.query("INSERT INTO table_b VALUES (500), (600)") + node_2_2.query("INSERT INTO table_b VALUES (700), (800)") + + backup_name = new_backup_name() + + node_1_1.query( + f"BACKUP TABLE table_a, TABLE table_b ON CLUSTER 'cluster_2x2' TO {backup_name}" + ) + + node_1_1.query(f"DROP TABLE table_a ON CLUSTER 'cluster_2x2' SYNC") + node_1_1.query(f"DROP TABLE table_b ON CLUSTER 'cluster_2x2' SYNC") + + node_1_1.query(f"RESTORE ALL ON CLUSTER 'cluster_2x2' FROM {backup_name}") + + node_1_1.query("SYSTEM SYNC REPLICA ON CLUSTER 'cluster_2x2' table_a") + node_1_1.query("SYSTEM SYNC REPLICA ON CLUSTER 'cluster_2x2' table_b") + + assert node_1_1.query("SELECT * FROM table_a ORDER BY x") == TSV([[100], [200]]) + assert node_1_2.query("SELECT * FROM table_a ORDER BY x") == TSV([[100], [200]]) + assert node_2_1.query("SELECT * FROM table_a ORDER BY x") == TSV([[300], [400]]) + assert node_2_2.query("SELECT * FROM table_a ORDER BY x") == TSV([[300], [400]]) + + assert node_1_1.query("SELECT * FROM table_b ORDER BY x") == TSV([[500], [600]]) + assert node_1_2.query("SELECT * FROM table_b ORDER BY x") == TSV([[500], [600]]) + assert node_2_1.query("SELECT * FROM table_b ORDER BY x") == TSV([[700], [800]]) + assert node_2_2.query("SELECT * FROM table_b ORDER BY x") == TSV([[700], [800]]) From f2ab23780fbcac3d15f6b6d5cbda1d013286c98e Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Fri, 26 Apr 2024 16:24:06 +0800 Subject: [PATCH 156/624] fix bug and add test case --- src/Interpreters/MergeJoin.cpp | 2 +- .../Transforms/MergeSortingTransform.cpp | 5 +- .../performance/external_sort_wide_table.xml | 354 ++++++++++++++++++ 3 files changed, 356 insertions(+), 5 deletions(-) create mode 100644 tests/performance/external_sort_wide_table.xml diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 24d403a782a..5bd49b3c971 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -604,7 +604,7 @@ void MergeJoin::mergeInMemoryRightBlocks() /// TODO: there should be no split keys by blocks for RIGHT|FULL JOIN builder.addTransform(std::make_shared( - builder.getHeader(), right_sort_description, 0, max_rows_in_right_block, 0, false, 0, 0, 0, nullptr, 0)); + builder.getHeader(), right_sort_description, max_rows_in_right_block, 0, 0, false, 0, 0, 0, nullptr, 0)); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); PullingPipelineExecutor executor(pipeline); diff --git a/src/Processors/Transforms/MergeSortingTransform.cpp b/src/Processors/Transforms/MergeSortingTransform.cpp index 304b79e6c59..ede13b29219 100644 --- a/src/Processors/Transforms/MergeSortingTransform.cpp +++ b/src/Processors/Transforms/MergeSortingTransform.cpp @@ -172,14 +172,11 @@ void MergeSortingTransform::consume(Chunk chunk) size_t reserve_size = sum_bytes_in_blocks + min_free_disk_space; auto & tmp_stream = tmp_data->createStream(header_without_constants, reserve_size); size_t max_merged_block_size = this->max_merged_block_size; - if (max_merged_block_size > 0) + if (max_block_bytes > 0 && sum_rows_in_blocks > 0 && sum_bytes_in_blocks > 0) { auto avg_row_bytes = sum_bytes_in_blocks / sum_rows_in_blocks; /// max_merged_block_size >= 128 max_merged_block_size = std::max(std::min(max_merged_block_size, max_block_bytes / avg_row_bytes), 128UL); - /// when max_block_size < 128, use max_block_size. - /// max_block_size still works. - max_merged_block_size = std::min(this->max_merged_block_size, max_merged_block_size); } merge_sorter = std::make_unique(header_without_constants, std::move(chunks), description, max_merged_block_size, limit); auto current_processor = std::make_shared(header_without_constants, tmp_stream, log); diff --git a/tests/performance/external_sort_wide_table.xml b/tests/performance/external_sort_wide_table.xml new file mode 100644 index 00000000000..4c6220a4503 --- /dev/null +++ b/tests/performance/external_sort_wide_table.xml @@ -0,0 +1,354 @@ + + + create table test_sort + ( + val00 Int64, + val01 Int64, + val02 Int64, + val03 Int64, + val04 Int64, + val05 Int64, + val06 Int64, + val07 Int64, + val08 Int64, + val09 Int64, + val10 Int64, + val11 Int64, + val12 Int64, + val13 Int64, + val14 Int64, + val15 Int64, + val16 Int64, + val17 Int64, + val18 Int64, + val19 Int64, + val20 Int64, + val21 Int64, + val22 Int64, + val23 Int64, + val24 Int64, + val25 Int64, + val26 Int64, + val27 Int64, + val28 Int64, + val29 Int64, + val30 Int64, + val31 Int64, + val32 Int64, + val33 Int64, + val34 Int64, + val35 Int64, + val36 Int64, + val37 Int64, + val38 Int64, + val39 Int64, + val40 Int64, + val41 Int64, + val42 Int64, + val43 Int64, + val44 Int64, + val45 Int64, + val46 Int64, + val47 Int64, + val48 Int64, + val49 Int64, + val50 Int64, + val51 Int64, + val52 Int64, + val53 Int64, + val54 Int64, + val55 Int64, + val56 Int64, + val57 Int64, + val58 Int64, + val59 Int64, + val60 Int64, + val61 Int64, + val62 Int64, + val63 Int64, + val64 Int64, + val65 Int64, + val66 Int64, + val67 Int64, + val68 Int64, + val69 Int64, + val70 Int64, + val71 Int64, + val72 Int64, + val73 Int64, + val74 Int64, + val75 Int64, + val76 Int64, + val77 Int64, + val78 Int64, + val79 Int64, + val80 Int64, + val81 Int64, + val82 Int64, + val83 Int64, + val84 Int64, + val85 Int64, + val86 Int64, + val87 Int64, + val88 Int64, + val89 Int64, + val90 Int64, + val91 Int64, + val92 Int64, + val93 Int64, + val94 Int64, + val95 Int64, + val96 Int64, + val97 Int64, + val98 Int64, + val99 Int64, + val100 Int64, + val101 Int64, + val102 Int64, + val103 Int64, + val104 Int64, + val105 Int64, + val106 Int64, + val107 Int64, + val108 Int64, + val109 Int64, + val110 Int64, + val111 Int64, + val112 Int64, + val113 Int64, + val114 Int64, + val115 Int64, + val116 Int64, + val117 Int64, + val118 Int64, + val119 Int64, + val120 Int64, + val121 Int64, + val122 Int64, + val123 Int64, + val124 Int64, + val125 Int64, + val126 Int64, + val127 Int64, + val128 Int64, + val129 Int64, + val130 Int64, + val131 Int64, + val132 Int64, + val133 Int64, + val134 Int64, + val135 Int64, + val136 Int64, + val137 Int64, + val138 Int64, + val139 Int64, + val140 Int64, + val141 Int64, + val142 Int64, + val143 Int64, + val144 Int64, + val145 Int64, + val146 Int64, + val147 Int64, + val148 Int64, + val149 Int64, + val150 Int64, + val151 Int64, + val152 Int64, + val153 Int64, + val154 Int64, + val155 Int64, + val156 Int64, + val157 Int64, + val158 Int64, + val159 Int64, + val160 Int64, + val161 Int64, + val162 Int64, + val163 Int64, + val164 Int64, + val165 Int64, + val166 Int64, + val167 Int64, + val168 Int64, + val169 Int64, + val170 Int64, + val171 Int64, + val172 Int64, + val173 Int64, + val174 Int64, + val175 Int64, + val176 Int64, + val177 Int64, + val178 Int64, + val179 Int64, + val180 Int64, + val181 Int64, + val182 Int64, + val183 Int64, + val184 Int64, + val185 Int64, + val186 Int64, + val187 Int64, + val188 Int64, + val189 Int64, + val190 Int64, + val191 Int64, + val192 Int64, + val193 Int64, + val194 Int64, + val195 Int64, + val196 Int64, + val197 Int64, + val198 Int64, + val199 Int64, + val200 Int64, + val201 Int64, + val202 Int64, + val203 Int64, + val204 Int64, + val205 Int64, + val206 Int64, + val207 Int64, + val208 Int64, + val209 Int64, + val210 Int64, + val211 Int64, + val212 Int64, + val213 Int64, + val214 Int64, + val215 Int64, + val216 Int64, + val217 Int64, + val218 Int64, + val219 Int64, + val220 Int64, + val221 Int64, + val222 Int64, + val223 Int64, + val224 Int64, + val225 Int64, + val226 Int64, + val227 Int64, + val228 Int64, + val229 Int64, + val230 Int64, + val231 Int64, + val232 Int64, + val233 Int64, + val234 Int64, + val235 Int64, + val236 Int64, + val237 Int64, + val238 Int64, + val239 Int64, + val240 Int64, + val241 Int64, + val242 Int64, + val243 Int64, + val244 Int64, + val245 Int64, + val246 Int64, + val247 Int64, + val248 Int64, + val249 Int64, + val250 Int64, + val251 Int64, + val252 Int64, + val253 Int64, + val254 Int64, + val255 Int64, + val256 Int64, + val257 Int64, + val258 Int64, + val259 Int64, + val260 Int64, + val261 Int64, + val262 Int64, + val263 Int64, + val264 Int64, + val265 Int64, + val266 Int64, + val267 Int64, + val268 Int64, + val269 Int64, + val270 Int64, + val271 Int64, + val272 Int64, + val273 Int64, + val274 Int64, + val275 Int64, + val276 Int64, + val277 Int64, + val278 Int64, + val279 Int64, + val280 Int64, + val281 Int64, + val282 Int64, + val283 Int64, + val284 Int64, + val285 Int64, + val286 Int64, + val287 Int64, + val288 Int64, + val289 Int64, + val290 Int64, + val291 Int64, + val292 Int64, + val293 Int64, + val294 Int64, + val295 Int64, + val296 Int64, + val297 Int64, + val298 Int64, + val299 Int64 + ) engine = MergeTree() order by tuple(); + + + + insert into test_sort select + number, number, number, number, number, number, number, number, number, number, + number, number, number, number, number, number, number, number, number, number, + number, number, number, number, number, number, number, number, number, number, + number, number, number, number, number, number, number, number, number, number, + number, number, number, number, number, number, number, number, number, number, + number, number, number, number, number, number, number, number, number, number, + number, number, number, number, number, number, number, number, number, number, + number, number, number, number, number, number, number, number, number, number, + number, number, number, number, number, number, number, number, number, number, + number, number, number, number, number, number, number, number, number, number, + number, number, number, number, number, number, number, number, number, number, + number, number, number, number, number, number, number, number, number, number, + number, number, number, number, number, number, number, number, number, number, + number, number, number, number, number, number, number, number, number, number, + number, number, number, number, number, number, number, number, number, number, + number, number, number, number, number, number, number, number, number, number, + number, number, number, number, number, number, number, number, number, number, + number, number, number, number, number, number, number, number, number, number, + number, number, number, number, number, number, number, number, number, number, + number, number, number, number, number, number, number, number, number, number, + number, number, number, number, number, number, number, number, number, number, + number, number, number, number, number, number, number, number, number, number, + number, number, number, number, number, number, number, number, number, number, + number, number, number, number, number, number, number, number, number, number, + number, number, number, number, number, number, number, number, number, number, + number, number, number, number, number, number, number, number, number, number, + number, number, number, number, number, number, number, number, number, number, + number, number, number, number, number, number, number, number, number, number, + number, number, number, number, number, number, number, number, number, number, + number, number, number, number, number, number, number, number, number, number + from numbers(1000000); + + + + 1 + + + + select * from test_sort order by val00 settings max_bytes_before_external_sort=50000000 format Null; + + + + drop table if exists test_sort + + From bff72f3b2753e88aaa1b703258ad904ad11fc4ee Mon Sep 17 00:00:00 2001 From: skyoct Date: Fri, 26 Apr 2024 09:06:17 +0000 Subject: [PATCH 157/624] batter --- src/Functions/clamp.cpp | 2 +- tests/queries/0_stateless/03036_clamp.sql | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/clamp.cpp b/src/Functions/clamp.cpp index 3438377afbf..448711aed27 100644 --- a/src/Functions/clamp.cpp +++ b/src/Functions/clamp.cpp @@ -47,7 +47,7 @@ public: for (size_t row_num = 0; row_num < input_rows_count; ++row_num) { if (converted_columns[1]->compareAt(row_num, row_num, *converted_columns[2], 1) > 0) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Function {} the minimum value cannot be greater than the maximum value", getName()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The minimum value cannot be greater than the maximum value for function {}", getName()); size_t best_arg = 0; if (converted_columns[1]->compareAt(row_num, row_num, *converted_columns[best_arg], 1) > 0) diff --git a/tests/queries/0_stateless/03036_clamp.sql b/tests/queries/0_stateless/03036_clamp.sql index d225be63f46..0ca1f99572a 100644 --- a/tests/queries/0_stateless/03036_clamp.sql +++ b/tests/queries/0_stateless/03036_clamp.sql @@ -10,6 +10,6 @@ select clamp(1, null, 5); select clamp(1, 6, null); select clamp(1, 5, nan); select clamp(toInt64(number), toInt64(number-1), toInt64(number+1)) from numbers(3); -select clamp(number, number-1, number+1) from numbers(3); -- { serverError 386 } -select clamp(1, 3, 2); -- { serverError 36 } +select clamp(number, number-1, number+1) from numbers(3); -- { serverError NO_COMMON_TYPE } +select clamp(1, 3, 2); -- { serverError BAD_ARGUMENTS } select clamp(1, data[1], data[2])from (select arrayJoin([[1, 2], [2,3], [3,2], [4, 4]]) as data); -- { serverError 36 } From d8ace22c3a585db0eab810bd5f871145402a11b7 Mon Sep 17 00:00:00 2001 From: Blargian Date: Sat, 27 Apr 2024 15:14:58 +0200 Subject: [PATCH 158/624] Implement review changes --- ...InputFormat.cpp => FormRowInputFormat.cpp} | 58 +++++++++---------- ...FormInputFormat.h => FormRowInputFormat.h} | 7 +-- 2 files changed, 30 insertions(+), 35 deletions(-) rename src/Processors/Formats/Impl/{FormInputFormat.cpp => FormRowInputFormat.cpp} (72%) rename src/Processors/Formats/Impl/{FormInputFormat.h => FormRowInputFormat.h} (83%) diff --git a/src/Processors/Formats/Impl/FormInputFormat.cpp b/src/Processors/Formats/Impl/FormRowInputFormat.cpp similarity index 72% rename from src/Processors/Formats/Impl/FormInputFormat.cpp rename to src/Processors/Formats/Impl/FormRowInputFormat.cpp index b3b62e21a99..242a14a7690 100644 --- a/src/Processors/Formats/Impl/FormInputFormat.cpp +++ b/src/Processors/Formats/Impl/FormRowInputFormat.cpp @@ -1,4 +1,4 @@ -#include +#include #include "Formats/EscapingRuleUtils.h" #include @@ -10,7 +10,18 @@ namespace ErrorCodes extern const int INCORRECT_DATA; } -FormInputFormat::FormInputFormat(ReadBuffer & in_, Block header_, Params params_, const FormatSettings & format_settings_) : IRowInputFormat(std::move(header_), in_, params_), format_settings(format_settings_) +namespace +{ + String readFieldName(ReadBuffer & buf) + { + String field; + readStringUntilEquals(field, buf); + assertChar('=', buf); + return field; + } +} + +FormRowInputFormat::FormRowInputFormat(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(); size_t num_columns = header.columns(); @@ -18,44 +29,36 @@ FormInputFormat::FormInputFormat(ReadBuffer & in_, Block header_, Params params_ name_map[header.getByPosition(i).name] = i; } -void FormInputFormat::readPrefix() +void FormRowInputFormat::readPrefix() { skipBOMIfExists(*in); } -const String & FormInputFormat::columnName(size_t i) const +const String & FormRowInputFormat::columnName(size_t i) const { return getPort().getHeader().getByPosition(i).name; } -void FormInputFormat::readField(size_t index, MutableColumns & columns) +void FormRowInputFormat::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] = read_columns[index] = true; + seen_columns[index] = true; const auto & serialization = serializations[index]; String encoded_str, decoded_str; readStringUntilAmpersand(encoded_str,*in); if (!in->eof()) - assertChar('&',*in); + assertChar('&', *in); Poco::URI::decode(encoded_str, decoded_str); ReadBufferFromString buf(decoded_str); serialization->deserializeWholeText(*columns[index], buf, format_settings); } -String readFieldName(ReadBuffer & buf) -{ - String field; - readStringUntilEquals(field, buf); - assertChar('=', buf); - return field; -} - -void FormInputFormat::readFormData(MutableColumns & columns) +void FormRowInputFormat::readFormData(MutableColumns & columns) { size_t index = 0; StringRef name_ref; @@ -74,15 +77,12 @@ void FormInputFormat::readFormData(MutableColumns & columns) throw Exception(ErrorCodes::INCORRECT_DATA, "Unknown field found while parsing Form format: {}", name_ref.toString()); /// Skip the value if key is not found. - NullOutput sink; String encoded_str; - readStringUntilAmpersand(encoded_str,*in); + readStringUntilAmpersand(encoded_str, *in); if (!in->eof()) assertChar('&',*in); - ReadBufferFromString buf(encoded_str); - readStringInto(sink, buf); } else { @@ -92,14 +92,12 @@ void FormInputFormat::readFormData(MutableColumns & columns) } } -bool FormInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext) +bool FormRowInputFormat::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); readFormData(columns); @@ -113,16 +111,15 @@ bool FormInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext) /// 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; + ext.read_columns = seen_columns; else - ext.read_columns.assign(read_columns.size(), true); + ext.read_columns.assign(seen_columns.size(), true); return true; } -void FormInputFormat::resetParser() +void FormRowInputFormat::resetParser() { IRowInputFormat::resetParser(); - read_columns.clear(); seen_columns.clear(); } @@ -134,12 +131,13 @@ FormSchemaReader::FormSchemaReader(ReadBuffer & in_, const FormatSettings & form NamesAndTypesList readRowAndGetNamesAndDataTypesForFormRow(ReadBuffer & in, const FormatSettings & settings) { NamesAndTypesList names_and_types; - String field, value; + String field, value, decoded_value; do { auto name = readFieldName(in); readStringUntilAmpersand(value,in); - auto type = tryInferDataTypeByEscapingRule(value, settings, FormatSettings::EscapingRule::Raw); + Poco::URI::decode(value, decoded_value); + auto type = tryInferDataTypeByEscapingRule(decoded_value, settings, FormatSettings::EscapingRule::Raw); names_and_types.emplace_back(name, type); } while (checkChar('&',in)); @@ -164,7 +162,7 @@ void registerInputFormatForm(FormatFactory & factory) IRowInputFormat::Params params, const FormatSettings & settings) { - return std::make_shared(buf, sample, std::move(params),settings); + return std::make_shared(buf, sample, std::move(params),settings); }); } diff --git a/src/Processors/Formats/Impl/FormInputFormat.h b/src/Processors/Formats/Impl/FormRowInputFormat.h similarity index 83% rename from src/Processors/Formats/Impl/FormInputFormat.h rename to src/Processors/Formats/Impl/FormRowInputFormat.h index 93edf49e258..00a90f39e8e 100644 --- a/src/Processors/Formats/Impl/FormInputFormat.h +++ b/src/Processors/Formats/Impl/FormRowInputFormat.h @@ -10,10 +10,10 @@ namespace DB class ReadBuffer; -class FormInputFormat final : public IRowInputFormat +class FormRowInputFormat final : public IRowInputFormat { public: - FormInputFormat(ReadBuffer & in_, Block header_, Params params_, const FormatSettings & format_settings_); + FormRowInputFormat(ReadBuffer & in_, Block header_, Params params_, const FormatSettings & format_settings_); String getName() const override { return "FormInputFormat"; } void resetParser() override; @@ -30,7 +30,6 @@ private: protected: const FormatSettings format_settings; - std::vector read_columns; std::vector seen_columns; }; @@ -43,7 +42,5 @@ private: NamesAndTypesList readRowAndGetNamesAndDataTypesForForm(ReadBuffer & in, const FormatSettings & settings); }; -String readFieldName(ReadBuffer & buf); - } From 6a8852f942e8b565c129f4e4e63bc3d81927a502 Mon Sep 17 00:00:00 2001 From: Blargian Date: Sat, 27 Apr 2024 15:26:35 +0200 Subject: [PATCH 159/624] Fix style check --- src/Processors/Formats/Impl/FormRowInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/FormRowInputFormat.cpp b/src/Processors/Formats/Impl/FormRowInputFormat.cpp index 242a14a7690..448c9e52cce 100644 --- a/src/Processors/Formats/Impl/FormRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/FormRowInputFormat.cpp @@ -10,7 +10,7 @@ namespace ErrorCodes extern const int INCORRECT_DATA; } -namespace +namespace { String readFieldName(ReadBuffer & buf) { From 14e32aa1a1029f25fd04d0b278ea8bfe661c0b4d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Apr 2024 18:23:14 +0200 Subject: [PATCH 160/624] Less directories in clickhouse-local --- programs/local/LocalServer.cpp | 18 +++++++----------- .../NamedCollections/NamedCollectionUtils.cpp | 15 +++++++++------ 2 files changed, 16 insertions(+), 17 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 612c62613ca..165800b6eed 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -211,7 +211,7 @@ void LocalServer::tryInitPath() else { // The path is not provided explicitly - use a unique path in the system temporary directory - // (or in the current dir if temporary don't exist) + // (or in the current dir if the temporary doesn't exist) LoggerRawPtr log = &logger(); std::filesystem::path parent_folder; std::filesystem::path default_path; @@ -249,27 +249,23 @@ void LocalServer::tryInitPath() LOG_DEBUG(log, "Working directory created: {}", path); } - if (path.back() != '/') - path += '/'; - - fs::create_directories(fs::path(path) / "user_defined/"); + fs::create_directories(config().getString("user_defined_path", fs::path(path) / "user_defined/")); fs::create_directories(fs::path(path) / "data/"); fs::create_directories(fs::path(path) / "metadata/"); fs::create_directories(fs::path(path) / "metadata_dropped/"); - global_context->setPath(path); + global_context->setPath(fs::path(path) / ""); - global_context->setTemporaryStoragePath(path + "tmp/", 0); - global_context->setFlagsPath(path + "flags"); + global_context->setTemporaryStoragePath(fs::path(path) / "tmp/", 0); + global_context->setFlagsPath(fs::path(path) / "flags"); - global_context->setUserFilesPath(""); // user's files are everywhere + global_context->setUserFilesPath(""); /// user's files are everywhere std::string user_scripts_path = config().getString("user_scripts_path", fs::path(path) / "user_scripts/"); global_context->setUserScriptsPath(user_scripts_path); - fs::create_directories(user_scripts_path); /// top_level_domains_lists - const std::string & top_level_domains_path = config().getString("top_level_domains_path", path + "top_level_domains/"); + const std::string & top_level_domains_path = config().getString("top_level_domains_path", fs::path(path) / "top_level_domains/"); if (!top_level_domains_path.empty()) TLDListsHolder::getInstance().parseConfig(fs::path(top_level_domains_path) / "", config()); } diff --git a/src/Common/NamedCollections/NamedCollectionUtils.cpp b/src/Common/NamedCollections/NamedCollectionUtils.cpp index 317ca3c84ac..eb851a23501 100644 --- a/src/Common/NamedCollections/NamedCollectionUtils.cpp +++ b/src/Common/NamedCollections/NamedCollectionUtils.cpp @@ -117,13 +117,14 @@ public: fs::canonical(context_->getPath()) / NAMED_COLLECTIONS_METADATA_DIRECTORY) { if (fs::exists(metadata_path)) - cleanUp(); - else - fs::create_directories(metadata_path); + cleanup(); } std::vector listCollections() const { + if (!fs::exists(metadata_path)) + return {}; + std::vector collection_names; fs::directory_iterator it{metadata_path}; for (; it != fs::directory_iterator{}; ++it) @@ -280,7 +281,7 @@ private: /// Delete .tmp files. They could be left undeleted in case of /// some exception or abrupt server restart. - void cleanUp() + void cleanup() { fs::directory_iterator it{metadata_path}; std::vector files_to_remove; @@ -308,11 +309,11 @@ private: return create_query; } - static void writeCreateQueryToMetadata( + void writeCreateQueryToMetadata( const ASTCreateNamedCollectionQuery & query, const std::string & path, const Settings & settings, - bool replace = false) + bool replace = false) const { if (!replace && fs::exists(path)) { @@ -322,6 +323,8 @@ private: path); } + fs::create_directories(metadata_path); + auto tmp_path = path + ".tmp"; String formatted_query = serializeAST(query); WriteBufferFromFile out(tmp_path, formatted_query.size(), O_WRONLY | O_CREAT | O_EXCL); From 2fafae45beeade23f0d0a15408fa164c373bcba4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Apr 2024 18:33:34 +0200 Subject: [PATCH 161/624] Fix removal of temporary directories --- programs/local/LocalServer.cpp | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 165800b6eed..dbf10e59038 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -295,10 +295,9 @@ void LocalServer::cleanup() // Delete the temporary directory if needed. if (temporary_directory_to_delete) { - const auto dir = *temporary_directory_to_delete; + LOG_DEBUG(&logger(), "Removing temporary directory: {}", temporary_directory_to_delete->string()); + fs::remove_all(*temporary_directory_to_delete); temporary_directory_to_delete.reset(); - LOG_DEBUG(&logger(), "Removing temporary directory: {}", dir.string()); - remove_all(dir); } } catch (...) @@ -477,6 +476,9 @@ try registerFormats(); processConfig(); + + SCOPE_EXIT({ cleanup(); }) + adjustSettings(); initTTYBuffer(toProgressOption(config().getString("progress", "default"))); ASTAlterCommand::setFormatAlterCommandsWithParentheses(true); @@ -527,16 +529,12 @@ try } catch (const DB::Exception & e) { - cleanup(); - bool need_print_stack_trace = config().getBool("stacktrace", false); std::cerr << getExceptionMessage(e, need_print_stack_trace, true) << std::endl; return e.code() ? e.code() : -1; } catch (...) { - cleanup(); - std::cerr << getCurrentExceptionMessage(false) << std::endl; return getCurrentExceptionCode(); } From 56102603994ec83ee5370e247a249d29dc7c3c11 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Apr 2024 18:36:46 +0200 Subject: [PATCH 162/624] Fix removal of temporary directories --- programs/local/LocalServer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index dbf10e59038..d6999aef195 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -477,7 +477,7 @@ try processConfig(); - SCOPE_EXIT({ cleanup(); }) + SCOPE_EXIT({ cleanup(); }); adjustSettings(); initTTYBuffer(toProgressOption(config().getString("progress", "default"))); From 39118ef4610c76071d1a3be6cabf22142dbddd3f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Apr 2024 18:47:03 +0200 Subject: [PATCH 163/624] Hopefully better --- programs/local/LocalServer.cpp | 11 +++++------ programs/server/Server.cpp | 6 +++--- src/Databases/DatabaseAtomic.cpp | 3 +++ src/Interpreters/DatabaseCatalog.cpp | 11 ++++++----- 4 files changed, 17 insertions(+), 14 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index d6999aef195..cb71886b522 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -249,15 +249,14 @@ void LocalServer::tryInitPath() LOG_DEBUG(log, "Working directory created: {}", path); } - fs::create_directories(config().getString("user_defined_path", fs::path(path) / "user_defined/")); - fs::create_directories(fs::path(path) / "data/"); - fs::create_directories(fs::path(path) / "metadata/"); - fs::create_directories(fs::path(path) / "metadata_dropped/"); + fs::create_directories(config().getString("user_defined_path", fs::path(path) / "user_defined" / "")); + fs::create_directories(fs::path(path) / "data" / ""); + fs::create_directories(fs::path(path) / "metadata" / ""); global_context->setPath(fs::path(path) / ""); - global_context->setTemporaryStoragePath(fs::path(path) / "tmp/", 0); - global_context->setFlagsPath(fs::path(path) / "flags"); + global_context->setTemporaryStoragePath(fs::path(path) / "tmp" / "", 0); + global_context->setFlagsPath(fs::path(path) / "flags" / ""); global_context->setUserFilesPath(""); /// user's files are everywhere diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 260dcbe4c16..387e68ae4be 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1205,11 +1205,11 @@ try } { - fs::create_directories(path / "data/"); - fs::create_directories(path / "metadata/"); + fs::create_directories(path / "data"); + fs::create_directories(path / "metadata"); /// Directory with metadata of tables, which was marked as dropped by Atomic database - fs::create_directories(path / "metadata_dropped/"); + fs::create_directories(path / "metadata_dropped"); } if (config().has("interserver_http_port") && config().has("interserver_https_port")) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index c2d0fbe1c00..ada06e3b035 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -137,6 +137,9 @@ void DatabaseAtomic::dropTableImpl(ContextPtr local_context, const String & tabl std::lock_guard lock(mutex); table = getTableUnlocked(table_name); table_metadata_path_drop = DatabaseCatalog::instance().getPathForDroppedMetadata(table->getStorageID()); + + fs::create_directory(fs::path(table_metadata_path_drop).parent_path()); + auto txn = local_context->getZooKeeperMetadataTransaction(); if (txn && !local_context->isInternalSubquery()) txn->commit(); /// Commit point (a sort of) for Replicated database diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 0caca88c283..a9b0e7e7a98 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -988,7 +988,7 @@ void DatabaseCatalog::loadMarkedAsDroppedTables() /// we should load them and enqueue cleanup to remove data from store/ and metadata from ZooKeeper std::map dropped_metadata; - String path = getContext()->getPath() + "metadata_dropped/"; + String path = std::filesystem::path(getContext()->getPath()) / "metadata_dropped" / ""; if (!std::filesystem::exists(path)) { @@ -1043,10 +1043,11 @@ void DatabaseCatalog::loadMarkedAsDroppedTables() String DatabaseCatalog::getPathForDroppedMetadata(const StorageID & table_id) const { - return getContext()->getPath() + "metadata_dropped/" + - escapeForFileName(table_id.getDatabaseName()) + "." + - escapeForFileName(table_id.getTableName()) + "." + - toString(table_id.uuid) + ".sql"; + return std::filesystem::path(getContext()->getPath()) / "metadata_dropped" / + fmt::format("{}.{}.{}.sql", + escapeForFileName(table_id.getDatabaseName()), + escapeForFileName(table_id.getTableName()), + toString(table_id.uuid)); } String DatabaseCatalog::getPathForMetadata(const StorageID & table_id) const From 089cc25d9bdd8b7f05335f2405766e77bc3a5d48 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Apr 2024 19:05:58 +0200 Subject: [PATCH 164/624] Less directories --- programs/local/LocalServer.cpp | 1 - .../UserDefined/createUserDefinedSQLObjectsStorage.cpp | 10 ++++++---- .../UserDefined/createUserDefinedSQLObjectsStorage.h | 1 + 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index cb71886b522..9efe80ad1ce 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -249,7 +249,6 @@ void LocalServer::tryInitPath() LOG_DEBUG(log, "Working directory created: {}", path); } - fs::create_directories(config().getString("user_defined_path", fs::path(path) / "user_defined" / "")); fs::create_directories(fs::path(path) / "data" / ""); fs::create_directories(fs::path(path) / "metadata" / ""); diff --git a/src/Functions/UserDefined/createUserDefinedSQLObjectsStorage.cpp b/src/Functions/UserDefined/createUserDefinedSQLObjectsStorage.cpp index f8847024508..0b1a74884cf 100644 --- a/src/Functions/UserDefined/createUserDefinedSQLObjectsStorage.cpp +++ b/src/Functions/UserDefined/createUserDefinedSQLObjectsStorage.cpp @@ -35,10 +35,12 @@ std::unique_ptr createUserDefinedSQLObjectsStorag } return std::make_unique(global_context, config.getString(zookeeper_path_key)); } - - String default_path = fs::path{global_context->getPath()} / "user_defined/"; - String path = config.getString(disk_path_key, default_path); - return std::make_unique(global_context, path); + else + { + String default_path = fs::path{global_context->getPath()} / "user_defined" / ""; + String path = config.getString(disk_path_key, default_path); + return std::make_unique(global_context, path); + } } } diff --git a/src/Functions/UserDefined/createUserDefinedSQLObjectsStorage.h b/src/Functions/UserDefined/createUserDefinedSQLObjectsStorage.h index 01659372dec..fc1d79adcf7 100644 --- a/src/Functions/UserDefined/createUserDefinedSQLObjectsStorage.h +++ b/src/Functions/UserDefined/createUserDefinedSQLObjectsStorage.h @@ -5,6 +5,7 @@ namespace DB { + class IUserDefinedSQLObjectsStorage; std::unique_ptr createUserDefinedSQLObjectsStorage(const ContextMutablePtr & global_context); From 6a083ebed0a695246e7f8d2eff4b0ef1f97a7f82 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Apr 2024 19:13:17 +0200 Subject: [PATCH 165/624] Even less directories --- programs/local/LocalServer.cpp | 3 --- src/Disks/DiskLocal.cpp | 4 ++-- src/Interpreters/Context.cpp | 2 -- 3 files changed, 2 insertions(+), 7 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 9efe80ad1ce..df27c46e56c 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -249,9 +249,6 @@ void LocalServer::tryInitPath() LOG_DEBUG(log, "Working directory created: {}", path); } - fs::create_directories(fs::path(path) / "data" / ""); - fs::create_directories(fs::path(path) / "metadata" / ""); - global_context->setPath(fs::path(path) / ""); global_context->setTemporaryStoragePath(fs::path(path) / "tmp" / "", 0); diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 33f7ca1ec19..f509f47499d 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -224,7 +224,7 @@ static UInt64 getTotalSpaceByName(const String & name, const String & disk_path, { struct statvfs fs; if (name == "default") /// for default disk we get space from path/data/ - fs = getStatVFS((fs::path(disk_path) / "data/").string()); + fs = getStatVFS((fs::path(disk_path) / "data" / "").string()); else fs = getStatVFS(disk_path); UInt64 total_size = fs.f_blocks * fs.f_frsize; @@ -248,7 +248,7 @@ std::optional DiskLocal::getAvailableSpace() const /// available for superuser only and for system purposes struct statvfs fs; if (name == "default") /// for default disk we get space from path/data/ - fs = getStatVFS((fs::path(disk_path) / "data/").string()); + fs = getStatVFS((fs::path(disk_path) / "data" / "").string()); else fs = getStatVFS(disk_path); UInt64 total_size = fs.f_bavail * fs.f_frsize; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 0b63904c3e5..41b2dcf148a 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1091,9 +1091,7 @@ void Context::setTemporaryStoragePath(const String & path, size_t max_size) VolumePtr volume = createLocalSingleDiskVolume(shared->tmp_path, shared->getConfigRefWithLock(lock)); for (const auto & disk : volume->getDisks()) - { setupTmpPath(shared->log, disk->getPath()); - } TemporaryDataOnDiskSettings temporary_data_on_disk_settings; temporary_data_on_disk_settings.max_size_on_disk = max_size; From 64fff163f7722a0ecb676ab3d5743ad00f85b63f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Apr 2024 19:54:37 +0200 Subject: [PATCH 166/624] Less directories --- src/Common/filesystemHelpers.cpp | 13 ++++++++++-- src/Common/filesystemHelpers.h | 2 +- src/Disks/DiskLocal.cpp | 1 - src/Disks/DiskLocal.h | 1 + src/Disks/TemporaryFileOnDisk.cpp | 3 +-- src/Interpreters/Context.cpp | 27 ++++++++++++------------ src/Interpreters/TemporaryDataOnDisk.cpp | 6 +++--- 7 files changed, 31 insertions(+), 22 deletions(-) diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 0d3b5cb83c8..2d053c615d9 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -42,14 +42,23 @@ namespace ErrorCodes extern const int CANNOT_CREATE_FILE; } -struct statvfs getStatVFS(const String & path) +struct statvfs getStatVFS(String path) { struct statvfs fs; while (statvfs(path.c_str(), &fs) != 0) { if (errno == EINTR) continue; - DB::ErrnoException::throwFromPath(DB::ErrorCodes::CANNOT_STATVFS, path, "Could not calculate available disk space (statvfs)"); + + /// Sometimes we create directories lazily, so we can request free space in a directory that yet to be created. + auto fs_path = std::filesystem::path(path); + if (errno == ENOENT && fs_path.has_parent_path()) + { + path = fs_path.parent_path(); + continue; + } + + ErrnoException::throwFromPath(ErrorCodes::CANNOT_STATVFS, path, "Could not calculate available disk space (statvfs)"); } return fs; } diff --git a/src/Common/filesystemHelpers.h b/src/Common/filesystemHelpers.h index 8591cd6cf92..a4eb212455d 100644 --- a/src/Common/filesystemHelpers.h +++ b/src/Common/filesystemHelpers.h @@ -56,7 +56,7 @@ std::filesystem::path getMountPoint(std::filesystem::path absolute_path); #endif String getFilesystemName([[maybe_unused]] const String & mount_point); -struct statvfs getStatVFS(const String & path); +struct statvfs getStatVFS(String path); /// Returns true if path starts with prefix path bool pathStartsWith(const std::filesystem::path & path, const std::filesystem::path & prefix_path); diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index f509f47499d..e5cde775f79 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -18,7 +18,6 @@ #include #include -#include #include #include diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index b9703019c19..e6088e21a3b 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -21,6 +21,7 @@ public: DiskLocal(const String & name_, const String & path_, UInt64 keep_free_space_bytes_, const Poco::Util::AbstractConfiguration & config, const String & config_prefix); + DiskLocal( const String & name_, const String & path_, diff --git a/src/Disks/TemporaryFileOnDisk.cpp b/src/Disks/TemporaryFileOnDisk.cpp index 92219a7f25f..91eb214d941 100644 --- a/src/Disks/TemporaryFileOnDisk.cpp +++ b/src/Disks/TemporaryFileOnDisk.cpp @@ -35,8 +35,7 @@ TemporaryFileOnDisk::TemporaryFileOnDisk(const DiskPtr & disk_, const String & p if (!disk) throw Exception(ErrorCodes::LOGICAL_ERROR, "Disk is not specified"); - if (fs::path prefix_path(prefix); prefix_path.has_parent_path()) - disk->createDirectories(prefix_path.parent_path()); + disk->createDirectories((fs::path("") / prefix).parent_path()); ProfileEvents::increment(ProfileEvents::ExternalProcessingFilesTotal); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 41b2dcf148a..e7971fbe2b1 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1044,29 +1044,30 @@ try { LOG_DEBUG(log, "Setting up {} to store temporary data in it", path); - fs::create_directories(path); - - /// Clearing old temporary files. - fs::directory_iterator dir_end; - for (fs::directory_iterator it(path); it != dir_end; ++it) + if (fs::exists(path)) { - if (it->is_regular_file()) + /// Clearing old temporary files. + fs::directory_iterator dir_end; + for (fs::directory_iterator it(path); it != dir_end; ++it) { - if (startsWith(it->path().filename(), "tmp")) + if (it->is_regular_file()) { - LOG_DEBUG(log, "Removing old temporary file {}", it->path().string()); - fs::remove(it->path()); + if (startsWith(it->path().filename(), "tmp")) + { + LOG_DEBUG(log, "Removing old temporary file {}", it->path().string()); + fs::remove(it->path()); + } + else + LOG_DEBUG(log, "Found unknown file in temporary path {}", it->path().string()); } - else - LOG_DEBUG(log, "Found unknown file in temporary path {}", it->path().string()); + /// We skip directories (for example, 'http_buffers' - it's used for buffering of the results) and all other file types. } - /// We skip directories (for example, 'http_buffers' - it's used for buffering of the results) and all other file types. } } catch (...) { DB::tryLogCurrentException(log, fmt::format( - "Caught exception while setup temporary path: {}. " + "Caught exception while setting up temporary path: {}. " "It is ok to skip this exception as cleaning old temporary files is not necessary", path)); } diff --git a/src/Interpreters/TemporaryDataOnDisk.cpp b/src/Interpreters/TemporaryDataOnDisk.cpp index 26a78d53aab..9a237738b3e 100644 --- a/src/Interpreters/TemporaryDataOnDisk.cpp +++ b/src/Interpreters/TemporaryDataOnDisk.cpp @@ -43,10 +43,10 @@ void TemporaryDataOnDiskScope::deltaAllocAndCheck(ssize_t compressed_delta, ssiz throw Exception(ErrorCodes::LOGICAL_ERROR, "Negative temporary data size"); } - size_t new_consumprion = stat.compressed_size + compressed_delta; - if (compressed_delta > 0 && settings.max_size_on_disk && new_consumprion > settings.max_size_on_disk) + size_t new_consumption = stat.compressed_size + compressed_delta; + if (compressed_delta > 0 && settings.max_size_on_disk && new_consumption > settings.max_size_on_disk) throw Exception(ErrorCodes::TOO_MANY_ROWS_OR_BYTES, - "Limit for temporary files size exceeded (would consume {} / {} bytes)", new_consumprion, settings.max_size_on_disk); + "Limit for temporary files size exceeded (would consume {} / {} bytes)", new_consumption, settings.max_size_on_disk); stat.compressed_size += compressed_delta; stat.uncompressed_size += uncompressed_delta; From 030f7114040654a6474e3bcb1655071e3c84982f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Apr 2024 20:01:54 +0200 Subject: [PATCH 167/624] Even less directories --- programs/local/LocalServer.cpp | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index df27c46e56c..c4cb5d4df9e 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -35,7 +36,6 @@ #include #include #include -#include #include #include #include @@ -48,7 +48,6 @@ #include #include #include -#include #include #include #include @@ -237,12 +236,12 @@ void LocalServer::tryInitPath() /// as we can't accurately distinguish those situations we don't touch any existent folders /// we just try to pick some free name for our working folder - default_path = parent_folder / fmt::format("clickhouse-local-{}-{}-{}", getpid(), time(nullptr), randomSeed()); + default_path = parent_folder / fmt::format("clickhouse-local-{}", UUIDHelpers::generateV4()); - if (exists(default_path)) - throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "Unsuccessful attempt to create working directory: {} already exists.", default_path.string()); + if (fs::exists(default_path)) + throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "Unsuccessful attempt to set up the working directory: {} already exists.", default_path.string()); - create_directory(default_path); + /// The directory can be created lazily during the runtime. temporary_directory_to_delete = default_path; path = default_path.string(); From d59175b6adf4b30c7f464b646876204ce366eb81 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Apr 2024 20:43:22 +0200 Subject: [PATCH 168/624] Even less directories --- src/Common/NamedCollections/NamedCollectionUtils.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Common/NamedCollections/NamedCollectionUtils.cpp b/src/Common/NamedCollections/NamedCollectionUtils.cpp index eb851a23501..e9b319f02a5 100644 --- a/src/Common/NamedCollections/NamedCollectionUtils.cpp +++ b/src/Common/NamedCollections/NamedCollectionUtils.cpp @@ -113,8 +113,7 @@ private: public: explicit LoadFromSQL(ContextPtr context_) : WithContext(context_) - , metadata_path( - fs::canonical(context_->getPath()) / NAMED_COLLECTIONS_METADATA_DIRECTORY) + , metadata_path(fs::weakly_canonical(context_->getPath()) / NAMED_COLLECTIONS_METADATA_DIRECTORY) { if (fs::exists(metadata_path)) cleanup(); From fae80801303f17e741e86281f07db1e98d89c160 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Apr 2024 04:39:44 +0200 Subject: [PATCH 169/624] Allow using clickhouse-local with a positional argument --- programs/main.cpp | 7 ++++++- src/Client/ClientBase.cpp | 28 ++++++++++++++++++++++++++-- 2 files changed, 32 insertions(+), 3 deletions(-) diff --git a/programs/main.cpp b/programs/main.cpp index 9ad8b016c82..8a8ba128cad 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -491,9 +491,14 @@ int main(int argc_, char ** argv_) /// clickhouse -q 'select 1' # use local /// clickhouse # spawn local /// clickhouse local # spawn local + /// clickhouse "select ..." # spawn local + /// clickhouse query.sql # spawn local /// - if (main_func == printHelp && !argv.empty() && (argv.size() == 1 || argv[1][0] == '-')) + if (main_func == printHelp && !argv.empty() && (argv.size() == 1 || argv[1][0] == '-' + || std::string_view(argv[1]).contains(' ') || std::string_view(argv[1]).contains('.'))) + { main_func = mainEntryClickHouseLocal; + } int exit_code = main_func(static_cast(argv.size()), argv.data()); diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 8d78c340626..826b9d75331 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2910,8 +2910,32 @@ void ClientBase::parseAndCheckOptions(OptionsDescription & options_description, } /// Check positional options. - if (std::ranges::count_if(parsed.options, [](const auto & op){ return !op.unregistered && op.string_key.empty() && !op.original_tokens[0].starts_with("--"); }) > 1) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional options are not supported."); + for (const auto & op : parsed.options) + { + if (!op.unregistered && op.string_key.empty() && !op.original_tokens[0].starts_with("--") + && !op.original_tokens[0].empty() && !op.value.empty()) + { + /// Two special cases for better usability: + /// - if the option contains a whitespace, it might be a query: clickhouse "SELECT 1" + /// - otherwise if it contains a dot it might be a queries file: clickhouse file.sql + /// These are relevant for interactive usage - user-friendly, but questionable in general. + /// In case of ambiguity or for scripts, prefer using proper options. + + const auto & token = op.original_tokens[0]; + po::variable_value value(boost::any(op.value), false); + + const char * option; + if (token.contains(' ')) + option = "query"; + else if (token.contains('.')) + option = "queries-file"; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional option `{}` is not supported.", token); + + if (!options.emplace(option, value).second) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional option `{}` is not supported.", token); + } + } po::store(parsed, options); } From c4e0cf2e7d02c5fe998bbaa9a890507f54ccf0f3 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 31 Jan 2024 06:41:32 +0000 Subject: [PATCH 170/624] rocksdb: implement ALTER SETTING Signed-off-by: Duc Canh Le --- .../RocksDB/EmbeddedRocksDBBulkSink.cpp | 9 ++-- .../RocksDB/EmbeddedRocksDBBulkSink.h | 1 + .../RocksDB/StorageEmbeddedRocksDB.cpp | 43 +++++++++++++++++-- src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 14 ++++-- 4 files changed, 55 insertions(+), 12 deletions(-) diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp index 6d9c320684d..dbaa5a8afea 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp @@ -200,17 +200,17 @@ void EmbeddedRocksDBBulkSink::consume(Chunk chunk_) auto [serialized_key_column, serialized_value_column] = serializeChunks(to_written); auto sst_file_path = getTemporarySSTFilePath(); - if (auto status = buildSSTFile(path, *serialized_key_column, *serialized_value_column); !status.ok()) + if (auto status = buildSSTFile(sst_file_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}, ingest_options); !status.ok()) + if (auto status = storage.rocksdb_ptr->IngestExternalFile({sst_file_path}, ingest_options); !status.ok()) throw Exception(ErrorCodes::ROCKSDB_ERROR, "RocksDB write error: {}", status.ToString()); - if (fs::exists(path)) - fs::remove(path); + if (fs::exists(sst_file_path)) + fs::remove(sst_file_path); } void EmbeddedRocksDBBulkSink::onFinish() @@ -220,7 +220,6 @@ void EmbeddedRocksDBBulkSink::onFinish() consume({}); } - 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 index fe28576a4a3..26a4f7c7fc3 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h +++ b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h @@ -14,6 +14,7 @@ namespace DB { +namespace fs = std::filesystem; class StorageEmbeddedRocksDB; class EmbeddedRocksDBBulkSink; diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index f72bc0e513b..af057b817d4 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -27,6 +27,7 @@ #include #include #include +#include #include #include #include @@ -179,7 +180,7 @@ StorageEmbeddedRocksDB::StorageEmbeddedRocksDB(const StorageID & table_id_, const StorageInMemoryMetadata & metadata_, LoadingStrictnessLevel mode, ContextPtr context_, - RocksDBSettings settings_, + std::unique_ptr settings_, const String & primary_key_, Int32 ttl_, String rocksdb_dir_, @@ -629,8 +630,20 @@ static StoragePtr create(const StorageFactory::Arguments & args) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "StorageEmbeddedRocksDB must require one column in primary key"); } - RocksDBSettings settings; - settings.loadFromQuery(*args.storage_def, args.getContext()); + auto settings = std::make_unique(); + settings->loadFromQuery(*args.storage_def, args.getContext()); + if (args.storage_def->settings) + metadata.settings_changes = args.storage_def->settings->ptr(); + else + { + /// A workaround because embedded rocksdb doesn't have default immutable settings + /// But InterpreterAlterQuery requires settings_changes to be set to run ALTER MODIFY + /// SETTING queries. So we just add a setting with its default value. + auto settings_changes = std::make_shared(); + settings_changes->is_standalone = false; + settings_changes->changes.insertSetting("optimize_for_bulk_insert", settings->optimize_for_bulk_insert.value); + metadata.settings_changes = settings_changes; + } return std::make_shared(args.table_id, args.relative_data_path, metadata, args.mode, args.getContext(), std::move(settings), primary_key_names[0], ttl, std::move(rocksdb_dir), read_only); } @@ -746,6 +759,22 @@ std::optional StorageEmbeddedRocksDB::totalBytes(const Settings & /*sett return estimated_bytes; } +void StorageEmbeddedRocksDB::alter( + const AlterCommands & params, + ContextPtr query_context, + AlterLockHolder & holder) +{ + IStorage::alter(params, query_context, holder); + auto new_metadata = getInMemoryMetadataPtr(); + if (new_metadata->settings_changes) + { + const auto & settings_changes = new_metadata->settings_changes->as(); + auto new_settings = std::make_unique(); + new_settings->applyChanges(settings_changes.changes); + setSettings(std::move(new_settings)); + } +} + void registerStorageEmbeddedRocksDB(StorageFactory & factory) { StorageFactory::StorageFeatures features{ @@ -757,4 +786,12 @@ void registerStorageEmbeddedRocksDB(StorageFactory & factory) factory.registerStorage("EmbeddedRocksDB", create, features); } + +void StorageEmbeddedRocksDB::checkAlterIsPossible(const AlterCommands & commands, ContextPtr /* context */) const +{ + for (const auto & command : commands) + if (!command.isCommentAlter() && !command.isSettingsAlter()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Alter of type '{}' is not supported by storage {}", command.type, getName()); +} + } diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index 456a912d8d5..9fc58ea6b38 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -37,7 +38,7 @@ public: const StorageInMemoryMetadata & metadata, LoadingStrictnessLevel mode, ContextPtr context_, - RocksDBSettings settings_, + std::unique_ptr settings_, const String & primary_key_, Int32 ttl_ = 0, String rocksdb_dir_ = "", @@ -63,6 +64,7 @@ public: void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override; void mutate(const MutationCommands &, ContextPtr) override; void drop() override; + void alter(const AlterCommands & params, ContextPtr query_context, AlterLockHolder &) override; bool optimize( const ASTPtr & query, @@ -103,12 +105,16 @@ public: std::optional totalBytes(const Settings & settings) const override; - const RocksDBSettings & getSettings() const { return settings; } + void checkAlterIsPossible(const AlterCommands & commands, ContextPtr /* context */) const override; - void setSettings(RocksDBSettings settings_) { settings = std::move(settings_); } + const RocksDBSettings & getSettings() const { return *storage_settings.get(); } + + void setSettings(std::unique_ptr && settings_) { storage_settings.set(std::move(settings_)); } private: - RocksDBSettings settings; + SinkToStoragePtr getSink(ContextPtr context, const StorageMetadataPtr & metadata_snapshot); + + MultiVersion storage_settings; const String primary_key; using RocksDBPtr = std::unique_ptr; RocksDBPtr rocksdb_ptr; From 72199a79be60dc9ebba33bf12adf00064c6aceb3 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Thu, 1 Feb 2024 09:26:59 +0000 Subject: [PATCH 171/624] better rocksdb test - add test for alter modify setting - make sure bulk insert kick in Signed-off-by: Duc Canh Le --- .../0_stateless/02956_rocksdb_bulk_sink.reference | 3 +++ .../queries/0_stateless/02956_rocksdb_bulk_sink.sql | 12 ++++++++---- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference index 83b33d238da..6232e00f9c9 100644 --- a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference +++ b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference @@ -1 +1,4 @@ +1 +1000 +0 1000 diff --git a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql index d685afadf81..34795546f7c 100644 --- a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql +++ b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql @@ -1,6 +1,10 @@ -- 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; -INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers_mt(1000) SETTINGS max_insert_threads = 2; +CREATE TABLE IF NOT EXISTS 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(1000); +SELECT sum(value) FROM system.rocksdb WHERE database = currentDatabase() AND table = 'rocksdb_worm' AND name = 'no.file.opens'; -- should be 1 +SELECT count() FROM rocksdb_worm; +TRUNCATE TABLE rocksdb_worm; +ALTER TABLE rocksdb_worm MODIFY SETTING optimize_for_bulk_insert = 0; +INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers(1000); +SELECT sum(value) FROM system.rocksdb WHERE database = currentDatabase() AND table = 'rocksdb_worm' AND name = 'no.file.opens'; -- should be 0 because all data is still in memtable SELECT count() FROM rocksdb_worm; From 3e4d9bf73abc072c2cb33b462cee5527212f71a1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 28 Apr 2024 14:04:16 +0200 Subject: [PATCH 172/624] Better result condition --- src/Interpreters/Cache/IFileCachePriority.h | 8 ++++++-- src/Interpreters/Cache/LRUFileCachePriority.cpp | 11 +++++++---- 2 files changed, 13 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/Cache/IFileCachePriority.h b/src/Interpreters/Cache/IFileCachePriority.h index 519e58fb107..bb7b220c1f9 100644 --- a/src/Interpreters/Cache/IFileCachePriority.h +++ b/src/Interpreters/Cache/IFileCachePriority.h @@ -137,7 +137,8 @@ public: virtual PriorityDumpPtr dump(const CachePriorityGuard::Lock &) = 0; - /// Collect eviction candidates sufficient to free `size` bytes. + /// Collect eviction candidates sufficient to free `size` bytes + /// and `elements` elements from cache. virtual bool collectCandidatesForEviction( size_t size, size_t elements, @@ -147,7 +148,10 @@ public: const UserID & user_id, const CachePriorityGuard::Lock &) = 0; - /// Collect eviction `candidates_num` candidates for eviction. + /// Collect eviction candidates sufficient to have `desired_size` + /// and `desired_elements_num` as current cache state. + /// Collect no more than `max_candidates_to_evict` elements. + /// Return `true` if the first condition is satisfied. virtual bool collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index c1ff9ce17ba..ec96eb14a8a 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -331,14 +331,17 @@ bool LRUFileCachePriority::collectCandidatesForEviction( EvictionCandidates & res, const CachePriorityGuard::Lock & lock) { - auto stop_condition = [&, this]() + auto desired_limits_satisfied = [&]() { return canFit(0, 0, stat.total_stat.releasable_size, stat.total_stat.releasable_count, - lock, &desired_size, &desired_elements_count) - || (max_candidates_to_evict && res.size() >= max_candidates_to_evict); + lock, &desired_size, &desired_elements_count); + }; + auto stop_condition = [&]() + { + return desired_limits_satisfied() || (max_candidates_to_evict && res.size() >= max_candidates_to_evict); }; iterateForEviction(res, stat, stop_condition, lock); - return stop_condition(); + return desired_limits_satisfied(); } void LRUFileCachePriority::iterateForEviction( From 8c8d30c2553e091f598e5d5d30bab85fe5d77606 Mon Sep 17 00:00:00 2001 From: Maxim Alexeev Date: Sun, 28 Apr 2024 15:55:51 +0300 Subject: [PATCH 173/624] Make test deterministic --- .../00181_cross_join_compression.reference | 2000 ++++++++--------- .../00181_cross_join_compression.sql | 4 +- 2 files changed, 1002 insertions(+), 1002 deletions(-) diff --git a/tests/queries/1_stateful/00181_cross_join_compression.reference b/tests/queries/1_stateful/00181_cross_join_compression.reference index da9f74397bd..2e2b18f3150 100644 --- a/tests/queries/1_stateful/00181_cross_join_compression.reference +++ b/tests/queries/1_stateful/00181_cross_join_compression.reference @@ -1,1000 +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 +33553863 http://ultival.1septs +33553862 http://full%2F4a%2Fartiry/prival Pandar &clid=4832.html?html&lang=ru&lr=2&msid=203&banerid +33553862 http://full%2F4a%2Fartiry/prival Pandar &clid=4832.html?html&lang=ru&lr=2&msid=203&banerid +33553742 https://myz.sutoclub +33553742 https://myz.sutocation{fontaktering=0&session.ru/index.ru/apltras[5]=0&engine_power/user-efender +33553742 https://myz.sutocation/search?text=коллей и солнце в +33553161 http://life-pri-burnoe-dlya-and-na-chudopoly +33553161 http://life-pri-burnoe-dlya-and-na-chudopoly +33553118 http://news%2F30%2F1.2&name":"30.475065&text=в контакте +33552544 http://m.yandsearch?text=юнис +33552455 http://maps.yandex.ru/fashionnya_10000940877/?action-lacost_mob_contakte,facenews/864490.html?accountryasam-full +33552449 http://top.rbc.ua/погода-харьков&img_url=http://xyyatsya.html&lang=ru&lr=2779077&text=арабатывающих +33552449 http://top.rbc.ru/yandsearch?text=вання +33552449 http://top.rbc.ru/yandsearch?text=ванные целевизора&lr=213&msid=2083696&text=как +33552449 http://top.rbc.ru/yandsearch?clid=90 +33552449 http://top.rbc.ru/onliner.play.php?s=8 +33552449 http://top.rbc.ru/online.ru/disk.ru/video/embed +33552449 http://top.rbc.ru/online.com.tr/guncel/frl-2/233 +33552449 http://top.rbc.ru/myfics.pulse.ru/yandsearch?lr=48&perialu.net +33552449 http://top.rbc.ru/myfics.pulscence/2014/91269707723/item/6961%26nid +33552328 http://kupit-i-lynn-na-vtorders/dokto.ru/touchshie_na_skladnaia-vlozhenshchadki +33552328 http://kupit-i-lynn-na-tastyagazeta.ru/tverdar.org/bin.ru/viewtopics/174287200029444776ffa1ba91d43030e +33552216 http://video/58785/?promo=10347&text=фильмы она для +33552216 http://video/58785/?promo=10347&text=фильмы она для +33552216 http://finam.intel +33552216 http://finam.inpearls.html&lang=ru&lr=959&text=вконтаж +33552099 http://rg.ru/remont_give-schet/shop.novosti/2014/9116-spalnyj-so +33551977 http://search +33551857 http://yandex.ua/uk/news=articles/485/mode=action=FR,B9 +33551857 http://yandex.ua/uk/news=articles/485/mode=action=FR,B9 +33551857 http://yandex.ua/uk/news=articles/485/mode=action=FR,B9 +33551857 http://yandex.ua/uk/news=articles/485/mode=action=FR,B9 +33551857 http://yandex.ru/public-tools/sanatify=nogeo=0&secret=52436 +33551857 http://yandex.ru/public-tools/sanatify=nogeo=0&secret=52436 +33551857 http://yandex.ru/public-tools/sanatify=nogeo=0&secret=52436 +33551857 http://yandex.ru/2402838/?content=on&_orders_num=4907.html#videos/doxodover/fotostransfer +33551274 http://mysweet-tavr.ru/yandex.ru/yandsearch?clid=13259827&spn +33551274 http://mysweet-tam-3-silya +33551274 http://mysweet-tam-3-silya +33550899 http://astroisshke +33550844 http://yandex.ru/cgi-bin/msglist/meshnyi-mezhdunarod +33550844 http://yandex.ru/?clid=20677839/pro-vospalnii-ot-mashnie_novosti/zakon +33550801 http://sibdomashnyj-cvet-telej-modanija +33550801 http://sibdom.net/ru/pozdrav.narutoprague/auto/geneva2014 +33550801 http://sibdom.net/ru/pozdrav.narutoprague/auto/geneva2014 +33550801 http://sibdom.com/top22013/summi-dlya-vsem-yumor_prezidents +33550801 http://sibdom.com/search +33550801 http://sibdom.com/search +33550801 http://sibdom.com/search +33550801 http://sibdom.com/search +33550801 http://sibdom.com/search +33550801 http://sibdom.com/search +33550801 http://sibdom.com/magayutsya-posle-yontent.ru +33550801 http://sibdom.29.ru/articles/ya-ukray/doktor +33550204 http://studio/dist_pink-dis +33549664 http://gde_timeout=144037&lr=50&from=direcommepage=0&ad_info=ElsdCQRfSVFsBgBTAw +33549406 http://yandex.ua/user_app_versianfood.com/search?cl4url=1primea-materisi-puloveplane +33549201 http://fashing.net/testdrive_key=506d9e3dfbd268e6b6630e58&doma +33549131 http://images/216377821524.13951616 +33549116 https://m.haberler.ru/braun/9408215.shtml&lang=ru +33549116 https://m.haberler.ru/braun/9408215.shtml&lang=ru +33549116 https://m.haberler.ru/Kovalidad.fisha.lebek tanka.com.ua/news +33549091 http://zagrams=bid%3D84841.shtml/ru/store/xml_catalog/view/2014/3 +33549091 http://zagradskaja/2011.18432133/11/maps.yandex.php?promo +33548853 http://predir%3Fid%3D0%26height%3Daf1ea8a2981 +33548697 http://yandex.ru/jobs/florange_key=&availa-vtornye +33548648 http://worldoftanks.ru/load.cgi%3Fsid +33548648 http://worldoftanks.ru/140-seasonvar +33548648 http://worldoftanks.aspx&refererleniya-bilgisa-roslanding_url +33548648 http://worldoftanks.aspx&referereezume/vorona.org/news.yandsearch +33548648 http://worldoftanks.aspx&referereezultator/anne_postami.ru/3628 +33548648 http://worldoftanks.aspx&referereezultator/anne_postami.ru/3628 +33548648 http://worldoftanks.aspx&referereezhischet_dorovideo/search?lr +33548648 http://worldoftanks.aspx&referereezhimost?q=грузкаопмо +33548648 http://worldoftanks.aspx&referereezhimost?q=грузкаопмо +33548648 http://worldoftanks.aspx&referereezhimost?p=5758415845392595002181 +33548648 http://worldoftanks.aspx&referereezhdunarodov.ru/igri7.ru/cher +33548648 http://worldoftanks.aspx&referereezhdugor.com/webhp?espv=1&ie=UTF +33548648 http://worldoftanks.aspx&referereezhda-devusher.html&langualeo +33548648 http://worldoftanks.aspx&referereezhda-devusher.html&langualeo +33548648 http://worldoftanks.aspx&referereezh&api_resuet +33548648 http://worldoftanks.aspx&referency=UAH&job_interzhalsya +33548361 http://wot/htm/wotlauncheskim_tatavni.ru%2F&is_mobile +33547846 http://yenisafak.com.ua/погода на можно ли скачать контакте&lr=194 +33547361 http://acunn.mk.ru +33546008 http://yandex.ru/syndyk.ru/yandex.ru/view&idtv=2182 +33545847 http://f5haber7.com/play.ru/2012/100135364&secret_besplate.net/gruzovik +33545847 http://f5haber7.com/play.ru/2012/100135364&secret_besplate.net/gruzovik +33545847 http://f5haber7.com/play.ru/2012/100135364&secret_besplate.net/gruzovik +33545847 http://f5haber7.com/play.ru/2012/100135364&secret_besplate.net/gruzovik +33545847 http://f5haber7.com/play.ru/2012/100135364&secret_besplate.net/gruzovik +33545847 http://f5haber7.com/play.ru/2012/100135364&secret_besplate.net/gruzovik +33545847 http://f5haber7.com/play.php?id=1989605&lr=213&msid=2749/?page +33545847 http://f5haber7.com/kayintov +33545847 http://f5haber7.com/kayintov +33545847 http://f5haber7.com/kayintov +33545847 http://f5haber7.com/?cid=19842783179368 +33545847 http://f5haber7.com&publications/mysel-omeganovsk/telegraf +33545847 http://f5haber7.com&publications/67459577859 +33545579 http://mamba.kg/#!/videntnogo-putin-show_doc_LAW +33545564 http://rnd.kz/poisk.com/iframe +33545404 http://home/shok_popundem/erohd +33545404 http://home/lp1/?p=4&t=583906723&text=пошив бюстгальтик +33545404 http://home/lp1/?p=10&lr=213&text=супер +33545397 http://mp3/20/139377/link7 +33544214 http://fanatik.ru/00003536430/?gcv_source=wizard&category/4dev.ru/polos.ru/kosmeteo.ua/polit/1741824424/?from=email.ru/yandex.ru +33543373 http://base.com.tr/firefox/295771534.shtml/ru/news.yandsearch?text=люблю 2 серия скачать +33543373 http://base.com.tr&user=user_all=76 +33542831 https://e.mail=125456_0.html_params=bid%3D1%26xdm_e=http://news/hotel_2_druzya-trana.ru/3275/?from=odnoklass/page=1#compila-ilici-ayakkan/friendsms.ru/registralizaciya-seriya-rosiya-tv.net/?next=/id/openphX21pbmlzdHZvery +33542831 https://e.mail.ru/my/#actions_510473759731&text=search.php?show_banner_click_id=-1&custoe +33542407 http://wot/htm/wot/golder-uyku-3653883720 +33542270 https://moscow/ru/novostranamadi-47312196&acce285092.html/ru/lisi-na-par-liga.net/download +33542270 https://moscow/ru/live/2014&sociologin=pros-pressage/vladtimeout=14403395141 +33542270 https://moscow/full-up-nnn.ru/index.kz/cars.autoplus-muthisweet.ru/00032014/03/melkova-uvere.html&lang=ru&lr=1946562&win +33542270 https://moscow/episode-com.tr/dizi-gunleri/kopevsk/?one=zoneid=2073/?frommanduyuruindex.ru/yandex.ru/yandex +33542270 https://hugesex.tv/th/ญี่ปุ่น-slozhnoe.ua/villa_199855362 +33542006 http://7ya.ru/Video/embed +33542002 http://gaz/2117/start.tv/?cutter&l10n=ru&lr=56&text=смотреть +33541260 http://yandex.ru/filmId=ydx&iv=ZLZHVUVlsySgV +33541260 http://yandex.ru/filmId=ydx&iv=ZLZHVUVlsySgV +33541260 http://fast-trybu-viyskiy-yagoji.com +33541260 http://fast-trybu-viyskiy-yagoji.com +33541260 http://fast-trybu-viyskiy-yagoji.com +33541260 http://fast-trybu-video/83506.html?themoscow/90/inciden sans-viborsa/#788 +33541222 http://mg.mgshare.com/r2/playerinvestions/629807703503541977079 +33541084 http://yandex.ru/news.yandsearch?text=нара 2 на начнеток&img_url +33541084 http://yandex.ru/news.yandex.ru/polovo_20014, это такой Александр по порно сантехник +33540897 http://small.ru/yandex.ru/photosession +33540564 http://wwww.forexpf.ru/Onliner.ru/spaces.ru/ +33540564 http://wwww.ford_id|8610871/train-danies/skachenie +33540564 http://wwww.ford-sparolevka.bigmir.net/ru/post/video-food/detskij-troit/ +33540506 http://news/bannetteyen-sostan.ua/kids=67774.1395223/19/chastnikapelleri.gen.tr/oauth_sig=d7e8342341&lr=25&secret=9c4c5ecf578b7f3 +33540506 http://news/bannettey.ru/retarutop&text=в контакте&clid=13953422.1395129/index.ru/my/messa +33539800 http://clubmansk.rabota +33539286 http://all/pala-deistvennoreask=1 +33538555 http://kler.ru/video/warplanberri.net/ +33538396 http://finans.html&tld=ru +33537935 http://yandex.ua/politics/17513810d4fgdTWw.. +33537265 http://yandex.ru/pers/1-0-671-kompyuterologinza.ru/id/111/149/milltext +33536919 http://aydan&type=2&sort=natik.ua/desktop/in/80949 +33536857 http://spartmentHelper&id=2738 +33536857 http://spartmentHelper&id=2738 +33536857 http://spartmentHelper&id=2738 +33536857 http://spartmentHelper&id=2738 +33536857 http://spartmentHelper&id=2738 +33536857 http://spartmentHelper&id=2738 +33536857 http://spartmentHelper&id=2738 +33536857 http://spartmentHelper&id=2738 +33536857 http://spartmentHelper&id=2738 +33536857 http://spartmentHelper&id=2738 +33536857 http://spartmentHelper&id=2738 +33536857 http://spartmentHelper&id=2738 +33536857 http://spartmentHelper&id=2738 +33536857 http://spartmentHelper&id=24086074727284&t=2&pos=29&hotels.html +33536857 http://spartmentHelper&id=240002152955821-deti/yandex.ru +33536857 http://spartmentHelper&id=240002152955821-deti/yandex.ru +33536857 http://spartmentHelper&id=240002152955821-deti/yandex.ru +33536857 http://spartmentHelper&id=240002152955821-deti/yandex.ru +33536857 http://spartmentHelper&id=240002152955821-deti/yandex.ru +33536857 http://spartmentHelper&id=240002152955821-deti/yandex.ru +33536857 http://spartmentHelper&id=240002152955821-deti/yandex.ru +33536857 http://spartmentHelper&id=240002152955821-deti/yandex.ru +33536857 http://spartmentHelper&id=240&text=стоит +33536857 http://spartmentHelper&id=240&text=герб россия, Республик +33536771 http://konster.ru/yandsearch?clid=1222/480786176723/?from +33536771 http://konster.ru/yandsearch?clid=1222/480786176723/?from +33536771 http://konster.ru/v-rossiya?n=2506461 +33536771 http://konster.ru/v-rossiya?n=2506461 +33536771 http://konster.php?Itemid=217867427535/?refreakinelegraf.html/ru/read.php +33536771 http://konster.gs.html/russichat/188077.html&former.ru/yandex.ru/0000000762 +33536771 http://konster.gs.html/russichat/188077.html&former.ru/yandex.ru/0000000762 +33536771 http://konster.com/?fb_source=vk&refererigroka-klub +33536771 http://konster.com/?fb_source=vk&refererigroka-klub +33536771 http://konster.com.tr/oauth=0&drivery/en/search=Отправильные фильм +33536771 http://konster.com.tr/newreg=1&auth.rosrees +33536771 http://konster.com%2Fnews.liga.net/journal +33536588 http://autone.html&gm=1&lang=ru +33536164 http://wotlaunchestvij-posts%2F2014/03/20/post.com/en/spravda.ru/neli.html?item_id=040103062&text=set +33535954 https://e.mail.yandex +33535840 http://yandex.ru/newsru.com.tr&user_id%3D84845.17.30.13953979&text=саламбрекены +33535746 http://wot/htm/wotlauncher.html&lang=ru +33535713 http://amk-widget/?overny +33535713 http://amk-widget/?overny +33534920 http://collash%3D2%26check=0&lang=ru&lr=1139&text=приколепный +33534613 http://topbest-fears.ru/nachali +33534534 http://kolesinda-nachinov-site_slot +33534469 http://small.com.tr/gunesia-mona-i-post.ru/yandex.ru/forbes +33534454 http://favto-referentop.ru/questink.com&public%2F20140317/218502595&metroveschits/24000092766 +33534116 https://male-hologinza.ru/Krist.org/radioveshilapii-2013-06-03-22 +33534056 http://cybers.com/index.ru/krashivai.ru/fp/derinburg/mototours +33534056 http://cybers.com.ua/sprashirsk +33533945 http://sefania/acers[]=3&ptl=0&clid=9403&lang=ru&lr=23&clid=993105-20338816573.18858.139539063&acce +33533945 http://sefania/acers/oley.html?html/ru/video/search?cl4url=http://home/politics/1700475_0&rnd=9582 +33533945 http://sefania/acerfax.ru/Suzuki_zavtrax.com/r/rbc.ru/Mukhi-spletnie-luchat.com/webcache/amk-windows +33533945 http://sefania/acereshebnik/102451185/?ref=http:%2F%2Fimage&lr=113&text=мультики&clid=1985544 +33533945 http://sefania/aceramtp.info=sw-1349-wh-768-ww-1351&rpt=simages/inbox/728x90/?from=7&templatotv +33533945 http://sefania/acerambler.ru/Отели +33533945 http://sefania/acerambler.ru/yandex.ru/yandex.ru/polikar.info +33533945 http://sefania/acerambler.ru/yandex.ru/hotels/leonliner.bjk.html&lang=ru&lr=213&oprnd=65864284.html?html?stat=0&category/stesi.com +33533945 http://sefania/acerambler.ru/yandex.ru +33533945 http://sefania/acerambler.ru/pyat-zhens.ru/recipes-topol-tri-degistemy/messages/index.ru/lg/lg_optik.com/view.html#.UxDJCdiRHsBCx9 +33533945 http://sefania/acerambler.ru/publiconnId=199291385fb7308ec3ee12d0c263bd778bfaf924d6a121291&lr=236&text +33533945 http://sefania/acerambler.ru/neo2/#inbox/?_1ld=1_10004182473169713.html?html/ru/kaluga.ru/yandex.ua +33533945 http://sefania/acerambler.ru/neo2/#inbox +33533945 http://sefania/acerambler.ru/moscowmap.php?p=3&clid=1989274 +33533945 http://sefania/acerambler.ru/menu.ru/yandex.ru/neo2/#inbox/?lang=ru&lr=13&clid=205519&text=закупки плакал на тильних +33533945 http://sefania/acerambler.ru/list?folder +33533945 http://sefania/acerambler.ru/inworld of MFP +33533945 http://sefania/acerambler.ru/from=formit +33533945 http://sefania/acerambler.ru/disk.ru/yandex +33533945 http://sefania/acerambler.ru/desk.spb.rabota +33533945 http://sefania/acerambler.ru/content.ws/grinews.yandsearch=FZ +33533945 http://sefania/acerambler.ru/commu-dva +33533945 http://sefania/acerambler.ru/cards.a5ltd +33533945 http://sefania/acerambler.ru/action=1.29081 +33533945 http://sefania/acerambler.ru/51/dollakov-pro.ru/neo2/#inbox/?lang=ru&lr=65&noreask=1&secret=cook-tv.ru/news-14546403357&a_id=vyZ1DKVQDYH08ZCc4FHhcJkMuAiYIHhMJWjIMES0QIhcpFx1xND5WWktYejZaOG9QWWt2ZAVfUFlATAV7QV9Gempe +33533945 http://sefania/acerambler.rotahabr.ru/korre/politics/game-online.at.ua/?ll=37.61767/?page=0&pos=6.0.2&vers.ru%2F2014-h154&noreask +33533945 http://sefania/acerambler.net/ru/theme=default +33533945 http://sefania/acerambler.mirovanie-foto +33533945 http://sefania/acerambler.html_part182434 +33533945 http://sefania/acerambler.com/web/webhp?espv=1&ie=UTF-8&l=vi&p=AhY_cQVTQQZLAVEDZFcJG1F4XldReWVtUEhEGHVfV1dnY3wPFWVyGmMFVRgHC0JdUyd7NGJzCHd7dVNSTlppOG1 +33533945 http://sefania/acerambler.com/index.ru +33533945 http://sefania/acerambler.com/futbol/Russing-skforummaonliner.raiserapidfilmId=qFYW5kZXgucnUvaGVua28 +33533945 http://sefania/acerambler-games.mail +33533945 http://sefania/acerambler&utm_campbell +33533945 http://sefania/acer_18_id448ecc0fddf4 +33533945 http://sefania/acer.ru/yandex.ru/?p=3&carfinCar%3D11921052307886.281500.159318&curre.ru/article +33533945 http://sefania/acer-sum=100&lr=213&noheader +33533945 http://sefania/acer-sum=100&lr=213&noheader +33533945 http://sefania/acer-sum=100&lr=213&noheader +33533945 http://sefania/acer-sum=100&lr=213&noheader +33533945 http://sefania/acer-sum=100&lr=213&noheader +33533945 http://sefania/acer-sum=100&lr=213&noheader +33533945 http://sefania/acer-sum=100&lr=213&noheader +33533945 http://sefania/acer-sum=100&lr=213&noheader +33533945 http://sefania/acer-sum=100&lr=213&noheader +33533945 http://sefania/acer-sum=100&lr=213&noheader +33533945 http://sefania/acer-sum=100&lr=213&noheader +33533945 http://sefania/acer-sum=100&lr=213&noheader +33533945 http://sefania/acer-sum=100&lr=213&noheader +33533945 http://sefania/acer-sum=100&lr=213&noheader +33533945 http://sefania/acer-sum=100&lr=213&noheader +33533945 http://sefania/acer-sum=100&lr=213&noheader +33533945 http://sefania/acer-sum=100&lr=213&noheader +33533945 http://sefania/acer-sum=100&lr=213&noheader +33533945 http://sefania/acer-sum=100&lr=213&noheader +33533945 http://sefania/acer-sum=100&lr=213&noheader +33533945 http://sefania/acer-dodgeraturalsk/motobralime.ru/yandsearch_kodelicheskim_kachej-i-pidzhanyy-za +33533945 http://sefania/acer-chto-vip-cakir-ilk-test-help.2gis.ru/postuchastnicyn-prezident.su/forum.ucoz +33533945 http://sefania/acer-amatvii-kili-pro-zhiznyu-needineniya-2012/04/hata.ru/name=d7f4a6a2bb32c889713463181048 +33533945 http://sefania/accountrysis +33533945 http://sefania/accountry":1,"bdate:descript/segodnya +33533945 http://sefania/accountry +33533945 http://sefania/accounter.start.airline/popular/1/o-patrix.ru/yandsearch?p=175&viewtopic/77-1-0 +33533945 http://sefania/accounter.start.airline.sberbankionline.html?id=1879974,55.7868151049.139540135 +33533945 http://sefania/accounter.ru/yandex.ru/neo2/#messages.yandex.ru +33533945 http://sefania/accounter.ru/yandex.ru/neo2/#messages.yandex.ru +33533945 http://sefania/accounter.ru/yandex.ru/neo2/#messages.yandex.ru +33533945 http://sefania/accounter.ru/yandex.ru/neo2/#messages.yandex.ru +33533945 http://sefania/accounter.ru/yandex.ru/neo2/#messages.yandex.ru +33533945 http://sefania/accounter.ru/yandex.ru/neo2/#messages.yandex.ru +33533945 http://sefania/accounter.ru/yandex.ru/neo2/#messages.yandex.ru +33533945 http://sefania/accounter.ru/yandex.ru/neo2/#messages.yandex.ru +33533945 http://sefania/accounter.ru/yandex.ru/neo2/#messages.yandex.ru +33533945 http://sefania/accounter.ru/yandex.ru/neo2/#messages.yandex.ru +33533945 http://sefania/accounter.ru/yandex.ru/neo2/#messages.yandex.ru +33533945 http://sefania/accounter.ru/yandex.ru/neo2/#messages.yandex.ru +33533945 http://sefania/accounter.ru/yandex.ru/neo2/#messages.yandex.ru +33533945 http://sefania/accounter.ru/yandex.ru/neo2/#messages.yandex.ru +33533945 http://sefania/accounter.ru/yandex.ru/neo2/#messages.yandex.ru +33533945 http://sefania/accounter.ru/yandex.ru/neo2/#messages.yandex.ru +33533945 http://sefania/accounter.ru/yandex.ru/neo2/#messages.yandex.ru +33533945 http://sefania/accounter.ru/yandex.ru/neo2/#messages.yandex.ru +33533945 http://sefania/accounter.ru/yandex.ru/neo2/#messages.yandex.ru +33533945 http://sefania/accounter.ru/yandex.ru/neo2/#messages.yandex.ru +33533945 http://sefania/account +33533945 http://sefania/account +33533945 http://sefania/account +33533945 http://sefania/accordsPerPage +33533945 http://sefania/accords&projects/chto-delat_rassyobse-v-zhit_list +33533945 http://sefania/accords&projects/12807.html?t=1106/#photo.qip.ru +33533945 http://sefania/accords&projectId=403170558%26customat-avtomashniy +33533945 http://sefania/accords&projectId=18827.html#47 +33533945 http://sefania/accords&project-syn-sonuclari +33533945 http://sefania/accords&project-strelyam-v-porner_click_id=2413714e82e72 +33533945 http://sefania/accords&project-site_id=40317%2F03%2F1000000.html +33533945 http://sefania/accords&project-site_id=0&is_app_id=20836&group=6079 +33533945 http://sefania/accords&project-site_id +33533945 http://sefania/accords&project-site=actic.livetv.sx/lenta.ru/moscow +33533945 http://sefania/accords&project-site.ru +33533945 http://sefania/accords&project-site-lyubvi +33533945 http://sefania/accords&project-sinin/?news +33533945 http://sefania/accords&project-sini-chi +33533945 http://sefania/accords&project-simite-nauchnistration +33533945 http://sefania/accords&project-proissha +33533945 http://sefania/accords&project-partnaya +33533945 http://sefania/accords&project-pamyatilsya.html/rus/news.yandsearch +33533945 http://sefania/accords&project-nauchenprosport +33533945 http://sefania/accords&project-natu-vodons/kalitics/5629499954055 +33533945 http://sefania/accords&project-namics%2F03 +33533945 http://sefania/accords&project-namer-4 +33533945 http://sefania/accords&project-name=aa642af6c9 +33533945 http://sefania/accords&project-name=&p +33533945 http://sefania/accords&project-name":"Красная +33533945 http://sefania/accords&project-nalog.com +33533945 http://sefania/accords&project-nalog-global +33533945 http://sefania/accords&project-nala.com +33533945 http://sefania/accords&project-na-zhelises.name=Zombi/25-pevchukovich +33533945 http://sefania/accords&project-na-photofunia +33533945 http://sefania/accords&project-na-kefirebitelstva_ukrainala_gosobytiya +33533945 http://sefania/accords&project-na-kachat +33533945 http://sefania/accords&project-na-detskie +33533945 http://sefania/accords&project-na-deputati +33533945 http://sefania/accords&project-na-dairy-iz-pena.com/obozrevatehomeapps +33533945 http://sefania/accords&project-female bold man/vacancy3446133&text +33533945 http://sefania/accords&project-favoru.com +33533945 http://sefania/accords&project-favori_derejit +33533945 http://sefania/accords&project-famme=08 +33533945 http://sefania/accords&project-familyspaces +33533945 http://sefania/accords&project-familyeva +33533945 http://sefania/accords&project-PF=214937 +33533945 http://sefania/accords&project-PF=214748165.html?ext=комедії +33533945 http://sefania/accords&project-PF=214283 +33533945 http://sefania/accords&project-PF=18084 +33533945 http://sefania/accords&project-PF=180&CarType +33533945 http://sefania/accords&project wars-3-sezon-2-bolumia 7107836/ +33533945 http://sefania/accords&project wanitelstvo_menenta-uaz-po-vinni +33533945 http://sefania/accords&project - Reckout/coverrikov-service-antier +33533945 http://sefania/accord +33533945 http://sefania/acciya +33533945 http://sefania/access_perekrasnogo +33533945 http://sefania/access_perekrasnogo +33533945 http://sefania/access_perekrasnogo +33533945 http://sefania/accebback_url=ria.ru/person/1126-yubimogo-obstvennok +33533945 http://sefania/accebback_url=ria.ru/person/1126-yubimogo-obstvennok +33533945 http://sefania/accebback_url=ria.ru/person/1126-yubimogo-obstvennok +33533945 http://sefania/accebback_url=http://ulogin?next=сс карта-pochi/zvezdov/teente_200599&api +33533945 http://sefania/accebback_url=http://rdfo.ru/neo2/#inbox/?back=1&source=googogic/start=0&costudia +33533945 http://sefania/accebback_url=http://bfm.ru/be.php?f=2892.2177/blogs.yandex.ua/category/pass.com +33533945 http://sefania/acceb9191d +33533945 http://sefania/acce163a15ca1cda8e +33533945 http://sefania/accbook +33533945 http://sefania/academika +33533945 http://sefania/academic.academo/mazdanie-parner.by/ +33533945 http://sefania/ac/auto.ru/neft-autoStart +33533945 http://sefania/ac/auto.ru/neft-autoStart +33533945 http://sefania/ac/auto.ru/neft-autoStart +33533945 http://sefania/ac/auto.ru/neft-autoStart +33533945 http://sefania/ac/auto.ru/neft-autoStart +33533945 http://sefania/ac/auto.ru/neft-autoStart +33533945 http://sefania/ac/auto.ru/neft-autoStart +33533945 http://sefania/ac/auto.ru/neft-autoStart +33533945 http://bonprix.ru/2014/912338.html&lang=ru&lr=20569 +33533815 http://gyps.ru/modul.rambler.ru/welcome/popgun.ru/yandsearch?clid=1923030318/1513475 +33533815 http://gyps.ru/modul.rambler.ru/welcome/popgun.ru/yandsearch?clid=1923030318/1513475 +33533815 http://gyps.ru/modul.rambler.ru/welcome/popgun.ru/yandsearch?clid=1923030318/1513475 +33533815 http://gyps.ru/modul.rambler.ru/deales-posobiyane_iz_kottei-200398904&text=как +33533815 http://gyps.ru/modul.rambler.ru/deales-posobiyane_iz_kottei-200398904&text=как +33533706 http://zp.php?carfinTpl%3D//adultmir.ru/yandex.ua/obyasnenie-vozduh +33533559 http://10kp +33533559 http://10kp +33533554 http://zhannaliz/yandex.ru/rubric/194-adekvator%2Fyayinskij-kaldirme +33533554 http://zhannaliz/showbiz +33533341 http://newsru.com/webhp?espv=1&ie=UTF +33532815 http://zerx.ru/moscow/ru/dzerzhd-vipstatuazh-prigotovij-neobraznye_chem/signup/?from=email.ru/sport.ru/news/tits-forums +33532556 https://expertyj_redir-1440&cation.html&lr=10347-v-rublya-pritsell +33532217 http://tulus-oblagovopoltsou.com/top100 +33532142 http://image&lr=4&text=часы +33532021 http://ld.ru/test1.radiorecheski-na-domachnomu-karpatents +33532021 http://ld.ru/test-dney-payi-teknology.ru/tzarin-durnals.tv/movies.ru/prom.news.ru%2Fimg1.livejournal +33531865 http://wotlauncher/frau-line +33530939 http://pda.net/downloader_ben_10_legend.ru +33530939 http://pda.net/download/zapgames.ru/vidyat-lyubov.com.tr&callback=1&search?lr=65&redircnt=13952471887 +33530939 http://pda.net/download/tv_taris-yerel-i-grania/sale/48828-fin.html +33530939 http://pda.net/download.html?html?id=1955405805/24/market.ru/inbox/?lang=ru&lr=213&text=санкции Детский +33530767 http://e.mail.rambler.ru/idei.ru/yandex +33530735 http://rambler.ru/viewer_id +33530351 http://minprokurortmanfirmudahan/an140318 +33530351 http://minprokopievsk.ru/kiev.pro/commepage=0&sid=8082872.139517112091842&win=111139518073.shmini-aviabil-footovit +33530351 http://minproklassniki.ru/mozila-fokushek.html&lang=ru&lr=14&l=map,trfe&trfm=cursii-otvetics%2Fpremika_karmens.com.html?pid=90991 +33530351 http://minproki.ru/new/adsensek_oon_objazanie/139515&clid=9582&text=адрес свежие на австрижак день секс +33530351 http://minproki.com/erike-dlya-bgg108348/1243669&sig=6baa7f9d6b799548339/?promeclipy/1-1-600-1-0-200713/?promo +33529744 http://medcezirovka-chet-album_id +33529744 http://medcezirovka-cher.html&language +33529548 http://tyzhdenii_kryim_v_hers/4608𗝖_14 +33529548 http://tyzhdenii_kryim_v_hers/4608𗝖_14 +33529548 http://tyzhdenii_kryim_v_hers/4608𗝖_14 +33529548 http://tyzhdenii_kryim_v_herost-perm=dista +33529548 http://tyzhdenii_kryim_v_herost-perm=dista +33529548 http://tyzhdenii_kryim_v_heroes/?id +33529548 http://tyzhdenii_kryim_v_heroes/?id +33529548 http://tyzhdenii_kryim_v_herkala.hh.ru/neo2/#inbox/?back=141&recipes +33529548 http://tyzhdenii_kryim_v_here=all&filmId=wDt3YsGMq3A +33529548 http://tyzhdenii_kryim_v_here=all&filmId=qEilHuuWUXI&where=all +33529548 http://tyzhdenii_kryim_v_here=all&filmId=qEilHuuWUXI&where=all +33529548 http://tyzhdenii_kryim_v_here=all&filmId=qEilHuuWUXI&where=all +33529548 http://tyzhdenii_kryim_v_here=all&filmId=qEilHuuWUXI&where=all +33529548 http://tyzhdenii_kryim_v_here=all&filmId=qEilHuuWUXI&where=all +33529548 http://tyzhdenii_kryim_v_here=all&filmId=qEilHuuWUXI&where=all +33529548 http://tyzhdenii_kryim_v_here=all&filmId=qEilHuuWUXI&where=all +33529548 http://tyzhdenii_kryim_v_here=all&filmId=qEilHuuWUXI&where=all +33529548 http://tyzhdenii_kryim_v_here=all&filmId=qEilHuuWUXI&where=all +33529548 http://tyzhdenii_kryim_v_here=all&filmId=qEilHuuWUXI&where=all +33529548 http://tyzhdenii_kryim_v_here=all&filmId=kBGsbFhZHmE +33529548 http://tyzhdenii_kryim_v_here=all&filmId=kBGsbFhZHmE +33529548 http://tyzhdenii_kryim_v_here=all&filmId=kBGsbFhZHmE +33529548 http://tyzhdenie.biz/arabatyur-percy/ +33529548 http://tyzhden.ua/news.tut +33529494 http://pozdravov/politics/101.184574 +33529494 http://pozdravlenii-v-rossiya +33529346 http://yandex.ru/image&lr=65&text=провение +33529346 http://yandex.ru/image&lr=65&text=жажданство +33529300 http://yandsearch?text=стасия","last/woid/igra.html/Article-16.html#object.ru/aim_minutes=0&page/13955768&spn=0.2373&is_settis +33529300 http://yandsearch?text=спертеристика +33529300 http://yandsearch?text=спертеристика +33529300 http://yandsearch?text=Где купить бесплатно&where=all&text=текст&lr=115&lr=213 +33529300 http://yandsearch?text=Где купить бесплатно&where=all&text=текст&lr=115&lr=213 +33529300 http://yandsearch?text=Где купить бесплатно по волчонок +33529300 http://yandsearch?cl4url=wwww.rg.ru/inregistercontent%3D52%26width%3Dhttp://tatskoy-bolum.html/ru/filmId=c172 +33529300 http://yandsearch?cl4url=wwww.rg.ru/inregion43=43&text=корая-страница&clid=9403&lr +33529300 http://yandsearch?cl4url=wwww.radikal.com.tr/Umutsu-ve-en3.html&sig=7c2bd3852134 +33528566 http://mudrosti.ru/cgi-bin/msglishero-vsemag.ru/yandsearch +33528516 http://amkspo-2010-03-2012-30-let-siti.ru/custoshok-video/searcher +33528246 http://auto.yandsearch?text=юлия мурмановидео майнкрафт картак 1 серия&clinic/psafe.com/diziizle +33527130 http://yandsearch?text=в конструкция +33527130 http://yandex.do&returnir_8833654666.html_parali-vo-ves-shop.com/dom-instroisam.com%2Fsmall.com.ua/neo2/#inbox +33527130 http://yandex.do&returney-yuho-voprogrammy/igra-ahedzhi/korole_hincidizi/medgorod +33527130 http://yandex.do&returney-yuho-voprogrammy/igra-ahedzhi/korole_hincidizi/medgorod +33527130 http://yandex.do&returney-yuho-voprogrammy/igra-ahedzhi/korole_hincidizi/medgorod +33527130 http://yandex.do&returney-yuho-voprogrammy/igra-ahedzhi/korole_hincidizi/medgorod +33527130 http://yandex.do&returney-yuho-voprogrammy/igra-ahedzhi/korole_hincidizi/medgorod +33527130 http://yandex.do&returnUrl=http://finance.turmovie/bolshoyvoprostok/62765-fh-585-viyskoy +33527130 http://yandex.do&returnUrl=%2Fhappy birthday=3713/hotelClass.ru/yandex +33527130 http://yandex.do&returnUrl=%2Fhappy birthday=3713/hotelClass.ru/yandex +33527130 http://yandex.do&returnUrl=%2Fhappy birthday=3713/hotelClass.ru/yandex +33527130 http://yandex.do&returnTo=Q833EiAcUXI&path=wizard%3D1027269.139521023600000&language +33527130 http://yandex.do&returnTo=Q833EiAcUXI&path=wizard%3D1027269.139521023600000&lang=ru&lr +33527130 http://yandex.do&returnDateTo=2419171971/?gcv_society/177-yatsya-v-dukhod +33527130 http://yandex.do&returnDateTo=2419171971/?gcv_society/177-yatsya-v-dukhod +33527130 http://yandex.do&return&from=link]-250-mercher.html/ru/land-parkompyuterermalinkvac.php/tophotostransonhaber7.com +33527130 http://yandex.do&return&from=link]-250-mercher.html/ru/land-parkompyuterermalinkvac.php/tophoto.html/ru +33527130 http://yandex.do&return&from=link]-250-mercher.html/ru/land-parkompyuterermalinkvac.php/tophoto.html/ru +33527130 http://yandex.do&referre=all&text=киногород&clid=1697451&lr=213&noreask=1&source +33527130 http://yandex.do&refererigroselki-dlya-terrovenirov/klerkel_zvuki-kofeynye-xrom +33527130 http://yandex.do&refererigroselki-dlya-terrovenirov/klerkel_zvuki-kofeynye-xrom +33527130 http://yandex.do&redition=RU,43,Lipetsins_crd_1991793-fw-1351-wh-634-aciklama-laman-online.ws +33527130 http://mama-terrobegom/cp/110026/biogram/676-tyis--gul-alinki/communit=-1&start=1&quickpay +33525856 http://yandex.ru/video/search?lr +33525856 http://yandex.ru/video/embed/659 +33525856 http://yandex.ru/video/embed/659 +33525822 http://yandex.ua/content&viewer_type=0&ad_info.com.ua/news.liga +33525822 http://yandex.ua/content&viewer.php?routeshant/regist/406-85AB-43F0-92f0-055fea7ep +33525822 http://yandex.ru/yandex.ru/onlinenie-baisvip.php?query=Беларусь +33525791 https://megafon_category/1-chtovo/item/3971/?from=wizard&films/family +33525532 http://xxxsuchku-i-shit_ligii +33525532 http://xxxsuchku-i-shit_ligii +33525532 http://xxxsuchku-i-shit_ligii +33525532 http://xxxsuchku-i-shit_ligii +33525419 https://rus.ru/Интернышов +33525419 https://rus.ru/Интернышов +33525419 https://rus.ru/statyi-i-i-bin/msglistvo +33525419 https://rus.ru/addriver.ru/?api_url=http://meteo +33525327 http://yandsearch?lr=213&msid=2083696&from +33525327 http://yandsearch?lr=213&msid=2&rg_viewcats.aspx +33525276 http://yandex.ru/neo2/#inbox/?back +33525276 http://yandex.ru/neo2/#inbox/?back +33524989 http://maximytishinfo/infodochok +33524989 http://maximytishinfo/infodochok +33524690 http://vampionalnyi-sux.com/fr/#ling.net/video/cfa.su/load +33523429 http://yandex.ru/yandex.ru/yandex +33523429 http://yandex.ru/yandex +33523429 http://yandex.ru/rates +33523021 http://yandex.ru/pro-po-rus-to/today/Visitor-site_id%3D1%26lang +33523021 http://yandex.ru/flirtina-zdorojnoe +33523021 http://yandex.ru/flirtina-zdorojnoe +33523021 http://yandex.php?t=1659.html%3Fid_clips.com +33523021 http://yandex.do&returnUrl=http +33523021 http://kinoptice-ruchni_207815367.shtml?html&language +33523021 http://home-provoloch.ru/yandsearch?text +33523021 http://home-provoloch.ru/yandex.ru/yandex +33523021 http://home-provoloch.ru/yandex.ru/views +33523021 http://home-provoloch.ru/yandex.ru/news.yandex +33523021 http://home-provoloch.ru/world.guns.all.com +33523021 http://home-provoloch.ru/word}&referers_android +33523021 http://home-provoloch.ru/vidomchannerstyle/begemennyy-love +33523021 http://home-provoloch.ru/video/search?lr=213&msid=292324%2F03%2F2014 +33523021 http://home-provoloch.ru/video.html?pid=b473b0c79b3ab2d0b0d18bd0b10747 +33523021 http://home-provoloch.ru/sluzhil-prografii +33523021 http://home-provoloch.ru/sluhar-in-cinka +33523021 http://home-provoloch.ru/semeyen-more/Default.ru/yandex.ru/naborisoedineniya +33523021 http://home-provoloch.ru/novostika-doktoraya-skoraya-griver.ru/politnyi +33523021 http://home-provoloch.ru/news.yandsearch?cl4url=wwww.hurriyet.com +33523021 http://home-provoloch.ru/meetindal_sub=26.04919705/reporno.com.ua +33523021 http://home-provoloch.ru/index.ru/11638455b9febd2 +33523021 http://home-provoloch.ru/home.net/range&text +33523021 http://home-provoloch.ru/?yaso-porniac.com/yozga_viyskaya-aksession +33523021 http://future +33523021 http://future +33523021 http://future +33523021 http://future +33523021 http://future +33523021 http://future +33523021 http://future +33523021 http://future +33523021 http://future +33523021 http://future +33522421 https://worldofwarplanshet.html/hu/priceli-yayconsultatistart&s=89628/?from=6009155113/ghost%3D6459119&nohead=2&job_index.ru/politics +33522421 https://worldoftankov-gormovie/6639/doram.ru/moscow/plama-alphant.ru/2327/univer +33522421 https://worldoftankov-gormovie/6639/doradonloads/Как_и_волжской_округловы +33522421 https://worldoftankov-gormovie/6639/doradonloads/Как_и_волжской_округловы +33522421 https://worldoftankov-355678268/?promo&limitings=27103052/?cid=5883-prodam +33522421 https://worldoftankov-355678268/?promo&limitine_6/ +33522421 https://worldoftankov-355000e43fd419fbf2c35cf +33522421 https://world/ukrasotyi-prog.html?uid=ajZ1DLp0/mark_app_id=vyZ1DLClUJZlcJG1F4XldSeWtvVkFkf38xIAEsQVgWCVtUekB +33522421 https://world/ukrasotyi-prog.html?uid=ajZ1DLp0/mark_app_id=vyZ1DLClUJZlcJG1F4XldSeWtvVkFkf38xIAEsQVgWCVtUekB +33522421 https://world.eu/index.ru/optik.ua/yandsearch?text=стать стихи для с мажор +33522421 https://world.eu/index.ru/optik.ua/yandsearch?text=смотреть +33521626 http://yandex.ru/yandsearch?lr=44&redict/season +33521536 http://russinskie_otnostno-vsegodnya +33520995 http://a1.s7.fsimg.ria.coccoc.com.br/instruktov_perm.ru/news_6_sot._snt_detay +33520995 http://a1.s7.fsimg.info/meden-seriya-hollassniki.ru/Oboi +33520995 http://a1.s7.fsimg.com/vse_seksklyuchenie-igry +33520995 http://a1.s7.fsimg.chitels/es-model.xml?typ=RET&anbietersburg +33520995 http://a1.s7.fsimg.avoncon.net.ru/rabota.ru/politics/174880%2F9999996657844.html#top_apress +33520925 http://yandex.ru/shop/wotlauncher +33520579 http://traveldudingsportal.ru/sverhosti.ua +33519953 https://e.mail=1&quickpay=38cd0b3d0b2d0bed180d177b-99f0-055f44e7f142bb41c743890 +33519953 https://e.mail.yandsearch?clid=479802000/000286567.html/ru/news.yandsearch?lr=43&noreask=1&source +33519953 https://e.mail.yandsearch?clid=479802000/000286567.html/ru/news.yandsearch?lr=43&noreask=1&source +33519953 https://e.mail.yandsearch?clid=479802000/000286567.html/ru/news.yandsearch?lr=43&noreask=1&source +33519953 https://e.mail.yandsearch?clid=479802000/000286567.html/ru/news.yandsearch?lr=43&noreask=1&source +33519953 https://e.mail.yandsearch?clid=1_30008434634632.html?vac_page3/out-of-tanceret545 +33519953 https://e.mail.yandsearch?clid=1_100002c&_cpofid=5310758&text=ГЕРОБРАЗОВОЕ +33519953 https://e.mail.yandsearch?cl4url=www.milii.ru/politics/1892.32279/product/134.4132/164310231099_2991eaealty +33519953 https://e.mail.yandex.ru/yandsearch&area=vac&subishi_dlya +33519953 https://e.mail.yandex.ru/msk/12812112538%26bn%3D3239658%26bt +33519953 https://e.mail.yandex.ru/msk/12812112538%26bn%3D3239658%26bt +33519953 https://e.mail.yandex.ru/medley7/article/14Mar2014/91243-v-kharniturgical-budet-haber/yandex.ru/kategory_from=odnoklassniki.ru +33519953 https://e.mail.yandex.ru/7/champaign=nw&utm +33519953 https://e.mail.ru/yandex.ru/c/m.exe/htm/wotlauncher.ru/naifl.ru/actualiday=1#video/search?textpesni_duhovke-vozil-ex.ru/?clid=13954055 +33519953 https://e.mail.ru/neo2/#inbox/ +33519953 https://e.mail.ru/neo2/#inbox/ +33519953 https://e.mail.ru/bit2bit.net/209228_900b9018.html?id=c19 +33519953 https://e.mail.ru/beremennok/sharelii.ru&devices-brics/89822/rosbalta.ru/mail.rambler.ru +33519953 https://e.mail.ru/Portal_onlayn-igrovya/29474?sid=246411740685&country_params=rhost%3D52%26pz%3D11555_Turkey.com/viewer_novily_dlja +33519674 http://best.kupit.ru/yandex.ru/sankt-petrovishi.ru/2004d3c53 +33519674 http://best.kupit.ru/yandex.ru/sankt-peter=814409631/?from=ohmb_city=0&uinfo=ElsdCQJUQxJThCwNSAxI3MSc4LiY4AxtuASgHIAgbOn98cV9TVkZNcQsyXkZ4 +33519236 http://yandex.ru/page.biz/17/99999/2014/19240&PrefFrend +33518904 http://rambler.ru/stattorgnut-Krimult +33518904 http://rambler.ru/stattorg.ua/hobbilein/msglistakeforex +33518904 http://rambler.ru/stattorg.ua/hobbile +33518904 http://rambler.ru/stattorg.ru/106460a +33518904 http://rambler.net/5927&text=говориль +33518904 http://rambler.net/5927&text=говориль +33518406 http://delta/htm/wot/htm/wotlauncher.ru/prography.html +33518406 http://delta/htm/wot/htm/wotlauncher.ru/prography.html +33518406 http://delta/htm/wot/htm/wotlauncher.ru/prography.html +33518384 http://svprestatusi_prede +33518384 http://svpressid=238&lr=63 +33518384 http://svpressages.yandex.ru/ekonov +33518384 http://svpressages.yandex.ru/ekonov +33518384 http://svpressages.yandex.ru/ekonov +33518384 http://svpressages.yandex.ru/ekonov +33518384 http://svpressages.yandex.ru/ekonov +33518384 http://svpressages.yandex.ru/ekonov +33518384 http://svpressages.yandex.ru/ekonov +33518384 http://svpressages.yandex.ru/ekonov +33518384 http://svpressages.yandex.ru/ekonov +33518384 http://svpressages.yandex.ru/ekonov +33518384 http://svpressages.yandex.ru/ekonov +33518384 http://svpressages.yandex.ru/ekonov +33518384 http://svpressages.yandex.ru/ekonov +33518384 http://svpressages.yandex.ru/ekonov +33518384 http://svpressages.yandex.ru/ekonov +33518384 http://svpressages.yandex.ru/ekonov +33518384 http://svpress.ykt.ru +33518384 http://svpress.yandex.ru/yandex.ru/memoryachi/zhere +33518384 http://svpress.yandex.ru/questions/volgogram/1-koman +33518384 http://svpress.yandex.ru/messabota +33518384 http://svpress.com/webhp?espv=1&ie=UTF-8&l=vi&p +33518384 http://svpress.com/webhp?espv=1&ie=UTF-8&l=vi&p +33518384 http://svpress.com/webhp?espv=1&ie=UTF-8&l=vi&p +33518384 http://svpress.com/webhp?espv=1&ie=UTF-8&l=vi&p +33518384 http://svpress.com/staforizaciya-anars.com.tr&user +33518384 http://svpress.com/staforizaciya-anars.com.tr&user +33518384 http://svpress.com/istanoy-v +33518384 http://svpress.com/apps&lc_name +33518384 http://svpress-chulkax-v-dengangbalta +33518384 http://svpress-chulkax-v-dengangbalta +33518384 http://svpress-chulkax-v-dengangbalta +33518384 http://svpress-chulkax-v-dengangbalta +33518384 http://svpress-chulkax-v-dengangbalta +33518384 http://svpress-chulkax-v-dengangbalta +33518384 http://svpress-chulkax-v-dengangbalta +33518384 http://svpress-chulkax-v-dengangbalta +33518384 http://svpress-chulkax-v-dengangbalta +33518384 http://svpress-chto-dizajn-trip.bg/новости телефоны +33518384 http://svpresennyy-bosch +33518384 http://svpresennyy-bosch +33518384 http://svpresennium.com/adam/istory +33518384 http://svpresennitsya_vzhe_pravkatalog +33518384 http://svpresennij-sposolstory +33518384 http://svpresennie-za-fenews.mail.ru/news/sankcii-products/vsepostupaemyj +33518384 http://svpresennie-inculatotv.tv +33518384 http://svpresennie-e-n-v/mediaget +33518384 http://svpresennie-1905-voevatelya +33518384 http://svpresennetteyzen.com/categoryId +33518384 http://svpresennet.ua/yandsearch?cl4url=rusnoe-komani +33518384 http://svpresennet.ua/news.yandex +33518384 http://svpresennet.com/load/gta_4_godetoedet +33518384 http://svpresennet.com/load/godonskim.html +33518384 http://svpresennet.com/ghitsya-vzjali +33518384 http://svpresennet.com/?cid=1_843 +33518384 http://svpresennet.com.tr/D/Genel +33518384 http://svpresennet.com.tr/?win=104&company=5 +33518384 http://svpresenner_viewertyvanlatım +33518384 http://svpresenner/5bd56db9c088f0 +33518384 http://svpresenne-vinny_frommail +33517550 http://telebek.com/notification.by/yandsearch.html +33516942 http://yandex.ua/describe.ru/yandex.ru/kak-managemennoletat +33516942 http://yandex.ua/Shou-biri-dnjatija-vo-vremennoafishin/zhensk.ua/ukranov/56 +33516934 http://news.yandex.ua/world +33516664 http://yandex.ru/yandex +33516516 http://skazan/applanet_seengliysk +33516436 http://childs=1000de +33516285 http://coccoc.com/Informalnaya-trax.com.ru/politics +33516126 http://forum.horo.mail.ru/volgoprove-did39 +33515878 http://yandex.ru/city=&name=Индийского развитие +33515401 http://rg.ru%2Fplatno-kurganizmantasy-loko.ru/games.mail.yandsearch?cl4url=http://worldoftanks.ru/news/231518 +33515401 http://rg.ru%2Fplatno-kurganizmantasy-loko.ru/games.mail.yandsearch?cl4url=http://worldoftanks.ru/news/231518 +33515401 http://rg.ru%2Fplatno-kurganizmantasy-lohanks.eu/play.ru/autop&text=чемпионов зодиака смотреть фиат альма новые-вычет зубна&clid=1979770d066e1 +33515401 http://rg.ru%2Fplanet.ru/ulyava +33515401 http://rg.ru%2Fplanet.ru/ulyava +33515401 http://rg.ru%2Fplanet.ru/ulyava +33515401 http://rg.ru%2Fplanet.ru/ulyava +33515401 http://rg.ru%2Fplanet.ru/ulyava +33515401 http://rg.ru%2Fplanet.ru/ulyava +33515401 http://rg.ru%2Fplanet.ru/ulyava +33515401 http://rg.ru%2Fplanet.ru/ulyava +33515401 http://rg.ru%2Fplanet.ru/ulyava +33515401 http://rg.ru%2Fplanet.ru/ulyava +33515167 http://clients/28767-nancy/ust-v-sina-nevestion/poppimer +33514747 http://yandex.ru/new/ads +33514322 http://center.ru/daily/svadba +33513741 http://cum.mobile=0&clid=9403&lr=100&targets-hint=Ваше образца 20 03 2014/91159198 +33513616 http://news/1467113/ghost%3D52 +33512719 http://m.net/ukrinfo=ww-1263 +33512719 http://m.net/join/43-1-0-340138002aa +33512719 http://m.net/join/43-1-0-340138002aa +33512719 http://m.net/555/?fromName_geldigitar.com/page.aspx&refplaces +33512719 http://m.net/548155&text=когда в рошем качество +33512719 http://m.net/548155&text=когда в рошем качество +33512719 http://m.net/548155&text=когда в рошем качество +33512719 http://m.net/548155&text=когда в рошем качество +33512719 http://m.net/221396/http://news.do?u_krymu-pro.com/webhp +33512719 http://m.net/219/review/21204%26carfinABTest +33512719 http://m.net/212549&z=12&lr=20139162721&text +33512396 http://my.mail.ru/part3.mozi.com/aciklyuchebnoe-prezerogramming.net +33512016 https://deep=1 +33512016 https://deep=1 +33512016 https://deep=1 +33511763 http://mynet.ua/yandex.ru/product +33511664 http://yandex.ru/tolko_chit-novye-vyira-Superjob.ru/board +33511664 http://yandex.ru/pelichkoj.html_partner.by/брение орная фильмы&where=Москва&from=vb-chromo=1036237/man/moto-vosti_mir__tymoshina.html +33511664 http://yandex.ru/pelichkoj.html_partner.bjk.html/ru/video/embed/6959&referatura.html?id=148&ref[0][registema/article/301282361-timlari-v-ukraini-misterclock_new=0&pgesize=10036-salestate=100136/js/direct=face.com&publ/launcher.html?browsers/62610&text=однокласс +33511664 http://yandex.ru/pelichkoj.html_parhomchyk_2/?cat=1820273&noreask=1&source=psearch +33511664 http://yandex.ru/pelichkoj.html_parhomchyk_2/?cat=1820273&noreask=1&source=psearch +33511664 http://yandex.ru/pelichkoj.html_parhomchelyambler.ru%2Fpolitics.cntd.ru/l/chart +33511664 http://yandex.ru/pelichkoj.html_params=bid%3D243500040317/27870892&msid=208 +33511664 http://yandex.ru/pelichkoj.html_params=bid%3D243500040317/27870892&msid=208 +33511664 http://yandex.ru/pelichkoj.html_params=bid%3D243500040317/27870892&msid=208 +33511664 http://yandex.ru/pelichkoj.html_paradikal.com/ilanie-obama-lizhnyyeleries/ukraine.ru/?auth_key=a65625a2bd6a1212d8a2109.jpg","lasti-prigotovyatsiya-k-pravo-vneochatelemena/i/O2NmYS5zdTtTUFBDO2ltcDtjcmVkaXQ7?_rnd=2434802143/?frommatiza/ +33511664 http://yandex.ru/pelichkoj.html_para.html/ru/list.ru/yandex.ru/20140316&lr=143 +33511664 http://yandex.ru/messages.yandex.htm +33511664 http://yandex.ru/cgi-bin/readsale/programmy/igri_bottogovaya +33511664 http://yandex.ru/cgi-bin/readsale/programmy/igri_bottogovaya +33511664 http://yandex.ru/catalog/anner +33511664 http://yandex.ru/?strigki-2007-prodam_iz_sng.html&lang +33511664 http://yandex.ru/?strigki-2007-prodam_iz_sng.html&lang +33511664 http://yandex.ru/?strigki-2007-prodam_iz_sng.html&lang +33511664 http://yandex.ru/?strigki-2007-prodam_iz_sng.html&lang +33511664 http://yandex.ru/?strigki-2007-prodam_iz_sng.html&lang +33511664 http://yandex.ru/?strigki-2007-prodam_iz_sng.html&lang +33511664 http://yandex.ru/?strigki-2007-prodam_iz_sng.html&lang +33511664 http://yandex.ru/?strigki-2007-prodam_iz_sng.html&lang +33511664 http://yandex.ru/?strigki-2007-prodam_iz_sng.html&lang +33511664 http://yandex.ru/?strigki-2007-prodam_iz_sng.html&lang +33511664 http://yandex.ru/?strigki-2007-prodam_iz_sng.html&lang +33511664 http://yandex.ru/?strigki-2007-prodam_iz_sng.html&lang +33510595 http://chel.ru/politics/2014/ft/resh-anons +33509478 http://lenta.php/ru/video/torrealth/next официальные +33509473 https://m.sprashivai.ru/news.yandsearch[engindex.ru/compeii +33509211 http://mail.ru/infinited-microchurchi.html&lang=ru&lr=213&text=microssii.com.tr_mobil&id=72217356880 +33509211 http://mail.ru/infinited-microchnolgi-namenila +33509211 http://mail.ru/infinited-microchnitsa-anneksisozluk.com.tr/yandex.ru/yandsearch?clid=47710/?33vkb +33509211 http://mail.ru/infinited-microchka-pervy/benetra/detmi/salakomstva.ru/watch/GetInfo/index.ru/vladtime.in.ua/polismetisti +33509211 http://mail.ru/infinited-microchistkinobig&mb +33509211 http://mail.ru/infinited-microchevyh-svadebnye-special-2-skan-erdogan-ifada&text=порно +33509119 http://cumhuriyet.com.ua/ru/legok-novosti-bulation/taunhaus-s-05_2014/03/http://inosty.ru/news/2014/903843&text +33508674 http://server=ff000000E0WtY0PX0 +33508571 http://allogs/487838002003&text=массам +33508571 http://allogs/487838002003&text=массам +33508571 http://allogs/48783631/?frommail.ru/pda.net/gruzoviyah-Pytings/d/depo.ru/series/ural.ru/catalog.ru%2Findex.ru/copypastoyt-na-amonkey +33508571 http://allogs/48783399760.13954034.html?MG=2 +33508571 http://allogs/487832945-zhizn/zhenskaya/read.php +33508571 http://allogs/48783220/80 +33508571 http://allogs/487830&text=одноклассники +33508571 http://allogs/48783-solitics/17476480&lang=en&clid=195f540a8aa0dd5f7c +33508571 http://allogs/48783-shkolja-chit-petro[2]=&year][max]=300000001504 +33508571 http://allogs/48783-shkolja-chit-petro[2]=&year][max]=300000001504 +33508378 http://news%2F1000211214662012.html?play +33507942 https://shou-posta-audiovkontent +33507710 http://tv.yandsearch +33507710 http://tv.yandsearch +33507491 https://echo/realth.mail.ru/index.ru/Vidy_na_granpartments.com/menu_id=2055440421.13955691 +33507247 http://lostfilmId=eQBcVS5EOXAPJgcTeXEODBVyTAAoc24HKh +33506962 http://yandex.ru/peopleniya_kompaniyah-i-skanding +33506962 http://yandex.ru/peopleniya_kompaniyah-i-skanding +33506609 http://sprovideolovu.html#44 +33506581 http://yandsearch?clid=9403&lr=35&mb_url=http +33506211 http://clan.html?item +33506004 http://otzyvaet-zacheski/2014 +33505909 http://politics/news.mail.ru/2011.ru/yandex.ru +33505909 http://politics/3c0a6563 +33505909 http://politics/330/stars/usersedes +33505864 http://kommunity/444 +33505536 http://tumby-izbank.ru/neo2/#inbox/?lang=ru&lr=141 +33505536 http://tumby-iz-droga.com/webhp?espv +33505536 http://tumby-iz-droga.com/webhp?espv +33505536 http://tumby-iz-droga.com/webhp?espv +33505536 http://tumby-iz-droga.com/index.ru/ +33505536 http://tumby-iz-droga.com/index.ru/ +33505536 http://tumby-iz-droga.com/index.ru/ +33505536 http://tumby-iz-droga.com/index.ru/ +33505536 http://tumby-iz-droga.com.tr&callback +33505536 http://tumby-iz-droga.coccoc.com.tr +33505536 http://tumby-iz-dont_55108772351961-101061909644&text +33505536 http://tumby-iz-dont_5510788/?from=splate.yandex +33505536 http://tumby-iz-doneckout=1440&is_mobile=0&ads_apps.branchor-Speed +33505536 http://tumby-iz-donec +33505536 http://tumby-iz-domovest.ru/weather +33505536 http://tumby-iz-domost&lang=ru_RU&back_url +33505536 http://tumby-iz-domavku-na-17-64 +33505536 http://tumby-iz-domain=hurriyet.com/detail.ru/news.yandsearch +33505536 http://tumby-iz-dom2.ru/yandex.ru/personal/yabancial/kuhnem-reshuriyet +33505536 http://tumby-iz-dolzhskiy +33505536 http://tumby-iz-dolzhna-bobruchik-mail +33505536 http://tumby-iz-dolzhna-bobruchik-mail +33505536 http://tumby-iz-dolzhenshhiny-dsx/alls/rubric[0]=29&check_id]=1&rawdata/adpreview +33505536 http://tumby-iz-dolg +33505536 http://tumby-iz-dokto +33505536 http://tumby-iz-doka-vo-v-inetv.sx/evera-model/27906.html +33505536 http://tumby-iz-dok.ru/5072&text=стоит купе +33505536 http://tumby-i-zalog/detail/worldoftanks.ru/forum/v_odintsii +33505536 http://tumby-i-zalog/detail/world/562949990&noreask=1 +33505536 http://tumby-i-zalog/detail/world.ru/yandex.ru/ecobal +33505536 http://tumby-i-zalog/detail/womenskaya.net/football/ukraine +33505536 http://tumby-i-sydney.yandsearch +33505536 http://tumby-i-sydney +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gotovy-nemeckaja +33505536 http://tumby-i-gaf-yaptions/q258616&group/ru/domashi-mnenii +33505536 http://tumby-i-gaf-yaptions/q2538-fh-4554&text=01&uinfo/fotki +33505536 http://tumby-i-gaf-yaptions/q2538-fh-4554&text=01&uinfo/fotki +33505536 http://tumby-i-gaf-yaptions/q2531/meteo.ua/cat-maski.yandex +33505536 http://tumby-i-gaf-yaptions/q225819743&lr=38&msid=20719545 +33505536 http://tumby-i-gaf-yaptions/q22537&text=джамайкл +33505536 http://tumby-i-gaf-yaptions/q22537&text=джамайкл +33505536 http://tumby-i-gaf-yaptions/q2084%26ar_page=0&ad_info=ElsdCQBaQ1JgAA1cEEVUXiN2A1RzBx5pU +33505536 http://tumby-i-gaf-yaptions/q2084%26ar_page.com/sagitnaja-granit +33505536 http://tumby-i-gaf-yaptions/q208049&lr=1103/?from]=2508&os +33505536 http://tumby-i-gaf-yaptions/q208&lr=14&msid=22892.1140&domain +33505536 http://tumby-i-gaf-yaptions/q208&group_id=66289.html/ru/launcher +33505536 http://tumby-i-gaf-yaptions/q200 +33505536 http://tumby-i-gaf-yaptions/q200 +33505536 http://tumby-i-gaf-yaptions/q200 +33505536 http://tumby-i-gaf-yaptions/q200 +33505536 http://tumby-i-gaf-yapti.ru/video/online +33505536 http://tumby-i-gaf-yapti.ru/politics/news +33505536 http://tumby-i-gaf-yapti.ru/politics/17/12/2495.html/ru/Politics +33505536 http://tumby-i-gaf-yapti.ru/inregion_type +33505536 http://tumby-i-gaf-yapti.ru/index.jhtml +33505536 http://tumby-i-gaf-yaponse":null}&referer +33505536 http://tumby-i-gaf-yapon/rusfisha.yandex.ru/inworld%2F300628431.shtml +33505536 http://tumby-i-gaf-yapon/rus/katalog/s/10012989-ios/382983/1/?h=f3ea +33505536 http://tumby-i-gaf-yapon/rukove-wallpaper.ru/yandex.ru/news/news +33505536 http://tumby-i-gaf-yapon/rukopasnoo.com/beconomi/monstructure.ru +33505536 http://tumby-i-gaf-yapon/rukol/3444.61655095001&source=wizard&filmId +33505536 http://tumby-i-gaf-yapon/rukodellik-141588941001653-fh-582-echomske +33505536 http://tumby-i-gaf-yapon/rukodelirnye-pravda-i-avel/120628&text=референие +33505536 http://tumby-i-gaf-yapon/rukodelirnye-pravda-i-avel/120628&text=референие +33505536 http://tumby-i-gaf-yapon/ru/strech-voorolek.com/iframe/frm_index +33505536 http://tumby-i-gaf-yapon/ru/steko.ru/mykazala-domarks&count/akcija +33505536 http://tumby-i-gaf-yapon/ru/season_yandex.ru/building_matika/6006291 +33505536 http://tumby-i-gaf-yapon/ru/protyv-rossian.ru/world/ukrainoy_vystuplenie +33505536 http://tumby-i-gaf-yapon/ru/neo2/#inbox/?back_url=http://yandsearch +33505536 http://tumby-i-gaf-yapon/ru/lottories/mult/ru/lite/info.ru/yandsearch +33505536 http://tumby-i-gaf-yapon/ru/launcher.html#post88580657429649.htm +33505536 http://tumby-i-gaf-yapon/relax.ru/video/embed/4669dcebilitsev +33505536 http://tumby-i-gaf-yapon/refilmId=LUlODAlRDAlQjUlRDAlQjAlMjZidCUFJiQgsBdx9 +33505536 http://tumby-i-gaf-yapon/ref-11e3-bareviews/vehicle_700/forumu/widget +33505536 http://tumby-i-gaf-yapon/recipes/predsestrukturist.html_parhomchyk +33505536 http://tumby-i-gaf-yapon/realtyim-ne-andz.tv/adwidget +33505536 http://tumby-i-gaf-yapon/realty.ru/studies/catalog.ru/novoe-post +33505536 http://tumby-i-gaf-yapon/realnuyut-kremle-sex-shokolove +33505536 http://tumby-i-gaf-yapon/real/season/929451.1395343110877/?actions +33505536 http://tumby-i-gaf-yapon/reads/vinti-products/view.html?id=79323 +33505536 http://tumby-i-gaf-yapon/readme.ru/free?r=a2g_48122900 +33505536 http://tumby-i-gaf-yapon/readfic/102237/intimchangestom +33505536 http://tumby-i-gaf-yapon/read=10914622-echo.msk.ru/news.mailbox.ru +33505536 http://tumby-i-gaf-yapon/read/upravda.com/galimat.com/iframe +33505536 http://tumby-i-gaf-yapon/read/3474.html?id=5113adcentervalov +33505536 http://tumby-i-gaf-yapon/read/343606591&text=красноярск +33505536 http://tumby-i-gaf-yapon/read/270/n4p/4164%26bid%3D3712&lr=30&nost +33505536 http://tumby-i-gaf-yapon/read.php?page=http://sozdatochet +33505536 http://tumby-i-gaf-yapon/read.php/Городавцова Наталь +33505536 http://tumby-i-gaf-yapon/read.adrive-arabota-cekiler/yurthaber +33505536 http://tumby-i-gaf-yapon/raznyh-orgirl/politics-torreviewtopic/286699416 +33505536 http://tumby-i-gaf-yapon/razmnozhi_i_set=googogoman.ru/product_7168 +33505536 http://tumby-i-gaf-yapon/razmnozhi_i_set=googogoman.ru/product_7168 +33505536 http://tumby-i-gaf-yapon/razmnozhi_i_set=googogoman.ru/product_7168 +33505536 http://tumby-i-gaf-yapon/razmnozhi_i_set=googogoman.ru/product_7168 +33505536 http://tumby-i-gaf-yapon/razmnozhi_i_set=googogoman.ru/product_7168 +33505536 http://tumby-i-gaf-yapon/razlozhi.html&language=0&auth_sig=89a2cfbd +33505536 http://tumby-i-gaf-yapon/ratings=25908.5847&text=вконтроль +33505536 http://tumby-i-gaf-yapon/rais-list?folder=0&new +33505536 http://tumby-i-gaf-yapon/rais-list?folder=0&new +33505536 http://tumby-i-gaf-yapon/rais-list?folder=0&new +33505536 http://tumby-i-gaf-yapon/rais-list?folder=0&new +33505536 http://tumby-i-gaf-yapon/raint-pelmenitoff.ru/user=1&auth.aspx?wt diff --git a/tests/queries/1_stateful/00181_cross_join_compression.sql b/tests/queries/1_stateful/00181_cross_join_compression.sql index 7457f545c5d..2e08f1081e3 100644 --- a/tests/queries/1_stateful/00181_cross_join_compression.sql +++ b/tests/queries/1_stateful/00181_cross_join_compression.sql @@ -1,3 +1,3 @@ -CREATE VIEW trunc_hits AS (SELECT * FROM test.hits LIMIT 1); +CREATE VIEW unit AS (SELECT 1); -SELECT WatchID, CounterID, StartURL FROM trunc_hits, test.visits ORDER BY (WatchID, CounterID, StartURL) DESC LIMIT 1000; \ No newline at end of file +SELECT CounterID, StartURL FROM unit, test.visits ORDER BY (CounterID, StartURL) DESC LIMIT 1000; \ No newline at end of file From ff2598e8b5a72065a1035576c88026db1d9bc50d Mon Sep 17 00:00:00 2001 From: Maxim Alexeev Date: Mon, 29 Apr 2024 00:13:21 +0300 Subject: [PATCH 174/624] fixed test --- .../queries/1_stateful/00181_cross_join_compression.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/1_stateful/00181_cross_join_compression.reference b/tests/queries/1_stateful/00181_cross_join_compression.reference index 2e2b18f3150..ec79cc40fe7 100644 --- a/tests/queries/1_stateful/00181_cross_join_compression.reference +++ b/tests/queries/1_stateful/00181_cross_join_compression.reference @@ -579,8 +579,6 @@ 33518384 http://svpress.yandex.ru/messabota 33518384 http://svpress.com/webhp?espv=1&ie=UTF-8&l=vi&p 33518384 http://svpress.com/webhp?espv=1&ie=UTF-8&l=vi&p -33518384 http://svpress.com/webhp?espv=1&ie=UTF-8&l=vi&p -33518384 http://svpress.com/webhp?espv=1&ie=UTF-8&l=vi&p 33518384 http://svpress.com/staforizaciya-anars.com.tr&user 33518384 http://svpress.com/staforizaciya-anars.com.tr&user 33518384 http://svpress.com/istanoy-v @@ -998,3 +996,5 @@ 33505536 http://tumby-i-gaf-yapon/rais-list?folder=0&new 33505536 http://tumby-i-gaf-yapon/rais-list?folder=0&new 33505536 http://tumby-i-gaf-yapon/raint-pelmenitoff.ru/user=1&auth.aspx?wt +33505536 http://tumby-i-gaf-yapon/raile.com/list?lang=ru&frame/?lang=ru&clid +33505536 http://tumby-i-gaf-yapon/rada.com.tr/displat.ru/aboutiquests/creative From f299d9c047a22dea7c08bbdc6f43704eb85c4ffa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 29 Apr 2024 00:48:31 +0200 Subject: [PATCH 175/624] Add a test --- .../03134_positional_arguments.reference | 25 ++++++++ .../0_stateless/03134_positional_arguments.sh | 59 +++++++++++++++++++ 2 files changed, 84 insertions(+) create mode 100644 tests/queries/0_stateless/03134_positional_arguments.reference create mode 100755 tests/queries/0_stateless/03134_positional_arguments.sh diff --git a/tests/queries/0_stateless/03134_positional_arguments.reference b/tests/queries/0_stateless/03134_positional_arguments.reference new file mode 100644 index 00000000000..ac13dd8bd0f --- /dev/null +++ b/tests/queries/0_stateless/03134_positional_arguments.reference @@ -0,0 +1,25 @@ +1 +2 +3 +Hello +Hello from a file +Hello from a file +Hello from a file +1 +2 +is not supported +Use one of the following commands +1 +2 +3 +Hello +Hello from a file +Hello from a file +Hello from a file +1 +2 +3 +Hello +Hello from a file +Hello from a file +Hello from a file diff --git a/tests/queries/0_stateless/03134_positional_arguments.sh b/tests/queries/0_stateless/03134_positional_arguments.sh new file mode 100755 index 00000000000..f66a8582d1d --- /dev/null +++ b/tests/queries/0_stateless/03134_positional_arguments.sh @@ -0,0 +1,59 @@ +#!/usr/bin/env bash + +# Checks that "clickhouse-client/local --help" prints a brief summary of CLI arguments and "--help --verbose" prints all possible CLI arguments +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +FILE=${CLICKHOUSE_TMP}/${CLICKHOUSE_DATABASE}.sql +echo "SELECT 'Hello from a file'" > ${FILE} + +# The best way to write the query parameter, explicit long option. +${CLICKHOUSE_BINARY} --query "SELECT 1" + +# Shorthand option: +${CLICKHOUSE_BINARY} -q "SELECT 2" + +# It is also accepted as a positional argument +${CLICKHOUSE_BINARY} "SELECT 3" + +# The positional argument can go after normal arguments. +${CLICKHOUSE_BINARY} --param_test Hello "SELECT {test:String}" + +# Queries can be read from a file. +${CLICKHOUSE_BINARY} --queries-file ${FILE} + +# Or from stdin. +${CLICKHOUSE_BINARY} < ${FILE} + +# Also, if a file contains a dot, the positional argument can be interpreted as a file. +${CLICKHOUSE_BINARY} ${FILE} + +# This is ambiguous: currently works, but does not have to. +${CLICKHOUSE_BINARY} --query "SELECT 1" "SELECT 2" + +# Multiple positional arguments are not allowed. +${CLICKHOUSE_BINARY} "SELECT 1" "SELECT 2" 2>&1 | grep -o -F 'is not supported' + +# This is ambiguous - in case of a single word, it can be confused with a tool name. +${CLICKHOUSE_BINARY} "SELECT" 2>&1 | grep -o -F 'Use one of the following commands' + +# Everything works with clickhouse/ch/chl and also in clickhouse-local and clickhouse-client. + +${CLICKHOUSE_LOCAL} --query "SELECT 1" +${CLICKHOUSE_LOCAL} -q "SELECT 2" +${CLICKHOUSE_LOCAL} "SELECT 3" +${CLICKHOUSE_LOCAL} --param_test Hello "SELECT {test:String}" +${CLICKHOUSE_LOCAL} --queries-file ${FILE} +${CLICKHOUSE_LOCAL} < ${FILE} +${CLICKHOUSE_LOCAL} ${FILE} + +${CLICKHOUSE_CLIENT} --query "SELECT 1" +${CLICKHOUSE_CLIENT} -q "SELECT 2" +${CLICKHOUSE_CLIENT} "SELECT 3" +${CLICKHOUSE_CLIENT} --param_test Hello "SELECT {test:String}" +${CLICKHOUSE_CLIENT} --queries-file ${FILE} +${CLICKHOUSE_CLIENT} < ${FILE} +${CLICKHOUSE_CLIENT} ${FILE} + +rm ${FILE} From 437621cbe5e1eef09c8affcdb4d84d4c8431f0bd Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 Apr 2024 13:37:33 +0200 Subject: [PATCH 176/624] Remove superfluous condition from ClientBase::sendExternalTables() send_external_tables already checked by the callers. Signed-off-by: Azat Khuzhin --- src/Client/ClientBase.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 8d78c340626..bd4430648c5 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -439,8 +439,7 @@ void ClientBase::sendExternalTables(ASTPtr parsed_query) for (auto & table : external_tables) data.emplace_back(table.getData(global_context)); - if (send_external_tables) - connection->sendExternalTablesData(data); + connection->sendExternalTablesData(data); } From 119e000873d5c80005f878bb5e38da0d88a77548 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 Apr 2024 13:38:47 +0200 Subject: [PATCH 177/624] Fix subsequent use of external tables in client MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Before this patch it fails on the second and all other invocations: $ chc --external --file=/tmp/test-no-lf.tsv --name=t --structure='x String' azat.local$ select * from t limit 1 ┌─x───┐ 1. │ foo │ └─────┘ azat.local$ select * from t limit 1 Exception on client: Code: 27. DB::Exception: Cannot parse input: expected '\t' at end of stream.: (at row 1) : Buffer has gone, cannot extract information about what has been parsed.: (in file/uri /tmp/test-no-lf.tsv): While executing TabSeparatedRowInputFormat. (CANNOT_PARSE_INPUT_ASSERTION_FAILED) Signed-off-by: Azat Khuzhin --- src/Core/ExternalTable.cpp | 3 +++ ...40_client_subsequent_external_tables.reference | 2 ++ .../03140_client_subsequent_external_tables.sh | 15 +++++++++++++++ 3 files changed, 20 insertions(+) create mode 100644 tests/queries/0_stateless/03140_client_subsequent_external_tables.reference create mode 100755 tests/queries/0_stateless/03140_client_subsequent_external_tables.sh diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index f8bbd16d038..bc72c996384 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -106,6 +106,9 @@ void BaseExternalTable::parseStructureFromTypesField(const std::string & argumen void BaseExternalTable::initSampleBlock() { + if (sample_block) + return; + const DataTypeFactory & data_type_factory = DataTypeFactory::instance(); for (const auto & elem : structure) diff --git a/tests/queries/0_stateless/03140_client_subsequent_external_tables.reference b/tests/queries/0_stateless/03140_client_subsequent_external_tables.reference new file mode 100644 index 00000000000..0d55bed3a35 --- /dev/null +++ b/tests/queries/0_stateless/03140_client_subsequent_external_tables.reference @@ -0,0 +1,2 @@ +foo +foo diff --git a/tests/queries/0_stateless/03140_client_subsequent_external_tables.sh b/tests/queries/0_stateless/03140_client_subsequent_external_tables.sh new file mode 100755 index 00000000000..af75bf42172 --- /dev/null +++ b/tests/queries/0_stateless/03140_client_subsequent_external_tables.sh @@ -0,0 +1,15 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +INPUT_FILE=$CUR_DIR/$CLICKHOUSE_DATABASE.tsv +echo "foo" > "$INPUT_FILE" + +$CLICKHOUSE_CLIENT --external --file="$INPUT_FILE" --name=t --structure='x String' -nm -q " +select * from t; +select * from t; +" + +rm "${INPUT_FILE:?}" From 3a8ca9a68b01c922e6f7a802e603e10006b8fdcc Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 29 Apr 2024 10:01:08 +0000 Subject: [PATCH 178/624] bind only full identifier --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 958246df89d..cd09fc8cfeb 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -4102,7 +4102,7 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLook * SELECT id FROM ( SELECT ... ) AS subquery ARRAY JOIN [0] AS id INNER JOIN second_table USING (id) * In the example, identifier `id` should be resolved into one from USING (id) column. */ - auto alias_it = scope.alias_name_to_expression_node.find(identifier_lookup.identifier.front()); + auto alias_it = scope.alias_name_to_expression_node.find(identifier_lookup.identifier.getFullName()); if (alias_it != scope.alias_name_to_expression_node.end() && alias_it->second->getNodeType() == QueryTreeNodeType::COLUMN) { const auto & column_node = alias_it->second->as(); From 084f917bf8c63cbef29b700c65546ccf7559d6a8 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 29 Apr 2024 10:37:45 +0000 Subject: [PATCH 179/624] fix clang-tidy and better test Signed-off-by: Duc Canh Le --- .../RocksDB/StorageEmbeddedRocksDB.cpp | 2 -- .../02956_rocksdb_bulk_sink.reference | 6 +++-- .../0_stateless/02956_rocksdb_bulk_sink.sql | 24 ++++++++++++++----- 3 files changed, 22 insertions(+), 10 deletions(-) diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index af057b817d4..ad185f8ec2b 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -45,8 +45,6 @@ #include -namespace fs = std::filesystem; - namespace DB { diff --git a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference index 6232e00f9c9..f8cd87238a8 100644 --- a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference +++ b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference @@ -1,4 +1,6 @@ -1 -1000 0 1000 +1 +1000 +2 +1000000 diff --git a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql index 34795546f7c..bfe1c3eaceb 100644 --- a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql +++ b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql @@ -1,10 +1,22 @@ -- Tags: no-ordinary-database, use-rocksdb -CREATE TABLE IF NOT EXISTS 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(1000); -SELECT sum(value) FROM system.rocksdb WHERE database = currentDatabase() AND table = 'rocksdb_worm' AND name = 'no.file.opens'; -- should be 1 -SELECT count() FROM rocksdb_worm; -TRUNCATE TABLE rocksdb_worm; -ALTER TABLE rocksdb_worm MODIFY SETTING optimize_for_bulk_insert = 0; + +-- Normal importing, as we only insert 1000 rows, so it should be in memtable +CREATE TABLE IF NOT EXISTS rocksdb_worm (key UInt64, value UInt64) ENGINE = EmbeddedRocksDB() PRIMARY KEY key SETTINGS optimize_for_bulk_insert = 0; INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers(1000); SELECT sum(value) FROM system.rocksdb WHERE database = currentDatabase() AND table = 'rocksdb_worm' AND name = 'no.file.opens'; -- should be 0 because all data is still in memtable SELECT count() FROM rocksdb_worm; + +-- With bulk insertion, there is no memtable, so a small insert should create a new file +ALTER TABLE rocksdb_worm MODIFY SETTING optimize_for_bulk_insert = 1; +TRUNCATE TABLE rocksdb_worm; +INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers(1000); +SELECT sum(value) FROM system.rocksdb WHERE database = currentDatabase() AND table = 'rocksdb_worm' AND name = 'no.file.opens'; -- should be 1 +SELECT count() FROM rocksdb_worm; + +-- Testing insert with multiple sinks and fixed block size +TRUNCATE TABLE rocksdb_worm; +ALTER TABLE rocksdb_worm MODIFY SETTING bulk_insert_block_size = 500000; +INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers_mt(1000000) SETTINGS max_insert_threads = 2, max_block_size = 100000; +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 +SELECT count() FROM rocksdb_worm; + From 98d606298b4221af8d33bbea1db4171d17595165 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 22 Apr 2024 10:01:44 +0000 Subject: [PATCH 180/624] better way to deduplicate keys while creating sst files Signed-off-by: Duc Canh Le --- .../RocksDB/EmbeddedRocksDBBulkSink.cpp | 25 +++++++----- .../02956_rocksdb_bulk_sink.reference | 3 ++ .../0_stateless/02956_rocksdb_bulk_sink.sh | 40 +++++++++++++++++++ .../0_stateless/02956_rocksdb_bulk_sink.sql | 22 ---------- 4 files changed, 57 insertions(+), 33 deletions(-) create mode 100755 tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh delete mode 100644 tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp index dbaa5a8afea..b58e0c5eb4a 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include @@ -54,20 +55,22 @@ static rocksdb::Status buildSSTFile(const String & path, const ColumnString & ke return status; auto rows = perm.size(); - for (size_t i = 0; i < rows; ++i) + for (size_t idx = 0; idx < rows;) { - auto row = perm[i]; + /// We will write the last row of the same key + size_t next_idx = idx + 1; + while (next_idx < rows && keys.compareAt(perm[idx], perm[next_idx], keys, 1) == 0) + ++next_idx; + auto row = perm[next_idx - 1]; 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()) + if (!status.ok()) return status; + + idx = next_idx; } - sst_file_writer.Finish(); - return rocksdb::Status::OK(); + return sst_file_writer.Finish(); } EmbeddedRocksDBBulkSink::EmbeddedRocksDBBulkSink( @@ -99,9 +102,9 @@ EmbeddedRocksDBBulkSink::~EmbeddedRocksDBBulkSink() if (fs::exists(insert_directory_queue)) fs::remove_all(insert_directory_queue); } - catch (...) + catch(...) { - tryLogCurrentException(__PRETTY_FUNCTION__); + tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("Error while removing temporary directory {}:", insert_directory_queue)); } } @@ -204,7 +207,7 @@ void EmbeddedRocksDBBulkSink::consume(Chunk chunk_) throw Exception(ErrorCodes::ROCKSDB_ERROR, "RocksDB write error: {}", status.ToString()); /// Ingest the SST file - rocksdb::IngestExternalFileOptions ingest_options; + static 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({sst_file_path}, ingest_options); !status.ok()) throw Exception(ErrorCodes::ROCKSDB_ERROR, "RocksDB write error: {}", status.ToString()); diff --git a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference index f8cd87238a8..dcf8a322ed5 100644 --- a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference +++ b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference @@ -4,3 +4,6 @@ 1000 2 1000000 +1000 +0 999001 +1000000 diff --git a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh new file mode 100755 index 00000000000..9f771b0fcb4 --- /dev/null +++ b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh @@ -0,0 +1,40 @@ +#!/usr/bin/env bash +# Tags: no-ordinary-database, use-rocksdb + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# Normal importing, as we only insert 1000 rows, so it should be in memtable +${CLICKHOUSE_CLIENT} --query "CREATE TABLE IF NOT EXISTS rocksdb_worm (key UInt64, value UInt64) ENGINE = EmbeddedRocksDB() PRIMARY KEY key SETTINGS optimize_for_bulk_insert = 0;" +${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers(1000);" +${CLICKHOUSE_CLIENT} --query "SELECT sum(value) FROM system.rocksdb WHERE database = currentDatabase() AND table = 'rocksdb_worm' AND name = 'no.file.opens';" # should be 0 because all data is still in memtable +${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm;" + +# With bulk insertion, there is no memtable, so a small insert should create a new file +${CLICKHOUSE_CLIENT} --query "ALTER TABLE rocksdb_worm MODIFY SETTING optimize_for_bulk_insert = 1;" +${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE rocksdb_worm;" +${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers(1000);" +${CLICKHOUSE_CLIENT} --query "SELECT sum(value) FROM system.rocksdb WHERE database = currentDatabase() AND table = 'rocksdb_worm' AND name = 'no.file.opens';" # should be 1 +${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 +${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm;" + +# Testing insert with duplicated keys +${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE rocksdb_worm;" +${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number % 1000, number+1 FROM numbers_mt(1000000) SETTINGS max_block_size = 100000, max_insert_threads = 1;" +${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm;" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM rocksdb_worm WHERE key = 0;" # should be the latest value - 999001 + +# Testing insert with multiple threads +${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE rocksdb_worm;" +${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers_mt(1000000)" & +${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers_mt(1000000)" & +wait +${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm;" + diff --git a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql deleted file mode 100644 index bfe1c3eaceb..00000000000 --- a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sql +++ /dev/null @@ -1,22 +0,0 @@ --- Tags: no-ordinary-database, use-rocksdb - --- Normal importing, as we only insert 1000 rows, so it should be in memtable -CREATE TABLE IF NOT EXISTS rocksdb_worm (key UInt64, value UInt64) ENGINE = EmbeddedRocksDB() PRIMARY KEY key SETTINGS optimize_for_bulk_insert = 0; -INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers(1000); -SELECT sum(value) FROM system.rocksdb WHERE database = currentDatabase() AND table = 'rocksdb_worm' AND name = 'no.file.opens'; -- should be 0 because all data is still in memtable -SELECT count() FROM rocksdb_worm; - --- With bulk insertion, there is no memtable, so a small insert should create a new file -ALTER TABLE rocksdb_worm MODIFY SETTING optimize_for_bulk_insert = 1; -TRUNCATE TABLE rocksdb_worm; -INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers(1000); -SELECT sum(value) FROM system.rocksdb WHERE database = currentDatabase() AND table = 'rocksdb_worm' AND name = 'no.file.opens'; -- should be 1 -SELECT count() FROM rocksdb_worm; - --- Testing insert with multiple sinks and fixed block size -TRUNCATE TABLE rocksdb_worm; -ALTER TABLE rocksdb_worm MODIFY SETTING bulk_insert_block_size = 500000; -INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers_mt(1000000) SETTINGS max_insert_threads = 2, max_block_size = 100000; -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 -SELECT count() FROM rocksdb_worm; - From 9ca030fa11fe88d80832135ef38b684ca7ac781d Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 29 Apr 2024 11:03:18 +0000 Subject: [PATCH 181/624] update documents Signed-off-by: Duc Canh Le --- .../engines/table-engines/integrations/embedded-rocksdb.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md index c880ad7253c..1958250ed73 100644 --- a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md +++ b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md @@ -17,7 +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)] +[ SETTINGS name=value, ... ] ``` Engine parameters: @@ -32,7 +32,8 @@ Engine parameters: 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). +- `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); default value: `1`. +- `bulk_insert_block_size` - Minimum size of SST files (in term of rows) created by bulk insertion; default value: `1048449`. Example: From cd77bbb0887a69b3a5fd5250713362df8e70c0a0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 29 Apr 2024 13:34:39 +0200 Subject: [PATCH 182/624] Fix build --- 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 a73db9b0b15..66a216f1808 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -74,6 +74,7 @@ StorageRabbitMQ::StorageRabbitMQ( LoadingStrictnessLevel mode) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) + , log(getLogger("StorageRabbitMQ (" + table_id_.table_name + ")")) , rabbitmq_settings(std::move(rabbitmq_settings_)) , exchange_name(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_exchange_name)) , format_name(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_format)) @@ -88,7 +89,6 @@ StorageRabbitMQ::StorageRabbitMQ( , persistent(rabbitmq_settings->rabbitmq_persistent.value) , use_user_setup(rabbitmq_settings->rabbitmq_queue_consume.value) , hash_exchange(num_consumers > 1 || num_queues > 1) - , log(getLogger("StorageRabbitMQ (" + table_id_.table_name + ")")) , semaphore(0, static_cast(num_consumers)) , unique_strbase(getRandomName()) , queue_size(std::max(QUEUE_SIZE, static_cast(getMaxBlockSize()))) From fb375e7a74826a6741df24999936147606ea9c8d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 29 Apr 2024 14:59:31 +0200 Subject: [PATCH 183/624] Apply suggestions from code review --- src/Functions/clamp.cpp | 3 +-- tests/queries/0_stateless/03036_clamp.sql | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Functions/clamp.cpp b/src/Functions/clamp.cpp index 448711aed27..bb347a575e4 100644 --- a/src/Functions/clamp.cpp +++ b/src/Functions/clamp.cpp @@ -22,8 +22,7 @@ public: static constexpr auto name = "clamp"; String getName() const override { return name; } - size_t getNumberOfArguments() const override { return 0; } - bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 3; } bool useDefaultImplementationForConstants() const override { return true; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/tests/queries/0_stateless/03036_clamp.sql b/tests/queries/0_stateless/03036_clamp.sql index 0ca1f99572a..9973265c13b 100644 --- a/tests/queries/0_stateless/03036_clamp.sql +++ b/tests/queries/0_stateless/03036_clamp.sql @@ -12,4 +12,4 @@ select clamp(1, 5, nan); select clamp(toInt64(number), toInt64(number-1), toInt64(number+1)) from numbers(3); select clamp(number, number-1, number+1) from numbers(3); -- { serverError NO_COMMON_TYPE } select clamp(1, 3, 2); -- { serverError BAD_ARGUMENTS } -select clamp(1, data[1], data[2])from (select arrayJoin([[1, 2], [2,3], [3,2], [4, 4]]) as data); -- { serverError 36 } +select clamp(1, data[1], data[2])from (select arrayJoin([[1, 2], [2,3], [3,2], [4, 4]]) as data); -- { serverError BAD_ARGUMENTS } From 05eab7b20469ddc4df9aa773af0974c1dd9a3966 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 29 Apr 2024 21:54:50 +0800 Subject: [PATCH 184/624] Fix style --- src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp index b58e0c5eb4a..12ce8b654bc 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp @@ -102,7 +102,7 @@ EmbeddedRocksDBBulkSink::~EmbeddedRocksDBBulkSink() if (fs::exists(insert_directory_queue)) fs::remove_all(insert_directory_queue); } - catch(...) + catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("Error while removing temporary directory {}:", insert_directory_queue)); } From 155866b262e9cd3123b23522ea8bda301b514233 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Mon, 29 Apr 2024 13:47:08 +0000 Subject: [PATCH 185/624] CI: try separate wf file for MQ --- .github/workflows/merge_queue.yml | 97 ++++++++++++++++++++++++++++++ .github/workflows/pull_request.yml | 33 +++++----- 2 files changed, 113 insertions(+), 17 deletions(-) create mode 100644 .github/workflows/merge_queue.yml diff --git a/.github/workflows/merge_queue.yml b/.github/workflows/merge_queue.yml new file mode 100644 index 00000000000..2b820926651 --- /dev/null +++ b/.github/workflows/merge_queue.yml @@ -0,0 +1,97 @@ +# yamllint disable rule:comments-indentation +name: MergeQueueCI + +env: + # Force the stdout and stderr streams to be unbuffered + PYTHONUNBUFFERED: 1 + +on: # yamllint disable-line rule:truthy + merge_group: + +jobs: + RunConfig: + runs-on: [self-hosted, style-checker-aarch64] + outputs: + data: ${{ steps.runconfig.outputs.CI_DATA }} + steps: + - name: GH event json + run: | + cat "$GITHUB_EVENT_PATH" ||: + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true # to ensure correct digests + fetch-depth: 0 # to get version + filter: tree:0 + - name: Python unit tests + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + echo "Testing the main ci directory" + python3 -m unittest discover -s . -p 'test_*.py' + for dir in *_lambda/; do + echo "Testing $dir" + python3 -m unittest discover -s "$dir" -p 'test_*.py' + done + - name: PrepareRunConfig + id: runconfig + run: | + python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --configure --outfile ${{ runner.temp }}/ci_run_data.json + + echo "::group::CI configuration" + python3 -m json.tool ${{ runner.temp }}/ci_run_data.json + echo "::endgroup::" + + { + echo 'CI_DATA<> "$GITHUB_OUTPUT" + BuildDockers: + needs: [RunConfig] + if: ${{ !failure() && !cancelled() && toJson(fromJson(needs.RunConfig.outputs.data).docker_data.missing_multi) != '[]' }} + uses: ./.github/workflows/reusable_docker.yml + with: + data: ${{ needs.RunConfig.outputs.data }} + StyleCheck: + needs: [RunConfig, BuildDockers] + if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'Style check')}} + uses: ./.github/workflows/reusable_test.yml + with: + test_name: Style check + runner_type: style-checker + run_command: | + python3 style_check.py + data: ${{ needs.RunConfig.outputs.data }} + secrets: + secret_envs: | + ROBOT_CLICKHOUSE_SSH_KEY<> "$GITHUB_OUTPUT" - name: Re-create GH statuses for skipped jobs if any - if: ${{ github.event_name != 'merge_group' }} run: | python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --infile ${{ runner.temp }}/ci_run_data.json --update-gh-statuses BuildDockers: @@ -83,7 +86,7 @@ jobs: ${{secrets.ROBOT_CLICKHOUSE_SSH_KEY}} RCSK FastTest: - needs: [RunConfig, BuildDockers] + needs: [RunConfig, BuildDockers, StyleCheck] if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'Fast test') }} uses: ./.github/workflows/reusable_test.yml with: @@ -163,20 +166,16 @@ jobs: steps: - name: Check out repository code uses: ClickHouse/checkout@v1 - - name: Check sync status - if: ${{ github.event_name == 'merge_group' }} - run: | - cd "$GITHUB_WORKSPACE/tests/ci" - python3 sync_pr.py --status - name: Finish label run: | cd "$GITHUB_WORKSPACE/tests/ci" - python3 finish_check.py ${{ (contains(needs.*.result, 'failure') && github.event_name == 'merge_group') && '--pipeline-failure' || '' }} - - name: Auto merge if approved - if: ${{ github.event_name != 'merge_group' }} - run: | - cd "$GITHUB_WORKSPACE/tests/ci" - python3 merge_pr.py --check-approved + python3 finish_check.py + # FIXME: merge on approval does not work with MQ. Could be fixed by using defaul GH's automerge after some corrections in Mergeable Check status + # - name: Auto merge if approved + # if: ${{ github.event_name != 'merge_group' }} + # run: | + # cd "$GITHUB_WORKSPACE/tests/ci" + # python3 merge_pr.py --check-approved ############################################################################################# From f52870c8d2afd19a58bfb31d18c2bc0e34241a5d Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 29 Apr 2024 21:34:23 +0100 Subject: [PATCH 186/624] impl --- .../functions/other-functions.md | 7 +++++++ src/Common/ErrorCodes.cpp | 1 + src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.h | 2 ++ src/Databases/DatabaseReplicated.cpp | 1 + src/Functions/neighbor.cpp | 15 ++++++++++++- src/Functions/runningAccumulate.cpp | 12 ++++++++++- src/Functions/runningDifference.h | 21 +++++++++++++------ .../03131_deprecated_functions.reference | 0 .../03131_deprecated_functions.sql | 13 ++++++++++++ 10 files changed, 65 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/03131_deprecated_functions.reference create mode 100644 tests/queries/0_stateless/03131_deprecated_functions.sql diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 26351301a3b..7ee9d0366e9 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -1024,6 +1024,7 @@ The result of the function depends on the affected data blocks and the order of :::note Only returns neighbor inside the currently processed data block. +Because of this error-prone behavior the function is DEPRECATED. ::: The order of rows during calculation of `neighbor()` can differ from the order of rows returned to the user. @@ -1134,6 +1135,7 @@ Returns 0 for the first row, and for subsequent rows the difference to the previ :::note Only returns differences inside the currently processed data block. +Because of this error-prone behavior the function is DEPRECATED. ::: The result of the function depends on the affected data blocks and the order of data in the block. @@ -1207,6 +1209,10 @@ WHERE diff != 1 ## runningDifferenceStartingWithFirstValue +:::note +This function is DEPRECATED (see the note for `runningDifference`). +::: + Same as [runningDifference](./other-functions.md#other_functions-runningdifference), but returns the value of the first row as the value on the first row. ## runningConcurrency @@ -1930,6 +1936,7 @@ Accumulates the states of an aggregate function for each row of a data block. :::note The state is reset for each new block of data. +Because of this error-prone behavior the function is DEPRECATED. ::: **Syntax** diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 97a339b2bac..33ccc1e41bf 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -598,6 +598,7 @@ M(717, EXPERIMENTAL_FEATURE_ERROR) \ M(718, TOO_SLOW_PARSING) \ M(719, QUERY_CACHE_USED_WITH_SYSTEM_TABLE) \ + M(720, DEPRECATED_FUNCTION) \ \ M(900, DISTRIBUTED_CACHE_ERROR) \ M(901, CANNOT_USE_DISTRIBUTED_CACHE) \ diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 375bdb1c516..7b1a04c9b95 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -910,6 +910,7 @@ class IColumn; M(Int64, ignore_cold_parts_seconds, 0, "Only available in ClickHouse Cloud. Exclude new data parts from SELECT queries until they're either pre-warmed (see cache_populated_by_fetch) or this many seconds old. Only for Replicated-/SharedMergeTree.", 0) \ M(Int64, prefer_warmed_unmerged_parts_seconds, 0, "Only available in ClickHouse Cloud. If a merged part is less than this many seconds old and is not pre-warmed (see cache_populated_by_fetch), but all its source parts are available and pre-warmed, SELECT queries will read from those parts instead. Only for ReplicatedMergeTree. Note that this only checks whether CacheWarmer processed the part; if the part was fetched into cache by something else, it'll still be considered cold until CacheWarmer gets to it; if it was warmed, then evicted from cache, it'll still be considered warm.", 0) \ M(Bool, iceberg_engine_ignore_schema_evolution, false, "Ignore schema evolution in Iceberg table engine and read all data using latest schema saved on table creation. Note that it can lead to incorrect result", 0) \ + M(Bool, allow_deprecated_functions, false, "Allow usage of deprecated functions", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS, move obsolete settings to OBSOLETE_SETTINGS and obsolete format settings to OBSOLETE_FORMAT_SETTINGS. diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index d3b5de06e70..35b81c50aea 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -85,6 +85,8 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { + {"24.5", {{"allow_deprecated_functions", true, false, "Allow usage of deprecated functions"}, + }}, {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, {"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"}, diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 57dbcad565f..01da242b24e 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -935,6 +935,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep query_context->setSetting("allow_experimental_window_functions", 1); query_context->setSetting("allow_experimental_geo_types", 1); query_context->setSetting("allow_experimental_map_type", 1); + query_context->setSetting("allow_deprecated_functions", 1); query_context->setSetting("allow_suspicious_low_cardinality_types", 1); query_context->setSetting("allow_suspicious_fixed_string_types", 1); diff --git a/src/Functions/neighbor.cpp b/src/Functions/neighbor.cpp index d367695448a..49b73aabe3d 100644 --- a/src/Functions/neighbor.cpp +++ b/src/Functions/neighbor.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include namespace DB @@ -13,6 +14,7 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ARGUMENT_OUT_OF_BOUND; + extern const int DEPRECATED_FUNCTION; } namespace @@ -31,7 +33,18 @@ class FunctionNeighbor : public IFunction { public: static constexpr auto name = "neighbor"; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + static FunctionPtr create(ContextPtr context) + { + if (!context->getSettingsRef().allow_deprecated_functions) + throw Exception( + ErrorCodes::DEPRECATED_FUNCTION, + "Function {} is deprecated since its usage is error-prone (see docs)." + "Set `allow_deprecated_functions` setting to enable it", + name); + + return std::make_shared(); + } /// Get the name of the function. String getName() const override { return name; } diff --git a/src/Functions/runningAccumulate.cpp b/src/Functions/runningAccumulate.cpp index 793e79cdf46..b2bc1ea02b9 100644 --- a/src/Functions/runningAccumulate.cpp +++ b/src/Functions/runningAccumulate.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -16,6 +17,7 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int DEPRECATED_FUNCTION; } namespace @@ -34,8 +36,16 @@ class FunctionRunningAccumulate : public IFunction { public: static constexpr auto name = "runningAccumulate"; - static FunctionPtr create(ContextPtr) + + static FunctionPtr create(ContextPtr context) { + if (!context->getSettingsRef().allow_deprecated_functions) + throw Exception( + ErrorCodes::DEPRECATED_FUNCTION, + "Function {} is deprecated since its usage is error-prone (see docs)." + "Set `allow_deprecated_functions` setting to enable it", + name); + return std::make_shared(); } diff --git a/src/Functions/runningDifference.h b/src/Functions/runningDifference.h index f1ec4f9e523..ea1408b4c08 100644 --- a/src/Functions/runningDifference.h +++ b/src/Functions/runningDifference.h @@ -1,16 +1,17 @@ #pragma once -#include -#include -#include #include -#include +#include #include #include #include #include +#include #include #include -#include +#include +#include +#include +#include namespace DB @@ -19,6 +20,7 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int DEPRECATED_FUNCTION; } @@ -135,8 +137,15 @@ private: public: static constexpr auto name = FunctionRunningDifferenceName::name; - static FunctionPtr create(ContextPtr) + static FunctionPtr create(ContextPtr context) { + if (!context->getSettingsRef().allow_deprecated_functions) + throw Exception( + ErrorCodes::DEPRECATED_FUNCTION, + "Function {} is deprecated since its usage is error-prone (see docs)." + "Set `allow_deprecated_functions` setting to enable it", + name); + return std::make_shared>(); } diff --git a/tests/queries/0_stateless/03131_deprecated_functions.reference b/tests/queries/0_stateless/03131_deprecated_functions.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03131_deprecated_functions.sql b/tests/queries/0_stateless/03131_deprecated_functions.sql new file mode 100644 index 00000000000..4cd7c0f473f --- /dev/null +++ b/tests/queries/0_stateless/03131_deprecated_functions.sql @@ -0,0 +1,13 @@ +SELECT number, neighbor(number, 2) FROM system.numbers LIMIT 10; -- { serverError 720 } + +SELECT runningDifference(number) FROM system.numbers LIMIT 10; -- { serverError 720 } + +SELECT k, runningAccumulate(sum_k) AS res FROM (SELECT number as k, sumState(k) AS sum_k FROM numbers(10) GROUP BY k ORDER BY k); -- { serverError 720 } + +SET allow_deprecated_functions=1; + +SELECT number, neighbor(number, 2) FROM system.numbers LIMIT 10 FORMAT Null; + +SELECT runningDifference(number) FROM system.numbers LIMIT 10 FORMAT Null; + +SELECT k, runningAccumulate(sum_k) AS res FROM (SELECT number as k, sumState(k) AS sum_k FROM numbers(10) GROUP BY k ORDER BY k) FORMAT Null; From 64ee0090cbc447e50d725e5f6f895d369c1e0996 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 29 Apr 2024 22:00:56 +0100 Subject: [PATCH 187/624] better --- docs/en/sql-reference/functions/other-functions.md | 6 +++--- src/Functions/neighbor.cpp | 2 +- src/Functions/runningAccumulate.cpp | 2 +- src/Functions/runningDifference.h | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 7ee9d0366e9..3d259d2b855 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -1024,7 +1024,7 @@ The result of the function depends on the affected data blocks and the order of :::note Only returns neighbor inside the currently processed data block. -Because of this error-prone behavior the function is DEPRECATED. +Because of this error-prone behavior the function is DEPRECATED, please use proper window functions instead. ::: The order of rows during calculation of `neighbor()` can differ from the order of rows returned to the user. @@ -1135,7 +1135,7 @@ Returns 0 for the first row, and for subsequent rows the difference to the previ :::note Only returns differences inside the currently processed data block. -Because of this error-prone behavior the function is DEPRECATED. +Because of this error-prone behavior the function is DEPRECATED, please use proper window functions instead. ::: The result of the function depends on the affected data blocks and the order of data in the block. @@ -1936,7 +1936,7 @@ Accumulates the states of an aggregate function for each row of a data block. :::note The state is reset for each new block of data. -Because of this error-prone behavior the function is DEPRECATED. +Because of this error-prone behavior the function is DEPRECATED, please use proper window functions instead. ::: **Syntax** diff --git a/src/Functions/neighbor.cpp b/src/Functions/neighbor.cpp index 49b73aabe3d..abe6d39422d 100644 --- a/src/Functions/neighbor.cpp +++ b/src/Functions/neighbor.cpp @@ -40,7 +40,7 @@ public: throw Exception( ErrorCodes::DEPRECATED_FUNCTION, "Function {} is deprecated since its usage is error-prone (see docs)." - "Set `allow_deprecated_functions` setting to enable it", + "Please use proper window function or set `allow_deprecated_functions` setting to enable it", name); return std::make_shared(); diff --git a/src/Functions/runningAccumulate.cpp b/src/Functions/runningAccumulate.cpp index b2bc1ea02b9..9bf387d3357 100644 --- a/src/Functions/runningAccumulate.cpp +++ b/src/Functions/runningAccumulate.cpp @@ -43,7 +43,7 @@ public: throw Exception( ErrorCodes::DEPRECATED_FUNCTION, "Function {} is deprecated since its usage is error-prone (see docs)." - "Set `allow_deprecated_functions` setting to enable it", + "Please use proper window function or set `allow_deprecated_functions` setting to enable it", name); return std::make_shared(); diff --git a/src/Functions/runningDifference.h b/src/Functions/runningDifference.h index ea1408b4c08..d3704aa97ca 100644 --- a/src/Functions/runningDifference.h +++ b/src/Functions/runningDifference.h @@ -143,7 +143,7 @@ public: throw Exception( ErrorCodes::DEPRECATED_FUNCTION, "Function {} is deprecated since its usage is error-prone (see docs)." - "Set `allow_deprecated_functions` setting to enable it", + "Please use proper window function or set `allow_deprecated_functions` setting to enable it", name); return std::make_shared>(); From c2297f6118de4847c7705af3a7314b5d87cf39eb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Apr 2024 01:52:39 +0200 Subject: [PATCH 188/624] Add a comment --- cmake/linux/default_libs.cmake | 1 + 1 file changed, 1 insertion(+) diff --git a/cmake/linux/default_libs.cmake b/cmake/linux/default_libs.cmake index 61d40b4fd04..e5ca8e296fc 100644 --- a/cmake/linux/default_libs.cmake +++ b/cmake/linux/default_libs.cmake @@ -8,6 +8,7 @@ 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) + # 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 () From 10073f9ae89447f02b263edd576bd357d3883c9a Mon Sep 17 00:00:00 2001 From: Xiaofei Hu <10776938+litlig@users.noreply.github.com> Date: Mon, 29 Apr 2024 17:40:05 -0700 Subject: [PATCH 189/624] init --- .../System/StorageSystemPartsColumns.cpp | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/src/Storages/System/StorageSystemPartsColumns.cpp b/src/Storages/System/StorageSystemPartsColumns.cpp index f34b0e0cfda..ad831426197 100644 --- a/src/Storages/System/StorageSystemPartsColumns.cpp +++ b/src/Storages/System/StorageSystemPartsColumns.cpp @@ -65,6 +65,8 @@ StorageSystemPartsColumns::StorageSystemPartsColumns(const StorageID & table_id_ {"column_data_uncompressed_bytes", std::make_shared(), "Total size of the decompressed data in the column, in bytes."}, {"column_marks_bytes", std::make_shared(), "The size of the marks for column, in bytes."}, {"column_modification_time", std::make_shared(std::make_shared()), "The last time the column was modified."}, + {"column_ttl_info.min", std::make_shared(std::make_shared()), "The minimum value of the calculated TTL expression of the column."} + {"column_ttl_info.max", std::make_shared(std::make_shared()), "The maximum value of the calculated TTL expression of the column."} {"serialization_kind", std::make_shared(), "Kind of serialization of a column"}, {"substreams", std::make_shared(std::make_shared()), "Names of substreams to which column is serialized"}, @@ -250,6 +252,21 @@ void StorageSystemPartsColumns::processNextStorage( else columns[res_index++]->insertDefault(); } + bool column_has_ttl = part->ttl_info.columns_ttl.contains(column.name); + if (columns_mask[src_index++] + { + if (column_has_ttl) + columns[res_index++]->insert(static_cast(part->ttl_infos.columns_ttl[column.name].min)); + else + columns[res_index++]->insertDefault(); + } + if (columns_mask[src_index++]) + { + if column_has_ttl + columns[res_index++]->insert(static_cast(part->ttl_infos.columns_ttl[column.name].max)); + else + columns[res_index++]->insertDefault(); + } auto serialization = part->getSerialization(column.name); if (columns_mask[src_index++]) From d7d38b14e8571ef0f1fa3c09d33563e297855e63 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Apr 2024 04:21:44 +0200 Subject: [PATCH 190/624] Fix tests --- programs/main.cpp | 3 +-- src/Client/ClientBase.cpp | 3 --- .../02751_multiquery_with_argument.reference | 12 +++++----- .../02751_multiquery_with_argument.sh | 12 +++++----- .../0_stateless/02784_connection_string.sh | 18 +++++++-------- .../03134_positional_arguments.reference | 9 -------- .../0_stateless/03134_positional_arguments.sh | 22 +------------------ 7 files changed, 24 insertions(+), 55 deletions(-) diff --git a/programs/main.cpp b/programs/main.cpp index 8a8ba128cad..65fcfb8fadf 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -492,10 +492,9 @@ int main(int argc_, char ** argv_) /// clickhouse # spawn local /// clickhouse local # spawn local /// clickhouse "select ..." # spawn local - /// clickhouse query.sql # spawn local /// if (main_func == printHelp && !argv.empty() && (argv.size() == 1 || argv[1][0] == '-' - || std::string_view(argv[1]).contains(' ') || std::string_view(argv[1]).contains('.'))) + || std::string_view(argv[1]).contains(' '))) { main_func = mainEntryClickHouseLocal; } diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 826b9d75331..80b59f00146 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2917,7 +2917,6 @@ void ClientBase::parseAndCheckOptions(OptionsDescription & options_description, { /// Two special cases for better usability: /// - if the option contains a whitespace, it might be a query: clickhouse "SELECT 1" - /// - otherwise if it contains a dot it might be a queries file: clickhouse file.sql /// These are relevant for interactive usage - user-friendly, but questionable in general. /// In case of ambiguity or for scripts, prefer using proper options. @@ -2927,8 +2926,6 @@ void ClientBase::parseAndCheckOptions(OptionsDescription & options_description, const char * option; if (token.contains(' ')) option = "query"; - else if (token.contains('.')) - option = "queries-file"; else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional option `{}` is not supported.", token); diff --git a/tests/queries/0_stateless/02751_multiquery_with_argument.reference b/tests/queries/0_stateless/02751_multiquery_with_argument.reference index 843fffb476c..4b0c199ed35 100644 --- a/tests/queries/0_stateless/02751_multiquery_with_argument.reference +++ b/tests/queries/0_stateless/02751_multiquery_with_argument.reference @@ -7,9 +7,10 @@ Empty query Empty query BAD_ARGUMENTS BAD_ARGUMENTS -BAD_ARGUMENTS -BAD_ARGUMENTS -BAD_ARGUMENTS +301 +302 +304 +305 Bad arguments Bad arguments Bad arguments @@ -17,7 +18,8 @@ Bad arguments Bad arguments Bad arguments Bad arguments -BAD_ARGUMENTS -BAD_ARGUMENTS +Bad arguments +320 +317 Bad arguments Bad arguments diff --git a/tests/queries/0_stateless/02751_multiquery_with_argument.sh b/tests/queries/0_stateless/02751_multiquery_with_argument.sh index d742cc0ad90..fea2978c116 100755 --- a/tests/queries/0_stateless/02751_multiquery_with_argument.sh +++ b/tests/queries/0_stateless/02751_multiquery_with_argument.sh @@ -18,10 +18,10 @@ $CLICKHOUSE_LOCAL --queries-file "queries.csv" --multiquery "SELECT 250;" 2>&1 | $CLICKHOUSE_CLIENT --queries-file "queries.csv" --multiquery "SELECT 251;" 2>&1 | grep -o 'BAD_ARGUMENTS' # Error expectation cases. -# -n is prohibited -$CLICKHOUSE_LOCAL -n "SELECT 301" 2>&1 | grep -o 'BAD_ARGUMENTS' -$CLICKHOUSE_LOCAL -n "SELECT 302;" 2>&1 | grep -o 'BAD_ARGUMENTS' -$CLICKHOUSE_LOCAL -n "SELECT 304;SELECT 305;" 2>&1 | grep -o 'BAD_ARGUMENTS' +# -n is also interpreted as a query +$CLICKHOUSE_LOCAL -n "SELECT 301" +$CLICKHOUSE_LOCAL -n "SELECT 302;" +$CLICKHOUSE_LOCAL -n "SELECT 304;SELECT 305;" $CLICKHOUSE_LOCAL --multiquery --multiquery 2>&1 | grep -o 'Bad arguments' $CLICKHOUSE_LOCAL -n --multiquery 2>&1 | grep -o 'Bad arguments' $CLICKHOUSE_LOCAL --multiquery -n 2>&1 | grep -o 'Bad arguments' @@ -29,7 +29,7 @@ $CLICKHOUSE_LOCAL --multiquery --multiquery "SELECT 306; SELECT 307;" 2>&1 | gre $CLICKHOUSE_LOCAL -n --multiquery "SELECT 307; SELECT 308;" 2>&1 | grep -o 'Bad arguments' $CLICKHOUSE_LOCAL --multiquery "SELECT 309; SELECT 310;" --multiquery 2>&1 | grep -o 'Bad arguments' $CLICKHOUSE_LOCAL --multiquery "SELECT 311;" --multiquery "SELECT 312;" 2>&1 | grep -o 'Bad arguments' -$CLICKHOUSE_LOCAL --multiquery "SELECT 313;" -n "SELECT 314;" 2>&1 | grep -o 'BAD_ARGUMENTS' -$CLICKHOUSE_LOCAL -n "SELECT 320" --query "SELECT 317;" 2>&1 | grep -o 'BAD_ARGUMENTS' +$CLICKHOUSE_LOCAL --multiquery "SELECT 313;" -n "SELECT 314;" 2>&1 | grep -o 'Bad arguments' +$CLICKHOUSE_LOCAL -n "SELECT 320" --query "SELECT 317;" $CLICKHOUSE_LOCAL --query -n "SELECT 400;" 2>&1 | grep -o 'Bad arguments' $CLICKHOUSE_LOCAL --query -n --multiquery "SELECT 401;" 2>&1 | grep -o 'Bad arguments' diff --git a/tests/queries/0_stateless/02784_connection_string.sh b/tests/queries/0_stateless/02784_connection_string.sh index 8353ac5b1e4..fd736e6838c 100755 --- a/tests/queries/0_stateless/02784_connection_string.sh +++ b/tests/queries/0_stateless/02784_connection_string.sh @@ -10,13 +10,13 @@ DATABASES=("$CLICKHOUSE_DATABASE" "") TEST_INDEX=0 -function runClient() +function runClient() { $CLICKHOUSE_CLIENT_BINARY "$@" -q "SELECT $TEST_INDEX" --log_comment 02766_connection_string.sh --send_logs_level=warning ((++TEST_INDEX)) } -function testConnectionString() +function testConnectionString() { if [ "$database" == "" ]; then runClient "clickhouse:$1" @@ -26,7 +26,7 @@ function testConnectionString() fi } -function testConnectionWithUserName() +function testConnectionWithUserName() { if [ "$user_info" == "" ] && [ "$host_port" == "" ]; then testConnectionString "//" @@ -53,9 +53,9 @@ TEST_USER_NAME="test_user_02771_$$" TEST_USER_EMAIL_NAME="test_user_02771_$$@some_mail.com" TEST_USER_EMAIL_NAME_ENCODED="test_user_02771_$$%40some_mail.com" -TEST_USER_PASSWORD="zyx%$&abc" +TEST_USER_PASSWORD="zyx%$&abc" # %, $, & percent encoded -TEST_USER_PASSWORD_ENCODED="zyx%25%24%26abc" +TEST_USER_PASSWORD_ENCODED="zyx%25%24%26abc" $CLICKHOUSE_CLIENT -q "CREATE USER '$TEST_USER_NAME'" $CLICKHOUSE_CLIENT -q "CREATE USER '$TEST_USER_EMAIL_NAME' IDENTIFIED WITH plaintext_password BY '$TEST_USER_PASSWORD'" @@ -66,14 +66,14 @@ runClient "clickhouse://$TEST_USER_EMAIL_NAME_ENCODED:$TEST_USER_PASSWORD_ENCODE $CLICKHOUSE_CLIENT -q "DROP USER '$TEST_USER_NAME'" $CLICKHOUSE_CLIENT -q "DROP USER '$TEST_USER_EMAIL_NAME'" -# Percent-encoded database in non-ascii symbols +# Percent-encoded database in non-ascii symbols UTF8_DATABASE="БазаДанных_$$" UTF8_DATABASE_PERCENT_ENCODED="%D0%91%D0%B0%D0%B7%D0%B0%D0%94%D0%B0%D0%BD%D0%BD%D1%8B%D1%85_$$" $CLICKHOUSE_CLIENT -q "CREATE DATABASE IF NOT EXISTS \`$UTF8_DATABASE\`" runClient "clickhouse://default@$CLICKHOUSE_HOST/$UTF8_DATABASE_PERCENT_ENCODED" $CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS \`$UTF8_DATABASE\`" -# clickhouse-client extra options cases +# clickhouse-client extra options cases TEST_INDEX=1000 runClient "clickhouse://$CLICKHOUSE_HOST/" --user 'default' @@ -114,7 +114,7 @@ runClient "clickhouse://" --port "$CLICKHOUSE_PORT_TCP" --host "$CLICKHOUSE_HOST runClient "clickhouse:///" --port "$CLICKHOUSE_PORT_TCP" --host "$CLICKHOUSE_HOST" 2>&1 | grep -o 'BAD_ARGUMENTS' runClient "clickhouse:///?" --port "$CLICKHOUSE_PORT_TCP" --host "$CLICKHOUSE_HOST" 2>&1 | grep -o 'BAD_ARGUMENTS' runClient "clickhouse://:/?" --port "$CLICKHOUSE_PORT_TCP" --host "$CLICKHOUSE_HOST" 2>&1 | grep -o 'BAD_ARGUMENTS' -runClient "clickhouse:" --database "$CLICKHOUSE_DATABASE" --port "$CLICKHOUSE_PORT_TCP" --host "$CLICKHOUSE_HOST" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse:" --database "$CLICKHOUSE_DATABASE" --port "$CLICKHOUSE_PORT_TCP" --host "$CLICKHOUSE_HOST" 2>&1 | grep -o 'BAD_ARGUMENTS' # Using clickhouse-client and connection is prohibited runClient "clickhouse:" --connection "connection" 2>&1 | grep -o 'BAD_ARGUMENTS' @@ -151,7 +151,7 @@ runClient "clickhouse://user1@localhost,default@localhost/" 2>&1 | grep -o 'BAD_ # Using '@' in user name is prohibited. User name should be percent-encoded. runClient "clickhouse://my_mail@email.com@host/" 2>&1 | grep -o 'BAD_ARGUMENTS' -# Wrong input cases +# Wrong input cases TEST_INDEX=100000 # Invalid user name runClient "clickhouse://non_exist_user@$CLICKHOUSE_HOST:$CLICKHOUSE_PORT_TCP/" 2>&1 | grep -o 'Authentication failed' diff --git a/tests/queries/0_stateless/03134_positional_arguments.reference b/tests/queries/0_stateless/03134_positional_arguments.reference index ac13dd8bd0f..f1cd5725b24 100644 --- a/tests/queries/0_stateless/03134_positional_arguments.reference +++ b/tests/queries/0_stateless/03134_positional_arguments.reference @@ -2,9 +2,6 @@ 2 3 Hello -Hello from a file -Hello from a file -Hello from a file 1 2 is not supported @@ -13,13 +10,7 @@ Use one of the following commands 2 3 Hello -Hello from a file -Hello from a file -Hello from a file 1 2 3 Hello -Hello from a file -Hello from a file -Hello from a file diff --git a/tests/queries/0_stateless/03134_positional_arguments.sh b/tests/queries/0_stateless/03134_positional_arguments.sh index f66a8582d1d..4390a8d5250 100755 --- a/tests/queries/0_stateless/03134_positional_arguments.sh +++ b/tests/queries/0_stateless/03134_positional_arguments.sh @@ -5,30 +5,18 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -FILE=${CLICKHOUSE_TMP}/${CLICKHOUSE_DATABASE}.sql -echo "SELECT 'Hello from a file'" > ${FILE} - # The best way to write the query parameter, explicit long option. ${CLICKHOUSE_BINARY} --query "SELECT 1" # Shorthand option: ${CLICKHOUSE_BINARY} -q "SELECT 2" -# It is also accepted as a positional argument +# It is also accepted as a positional argument ${CLICKHOUSE_BINARY} "SELECT 3" # The positional argument can go after normal arguments. ${CLICKHOUSE_BINARY} --param_test Hello "SELECT {test:String}" -# Queries can be read from a file. -${CLICKHOUSE_BINARY} --queries-file ${FILE} - -# Or from stdin. -${CLICKHOUSE_BINARY} < ${FILE} - -# Also, if a file contains a dot, the positional argument can be interpreted as a file. -${CLICKHOUSE_BINARY} ${FILE} - # This is ambiguous: currently works, but does not have to. ${CLICKHOUSE_BINARY} --query "SELECT 1" "SELECT 2" @@ -44,16 +32,8 @@ ${CLICKHOUSE_LOCAL} --query "SELECT 1" ${CLICKHOUSE_LOCAL} -q "SELECT 2" ${CLICKHOUSE_LOCAL} "SELECT 3" ${CLICKHOUSE_LOCAL} --param_test Hello "SELECT {test:String}" -${CLICKHOUSE_LOCAL} --queries-file ${FILE} -${CLICKHOUSE_LOCAL} < ${FILE} -${CLICKHOUSE_LOCAL} ${FILE} ${CLICKHOUSE_CLIENT} --query "SELECT 1" ${CLICKHOUSE_CLIENT} -q "SELECT 2" ${CLICKHOUSE_CLIENT} "SELECT 3" ${CLICKHOUSE_CLIENT} --param_test Hello "SELECT {test:String}" -${CLICKHOUSE_CLIENT} --queries-file ${FILE} -${CLICKHOUSE_CLIENT} < ${FILE} -${CLICKHOUSE_CLIENT} ${FILE} - -rm ${FILE} From 7d53b4fd387a2dd9a13ebd638126e75360cafe33 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Apr 2024 05:45:05 +0300 Subject: [PATCH 191/624] Update programs/local/LocalServer.cpp Co-authored-by: Konstantin Bogdanov --- programs/local/LocalServer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index c4cb5d4df9e..23eb78a4534 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -210,7 +210,7 @@ void LocalServer::tryInitPath() else { // The path is not provided explicitly - use a unique path in the system temporary directory - // (or in the current dir if the temporary doesn't exist) + // (or in the current dir if a temporary doesn't exist) LoggerRawPtr log = &logger(); std::filesystem::path parent_folder; std::filesystem::path default_path; From 1546c9e3b8eb8a66da4743347940d1a8134686a6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Apr 2024 05:46:06 +0300 Subject: [PATCH 192/624] Update LocalServer.cpp --- programs/local/LocalServer.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index a005d9b307b..ac63090f007 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -473,7 +473,6 @@ try SCOPE_EXIT({ cleanup(); }); - adjustSettings(); initTTYBuffer(toProgressOption(config().getString("progress", "default"))); ASTAlterCommand::setFormatAlterCommandsWithParentheses(true); From a501887f15e1e21a7f35740bc4bf82f3e2b2df52 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Apr 2024 05:15:35 +0200 Subject: [PATCH 193/624] Fancy quotes --- src/Parsers/Lexer.cpp | 52 ++++++++++++++++++++++++++++++++++++++----- 1 file changed, 47 insertions(+), 5 deletions(-) diff --git a/src/Parsers/Lexer.cpp b/src/Parsers/Lexer.cpp index 0e6db7a40e5..c3b503404a3 100644 --- a/src/Parsers/Lexer.cpp +++ b/src/Parsers/Lexer.cpp @@ -11,8 +11,9 @@ namespace { /// This must be consistent with functions in ReadHelpers.h -template -Token quotedString(const char *& pos, const char * const token_begin, const char * const end) +template +Token quotedString(const char *& pos, const char * const token_begin, const char * const end, + TokenType success_token, TokenType error_token) { ++pos; while (true) @@ -45,6 +46,37 @@ Token quotedString(const char *& pos, const char * const token_begin, const char } } +Token quotedStringWithUnicodeQuotes(const char *& pos, const char * const token_begin, const char * const end, + char expected_end_byte, TokenType success_token, TokenType error_token) +{ + /// ‘: e2 80 98 + /// ’: e2 80 99 + /// “: e2 80 9c + /// ”: e2 80 9d + + while (true) + { + pos = find_first_symbols<'\xE2', '\\'>(pos, end); + if (pos + 2 >= end) + return Token(error_token, token_begin, end); + + if (pos[0] == '\xE2' && pos[1] == '\x80' && pos[2] == expected_end_byte) + { + pos += 3; + return Token(success_token, token_begin, pos); + } + + if (*pos == '\\') + { + ++pos; + if (pos >= end) + return Token(error_token, token_begin, end); + ++pos; + continue; + } + } +} + Token quotedHexOrBinString(const char *& pos, const char * const token_begin, const char * const end) { constexpr char quote = '\''; @@ -224,11 +256,11 @@ Token Lexer::nextTokenImpl() } case '\'': - return quotedString<'\'', TokenType::StringLiteral, TokenType::ErrorSingleQuoteIsNotClosed>(pos, token_begin, end); + return quotedString<'\''>(pos, token_begin, end, TokenType::StringLiteral, TokenType::ErrorSingleQuoteIsNotClosed); case '"': - return quotedString<'"', TokenType::QuotedIdentifier, TokenType::ErrorDoubleQuoteIsNotClosed>(pos, token_begin, end); + return quotedString<'"'>(pos, token_begin, end, TokenType::QuotedIdentifier, TokenType::ErrorDoubleQuoteIsNotClosed); case '`': - return quotedString<'`', TokenType::QuotedIdentifier, TokenType::ErrorBackQuoteIsNotClosed>(pos, token_begin, end); + return quotedString<'`'>(pos, token_begin, end, TokenType::QuotedIdentifier, TokenType::ErrorBackQuoteIsNotClosed); case '(': return Token(TokenType::OpeningRoundBracket, token_begin, ++pos); @@ -434,6 +466,16 @@ Token Lexer::nextTokenImpl() pos += 3; return Token(TokenType::Minus, token_begin, pos); } + /// Unicode quoted string, ‘Hello’ or “World”. + if (pos + 5 < end && pos[0] == '\xE2' && pos[1] == '\x80' && (pos[2] == '\x98' || pos[2] == '\x9C')) + { + const char expected_end_byte = pos[2] + 1; + pos += 3; + + TokenType success_token = pos[2] == '\x98' ? TokenType::StringLiteral : TokenType::QuotedIdentifier; + TokenType error_token = pos[2] == '\x98' ? TokenType::ErrorSingleQuoteIsNotClosed : TokenType::ErrorDoubleQuoteIsNotClosed; + return quotedStringWithUnicodeQuotes(pos, token_begin, end, expected_end_byte, success_token, error_token); + } /// Other characters starting at E2 can be parsed, see skipWhitespacesUTF8 [[fallthrough]]; } From 10fb28c6ea52be948a0e14b58eb12e26b6137c9e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 30 Apr 2024 13:36:20 +0200 Subject: [PATCH 194/624] Fix crash with untuple and unresolved lambda --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 4 ++++ tests/queries/0_stateless/03142_untuple_crash.reference | 0 tests/queries/0_stateless/03142_untuple_crash.sql | 1 + 3 files changed, 5 insertions(+) create mode 100644 tests/queries/0_stateless/03142_untuple_crash.reference create mode 100644 tests/queries/0_stateless/03142_untuple_crash.sql diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 73d1b9df5f6..04b2c6de19e 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -5725,6 +5725,10 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi checkFunctionNodeHasEmptyNullsAction(function_node); const auto & untuple_argument = function_arguments[0]; + /// Handle this special case first as `getResultType()` might return nullptr + if (untuple_argument->as()) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function untuple can't have lambda-expressions as arguments"); + auto result_type = untuple_argument->getResultType(); const auto * tuple_data_type = typeid_cast(result_type.get()); if (!tuple_data_type) diff --git a/tests/queries/0_stateless/03142_untuple_crash.reference b/tests/queries/0_stateless/03142_untuple_crash.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03142_untuple_crash.sql b/tests/queries/0_stateless/03142_untuple_crash.sql new file mode 100644 index 00000000000..e0f4225dbac --- /dev/null +++ b/tests/queries/0_stateless/03142_untuple_crash.sql @@ -0,0 +1 @@ +SELECT untuple(x -> 0) -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } From 2263291fd8893c5d16c7e3d613cb43d6f8b2ce57 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 30 Apr 2024 13:28:29 +0100 Subject: [PATCH 195/624] fix tests 1 --- src/Storages/System/StorageSystemFunctions.cpp | 4 +++- tests/queries/0_stateless/01353_neighbor_overflow.sql | 1 + tests/queries/0_stateless/01670_neighbor_lc_bug.sql | 1 + 3 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemFunctions.cpp b/src/Storages/System/StorageSystemFunctions.cpp index c5c91294801..6e4ac8b2747 100644 --- a/src/Storages/System/StorageSystemFunctions.cpp +++ b/src/Storages/System/StorageSystemFunctions.cpp @@ -23,6 +23,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int SUPPORT_IS_DISABLED; extern const int ACCESS_DENIED; + extern const int DEPRECATED_FUNCTION; }; enum class FunctionOrigin : int8_t @@ -152,7 +153,8 @@ void StorageSystemFunctions::fillData(MutableColumns & res_columns, ContextPtr c || e.code() == ErrorCodes::FUNCTION_NOT_ALLOWED || e.code() == ErrorCodes::NOT_IMPLEMENTED || e.code() == ErrorCodes::SUPPORT_IS_DISABLED - || e.code() == ErrorCodes::ACCESS_DENIED) + || e.code() == ErrorCodes::ACCESS_DENIED + || e.code() == ErrorCodes::DEPRECATED_FUNCTION) { /// Ignore exception, show is_deterministic = NULL. } diff --git a/tests/queries/0_stateless/01353_neighbor_overflow.sql b/tests/queries/0_stateless/01353_neighbor_overflow.sql index dee18d52ae6..ac168cb3305 100644 --- a/tests/queries/0_stateless/01353_neighbor_overflow.sql +++ b/tests/queries/0_stateless/01353_neighbor_overflow.sql @@ -1,2 +1,3 @@ +SET allow_deprecated_functions = 1; SELECT neighbor(toString(number), -9223372036854775808) FROM numbers(100); -- { serverError 69 } WITH neighbor(toString(number), toInt64(rand64())) AS x SELECT * FROM system.numbers WHERE NOT ignore(x); -- { serverError 69 } diff --git a/tests/queries/0_stateless/01670_neighbor_lc_bug.sql b/tests/queries/0_stateless/01670_neighbor_lc_bug.sql index 2c95e16e916..b665c0b48fd 100644 --- a/tests/queries/0_stateless/01670_neighbor_lc_bug.sql +++ b/tests/queries/0_stateless/01670_neighbor_lc_bug.sql @@ -1,3 +1,4 @@ +SET allow_deprecated_functions = 1; SET output_format_pretty_row_numbers = 0; SELECT From 8f93df354ec51a6c8393cdeae67016dbb89aa217 Mon Sep 17 00:00:00 2001 From: HowePa <2873679104@qq.com> Date: Tue, 30 Apr 2024 20:34:08 +0800 Subject: [PATCH 196/624] fix end symbol --- src/Processors/Formats/Impl/NpyOutputFormat.cpp | 6 +++++- tests/queries/0_stateless/02895_npy_output_format.reference | 6 +++--- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/src/Processors/Formats/Impl/NpyOutputFormat.cpp b/src/Processors/Formats/Impl/NpyOutputFormat.cpp index b3d5042aa79..e02787b4f70 100644 --- a/src/Processors/Formats/Impl/NpyOutputFormat.cpp +++ b/src/Processors/Formats/Impl/NpyOutputFormat.cpp @@ -40,7 +40,11 @@ void writeNumpyStrings(const ColumnPtr & column, size_t length, WriteBuffer & bu { const auto * string_column = assert_cast(column.get()); for (size_t i = 0; i < string_column->size(); ++i) - buf.write(string_column->getDataAt(i).data, length); + { + auto data = string_column->getDataAt(i); + buf.write(data.data, data.size); + writeChar(0, length - data.size, buf); + } } } diff --git a/tests/queries/0_stateless/02895_npy_output_format.reference b/tests/queries/0_stateless/02895_npy_output_format.reference index b599f1dceea..77d9b55ca9f 100644 --- a/tests/queries/0_stateless/02895_npy_output_format.reference +++ b/tests/queries/0_stateless/02895_npy_output_format.reference @@ -52,9 +52,9 @@ array String [[0.1],[0.2]] [[0.1],[0.2]] [[0.1],[0.2]] -[['abb','bbc'],['ccc','dddd']] -[['abb','bbc'],['ccc','dddd']] -[['abb','bbc'],['ccc','dddd']] +[['a','bb'],['ccc','dddd']] +[['a','bb'],['ccc','dddd']] +[['a','bb'],['ccc','dddd']] array Array(Array(Array(Int8))) array Array(Array(Float64)) array Array(Array(String)) From 72ef106f471e39a7edd70f8a6ed567c97d0dd497 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 30 Apr 2024 14:09:44 +0100 Subject: [PATCH 197/624] fix tests 2 --- .../0_stateless/00166_functions_of_aggregation_states.sql | 1 + .../0_stateless/00410_aggregation_combinators_with_arenas.sql | 1 + tests/queries/0_stateless/00653_running_difference.sql | 1 + tests/queries/0_stateless/00808_not_optimize_predicate.sql | 1 + tests/queries/0_stateless/00957_neighbor.sql | 3 ++- tests/queries/0_stateless/00996_neighbor.sql | 1 + tests/queries/0_stateless/01012_reset_running_accumulate.sql | 3 ++- tests/queries/0_stateless/01051_aggregate_function_crash.sql | 4 +++- tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql | 1 + .../0_stateless/01455_optimize_trivial_insert_select.sql | 1 + tests/queries/0_stateless/01665_running_difference_ubsan.sql | 1 + tests/queries/0_stateless/02496_remove_redundant_sorting.sh | 3 ++- .../02496_remove_redundant_sorting_analyzer.reference | 1 + .../0_stateless/02788_fix_logical_error_in_sorting.sql | 2 ++ .../02842_largestTriangleThreeBuckets_aggregate_function.sql | 3 ++- .../0_stateless/02901_predicate_pushdown_cte_stateful.sql | 2 ++ 16 files changed, 24 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/00166_functions_of_aggregation_states.sql b/tests/queries/0_stateless/00166_functions_of_aggregation_states.sql index 0a5a84bbb46..85f26d4e206 100644 --- a/tests/queries/0_stateless/00166_functions_of_aggregation_states.sql +++ b/tests/queries/0_stateless/00166_functions_of_aggregation_states.sql @@ -1,4 +1,5 @@ -- Disable external aggregation because the state is reset for each new block of data in 'runningAccumulate' function. SET max_bytes_before_external_group_by = 0; +SET allow_deprecated_functions = 1; SELECT k, finalizeAggregation(sum_state), runningAccumulate(sum_state) FROM (SELECT intDiv(number, 50000) AS k, sumState(number) AS sum_state FROM (SELECT number FROM system.numbers LIMIT 1000000) GROUP BY k ORDER BY k); diff --git a/tests/queries/0_stateless/00410_aggregation_combinators_with_arenas.sql b/tests/queries/0_stateless/00410_aggregation_combinators_with_arenas.sql index a3abbb9fd58..99091878d90 100644 --- a/tests/queries/0_stateless/00410_aggregation_combinators_with_arenas.sql +++ b/tests/queries/0_stateless/00410_aggregation_combinators_with_arenas.sql @@ -1,3 +1,4 @@ +SET allow_deprecated_functions = 1; DROP TABLE IF EXISTS arena; CREATE TABLE arena (k UInt8, d String) ENGINE = Memory; INSERT INTO arena SELECT number % 10 AS k, hex(intDiv(number, 10) % 1000) AS d FROM system.numbers LIMIT 10000000; diff --git a/tests/queries/0_stateless/00653_running_difference.sql b/tests/queries/0_stateless/00653_running_difference.sql index 1f18cfc42a7..d210e04a3a4 100644 --- a/tests/queries/0_stateless/00653_running_difference.sql +++ b/tests/queries/0_stateless/00653_running_difference.sql @@ -1,3 +1,4 @@ +SET allow_deprecated_functions = 1; select runningDifference(x) from (select arrayJoin([0, 1, 5, 10]) as x); select '-'; select runningDifference(x) from (select arrayJoin([2, Null, 3, Null, 10]) as x); diff --git a/tests/queries/0_stateless/00808_not_optimize_predicate.sql b/tests/queries/0_stateless/00808_not_optimize_predicate.sql index d0dda14e026..c39f1ff2ad1 100644 --- a/tests/queries/0_stateless/00808_not_optimize_predicate.sql +++ b/tests/queries/0_stateless/00808_not_optimize_predicate.sql @@ -1,5 +1,6 @@ SET send_logs_level = 'fatal'; SET convert_query_to_cnf = 0; +SET allow_deprecated_functions = 1; DROP TABLE IF EXISTS test_00808; CREATE TABLE test_00808(date Date, id Int8, name String, value Int64, sign Int8) ENGINE = CollapsingMergeTree(sign) ORDER BY (id, date); diff --git a/tests/queries/0_stateless/00957_neighbor.sql b/tests/queries/0_stateless/00957_neighbor.sql index c4b801c80cb..8c40f0aab47 100644 --- a/tests/queries/0_stateless/00957_neighbor.sql +++ b/tests/queries/0_stateless/00957_neighbor.sql @@ -1,3 +1,4 @@ +SET allow_deprecated_functions = 1; -- no arguments select neighbor(); -- { serverError 42 } -- single argument @@ -39,4 +40,4 @@ select number, -number * 2 as offset, neighbor(number, offset, number * 10) from select 'Dynamic column and offset, without defaults'; select number, -(number - 2) * 2 as offset, neighbor(number, offset) from numbers(6); select 'Constant column'; -select number, neighbor(1000, 10) from numbers(3); \ No newline at end of file +select number, neighbor(1000, 10) from numbers(3); diff --git a/tests/queries/0_stateless/00996_neighbor.sql b/tests/queries/0_stateless/00996_neighbor.sql index 25c20b1b896..50b07242eac 100644 --- a/tests/queries/0_stateless/00996_neighbor.sql +++ b/tests/queries/0_stateless/00996_neighbor.sql @@ -1,3 +1,4 @@ +SET allow_deprecated_functions = 1; SELECT number, neighbor(toString(number), 0) FROM numbers(10); SELECT number, neighbor(toString(number), 5) FROM numbers(10); diff --git a/tests/queries/0_stateless/01012_reset_running_accumulate.sql b/tests/queries/0_stateless/01012_reset_running_accumulate.sql index c2c5bf6f87d..eed653cc629 100644 --- a/tests/queries/0_stateless/01012_reset_running_accumulate.sql +++ b/tests/queries/0_stateless/01012_reset_running_accumulate.sql @@ -1,5 +1,6 @@ -- Disable external aggregation because the state is reset for each new block of data in 'runningAccumulate' function. SET max_bytes_before_external_group_by = 0; +SET allow_deprecated_functions = 1; SELECT grouping, item, @@ -11,4 +12,4 @@ FROM ( FROM (SELECT number FROM system.numbers LIMIT 30) GROUP BY grouping, item ORDER BY grouping, item -); \ No newline at end of file +); diff --git a/tests/queries/0_stateless/01051_aggregate_function_crash.sql b/tests/queries/0_stateless/01051_aggregate_function_crash.sql index 4570662ca90..c50c275d834 100644 --- a/tests/queries/0_stateless/01051_aggregate_function_crash.sql +++ b/tests/queries/0_stateless/01051_aggregate_function_crash.sql @@ -1,4 +1,6 @@ +SET allow_deprecated_functions = 1; + SELECT runningAccumulate(string_state) FROM ( SELECT argMaxState(repeat('a', 48), 1) AS string_state -) \ No newline at end of file +) diff --git a/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql b/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql index 13d90b9f43e..6ea42ec32b0 100644 --- a/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql +++ b/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql @@ -1,6 +1,7 @@ SET enable_optimize_predicate_expression = 1; SET joined_subquery_requires_alias = 0; SET convert_query_to_cnf = 0; +SET allow_deprecated_functions = 1; -- https://github.com/ClickHouse/ClickHouse/issues/3885 -- https://github.com/ClickHouse/ClickHouse/issues/5485 diff --git a/tests/queries/0_stateless/01455_optimize_trivial_insert_select.sql b/tests/queries/0_stateless/01455_optimize_trivial_insert_select.sql index 5b59bc065dd..466c9aa3707 100644 --- a/tests/queries/0_stateless/01455_optimize_trivial_insert_select.sql +++ b/tests/queries/0_stateless/01455_optimize_trivial_insert_select.sql @@ -1,4 +1,5 @@ SET max_insert_threads = 1, max_threads = 100, min_insert_block_size_rows = 1048576, max_block_size = 65536; +SET allow_deprecated_functions = 1; DROP TABLE IF EXISTS t; CREATE TABLE t (x UInt64) ENGINE = StripeLog; -- For trivial INSERT SELECT, max_threads is lowered to max_insert_threads and max_block_size is changed to min_insert_block_size_rows. diff --git a/tests/queries/0_stateless/01665_running_difference_ubsan.sql b/tests/queries/0_stateless/01665_running_difference_ubsan.sql index fa09502a105..504cb0269f8 100644 --- a/tests/queries/0_stateless/01665_running_difference_ubsan.sql +++ b/tests/queries/0_stateless/01665_running_difference_ubsan.sql @@ -1 +1,2 @@ +SET allow_deprecated_functions = 1; SELECT k, d, i FROM (SELECT t.1 AS k, t.2 AS v, runningDifference(v) AS d, runningDifference(cityHash64(t.1)) AS i FROM (SELECT arrayJoin([(NULL, 65535), ('a', 7), ('a', 3), ('b', 11), ('b', 2), ('', -9223372036854775808)]) AS t)) WHERE i = 9223372036854775807; diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting.sh b/tests/queries/0_stateless/02496_remove_redundant_sorting.sh index 23eccb4e782..31d2936628b 100755 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting.sh +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting.sh @@ -314,7 +314,8 @@ FROM FROM numbers(10) ORDER BY number DESC ) -ORDER BY number ASC" +ORDER BY number ASC +SETTINGS allow_deprecated_functions = 1" run_query "$query" echo "-- non-stateful function does _not_ prevent removing inner ORDER BY" diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference b/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference index 47348651025..d74ef70a23f 100644 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference @@ -477,6 +477,7 @@ FROM ORDER BY number DESC ) ORDER BY number ASC +SETTINGS allow_deprecated_functions = 1 -- explain Expression (Project names) Sorting (Sorting for ORDER BY) diff --git a/tests/queries/0_stateless/02788_fix_logical_error_in_sorting.sql b/tests/queries/0_stateless/02788_fix_logical_error_in_sorting.sql index 60905e2634f..6964d8cf47d 100644 --- a/tests/queries/0_stateless/02788_fix_logical_error_in_sorting.sql +++ b/tests/queries/0_stateless/02788_fix_logical_error_in_sorting.sql @@ -1,3 +1,5 @@ +SET allow_deprecated_functions = 1; + DROP TABLE IF EXISTS session_events; DROP TABLE IF EXISTS event_types; diff --git a/tests/queries/0_stateless/02842_largestTriangleThreeBuckets_aggregate_function.sql b/tests/queries/0_stateless/02842_largestTriangleThreeBuckets_aggregate_function.sql index 8a6a2358918..254875ba041 100644 --- a/tests/queries/0_stateless/02842_largestTriangleThreeBuckets_aggregate_function.sql +++ b/tests/queries/0_stateless/02842_largestTriangleThreeBuckets_aggregate_function.sql @@ -1,3 +1,4 @@ +SET allow_deprecated_functions = 1; drop table if exists largestTriangleThreeBucketsTestFloat64Float64; CREATE TABLE largestTriangleThreeBucketsTestFloat64Float64 @@ -60,4 +61,4 @@ SELECT point_x - neighbor(point_x, -1) AS point_x_diff_with_previous_row FROM largestTriangleTreeBucketsBucketSizeTest LIMIT 990, 10; -DROP TABLE largestTriangleTreeBucketsBucketSizeTest; \ No newline at end of file +DROP TABLE largestTriangleTreeBucketsBucketSizeTest; diff --git a/tests/queries/0_stateless/02901_predicate_pushdown_cte_stateful.sql b/tests/queries/0_stateless/02901_predicate_pushdown_cte_stateful.sql index f63e1d59471..a208519b655 100644 --- a/tests/queries/0_stateless/02901_predicate_pushdown_cte_stateful.sql +++ b/tests/queries/0_stateless/02901_predicate_pushdown_cte_stateful.sql @@ -1,3 +1,5 @@ +SET allow_deprecated_functions = 1; + CREATE TABLE t ( `rDate` String, From 2d7a04a68418e7ad63795a29fe8249055079434c Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 30 Apr 2024 14:41:50 +0100 Subject: [PATCH 198/624] fix tests 3 --- .../queries/0_stateless/02496_remove_redundant_sorting.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting.reference b/tests/queries/0_stateless/02496_remove_redundant_sorting.reference index bc22ae23ee1..dbb8ad02293 100644 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting.reference +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting.reference @@ -478,6 +478,7 @@ FROM ORDER BY number DESC ) ORDER BY number ASC +SETTINGS allow_deprecated_functions = 1 -- explain Expression (Projection) Sorting (Sorting for ORDER BY) From 0d6cac4068e077681c9a214df0dcef400a8bae64 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 30 Apr 2024 14:25:19 +0000 Subject: [PATCH 199/624] Fix: simplify test --- .../test_parallel_replicas_failover/test.py | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/tests/integration/test_parallel_replicas_failover/test.py b/tests/integration/test_parallel_replicas_failover/test.py index fa9e1488b20..bf25136bff7 100644 --- a/tests/integration/test_parallel_replicas_failover/test.py +++ b/tests/integration/test_parallel_replicas_failover/test.py @@ -84,16 +84,9 @@ def test_skip_replicas_without_table(start_cluster): node1.query("SYSTEM FLUSH LOGS") assert ( node1.query( - f"SELECT ProfileEvents['DistributedConnectionMissingTable'] FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '{log_comment}' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0" + f"SELECT ProfileEvents['DistributedConnectionMissingTable'], ProfileEvents['ParallelReplicasUnavailableCount'] FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '{log_comment}' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0" ) - == "1\n" - ) - - assert ( - node1.query( - f"SELECT ProfileEvents['ParallelReplicasUnavailableCount'] FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '{log_comment}' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0" - ) - == "1\n" + == "1\t1\n" ) From b66c53c82e1994951a9972c72cff2d65553a3d12 Mon Sep 17 00:00:00 2001 From: Xiaofei Hu <10776938+litlig@users.noreply.github.com> Date: Tue, 30 Apr 2024 10:00:11 -0700 Subject: [PATCH 200/624] Add ttl info to system table --- .../System/StorageSystemPartsColumns.cpp | 10 ++++---- .../02117_show_create_table_system.reference | 2 ++ ...tl_in_system_parts_columns_table.reference | 4 +++ ...3143_ttl_in_system_parts_columns_table.sql | 25 +++++++++++++++++++ 4 files changed, 36 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/03143_ttl_in_system_parts_columns_table.reference create mode 100644 tests/queries/0_stateless/03143_ttl_in_system_parts_columns_table.sql diff --git a/src/Storages/System/StorageSystemPartsColumns.cpp b/src/Storages/System/StorageSystemPartsColumns.cpp index ad831426197..e09ce02faea 100644 --- a/src/Storages/System/StorageSystemPartsColumns.cpp +++ b/src/Storages/System/StorageSystemPartsColumns.cpp @@ -65,8 +65,8 @@ StorageSystemPartsColumns::StorageSystemPartsColumns(const StorageID & table_id_ {"column_data_uncompressed_bytes", std::make_shared(), "Total size of the decompressed data in the column, in bytes."}, {"column_marks_bytes", std::make_shared(), "The size of the marks for column, in bytes."}, {"column_modification_time", std::make_shared(std::make_shared()), "The last time the column was modified."}, - {"column_ttl_info.min", std::make_shared(std::make_shared()), "The minimum value of the calculated TTL expression of the column."} - {"column_ttl_info.max", std::make_shared(std::make_shared()), "The maximum value of the calculated TTL expression of the column."} + {"column_ttl_info.min", std::make_shared(std::make_shared()), "The minimum value of the calculated TTL expression of the column."}, + {"column_ttl_info.max", std::make_shared(std::make_shared()), "The maximum value of the calculated TTL expression of the column."}, {"serialization_kind", std::make_shared(), "Kind of serialization of a column"}, {"substreams", std::make_shared(std::make_shared()), "Names of substreams to which column is serialized"}, @@ -252,8 +252,8 @@ void StorageSystemPartsColumns::processNextStorage( else columns[res_index++]->insertDefault(); } - bool column_has_ttl = part->ttl_info.columns_ttl.contains(column.name); - if (columns_mask[src_index++] + bool column_has_ttl = part->ttl_infos.columns_ttl.contains(column.name); + if (columns_mask[src_index++]) { if (column_has_ttl) columns[res_index++]->insert(static_cast(part->ttl_infos.columns_ttl[column.name].min)); @@ -262,7 +262,7 @@ void StorageSystemPartsColumns::processNextStorage( } if (columns_mask[src_index++]) { - if column_has_ttl + if (column_has_ttl) columns[res_index++]->insert(static_cast(part->ttl_infos.columns_ttl[column.name].max)); else columns[res_index++]->insertDefault(); diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index bdd0da7d166..8ba7dbe58e5 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -565,6 +565,8 @@ CREATE TABLE system.parts_columns `column_data_uncompressed_bytes` UInt64, `column_marks_bytes` UInt64, `column_modification_time` Nullable(DateTime), + `column_ttl_info.min` Nullable(DateTime), + `column_ttl_info.max` Nullable(DateTime), `serialization_kind` String, `substreams` Array(String), `filenames` Array(String), diff --git a/tests/queries/0_stateless/03143_ttl_in_system_parts_columns_table.reference b/tests/queries/0_stateless/03143_ttl_in_system_parts_columns_table.reference new file mode 100644 index 00000000000..b67f6764969 --- /dev/null +++ b/tests/queries/0_stateless/03143_ttl_in_system_parts_columns_table.reference @@ -0,0 +1,4 @@ +all_1_1_0 timestamp DateTime \N \N +all_1_1_0 x UInt32 2024-05-23 00:00:00 2024-05-23 00:00:00 +all_1_1_0 y String 2024-04-24 00:00:00 2024-04-24 00:00:00 +all_1_1_0 z String \N \N diff --git a/tests/queries/0_stateless/03143_ttl_in_system_parts_columns_table.sql b/tests/queries/0_stateless/03143_ttl_in_system_parts_columns_table.sql new file mode 100644 index 00000000000..d3b372b137d --- /dev/null +++ b/tests/queries/0_stateless/03143_ttl_in_system_parts_columns_table.sql @@ -0,0 +1,25 @@ +DROP TABLE IF EXISTS test_03143; + +CREATE TABLE test_03143 ( + timestamp DateTime, + x UInt32 TTL timestamp + INTERVAL 1 MONTH, + y String TTL timestamp + INTERVAL 1 DAY, + z String +) +ENGINE = MergeTree +ORDER BY tuple(); + + +INSERT INTO test_03143 VALUES ('2024-04-23', 123, 'Hello, world!', 'xxx yyy'); + +SELECT + name, + column, + type, + column_ttl_info.min, + column_ttl_info.max +FROM system.parts_columns +WHERE table = 'test_03143' and database = currentDatabase() +ORDER BY name, column; + +DROP TABLE IF EXISTS test_03143; From a4d811eb1b7bdf0b7eb95d4ec085b55a34b55f2f Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 30 Apr 2024 20:46:03 +0100 Subject: [PATCH 201/624] fix tests 4 --- tests/queries/0_stateless/03131_deprecated_functions.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03131_deprecated_functions.sql b/tests/queries/0_stateless/03131_deprecated_functions.sql index 4cd7c0f473f..35cfe648c00 100644 --- a/tests/queries/0_stateless/03131_deprecated_functions.sql +++ b/tests/queries/0_stateless/03131_deprecated_functions.sql @@ -1,8 +1,8 @@ -SELECT number, neighbor(number, 2) FROM system.numbers LIMIT 10; -- { serverError 720 } +SELECT number, neighbor(number, 2) FROM system.numbers LIMIT 10; -- { serverError 721 } -SELECT runningDifference(number) FROM system.numbers LIMIT 10; -- { serverError 720 } +SELECT runningDifference(number) FROM system.numbers LIMIT 10; -- { serverError 721 } -SELECT k, runningAccumulate(sum_k) AS res FROM (SELECT number as k, sumState(k) AS sum_k FROM numbers(10) GROUP BY k ORDER BY k); -- { serverError 720 } +SELECT k, runningAccumulate(sum_k) AS res FROM (SELECT number as k, sumState(k) AS sum_k FROM numbers(10) GROUP BY k ORDER BY k); -- { serverError 721 } SET allow_deprecated_functions=1; From aeafb40b2b7bf7293b32f369a952ace67d32c0cd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 30 Apr 2024 22:22:03 +0200 Subject: [PATCH 202/624] Update 03142_untuple_crash.sql --- tests/queries/0_stateless/03142_untuple_crash.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03142_untuple_crash.sql b/tests/queries/0_stateless/03142_untuple_crash.sql index e0f4225dbac..ac5dbba0de1 100644 --- a/tests/queries/0_stateless/03142_untuple_crash.sql +++ b/tests/queries/0_stateless/03142_untuple_crash.sql @@ -1 +1,2 @@ +SET allow_experimental_analyzer=1; SELECT untuple(x -> 0) -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } From 685fb273e382dab8e507e6f14a27d80d5684ed15 Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 30 Apr 2024 22:39:39 +0200 Subject: [PATCH 203/624] Escape ANSI sequences in UTF8::computeWidth, add a test --- src/Common/StringUtils/StringUtils.h | 6 ++++ src/Common/UTF8Helpers.cpp | 36 ++++++++++++------- ..._skip_ANSI_in_UTF8_compute_width.reference | 18 ++++++++++ .../03142_skip_ANSI_in_UTF8_compute_width.sql | 18 ++++++++++ 4 files changed, 65 insertions(+), 13 deletions(-) create mode 100644 tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.reference create mode 100644 tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql diff --git a/src/Common/StringUtils/StringUtils.h b/src/Common/StringUtils/StringUtils.h index 2c9331a43e2..16fba0370f5 100644 --- a/src/Common/StringUtils/StringUtils.h +++ b/src/Common/StringUtils/StringUtils.h @@ -138,6 +138,12 @@ inline bool isPrintableASCII(char c) return uc >= 32 && uc <= 126; /// 127 is ASCII DEL. } +inline bool isCSIFinalByte(char c) +{ + uint8_t uc = c; + return uc >= 0x40 && uc <= 0x7E; /// ASCII @A–Z[\]^_`a–z{|}~ +} + inline bool isPunctuationASCII(char c) { uint8_t uc = c; diff --git a/src/Common/UTF8Helpers.cpp b/src/Common/UTF8Helpers.cpp index 0af31726f40..9b9c08afbaa 100644 --- a/src/Common/UTF8Helpers.cpp +++ b/src/Common/UTF8Helpers.cpp @@ -97,13 +97,14 @@ namespace enum ComputeWidthMode { Width, /// Calculate and return visible width - BytesBeforLimit /// Calculate and return the maximum number of bytes when substring fits in visible width. + BytesBeforeLimit /// Calculate and return the maximum number of bytes when substring fits in visible width. }; template size_t computeWidthImpl(const UInt8 * data, size_t size, size_t prefix, size_t limit) noexcept { UTF8Decoder decoder; + int isEscapeSequence = false; size_t width = 0; size_t rollback = 0; for (size_t i = 0; i < size; ++i) @@ -132,21 +133,32 @@ size_t computeWidthImpl(const UInt8 * data, size_t size, size_t prefix, size_t l } else { - i += 16; - width += 16; + if (isEscapeSequence) + { + break; + } + else + { + i += 16; + width += 16; + } } } #endif while (i < size && isPrintableASCII(data[i])) { - ++width; + if (!isEscapeSequence) + ++width; + if (isCSIFinalByte(data[i]) && data[i-1]!='\x1b') + isEscapeSequence = false; /// end of CSI escape sequence reached ++i; } /// Now i points to position in bytes after regular ASCII sequence /// and if width > limit, then (width - limit) is the number of extra ASCII characters after width limit. - if (mode == BytesBeforLimit && width > limit) + + if (mode == BytesBeforeLimit && width > limit) return i - (width - limit); switch (decoder.decode(data[i])) @@ -162,20 +174,18 @@ size_t computeWidthImpl(const UInt8 * data, size_t size, size_t prefix, size_t l } case UTF8Decoder::ACCEPT: { - // there are special control characters that manipulate the terminal output. - // (`0x08`, `0x09`, `0x0a`, `0x0b`, `0x0c`, `0x0d`, `0x1b`) - // Since we don't touch the original column data, there is no easy way to escape them. - // TODO: escape control characters // TODO: multiline support for '\n' - // special treatment for '\t' + // special treatment for '\t' and for ESC size_t next_width = width; + if (decoder.codepoint == '\x1b') + isEscapeSequence = true; if (decoder.codepoint == '\t') next_width += 8 - (prefix + width) % 8; else next_width += wcwidth(decoder.codepoint); - if (mode == BytesBeforLimit && next_width > limit) + if (mode == BytesBeforeLimit && next_width > limit) return i - rollback; width = next_width; @@ -189,7 +199,7 @@ size_t computeWidthImpl(const UInt8 * data, size_t size, size_t prefix, size_t l } // no need to handle trailing sequence as they have zero width - return (mode == BytesBeforLimit) ? size : width; + return (mode == BytesBeforeLimit) ? size : width; } } @@ -202,7 +212,7 @@ size_t computeWidth(const UInt8 * data, size_t size, size_t prefix) noexcept size_t computeBytesBeforeWidth(const UInt8 * data, size_t size, size_t prefix, size_t limit) noexcept { - return computeWidthImpl(data, size, prefix, limit); + return computeWidthImpl(data, size, prefix, limit); } } diff --git a/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.reference b/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.reference new file mode 100644 index 00000000000..4e1caa85ac7 --- /dev/null +++ b/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.reference @@ -0,0 +1,18 @@ +█ +█ +█ +█ +█ +█ +█ +█ +█ +█ +█ +█ +█ +█ +█ +█ +█ +█ diff --git a/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql b/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql new file mode 100644 index 00000000000..6487e9d5140 --- /dev/null +++ b/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql @@ -0,0 +1,18 @@ +SELECT format('\x1b[38;2;{0};{1};{2}m█\x1b[0m', 255, 128, 0) AS x; +SELECT '\x1b[1A█' AS x; +SELECT '\x1b[1C█' AS x; +SELECT '\x1b[1D█' AS x; +SELECT '\x1b[1E█' AS x; +SELECT '\x1b[1F█' AS x; +SELECT '\x1b[1G█' AS x; +SELECT '\x1b[1;1H█' AS x; +SELECT '\x1b[1J█' AS x; +SELECT '\x1b[1J█' AS x; +SELECT '\x1b[1K█' AS x; +SELECT '\x1b[1S█' AS x; +SELECT '\x1b[1K█' AS x; +SELECT '\x1b[1;1f█' AS x; +SELECT '\x1b[1;1f█' AS x; +SELECT '\x1b[5i█' AS x; +SELECT '\x1b[4i█' AS x; +SELECT '\x1b[6n█' AS x; From c1e465b6fb6543e9cc72c2d434da73b9837e2ea8 Mon Sep 17 00:00:00 2001 From: pufit Date: Tue, 30 Apr 2024 17:00:05 -0400 Subject: [PATCH 204/624] Correct load for SQL security defaults during startup --- src/Databases/DatabaseOnDisk.cpp | 6 +++ ...mpty_sql_security_in_create_view_query.xml | 3 ++ .../test_ignore_empty_sql_security.py | 48 +++++++++++++++++++ 3 files changed, 57 insertions(+) create mode 100644 tests/integration/test_sql_security/configs/ignore_empty_sql_security_in_create_view_query.xml create mode 100644 tests/integration/test_sql_security/test_ignore_empty_sql_security.py diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 674e9afa8ac..93bc119009f 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -67,6 +68,11 @@ std::pair createTableFromAST( ast_create_query.attach = true; ast_create_query.setDatabase(database_name); + if (!ast_create_query.sql_security && ast_create_query.supportSQLSecurity() && !context->getServerSettings().ignore_empty_sql_security_in_create_view_query) + { + ast_create_query.sql_security = std::make_shared(); + InterpreterCreateQuery::processSQLSecurityOption(context, ast_create_query.sql_security->as(), true, ast_create_query.is_materialized_view); + } if (ast_create_query.select && ast_create_query.isView()) ApplyWithSubqueryVisitor::visit(*ast_create_query.select); diff --git a/tests/integration/test_sql_security/configs/ignore_empty_sql_security_in_create_view_query.xml b/tests/integration/test_sql_security/configs/ignore_empty_sql_security_in_create_view_query.xml new file mode 100644 index 00000000000..99819f58630 --- /dev/null +++ b/tests/integration/test_sql_security/configs/ignore_empty_sql_security_in_create_view_query.xml @@ -0,0 +1,3 @@ + + 1 + diff --git a/tests/integration/test_sql_security/test_ignore_empty_sql_security.py b/tests/integration/test_sql_security/test_ignore_empty_sql_security.py new file mode 100644 index 00000000000..cf6b29ec717 --- /dev/null +++ b/tests/integration/test_sql_security/test_ignore_empty_sql_security.py @@ -0,0 +1,48 @@ +#!/usr/bin/env python3 +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node1", + main_configs=["configs/ignore_empty_sql_security_in_create_view_query.xml"], + stay_alive=True, +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + + +def test_load_mv_with_security_none(started_cluster: ClickHouseCluster): + node.query( + "CREATE TABLE test_table (s String) ENGINE = MergeTree ORDER BY s" + ) + node.query( + "CREATE MATERIALIZED VIEW test_mv_1 (s String) ENGINE = MergeTree ORDER BY s AS SELECT * FROM test_table" + ) + node.query("INSERT INTO test_table VALUES ('foo'), ('bar')") + + node.query("CREATE USER test_user") + node.query("GRANT SELECT ON test_mv_1 TO test_user") + + with pytest.raises(Exception, match="Not enough privileges"): + node.query("SELECT count() FROM test_mv_1", user="test_user") + + node.replace_in_config( + "/etc/clickhouse-server/config.d/ignore_empty_sql_security_in_create_view_query.xml", + "1", + "0", + ) + + node.restart_clickhouse() + + assert node.query("SELECT count() FROM test_mv_1", user="test_user") == "2\n" + From a27f29f10c46784773779d92c39e27b0ac12c1e0 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 30 Apr 2024 21:38:56 +0000 Subject: [PATCH 205/624] Automatic style fix --- .../test_sql_security/test_ignore_empty_sql_security.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/tests/integration/test_sql_security/test_ignore_empty_sql_security.py b/tests/integration/test_sql_security/test_ignore_empty_sql_security.py index cf6b29ec717..d6349242cfd 100644 --- a/tests/integration/test_sql_security/test_ignore_empty_sql_security.py +++ b/tests/integration/test_sql_security/test_ignore_empty_sql_security.py @@ -22,9 +22,7 @@ def started_cluster(): def test_load_mv_with_security_none(started_cluster: ClickHouseCluster): - node.query( - "CREATE TABLE test_table (s String) ENGINE = MergeTree ORDER BY s" - ) + node.query("CREATE TABLE test_table (s String) ENGINE = MergeTree ORDER BY s") node.query( "CREATE MATERIALIZED VIEW test_mv_1 (s String) ENGINE = MergeTree ORDER BY s AS SELECT * FROM test_table" ) @@ -45,4 +43,3 @@ def test_load_mv_with_security_none(started_cluster: ClickHouseCluster): node.restart_clickhouse() assert node.query("SELECT count() FROM test_mv_1", user="test_user") == "2\n" - From 3a6136d0e650dee0a660ca9d14d47b77585df3fc Mon Sep 17 00:00:00 2001 From: pufit Date: Tue, 30 Apr 2024 18:42:12 -0400 Subject: [PATCH 206/624] fix style --- tests/integration/test_sql_security/__init__.py | 0 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 tests/integration/test_sql_security/__init__.py diff --git a/tests/integration/test_sql_security/__init__.py b/tests/integration/test_sql_security/__init__.py new file mode 100644 index 00000000000..e69de29bb2d From 424e3f12e209608bfba00b3c5056f004f4c33192 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 1 May 2024 00:52:15 +0000 Subject: [PATCH 207/624] Revert "Merge pull request #62577 from ClickHouse/revert-61551-better-marks-loading" This reverts commit 17315e53b89ef7a8c1bea118b0fb75ec5ab51f9f, reversing changes made to ed35fddddd28d4109fa1934db16e93c41b107343. --- src/Storages/MergeTree/MarkRange.cpp | 5 ++ src/Storages/MergeTree/MarkRange.h | 1 + .../MergeTree/MergeTreeIndexReader.cpp | 3 + .../MergeTree/MergeTreeMarksLoader.cpp | 8 +++ src/Storages/MergeTree/MergeTreeMarksLoader.h | 1 + .../MergeTree/MergeTreeReaderCompact.cpp | 1 + .../MergeTree/MergeTreeReaderStream.cpp | 72 ++++++++++++++----- .../MergeTree/MergeTreeReaderStream.h | 38 +++++++--- .../MergeTree/MergeTreeReaderWide.cpp | 32 ++++++--- src/Storages/MergeTree/MergeTreeReaderWide.h | 1 + .../MergeTree/MergeTreeSequentialSource.cpp | 9 +-- .../test_merge_tree_load_marks/__init__.py | 0 .../configs/config.xml | 12 ++++ .../test_merge_tree_load_marks/test.py | 62 ++++++++++++++++ .../02532_send_logs_level_test.reference | 1 + .../0_stateless/02532_send_logs_level_test.sh | 2 +- 16 files changed, 207 insertions(+), 41 deletions(-) create mode 100644 tests/integration/test_merge_tree_load_marks/__init__.py create mode 100644 tests/integration/test_merge_tree_load_marks/configs/config.xml create mode 100644 tests/integration/test_merge_tree_load_marks/test.py diff --git a/src/Storages/MergeTree/MarkRange.cpp b/src/Storages/MergeTree/MarkRange.cpp index bd8546f04cc..c6e98b4e5a1 100644 --- a/src/Storages/MergeTree/MarkRange.cpp +++ b/src/Storages/MergeTree/MarkRange.cpp @@ -81,6 +81,11 @@ size_t MarkRanges::getNumberOfMarks() const return result; } +bool MarkRanges::isOneRangeForWholePart(size_t num_marks_in_part) const +{ + return size() == 1 && front().begin == 0 && front().end == num_marks_in_part; +} + void MarkRanges::serialize(WriteBuffer & out) const { writeBinaryLittleEndian(this->size(), out); diff --git a/src/Storages/MergeTree/MarkRange.h b/src/Storages/MergeTree/MarkRange.h index 1d9d0a1e27e..f36d5d89825 100644 --- a/src/Storages/MergeTree/MarkRange.h +++ b/src/Storages/MergeTree/MarkRange.h @@ -36,6 +36,7 @@ struct MarkRanges : public std::deque using std::deque::deque; /// NOLINT(modernize-type-traits) size_t getNumberOfMarks() const; + bool isOneRangeForWholePart(size_t num_marks_in_part) const; void serialize(WriteBuffer & out) const; String describe() const; diff --git a/src/Storages/MergeTree/MergeTreeIndexReader.cpp b/src/Storages/MergeTree/MergeTreeIndexReader.cpp index 6012994b46d..e7ae1fc5c13 100644 --- a/src/Storages/MergeTree/MergeTreeIndexReader.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexReader.cpp @@ -31,6 +31,8 @@ std::unique_ptr makeIndexReader( load_marks_threadpool, /*num_columns_in_mark=*/ 1); + marks_loader->startAsyncLoad(); + return std::make_unique( part->getDataPartStoragePtr(), index->getFileName(), extension, marks_count, @@ -65,6 +67,7 @@ MergeTreeIndexReader::MergeTreeIndexReader( mark_cache, uncompressed_cache, std::move(settings)); + version = index_format.version; stream->adjustRightMark(getLastMark(all_mark_ranges_)); diff --git a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp index eae7594448a..168134a329f 100644 --- a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp +++ b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp @@ -64,6 +64,10 @@ MergeTreeMarksLoader::MergeTreeMarksLoader( , read_settings(read_settings_) , num_columns_in_mark(num_columns_in_mark_) , load_marks_threadpool(load_marks_threadpool_) +{ +} + +void MergeTreeMarksLoader::startAsyncLoad() { if (load_marks_threadpool) future = loadMarksAsync(); @@ -102,6 +106,8 @@ MergeTreeMarksGetterPtr MergeTreeMarksLoader::loadMarks() MarkCache::MappedPtr MergeTreeMarksLoader::loadMarksImpl() { + LOG_TEST(getLogger("MergeTreeMarksLoader"), "Loading marks from path {}", mrk_path); + /// Memory for marks must not be accounted as memory usage for query, because they are stored in shared cache. MemoryTrackerBlockerInThread temporarily_disable_memory_tracker; @@ -218,7 +224,9 @@ MarkCache::MappedPtr MergeTreeMarksLoader::loadMarksSync() } } else + { loaded_marks = loadMarksImpl(); + } if (!loaded_marks) { diff --git a/src/Storages/MergeTree/MergeTreeMarksLoader.h b/src/Storages/MergeTree/MergeTreeMarksLoader.h index 73dd462f2fa..2aa4474e1c5 100644 --- a/src/Storages/MergeTree/MergeTreeMarksLoader.h +++ b/src/Storages/MergeTree/MergeTreeMarksLoader.h @@ -50,6 +50,7 @@ public: ~MergeTreeMarksLoader(); + void startAsyncLoad(); MergeTreeMarksGetterPtr loadMarks(); size_t getNumColumns() const { return num_columns_in_mark; } diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 53acfd539fb..643a1c31474 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -48,6 +48,7 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( , profile_callback(profile_callback_) , clock_type(clock_type_) { + marks_loader->startAsyncLoad(); } void MergeTreeReaderCompact::fillColumnPositions() diff --git a/src/Storages/MergeTree/MergeTreeReaderStream.cpp b/src/Storages/MergeTree/MergeTreeReaderStream.cpp index 40a16176c69..15ef02440cb 100644 --- a/src/Storages/MergeTree/MergeTreeReaderStream.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderStream.cpp @@ -13,6 +13,7 @@ namespace ErrorCodes { extern const int ARGUMENT_OUT_OF_BOUND; extern const int CANNOT_READ_ALL_DATA; + extern const int LOGICAL_ERROR; } MergeTreeReaderStream::MergeTreeReaderStream( @@ -41,14 +42,17 @@ MergeTreeReaderStream::MergeTreeReaderStream( { } +void MergeTreeReaderStream::loadMarks() +{ + if (!marks_getter) + marks_getter = marks_loader->loadMarks(); +} + void MergeTreeReaderStream::init() { if (initialized) return; - initialized = true; - marks_getter = marks_loader->loadMarks(); - /// Compute the size of the buffer. auto [max_mark_range_bytes, sum_mark_range_bytes] = estimateMarkRangeBytes(all_mark_ranges); @@ -110,11 +114,15 @@ void MergeTreeReaderStream::init() data_buffer = non_cached_buffer.get(); compressed_data_buffer = non_cached_buffer.get(); } + + initialized = true; } void MergeTreeReaderStream::seekToMarkAndColumn(size_t row_index, size_t column_position) { init(); + loadMarks(); + const auto & mark = marks_getter->getMark(row_index, column_position); try @@ -193,7 +201,7 @@ CompressedReadBufferBase * MergeTreeReaderStream::getCompressedDataBuffer() return compressed_data_buffer; } -size_t MergeTreeReaderStreamSingleColumn::getRightOffset(size_t right_mark) const +size_t MergeTreeReaderStreamSingleColumn::getRightOffset(size_t right_mark) { /// NOTE: if we are reading the whole file, then right_mark == marks_count /// and we will use max_read_buffer_size for buffer size, thus avoiding the need to load marks. @@ -202,7 +210,8 @@ size_t MergeTreeReaderStreamSingleColumn::getRightOffset(size_t right_mark) cons if (marks_count == 0) return 0; - assert(right_mark <= marks_count); + chassert(right_mark <= marks_count); + loadMarks(); if (right_mark == 0) return marks_getter->getMark(right_mark, 0).offset_in_compressed_file; @@ -281,9 +290,9 @@ size_t MergeTreeReaderStreamSingleColumn::getRightOffset(size_t right_mark) cons return file_size; } -std::pair MergeTreeReaderStreamSingleColumn::estimateMarkRangeBytes(const MarkRanges & mark_ranges) const +std::pair MergeTreeReaderStreamSingleColumn::estimateMarkRangeBytes(const MarkRanges & mark_ranges) { - assert(marks_getter != nullptr); + loadMarks(); size_t max_range_bytes = 0; size_t sum_range_bytes = 0; @@ -302,7 +311,34 @@ std::pair MergeTreeReaderStreamSingleColumn::estimateMarkRangeBy return {max_range_bytes, sum_range_bytes}; } -size_t MergeTreeReaderStreamMultipleColumns::getRightOffsetOneColumn(size_t right_mark_non_included, size_t column_position) const +size_t MergeTreeReaderStreamSingleColumnWholePart::getRightOffset(size_t right_mark) +{ + if (right_mark != marks_count) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expected one right mark: {}, got: {}", + marks_count, right_mark); + } + return file_size; +} + +std::pair MergeTreeReaderStreamSingleColumnWholePart::estimateMarkRangeBytes(const MarkRanges & mark_ranges) +{ + if (!mark_ranges.isOneRangeForWholePart(marks_count)) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expected one mark range that covers the whole part, got: {}", + mark_ranges.describe()); + } + return {file_size, file_size}; +} + +void MergeTreeReaderStreamSingleColumnWholePart::seekToMark(size_t) +{ + throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeTreeReaderStreamSingleColumnWholePart cannot seek to marks"); +} + +size_t MergeTreeReaderStreamMultipleColumns::getRightOffsetOneColumn(size_t right_mark_non_included, size_t column_position) { /// NOTE: if we are reading the whole file, then right_mark == marks_count /// and we will use max_read_buffer_size for buffer size, thus avoiding the need to load marks. @@ -311,7 +347,8 @@ size_t MergeTreeReaderStreamMultipleColumns::getRightOffsetOneColumn(size_t righ if (marks_count == 0) return 0; - assert(right_mark_non_included <= marks_count); + chassert(right_mark_non_included <= marks_count); + loadMarks(); if (right_mark_non_included == 0) return marks_getter->getMark(right_mark_non_included, column_position).offset_in_compressed_file; @@ -347,9 +384,9 @@ size_t MergeTreeReaderStreamMultipleColumns::getRightOffsetOneColumn(size_t righ } std::pair -MergeTreeReaderStreamMultipleColumns::estimateMarkRangeBytesOneColumn(const MarkRanges & mark_ranges, size_t column_position) const +MergeTreeReaderStreamMultipleColumns::estimateMarkRangeBytesOneColumn(const MarkRanges & mark_ranges, size_t column_position) { - assert(marks_getter != nullptr); + loadMarks(); /// As a maximal range we return the maximal size of a whole stripe. size_t max_range_bytes = 0; @@ -386,8 +423,9 @@ MergeTreeReaderStreamMultipleColumns::estimateMarkRangeBytesOneColumn(const Mark return {max_range_bytes, sum_range_bytes}; } -MarkInCompressedFile MergeTreeReaderStreamMultipleColumns::getStartOfNextStripeMark(size_t row_index, size_t column_position) const +MarkInCompressedFile MergeTreeReaderStreamMultipleColumns::getStartOfNextStripeMark(size_t row_index, size_t column_position) { + loadMarks(); const auto & current_mark = marks_getter->getMark(row_index, column_position); if (marks_getter->getNumColumns() == 1) @@ -434,27 +472,27 @@ MarkInCompressedFile MergeTreeReaderStreamMultipleColumns::getStartOfNextStripeM return marks_getter->getMark(mark_index + 1, column_position + 1); } -size_t MergeTreeReaderStreamOneOfMultipleColumns::getRightOffset(size_t right_mark_non_included) const +size_t MergeTreeReaderStreamOneOfMultipleColumns::getRightOffset(size_t right_mark_non_included) { return getRightOffsetOneColumn(right_mark_non_included, column_position); } -std::pair MergeTreeReaderStreamOneOfMultipleColumns::estimateMarkRangeBytes(const MarkRanges & mark_ranges) const +std::pair MergeTreeReaderStreamOneOfMultipleColumns::estimateMarkRangeBytes(const MarkRanges & mark_ranges) { return estimateMarkRangeBytesOneColumn(mark_ranges, column_position); } -size_t MergeTreeReaderStreamAllOfMultipleColumns::getRightOffset(size_t right_mark_non_included) const +size_t MergeTreeReaderStreamAllOfMultipleColumns::getRightOffset(size_t right_mark_non_included) { return getRightOffsetOneColumn(right_mark_non_included, marks_loader->getNumColumns() - 1); } -std::pair MergeTreeReaderStreamAllOfMultipleColumns::estimateMarkRangeBytes(const MarkRanges & mark_ranges) const +std::pair MergeTreeReaderStreamAllOfMultipleColumns::estimateMarkRangeBytes(const MarkRanges & mark_ranges) { size_t max_range_bytes = 0; size_t sum_range_bytes = 0; - for (size_t i = 0; i < marks_getter->getNumColumns(); ++i) + for (size_t i = 0; i < marks_loader->getNumColumns(); ++i) { auto [current_max, current_sum] = estimateMarkRangeBytesOneColumn(mark_ranges, i); diff --git a/src/Storages/MergeTree/MergeTreeReaderStream.h b/src/Storages/MergeTree/MergeTreeReaderStream.h index f3ca6953ceb..05341cd8acc 100644 --- a/src/Storages/MergeTree/MergeTreeReaderStream.h +++ b/src/Storages/MergeTree/MergeTreeReaderStream.h @@ -40,6 +40,7 @@ public: /// Seeks to exact mark in file. void seekToMarkAndColumn(size_t row_index, size_t column_position); + /// Seeks to the start of the file. void seekToStart(); /** @@ -53,11 +54,11 @@ public: private: /// Returns offset in file up to which it's needed to read file to read all rows up to @right_mark mark. - virtual size_t getRightOffset(size_t right_mark) const = 0; + virtual size_t getRightOffset(size_t right_mark) = 0; /// Returns estimated max amount of bytes to read among mark ranges (which is used as size for read buffer) /// and total amount of bytes to read in all mark ranges. - virtual std::pair estimateMarkRangeBytes(const MarkRanges & mark_ranges) const = 0; + virtual std::pair estimateMarkRangeBytes(const MarkRanges & mark_ranges) = 0; const ReadBufferFromFileBase::ProfileCallback profile_callback; const clockid_t clock_type; @@ -80,6 +81,7 @@ private: protected: void init(); + void loadMarks(); const MergeTreeReaderSettings settings; const size_t marks_count; @@ -100,11 +102,25 @@ public: { } - size_t getRightOffset(size_t right_mark_non_included) const override; - std::pair estimateMarkRangeBytes(const MarkRanges & mark_ranges) const override; + size_t getRightOffset(size_t right_mark_non_included) override; + std::pair estimateMarkRangeBytes(const MarkRanges & mark_ranges) override; void seekToMark(size_t row_index) override { seekToMarkAndColumn(row_index, 0); } }; +class MergeTreeReaderStreamSingleColumnWholePart : public MergeTreeReaderStream +{ +public: + template + explicit MergeTreeReaderStreamSingleColumnWholePart(Args &&... args) + : MergeTreeReaderStream{std::forward(args)...} + { + } + + size_t getRightOffset(size_t right_mark_non_included) override; + std::pair estimateMarkRangeBytes(const MarkRanges & mark_ranges) override; + void seekToMark(size_t row_index) override; +}; + /// Base class for reading from file that contains multiple columns. /// It is used to read from compact parts. /// See more details about data layout in MergeTreeDataPartCompact.h. @@ -118,9 +134,9 @@ public: } protected: - size_t getRightOffsetOneColumn(size_t right_mark_non_included, size_t column_position) const; - std::pair estimateMarkRangeBytesOneColumn(const MarkRanges & mark_ranges, size_t column_position) const; - MarkInCompressedFile getStartOfNextStripeMark(size_t row_index, size_t column_position) const; + size_t getRightOffsetOneColumn(size_t right_mark_non_included, size_t column_position); + std::pair estimateMarkRangeBytesOneColumn(const MarkRanges & mark_ranges, size_t column_position); + MarkInCompressedFile getStartOfNextStripeMark(size_t row_index, size_t column_position); }; /// Class for reading a single column from file that contains multiple columns @@ -135,8 +151,8 @@ public: { } - size_t getRightOffset(size_t right_mark_non_included) const override; - std::pair estimateMarkRangeBytes(const MarkRanges & mark_ranges) const override; + size_t getRightOffset(size_t right_mark_non_included) override; + std::pair estimateMarkRangeBytes(const MarkRanges & mark_ranges) override; void seekToMark(size_t row_index) override { seekToMarkAndColumn(row_index, column_position); } private: @@ -154,8 +170,8 @@ public: { } - size_t getRightOffset(size_t right_mark_non_included) const override; - std::pair estimateMarkRangeBytes(const MarkRanges & mark_ranges) const override; + size_t getRightOffset(size_t right_mark_non_included) override; + std::pair estimateMarkRangeBytes(const MarkRanges & mark_ranges) override; void seekToMark(size_t row_index) override { seekToMarkAndColumn(row_index, 0); } }; diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 394a22835f1..d398668d5c8 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -43,6 +43,7 @@ MergeTreeReaderWide::MergeTreeReaderWide( mark_ranges_, settings_, avg_value_size_hints_) + , read_whole_part(all_mark_ranges.isOneRangeForWholePart(data_part_info_for_read->getMarksCount())) { try { @@ -227,12 +228,13 @@ void MergeTreeReaderWide::addStreams( auto context = data_part_info_for_read->getContext(); auto * load_marks_threadpool = settings.read_settings.load_marks_asynchronously ? &context->getLoadMarksThreadpool() : nullptr; + size_t num_marks_in_part = data_part_info_for_read->getMarksCount(); auto marks_loader = std::make_shared( data_part_info_for_read, mark_cache, data_part_info_for_read->getIndexGranularityInfo().getMarksFilePath(*stream_name), - data_part_info_for_read->getMarksCount(), + num_marks_in_part, data_part_info_for_read->getIndexGranularityInfo(), settings.save_marks_in_cache, settings.read_settings, @@ -243,11 +245,24 @@ void MergeTreeReaderWide::addStreams( auto stream_settings = settings; stream_settings.is_low_cardinality_dictionary = substream_path.size() > 1 && substream_path[substream_path.size() - 2].type == ISerialization::Substream::Type::DictionaryKeys; - streams.emplace(*stream_name, std::make_unique( - data_part_info_for_read->getDataPartStorage(), *stream_name, DATA_FILE_EXTENSION, - data_part_info_for_read->getMarksCount(), all_mark_ranges, stream_settings, - uncompressed_cache, data_part_info_for_read->getFileSizeOrZero(*stream_name + DATA_FILE_EXTENSION), - std::move(marks_loader), profile_callback, clock_type)); + auto create_stream = [&]() + { + return std::make_unique( + data_part_info_for_read->getDataPartStorage(), *stream_name, DATA_FILE_EXTENSION, + num_marks_in_part, all_mark_ranges, stream_settings, + uncompressed_cache, data_part_info_for_read->getFileSizeOrZero(*stream_name + DATA_FILE_EXTENSION), + std::move(marks_loader), profile_callback, clock_type); + }; + + if (read_whole_part) + { + streams.emplace(*stream_name, create_stream.operator()()); + } + else + { + marks_loader->startAsyncLoad(); + streams.emplace(*stream_name, create_stream.operator()()); + } }; serialization->enumerateStreams(callback); @@ -325,7 +340,8 @@ void MergeTreeReaderWide::prefetchForColumn( if (stream_name && !prefetched_streams.contains(*stream_name)) { - bool seek_to_mark = !continue_reading; + bool seek_to_mark = !continue_reading && !read_whole_part; + if (ReadBuffer * buf = getStream(false, substream_path, data_part_info_for_read->getChecksums(), streams, name_and_type, from_mark, seek_to_mark, current_task_last_mark, cache)) { buf->prefetch(priority); @@ -349,7 +365,7 @@ void MergeTreeReaderWide::readData( deserialize_settings.getter = [&](const ISerialization::SubstreamPath & substream_path) { - bool seek_to_mark = !was_prefetched && !continue_reading; + bool seek_to_mark = !was_prefetched && !continue_reading && !read_whole_part; return getStream( /* seek_to_start = */false, substream_path, diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.h b/src/Storages/MergeTree/MergeTreeReaderWide.h index a9a5526dd65..7ffe565d262 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.h +++ b/src/Storages/MergeTree/MergeTreeReaderWide.h @@ -73,6 +73,7 @@ private: std::unordered_map caches; std::unordered_set prefetched_streams; ssize_t prefetched_from_mark = -1; + bool read_whole_part = false; }; } diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index c022cfe3861..47661a3ff93 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -184,12 +184,12 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( storage_snapshot, *mark_ranges, /*virtual_fields=*/ {}, - /*uncompressed_cache=*/{}, + /*uncompressed_cache=*/ {}, mark_cache.get(), alter_conversions, reader_settings, - {}, - {}); + /*avg_value_size_hints=*/ {}, + /*profile_callback=*/ {}); } static void fillBlockNumberColumns( @@ -230,6 +230,7 @@ try const auto & header = getPort().getHeader(); /// Part level is useful for next step for merging non-merge tree table bool add_part_level = storage.merging_params.mode != MergeTreeData::MergingParams::Ordinary; + size_t num_marks_in_part = data_part->getMarksCount(); if (!isCancelled() && current_row < data_part->rows_count) { @@ -238,7 +239,7 @@ try const auto & sample = reader->getColumns(); Columns columns(sample.size()); - size_t rows_read = reader->readRows(current_mark, data_part->getMarksCount(), continue_reading, rows_to_read, columns); + size_t rows_read = reader->readRows(current_mark, num_marks_in_part, continue_reading, rows_to_read, columns); if (rows_read) { diff --git a/tests/integration/test_merge_tree_load_marks/__init__.py b/tests/integration/test_merge_tree_load_marks/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_merge_tree_load_marks/configs/config.xml b/tests/integration/test_merge_tree_load_marks/configs/config.xml new file mode 100644 index 00000000000..1c9ee8d698f --- /dev/null +++ b/tests/integration/test_merge_tree_load_marks/configs/config.xml @@ -0,0 +1,12 @@ + + + system + text_log
+ 7500 + 1048576 + 8192 + 524288 + false + test +
+
diff --git a/tests/integration/test_merge_tree_load_marks/test.py b/tests/integration/test_merge_tree_load_marks/test.py new file mode 100644 index 00000000000..b066b2a6ec0 --- /dev/null +++ b/tests/integration/test_merge_tree_load_marks/test.py @@ -0,0 +1,62 @@ +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node", + main_configs=["configs/config.xml"], +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +# This test is bad and it should be a functional test but S3 metrics +# are accounted incorrectly for merges in part_log and query_log. +# Also we have text_log with level 'trace' in functional tests +# but this test requeires text_log with level 'test'. + + +@pytest.mark.parametrize("min_bytes_for_wide_part", [0, 1000000000]) +def test_merge_load_marks(started_cluster, min_bytes_for_wide_part): + node.query( + f""" + DROP TABLE IF EXISTS t_load_marks; + + CREATE TABLE t_load_marks (a UInt64, b UInt64) + ENGINE = MergeTree ORDER BY a + SETTINGS min_bytes_for_wide_part = {min_bytes_for_wide_part}; + + INSERT INTO t_load_marks SELECT number, number FROM numbers(1000); + INSERT INTO t_load_marks SELECT number, number FROM numbers(1000); + + OPTIMIZE TABLE t_load_marks FINAL; + SYSTEM FLUSH LOGS; + """ + ) + + uuid = node.query( + "SELECT uuid FROM system.tables WHERE table = 't_prewarm_merge'" + ).strip() + + result = node.query( + f""" + SELECT count() + FROM system.text_log + WHERE (query_id LIKE '%{uuid}::all_1_2_1%') AND (message LIKE '%Loading marks%') + """ + ).strip() + + result = int(result) + + is_wide = min_bytes_for_wide_part == 0 + not_loaded = result == 0 + + assert is_wide == not_loaded diff --git a/tests/queries/0_stateless/02532_send_logs_level_test.reference b/tests/queries/0_stateless/02532_send_logs_level_test.reference index dbd49cfc0a4..7e51b888d9c 100644 --- a/tests/queries/0_stateless/02532_send_logs_level_test.reference +++ b/tests/queries/0_stateless/02532_send_logs_level_test.reference @@ -1,2 +1,3 @@ + MergeTreeMarksLoader: Loading marks from path data.cmrk3 MergeTreeRangeReader: First reader returned: num_rows: 1, columns: 1, total_rows_per_granule: 1, no filter, column[0]: Int32(size = 1), requested columns: key MergeTreeRangeReader: read() returned num_rows: 1, columns: 1, total_rows_per_granule: 1, no filter, column[0]: Int32(size = 1), sample block key diff --git a/tests/queries/0_stateless/02532_send_logs_level_test.sh b/tests/queries/0_stateless/02532_send_logs_level_test.sh index f65d8705569..4afc6d4496b 100755 --- a/tests/queries/0_stateless/02532_send_logs_level_test.sh +++ b/tests/queries/0_stateless/02532_send_logs_level_test.sh @@ -9,7 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -nm -q " drop table if exists data; - create table data (key Int) engine=MergeTree order by tuple(); + create table data (key Int) engine=MergeTree order by tuple() settings min_bytes_for_wide_part = '1G', compress_marks = 1; insert into data values (1); " From 90d6c165e32ba7ba4d4c4db86a5f93187b922d34 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 1 May 2024 10:05:49 +0800 Subject: [PATCH 208/624] Update src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp Co-authored-by: vdimir --- src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp index 12ce8b654bc..decda5c7804 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp @@ -113,11 +113,7 @@ std::vector EmbeddedRocksDBBulkSink::squash(Chunk chunk) /// End of input stream if (chunk.getNumRows() == 0) { - if (chunks.empty()) - return {}; - std::vector to_return; - std::swap(to_return, chunks); - return to_return; + return std::move(chunks); } /// Just read block is already enough. From 6e235ad8235b4bd8b0b15dee7a5d9aedd58ec42c Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 1 May 2024 02:06:44 +0000 Subject: [PATCH 209/624] address reviews Signed-off-by: Duc Canh Le --- src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp | 3 ++- src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h | 3 ++- .../0_stateless/02956_rocksdb_bulk_sink.reference | 1 + tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh | 10 +++++++++- 4 files changed, 14 insertions(+), 3 deletions(-) diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp index decda5c7804..1aca0edc223 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp @@ -90,7 +90,7 @@ EmbeddedRocksDBBulkSink::EmbeddedRocksDBBulkSink( /// 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 - auto base_directory_name = sipHash128String(getContext()->getCurrentQueryId()); + auto base_directory_name = TMP_INSERT_PREFIX + sipHash128String(getContext()->getCurrentQueryId()); insert_directory_queue = fs::path(storage.getDataPaths()[0]) / (base_directory_name + "-" + getRandomASCIIString(8)); fs::create_directory(insert_directory_queue); } @@ -176,6 +176,7 @@ std::pair EmbeddedRocksDBBulkSink::seriali { for (size_t idx = 0; idx < columns.size(); ++idx) serializations[idx]->serializeBinary(*columns[idx], i, idx == primary_key_pos ? writer_key : writer_value, {}); + /// String in ColumnString must be null-terminated writeChar('\0', writer_key); writeChar('\0', writer_value); serialized_key_offsets.emplace_back(writer_key.count()); diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h index 26a4f7c7fc3..e3ca2013154 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h +++ b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h @@ -61,7 +61,8 @@ private: size_t min_block_size_rows = 0; /// For writing SST files - std::atomic_size_t file_counter = 0; + size_t file_counter = 0; + static constexpr auto TMP_INSERT_PREFIX = "tmp_insert_"; String insert_directory_queue; }; diff --git a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference index dcf8a322ed5..74c71827e6e 100644 --- a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference +++ b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference @@ -1,5 +1,6 @@ 0 1000 +1000 1 1000 2 diff --git a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh index 9f771b0fcb4..8acc83fc86c 100755 --- a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh +++ b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh @@ -11,8 +11,15 @@ ${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 F ${CLICKHOUSE_CLIENT} --query "SELECT sum(value) FROM system.rocksdb WHERE database = currentDatabase() AND table = 'rocksdb_worm' AND name = 'no.file.opens';" # should be 0 because all data is still in memtable ${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm;" -# With bulk insertion, there is no memtable, so a small insert should create a new file +# Enabling bulk insertion ${CLICKHOUSE_CLIENT} --query "ALTER TABLE rocksdb_worm MODIFY SETTING optimize_for_bulk_insert = 1;" + +# Testing that key serialization is identical w. and w/o bulk sink +${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE rocksdb_worm;" +${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+2 FROM numbers(1000);" # should override previous keys +${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm WHERE value = key + 2;" + +# With bulk insertion, there is no memtable, so a small insert should create a new file ${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE rocksdb_worm;" ${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers(1000);" ${CLICKHOUSE_CLIENT} --query "SELECT sum(value) FROM system.rocksdb WHERE database = currentDatabase() AND table = 'rocksdb_worm' AND name = 'no.file.opens';" # should be 1 @@ -31,6 +38,7 @@ ${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number % 1000, num ${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm;" ${CLICKHOUSE_CLIENT} --query "SELECT * FROM rocksdb_worm WHERE key = 0;" # should be the latest value - 999001 + # Testing insert with multiple threads ${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE rocksdb_worm;" ${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers_mt(1000000)" & From c462d003af0293b2ffaf0de701cd98a099532b9b Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 1 May 2024 02:11:02 +0000 Subject: [PATCH 210/624] update comments Signed-off-by: Duc Canh Le --- src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h index e3ca2013154..19ce1e3b83e 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h +++ b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h @@ -23,7 +23,7 @@ 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 +/// 2. Squash chunks to reduce the number of SST files class EmbeddedRocksDBBulkSink : public SinkToStorage, public WithContext { public: From deb47060564c9ffe6cabd4c386d45455eb20bf5b Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 1 May 2024 11:19:51 +0200 Subject: [PATCH 211/624] Review fixes --- src/Common/ProfileEvents.cpp | 2 ++ src/Interpreters/Cache/FileCache.cpp | 19 +++++++++++++------ 2 files changed, 15 insertions(+), 6 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index c00d1017586..ed0b29c7b44 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -489,6 +489,8 @@ The server successfully detected this situation and will download merged part fr M(FilesystemCacheFailToReserveSpaceBecauseOfLockContention, "Number of times space reservation was skipped due to a high contention on the cache lock") \ M(FilesystemCacheHoldFileSegments, "Filesystem cache file segments count, which were hold") \ M(FilesystemCacheUnusedHoldFileSegments, "Filesystem cache file segments count, which were hold, but not used (because of seek or LIMIT n, etc)") \ + M(FilesystemCacheFreeSpaceKeepingThreadRun, "Number of times background thread executed free space keeping job") \ + M(FilesystemCacheFreeSpaceKeepingThreadWorkMilliseconds, "Time for which background thread executed free space keeping job") \ \ M(RemoteFSSeeks, "Total number of seeks for async buffer") \ M(RemoteFSPrefetches, "Number of prefetches made with asynchronous reading from remote filesystem") \ diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 0876ce8e398..1ded737941c 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -28,6 +28,8 @@ namespace ProfileEvents extern const Event FilesystemCacheGetOrSetMicroseconds; extern const Event FilesystemCacheGetMicroseconds; extern const Event FilesystemCacheFailToReserveSpaceBecauseOfLockContention; + extern const Event FilesystemCacheFreeSpaceKeepingThreadRun; + extern const Event FilesystemCacheFreeSpaceKeepingThreadWorkMilliseconds; } namespace DB @@ -990,6 +992,8 @@ void FileCache::freeSpaceRatioKeepingThreadFunc() return; } + ProfileEvents::increment(ProfileEvents::FilesystemCacheFreeSpaceKeepingThreadRun); + FileCacheReserveStat stat; EvictionCandidates eviction_candidates; @@ -1009,14 +1013,12 @@ void FileCache::freeSpaceRatioKeepingThreadFunc() 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); + || 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); + || current_elements_count - stat.total_stat.releasable_count <= desired_elements_num); } #endif @@ -1057,12 +1059,17 @@ void FileCache::freeSpaceRatioKeepingThreadFunc() chassert(false); } + watch.stop(); + ProfileEvents::increment(ProfileEvents::FilesystemCacheFreeSpaceKeepingThreadWorkMilliseconds, watch.elapsedMilliseconds()); + LOG_TRACE(log, "Free space ratio keeping thread finished in {} ms", watch.elapsedMilliseconds()); + [[maybe_unused]] bool scheduled = false; if (limits_satisfied) - keep_up_free_space_ratio_task->scheduleAfter(general_reschedule_ms); + scheduled = keep_up_free_space_ratio_task->scheduleAfter(general_reschedule_ms); else - keep_up_free_space_ratio_task->schedule(); + scheduled = keep_up_free_space_ratio_task->schedule(); + chassert(scheduled); } void FileCache::iterate(IterateFunc && func, const UserID & user_id) From 2bcb3d975b136df189614a52e5a3ad90dcdae370 Mon Sep 17 00:00:00 2001 From: Volodya Giro Date: Tue, 13 Feb 2024 15:17:52 +0300 Subject: [PATCH 212/624] Better rendering of multiline strings in Pretty formats --- .../Formats/Impl/PrettyBlockOutputFormat.cpp | 94 ++++++++++++++++++- .../Formats/Impl/PrettyBlockOutputFormat.h | 5 +- .../Impl/PrettyCompactBlockOutputFormat.cpp | 20 +++- .../Impl/PrettySpaceBlockOutputFormat.cpp | 23 ++++- 4 files changed, 136 insertions(+), 6 deletions(-) diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index 086b5bfada2..55147ccba44 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -68,6 +69,17 @@ void PrettyBlockOutputFormat::calculateWidths( } widths[i][j] = UTF8::computeWidth(reinterpret_cast(serialized_value.data()), serialized_value.size(), prefix); + if (serialized_value.contains('\n')) { + size_t row_width = 0; + size_t row_start = 0; + for (size_t k = 0; k < serialized_value.size(); ++k) { + if (serialized_value[k] == '\n') { + row_width = std::max(row_width, k - row_start + 1 + (row_start != 0)); + row_start = k + 1; + } + } + widths[i][j] = std::max(row_width, serialized_value.size() - row_start + 1); + } max_padded_widths[i] = std::max(max_padded_widths[i], std::min(format_settings.pretty.max_column_pad_width, std::min(format_settings.pretty.max_value_width, widths[i][j]))); @@ -303,19 +315,38 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind writeCString(grid_symbols.bar, out); + std::vector transferred_row(num_columns); + bool has_transferred_row = false; + for (size_t j = 0; j < num_columns; ++j) { if (j != 0) writeCString(grid_symbols.bar, out); const auto & type = *header.getByPosition(j).type; + bool has_break_line = false; writeValueWithPadding(*columns[j], *serializations[j], i, widths[j].empty() ? max_widths[j] : widths[j][i], - max_widths[j], cut_to_width, type.shouldAlignRightInPrettyFormats(), isNumber(type)); + max_widths[j], cut_to_width, type.shouldAlignRightInPrettyFormats(), isNumber(type), has_break_line); + + if (has_break_line) { + has_transferred_row = true; + String serialized_value = " "; + { + WriteBufferFromString out_serialize(serialized_value, AppendModeTag()); + serializations[j]->serializeText(*columns[j], i, out_serialize, format_settings); + } + size_t break_line_pos = serialized_value.find_first_of('\n'); + transferred_row[j] = serialized_value.substr(break_line_pos + 1); + } } writeCString(grid_symbols.bar, out); writeReadableNumberTip(chunk); writeCString("\n", out); + + if (has_transferred_row) { + writeTransferredRow(max_widths, transferred_row); + } } if (format_settings.pretty.output_format_pretty_row_numbers) @@ -397,7 +428,7 @@ static String highlightDigitGroups(String source) void PrettyBlockOutputFormat::writeValueWithPadding( const IColumn & column, const ISerialization & serialization, size_t row_num, - size_t value_width, size_t pad_to_width, size_t cut_to_width, bool align_right, bool is_number) + size_t value_width, size_t pad_to_width, size_t cut_to_width, bool align_right, bool is_number, bool & has_line_breake) { String serialized_value = " "; { @@ -405,6 +436,12 @@ void PrettyBlockOutputFormat::writeValueWithPadding( serialization.serializeText(column, row_num, out_serialize, format_settings); } + if (size_t line_breake_pos = serialized_value.find_first_of('\n'); line_breake_pos != String::npos) { + has_line_breake = true; + serialized_value = serialized_value.substr(0, line_breake_pos) + "…"; + value_width = serialized_value.size() - 3; + } + if (cut_to_width && value_width > cut_to_width) { serialized_value.resize(UTF8::computeBytesBeforeWidth( @@ -448,6 +485,59 @@ void PrettyBlockOutputFormat::writeValueWithPadding( } } +void PrettyBlockOutputFormat::writeTransferredRow(const Widths & max_widths, const std::vector & transferred_row) { + const GridSymbols & grid_symbols = format_settings.pretty.charset == FormatSettings::Pretty::Charset::UTF8 ? + utf8_grid_symbols : + ascii_grid_symbols; + + size_t num_columns = max_widths.size(); + + writeCString(grid_symbols.bar, out); + std::vector new_transferred_row(num_columns); + bool has_transferred_row = false; + size_t cur_width = 0; + + for (size_t j = 0; j < num_columns; ++j) + { + if (j != 0) + writeCString(grid_symbols.bar, out); + + String value = transferred_row[j]; + cur_width = value.size(); + + if (size_t break_line_pos = value.find_first_of('\n'); break_line_pos != String::npos) { + has_transferred_row = true; + new_transferred_row[j] = value.substr(break_line_pos + 1); + value = value.substr(0, break_line_pos) + "…"; + cur_width = value.size() - 2; + } + + if (!value.empty()) { + value = "…" + value; + cur_width += 1; + } + + value = " " + value + " "; + + auto write_padding = [&]() + { + if (max_widths[j] > cur_width) + for (size_t k = 0; k < max_widths[j] - cur_width; ++k) + writeChar(' ', out); + }; + + out.write(value.data(), value.size()); + write_padding(); + } + + writeCString(grid_symbols.bar, out); + writeCString("\n", out); + + if (has_transferred_row) { + writeTransferredRow(max_widths, new_transferred_row); + } +} + void PrettyBlockOutputFormat::consume(Chunk chunk) { diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h index 4c52300fbd1..538328979e9 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -48,7 +49,9 @@ protected: void writeValueWithPadding( const IColumn & column, const ISerialization & serialization, size_t row_num, - size_t value_width, size_t pad_to_width, size_t cut_to_width, bool align_right, bool is_number); + size_t value_width, size_t pad_to_width, size_t cut_to_width, bool align_right, bool is_number, bool & has_line_breake); + + void writeTransferredRow(const Widths & max_widths, const std::vector & transferred_row); void resetFormatterImpl() override { diff --git a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp index e1cbf69dbf0..c7890c64bb6 100644 --- a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp @@ -168,6 +168,8 @@ void PrettyCompactBlockOutputFormat::writeRow( writeCString(grid_symbols.bar, out); + std::vector transferred_row(num_columns); + bool has_transferred_row = false; for (size_t j = 0; j < num_columns; ++j) { if (j != 0) @@ -175,12 +177,28 @@ void PrettyCompactBlockOutputFormat::writeRow( const auto & type = *header.getByPosition(j).type; const auto & cur_widths = widths[j].empty() ? max_widths[j] : widths[j][row_num]; - writeValueWithPadding(*columns[j], *serializations[j], row_num, cur_widths, max_widths[j], cut_to_width, type.shouldAlignRightInPrettyFormats(), isNumber(type)); + bool has_break_line = false; + writeValueWithPadding(*columns[j], *serializations[j], row_num, cur_widths, max_widths[j], cut_to_width, type.shouldAlignRightInPrettyFormats(), isNumber(type), has_break_line); + + if (has_break_line) { + has_transferred_row = true; + String serialized_value = " "; + { + WriteBufferFromString out_serialize(serialized_value, AppendModeTag()); + serializations[j]->serializeText(*columns[j], row_num, out_serialize, format_settings); + } + size_t break_line_pos = serialized_value.find_first_of('\n'); + transferred_row[j] = serialized_value.substr(break_line_pos + 1); + } } writeCString(grid_symbols.bar, out); writeReadableNumberTip(chunk); writeCString("\n", out); + + if (has_transferred_row) { + writeTransferredRow(max_widths, transferred_row); + } } void PrettyCompactBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind) diff --git a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp index 3f224f034aa..4604704cab6 100644 --- a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp @@ -70,6 +70,9 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port } writeCString("\n\n", out); + std::vector transferred_row(num_columns); + bool has_transferred_row = false; + for (size_t row = 0; row < num_rows && total_rows + row < max_rows; ++row) { if (format_settings.pretty.output_format_pretty_row_numbers) @@ -92,12 +95,28 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port const auto & type = *header.getByPosition(column).type; auto & cur_width = widths[column].empty() ? max_widths[column] : widths[column][row]; - writeValueWithPadding( - *columns[column], *serializations[column], row, cur_width, max_widths[column], cut_to_width, type.shouldAlignRightInPrettyFormats(), isNumber(type)); + bool has_break_line = false; + writeValueWithPadding(*columns[column], *serializations[column], + row, cur_width, max_widths[column], cut_to_width, type.shouldAlignRightInPrettyFormats(), isNumber(type), has_break_line); + + if (has_break_line) { + has_transferred_row = true; + String serialized_value = " "; + { + WriteBufferFromString out_serialize(serialized_value, AppendModeTag()); + serializations[column]->serializeText(*columns[column], row, out_serialize, format_settings); + } + size_t break_line_pos = serialized_value.find_first_of('\n'); + transferred_row[column] = serialized_value.substr(break_line_pos + 1); + } } writeReadableNumberTip(chunk); writeChar('\n', out); + + if (has_transferred_row) { + writeTransferredRow(max_widths, transferred_row); + } } total_rows += num_rows; From a8f1d689aabcf41f2edfe19c5876d6d890865ee1 Mon Sep 17 00:00:00 2001 From: Volodya Giro Date: Tue, 13 Feb 2024 18:05:09 +0300 Subject: [PATCH 213/624] Code style --- .../Formats/Impl/PrettyBlockOutputFormat.cpp | 27 ++++++++++--------- .../Formats/Impl/PrettyBlockOutputFormat.h | 1 - .../Impl/PrettyCompactBlockOutputFormat.cpp | 6 ++--- .../Impl/PrettySpaceBlockOutputFormat.cpp | 3 +-- 4 files changed, 19 insertions(+), 18 deletions(-) diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index 55147ccba44..d3b812b3509 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -1,4 +1,3 @@ -#include #include #include #include @@ -69,11 +68,14 @@ void PrettyBlockOutputFormat::calculateWidths( } widths[i][j] = UTF8::computeWidth(reinterpret_cast(serialized_value.data()), serialized_value.size(), prefix); - if (serialized_value.contains('\n')) { + if (serialized_value.contains('\n')) + { size_t row_width = 0; size_t row_start = 0; - for (size_t k = 0; k < serialized_value.size(); ++k) { - if (serialized_value[k] == '\n') { + for (size_t k = 0; k < serialized_value.size(); ++k) + { + if (serialized_value[k] == '\n') + { row_width = std::max(row_width, k - row_start + 1 + (row_start != 0)); row_start = k + 1; } @@ -344,9 +346,8 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind writeReadableNumberTip(chunk); writeCString("\n", out); - if (has_transferred_row) { + if (has_transferred_row) writeTransferredRow(max_widths, transferred_row); - } } if (format_settings.pretty.output_format_pretty_row_numbers) @@ -485,7 +486,8 @@ void PrettyBlockOutputFormat::writeValueWithPadding( } } -void PrettyBlockOutputFormat::writeTransferredRow(const Widths & max_widths, const std::vector & transferred_row) { +void PrettyBlockOutputFormat::writeTransferredRow(const Widths & max_widths, const std::vector & transferred_row) +{ const GridSymbols & grid_symbols = format_settings.pretty.charset == FormatSettings::Pretty::Charset::UTF8 ? utf8_grid_symbols : ascii_grid_symbols; @@ -505,20 +507,22 @@ void PrettyBlockOutputFormat::writeTransferredRow(const Widths & max_widths, con String value = transferred_row[j]; cur_width = value.size(); - if (size_t break_line_pos = value.find_first_of('\n'); break_line_pos != String::npos) { + if (size_t break_line_pos = value.find_first_of('\n'); break_line_pos != String::npos) + { has_transferred_row = true; new_transferred_row[j] = value.substr(break_line_pos + 1); value = value.substr(0, break_line_pos) + "…"; cur_width = value.size() - 2; } - if (!value.empty()) { + if (!value.empty()) + { value = "…" + value; cur_width += 1; } value = " " + value + " "; - + auto write_padding = [&]() { if (max_widths[j] > cur_width) @@ -533,9 +537,8 @@ void PrettyBlockOutputFormat::writeTransferredRow(const Widths & max_widths, con writeCString(grid_symbols.bar, out); writeCString("\n", out); - if (has_transferred_row) { + if (has_transferred_row) writeTransferredRow(max_widths, new_transferred_row); - } } diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h index 538328979e9..431a9038cd2 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include diff --git a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp index c7890c64bb6..16589e053c4 100644 --- a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp @@ -180,7 +180,8 @@ void PrettyCompactBlockOutputFormat::writeRow( bool has_break_line = false; writeValueWithPadding(*columns[j], *serializations[j], row_num, cur_widths, max_widths[j], cut_to_width, type.shouldAlignRightInPrettyFormats(), isNumber(type), has_break_line); - if (has_break_line) { + if (has_break_line) + { has_transferred_row = true; String serialized_value = " "; { @@ -196,9 +197,8 @@ void PrettyCompactBlockOutputFormat::writeRow( writeReadableNumberTip(chunk); writeCString("\n", out); - if (has_transferred_row) { + if (has_transferred_row) writeTransferredRow(max_widths, transferred_row); - } } void PrettyCompactBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind) diff --git a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp index 4604704cab6..757e9d2f837 100644 --- a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp @@ -114,9 +114,8 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port writeReadableNumberTip(chunk); writeChar('\n', out); - if (has_transferred_row) { + if (has_transferred_row) writeTransferredRow(max_widths, transferred_row); - } } total_rows += num_rows; From 254fde8f3a9a517abaafe4cec5e90a127618f878 Mon Sep 17 00:00:00 2001 From: Volodya Giro Date: Tue, 13 Feb 2024 18:39:58 +0300 Subject: [PATCH 214/624] Code style 2.0 --- src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index d3b812b3509..804c9fc2cf2 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -437,7 +437,8 @@ void PrettyBlockOutputFormat::writeValueWithPadding( serialization.serializeText(column, row_num, out_serialize, format_settings); } - if (size_t line_breake_pos = serialized_value.find_first_of('\n'); line_breake_pos != String::npos) { + if (size_t line_breake_pos = serialized_value.find_first_of('\n'); line_breake_pos != String::npos) + { has_line_breake = true; serialized_value = serialized_value.substr(0, line_breake_pos) + "…"; value_width = serialized_value.size() - 3; From d8859e93163bcb11e0c29c2b50409132beb928c4 Mon Sep 17 00:00:00 2001 From: Volodya Giro Date: Fri, 26 Apr 2024 17:46:18 +0300 Subject: [PATCH 215/624] code style and break_line --- src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp | 9 +++++---- src/Processors/Formats/Impl/PrettyBlockOutputFormat.h | 4 ++-- .../Formats/Impl/PrettySpaceBlockOutputFormat.cpp | 5 +++-- 3 files changed, 10 insertions(+), 8 deletions(-) diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index 804c9fc2cf2..b40ac69f62c 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -329,8 +329,9 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind writeValueWithPadding(*columns[j], *serializations[j], i, widths[j].empty() ? max_widths[j] : widths[j][i], max_widths[j], cut_to_width, type.shouldAlignRightInPrettyFormats(), isNumber(type), has_break_line); - - if (has_break_line) { + + if (has_break_line) + { has_transferred_row = true; String serialized_value = " "; { @@ -429,7 +430,7 @@ static String highlightDigitGroups(String source) void PrettyBlockOutputFormat::writeValueWithPadding( const IColumn & column, const ISerialization & serialization, size_t row_num, - size_t value_width, size_t pad_to_width, size_t cut_to_width, bool align_right, bool is_number, bool & has_line_breake) + size_t value_width, size_t pad_to_width, size_t cut_to_width, bool align_right, bool is_number, bool & has_break_line) { String serialized_value = " "; { @@ -439,7 +440,7 @@ void PrettyBlockOutputFormat::writeValueWithPadding( if (size_t line_breake_pos = serialized_value.find_first_of('\n'); line_breake_pos != String::npos) { - has_line_breake = true; + has_break_line = true; serialized_value = serialized_value.substr(0, line_breake_pos) + "…"; value_width = serialized_value.size() - 3; } diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h index 431a9038cd2..794a89260c4 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h @@ -48,8 +48,8 @@ protected: void writeValueWithPadding( const IColumn & column, const ISerialization & serialization, size_t row_num, - size_t value_width, size_t pad_to_width, size_t cut_to_width, bool align_right, bool is_number, bool & has_line_breake); - + size_t value_width, size_t pad_to_width, size_t cut_to_width, bool align_right, bool is_number, bool & has_break_line); + void writeTransferredRow(const Widths & max_widths, const std::vector & transferred_row); void resetFormatterImpl() override diff --git a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp index 757e9d2f837..aab6f61eb51 100644 --- a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp @@ -98,8 +98,9 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port bool has_break_line = false; writeValueWithPadding(*columns[column], *serializations[column], row, cur_width, max_widths[column], cut_to_width, type.shouldAlignRightInPrettyFormats(), isNumber(type), has_break_line); - - if (has_break_line) { + + if (has_break_line) + { has_transferred_row = true; String serialized_value = " "; { From 983b460c6bf8e8bcdb50a9f938eb724357ced070 Mon Sep 17 00:00:00 2001 From: Volodya Giro Date: Sat, 27 Apr 2024 00:49:46 +0300 Subject: [PATCH 216/624] =?UTF-8?q?=E2=80=A6=20in=20the=20position=20where?= =?UTF-8?q?=20the=20value=20cannot=20appear?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../Formats/Impl/PrettyBlockOutputFormat.cpp | 37 ++++++++++++------- 1 file changed, 24 insertions(+), 13 deletions(-) diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index b40ac69f62c..3d7adb77aff 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -76,11 +76,11 @@ void PrettyBlockOutputFormat::calculateWidths( { if (serialized_value[k] == '\n') { - row_width = std::max(row_width, k - row_start + 1 + (row_start != 0)); + row_width = std::max(row_width, k - row_start); row_start = k + 1; } } - widths[i][j] = std::max(row_width, serialized_value.size() - row_start + 1); + widths[i][j] = std::max(row_width, serialized_value.size() - row_start); } max_padded_widths[i] = std::max(max_padded_widths[i], std::min(format_settings.pretty.max_column_pad_width, @@ -441,8 +441,8 @@ void PrettyBlockOutputFormat::writeValueWithPadding( if (size_t line_breake_pos = serialized_value.find_first_of('\n'); line_breake_pos != String::npos) { has_break_line = true; - serialized_value = serialized_value.substr(0, line_breake_pos) + "…"; - value_width = serialized_value.size() - 3; + serialized_value = serialized_value.substr(0, line_breake_pos); + value_width = serialized_value.size() - 1; } if (cut_to_width && value_width > cut_to_width) @@ -462,7 +462,7 @@ void PrettyBlockOutputFormat::writeValueWithPadding( value_width = format_settings.pretty.max_value_width; } - else + else if (!has_break_line) serialized_value += ' '; auto write_padding = [&]() @@ -486,6 +486,9 @@ void PrettyBlockOutputFormat::writeValueWithPadding( out.write(serialized_value.data(), serialized_value.size()); write_padding(); } + + if (has_break_line) + writeString("…", out); } void PrettyBlockOutputFormat::writeTransferredRow(const Widths & max_widths, const std::vector & transferred_row) @@ -496,6 +499,10 @@ void PrettyBlockOutputFormat::writeTransferredRow(const Widths & max_widths, con size_t num_columns = max_widths.size(); + if (format_settings.pretty.output_format_pretty_row_numbers) + for (size_t i = 0; i < row_number_width; ++i) + writeChar(' ', out); + writeCString(grid_symbols.bar, out); std::vector new_transferred_row(num_columns); bool has_transferred_row = false; @@ -508,22 +515,21 @@ void PrettyBlockOutputFormat::writeTransferredRow(const Widths & max_widths, con String value = transferred_row[j]; cur_width = value.size(); + bool has_break_line = false; if (size_t break_line_pos = value.find_first_of('\n'); break_line_pos != String::npos) { has_transferred_row = true; new_transferred_row[j] = value.substr(break_line_pos + 1); - value = value.substr(0, break_line_pos) + "…"; - cur_width = value.size() - 2; + value = value.substr(0, break_line_pos); + cur_width = value.size(); + has_break_line = true; } if (!value.empty()) - { - value = "…" + value; - cur_width += 1; - } - - value = " " + value + " "; + writeString("…", out); + else + writeChar(' ', out); auto write_padding = [&]() { @@ -534,6 +540,11 @@ void PrettyBlockOutputFormat::writeTransferredRow(const Widths & max_widths, con out.write(value.data(), value.size()); write_padding(); + + if (has_break_line) + writeString("…", out); + else + writeChar(' ', out); } writeCString(grid_symbols.bar, out); From cb09f0c7c10ba02aebbe17b59e640669aba2d0cb Mon Sep 17 00:00:00 2001 From: Volodya Giro Date: Sat, 27 Apr 2024 00:52:31 +0300 Subject: [PATCH 217/624] canonize test --- ...2026_describe_include_subcolumns.reference | 66 +++++++++---------- 1 file changed, 33 insertions(+), 33 deletions(-) diff --git a/tests/queries/0_stateless/02026_describe_include_subcolumns.reference b/tests/queries/0_stateless/02026_describe_include_subcolumns.reference index dec65f62748..082aa29de83 100644 --- a/tests/queries/0_stateless/02026_describe_include_subcolumns.reference +++ b/tests/queries/0_stateless/02026_describe_include_subcolumns.reference @@ -1,33 +1,33 @@ - ┌─name─┬─type──────────────────────────────────────────────────────────────────────┬─default_type─┬─default_expression─┬─comment─────────────────┬─codec_expression─┬─ttl_expression───────┐ -1. │ d │ Date │ │ │ │ │ │ -2. │ n │ Nullable(String) │ │ │ It is a nullable column │ │ │ -3. │ arr1 │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ -4. │ arr2 │ Array(Array(String)) │ │ │ │ │ d + toIntervalDay(1) │ -5. │ t │ Tuple( - s String, - a Array(Tuple( - a UInt32, - b UInt32))) │ │ │ │ ZSTD(1) │ │ - └──────┴───────────────────────────────────────────────────────────────────────────┴──────────────┴────────────────────┴─────────────────────────┴──────────────────┴──────────────────────┘ - ┌─name───────┬─type──────────────────────────────────────────────────────────────────────┬─default_type─┬─default_expression─┬─comment─────────────────┬─codec_expression─┬─ttl_expression───────┬─is_subcolumn─┐ - 1. │ d │ Date │ │ │ │ │ │ 0 │ - 2. │ n │ Nullable(String) │ │ │ It is a nullable column │ │ │ 0 │ - 3. │ arr1 │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ 0 │ - 4. │ arr2 │ Array(Array(String)) │ │ │ │ │ d + toIntervalDay(1) │ 0 │ - 5. │ t │ Tuple( - s String, - a Array(Tuple( - a UInt32, - b UInt32))) │ │ │ │ ZSTD(1) │ │ 0 │ - 6. │ n.null │ UInt8 │ │ │ It is a nullable column │ │ │ 1 │ - 7. │ arr1.size0 │ UInt64 │ │ │ │ │ │ 1 │ - 8. │ arr2.size0 │ UInt64 │ │ │ │ │ d + toIntervalDay(1) │ 1 │ - 9. │ arr2.size1 │ Array(UInt64) │ │ │ │ │ d + toIntervalDay(1) │ 1 │ -10. │ t.s │ String │ │ │ │ ZSTD(1) │ │ 1 │ -11. │ t.a │ Array(Tuple( - a UInt32, - b UInt32)) │ │ │ │ │ │ 1 │ -12. │ t.a.size0 │ UInt64 │ │ │ │ │ │ 1 │ -13. │ t.a.a │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ 1 │ -14. │ t.a.b │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ 1 │ - └────────────┴───────────────────────────────────────────────────────────────────────────┴──────────────┴────────────────────┴─────────────────────────┴──────────────────┴──────────────────────┴──────────────┘ + ┌─name─┬─type─────────────────┬─default_type─┬─default_expression─┬─comment─────────────────┬─codec_expression─┬─ttl_expression───────┐ +1. │ d │ Date │ │ │ │ │ │ +2. │ n │ Nullable(String) │ │ │ It is a nullable column │ │ │ +3. │ arr1 │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ +4. │ arr2 │ Array(Array(String)) │ │ │ │ │ d + toIntervalDay(1) │ +5. │ t │ Tuple( …│ │ │ │ ZSTD(1) │ │ + │ │… s String, …│ │ │ │ │ │ + │ │… a Array(Tuple( …│ │ │ │ │ │ + │ │… a UInt32, …│ │ │ │ │ │ + │ │… b UInt32))) │ │ │ │ │ │ + └──────┴──────────────────────┴──────────────┴────────────────────┴─────────────────────────┴──────────────────┴──────────────────────┘ + ┌─name───────┬─type─────────────────┬─default_type─┬─default_expression─┬─comment─────────────────┬─codec_expression─┬─ttl_expression───────┬─is_subcolumn─┐ + 1. │ d │ Date │ │ │ │ │ │ 0 │ + 2. │ n │ Nullable(String) │ │ │ It is a nullable column │ │ │ 0 │ + 3. │ arr1 │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ 0 │ + 4. │ arr2 │ Array(Array(String)) │ │ │ │ │ d + toIntervalDay(1) │ 0 │ + 5. │ t │ Tuple( …│ │ │ │ ZSTD(1) │ │ 0 │ + │ │… s String, …│ │ │ │ │ │ │ + │ │… a Array(Tuple( …│ │ │ │ │ │ │ + │ │… a UInt32, …│ │ │ │ │ │ │ + │ │… b UInt32))) │ │ │ │ │ │ │ + 6. │ n.null │ UInt8 │ │ │ It is a nullable column │ │ │ 1 │ + 7. │ arr1.size0 │ UInt64 │ │ │ │ │ │ 1 │ + 8. │ arr2.size0 │ UInt64 │ │ │ │ │ d + toIntervalDay(1) │ 1 │ + 9. │ arr2.size1 │ Array(UInt64) │ │ │ │ │ d + toIntervalDay(1) │ 1 │ +10. │ t.s │ String │ │ │ │ ZSTD(1) │ │ 1 │ +11. │ t.a │ Array(Tuple( …│ │ │ │ │ │ 1 │ + │ │… a UInt32, …│ │ │ │ │ │ │ + │ │… b UInt32)) │ │ │ │ │ │ │ +12. │ t.a.size0 │ UInt64 │ │ │ │ │ │ 1 │ +13. │ t.a.a │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ 1 │ +14. │ t.a.b │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ 1 │ + └────────────┴──────────────────────┴──────────────┴────────────────────┴─────────────────────────┴──────────────────┴──────────────────────┴──────────────┘ From da0c6413b3838e6ecb59fff21663af159e4008f1 Mon Sep 17 00:00:00 2001 From: Volodya Giro Date: Sat, 27 Apr 2024 04:03:08 +0300 Subject: [PATCH 218/624] single value without cut --- src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index 3d7adb77aff..3cda0132ab8 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -38,6 +38,8 @@ void PrettyBlockOutputFormat::calculateWidths( max_padded_widths.resize_fill(num_columns); name_widths.resize(num_columns); + const bool no_need_cut_to_width = !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; size_t prefix = 2; // Tab character adjustment @@ -68,7 +70,7 @@ void PrettyBlockOutputFormat::calculateWidths( } widths[i][j] = UTF8::computeWidth(reinterpret_cast(serialized_value.data()), serialized_value.size(), prefix); - if (serialized_value.contains('\n')) + if (serialized_value.contains('\n') && !no_need_cut_to_width) { size_t row_width = 0; size_t row_start = 0; @@ -438,7 +440,7 @@ void PrettyBlockOutputFormat::writeValueWithPadding( serialization.serializeText(column, row_num, out_serialize, format_settings); } - if (size_t line_breake_pos = serialized_value.find_first_of('\n'); line_breake_pos != String::npos) + if (size_t line_breake_pos = serialized_value.find_first_of('\n'); line_breake_pos != String::npos && cut_to_width) { has_break_line = true; serialized_value = serialized_value.substr(0, line_breake_pos); @@ -462,7 +464,7 @@ void PrettyBlockOutputFormat::writeValueWithPadding( value_width = format_settings.pretty.max_value_width; } - else if (!has_break_line) + else if (!has_break_line && cut_to_width) serialized_value += ' '; auto write_padding = [&]() From fd36c6080320dd28528a5e2fc5868247bc795bcc Mon Sep 17 00:00:00 2001 From: Volodya Giro Date: Sat, 27 Apr 2024 04:07:53 +0300 Subject: [PATCH 219/624] add test --- .../03132_pretty_format_break_line.reference | 8 ++++++++ .../0_stateless/03132_pretty_format_break_line.sql | 10 ++++++++++ 2 files changed, 18 insertions(+) create mode 100644 tests/queries/0_stateless/03132_pretty_format_break_line.reference create mode 100644 tests/queries/0_stateless/03132_pretty_format_break_line.sql diff --git a/tests/queries/0_stateless/03132_pretty_format_break_line.reference b/tests/queries/0_stateless/03132_pretty_format_break_line.reference new file mode 100644 index 00000000000..0f87a0c3c22 --- /dev/null +++ b/tests/queries/0_stateless/03132_pretty_format_break_line.reference @@ -0,0 +1,8 @@ +┌─id─┬─value─┐ +│ 0 │ hello…│ +│ │…world │ +└────┴───────┘ + ┌─id─┬─value─┐ +1. │ 0 │ hello…│ + │ │…world │ + └────┴───────┘ diff --git a/tests/queries/0_stateless/03132_pretty_format_break_line.sql b/tests/queries/0_stateless/03132_pretty_format_break_line.sql new file mode 100644 index 00000000000..1c8b75eb64b --- /dev/null +++ b/tests/queries/0_stateless/03132_pretty_format_break_line.sql @@ -0,0 +1,10 @@ +DROP TABLE IF EXISTS t_break_line; + +CREATE TABLE t_break_line (id UInt64, value String) ENGINE=MergeTree ORDER BY id; + +INSERT INTO t_break_line VALUES(0, 'hello\nworld'); + +SELECT * FROM t_break_line FORMAT PrettyCompactNoEscapes SETTINGS output_format_pretty_row_numbers = 0; +SELECT * FROM t_break_line FORMAT PrettyCompactNoEscapes; + +DROP TABLE t_break_line; \ No newline at end of file From b1c2ccecc4eff06912f7dd2d44ae290ec88751cb Mon Sep 17 00:00:00 2001 From: Volodya Giro Date: Sat, 27 Apr 2024 04:44:21 +0300 Subject: [PATCH 220/624] fix --- 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 3cda0132ab8..1d713fef06a 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -464,7 +464,7 @@ void PrettyBlockOutputFormat::writeValueWithPadding( value_width = format_settings.pretty.max_value_width; } - else if (!has_break_line && cut_to_width) + else if (!has_break_line) serialized_value += ' '; auto write_padding = [&]() From 156649a0873200d7c67323ebe655de7837b5f565 Mon Sep 17 00:00:00 2001 From: Volodya Date: Sat, 27 Apr 2024 10:25:42 +0000 Subject: [PATCH 221/624] check cut_to_width first --- src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index 1d713fef06a..57d2cf0a00c 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -70,7 +70,7 @@ void PrettyBlockOutputFormat::calculateWidths( } widths[i][j] = UTF8::computeWidth(reinterpret_cast(serialized_value.data()), serialized_value.size(), prefix); - if (serialized_value.contains('\n') && !no_need_cut_to_width) + if (!no_need_cut_to_width && serialized_value.contains('\n')) { size_t row_width = 0; size_t row_start = 0; @@ -440,7 +440,10 @@ void PrettyBlockOutputFormat::writeValueWithPadding( serialization.serializeText(column, row_num, out_serialize, format_settings); } - if (size_t line_breake_pos = serialized_value.find_first_of('\n'); line_breake_pos != String::npos && cut_to_width) + size_t line_breake_pos = String::npos; + if (cut_to_width) + line_breake_pos = serialized_value.find_first_of('\n'); + if (line_breake_pos != String::npos) { has_break_line = true; serialized_value = serialized_value.substr(0, line_breake_pos); From 77317c0514b891f7c935dba297ee5cd9fcbbec28 Mon Sep 17 00:00:00 2001 From: Volodya Date: Mon, 29 Apr 2024 18:28:32 +0000 Subject: [PATCH 222/624] new tests --- .../Formats/Impl/PrettyBlockOutputFormat.cpp | 25 +++++---- .../Formats/Impl/PrettyBlockOutputFormat.h | 2 +- .../Impl/PrettyCompactBlockOutputFormat.cpp | 2 +- .../Impl/PrettySpaceBlockOutputFormat.cpp | 2 +- .../03132_pretty_format_break_line.reference | 52 ++++++++++++++++--- .../03132_pretty_format_break_line.sql | 12 ++++- 6 files changed, 72 insertions(+), 23 deletions(-) diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index 57d2cf0a00c..f2ca63de2f2 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -350,7 +350,7 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind writeCString("\n", out); if (has_transferred_row) - writeTransferredRow(max_widths, transferred_row); + writeTransferredRow(max_widths, transferred_row, false); } if (format_settings.pretty.output_format_pretty_row_numbers) @@ -440,13 +440,13 @@ void PrettyBlockOutputFormat::writeValueWithPadding( serialization.serializeText(column, row_num, out_serialize, format_settings); } - size_t line_breake_pos = String::npos; + size_t break_line_pos = String::npos; if (cut_to_width) - line_breake_pos = serialized_value.find_first_of('\n'); - if (line_breake_pos != String::npos) + break_line_pos = serialized_value.find_first_of('\n'); + if (break_line_pos != String::npos) { has_break_line = true; - serialized_value = serialized_value.substr(0, line_breake_pos); + serialized_value = serialized_value.substr(0, break_line_pos); value_width = serialized_value.size() - 1; } @@ -496,7 +496,7 @@ void PrettyBlockOutputFormat::writeValueWithPadding( writeString("…", out); } -void PrettyBlockOutputFormat::writeTransferredRow(const Widths & max_widths, const std::vector & transferred_row) +void PrettyBlockOutputFormat::writeTransferredRow(const Widths & max_widths, const std::vector & transferred_row, const bool & space_block) { const GridSymbols & grid_symbols = format_settings.pretty.charset == FormatSettings::Pretty::Charset::UTF8 ? utf8_grid_symbols : @@ -508,15 +508,19 @@ void PrettyBlockOutputFormat::writeTransferredRow(const Widths & max_widths, con for (size_t i = 0; i < row_number_width; ++i) writeChar(' ', out); - writeCString(grid_symbols.bar, out); + if (!space_block) + writeCString(grid_symbols.bar, out); + std::vector new_transferred_row(num_columns); bool has_transferred_row = false; size_t cur_width = 0; for (size_t j = 0; j < num_columns; ++j) { - if (j != 0) + if (j != 0 && !space_block) writeCString(grid_symbols.bar, out); + else if (j != 0) + writeCString(" ", out); String value = transferred_row[j]; cur_width = value.size(); @@ -552,11 +556,12 @@ void PrettyBlockOutputFormat::writeTransferredRow(const Widths & max_widths, con writeChar(' ', out); } - writeCString(grid_symbols.bar, out); + if (!space_block) + writeCString(grid_symbols.bar, out); writeCString("\n", out); if (has_transferred_row) - writeTransferredRow(max_widths, new_transferred_row); + writeTransferredRow(max_widths, new_transferred_row, space_block); } diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h index 794a89260c4..5173610024c 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h @@ -50,7 +50,7 @@ protected: const IColumn & column, const ISerialization & serialization, size_t row_num, size_t value_width, size_t pad_to_width, size_t cut_to_width, bool align_right, bool is_number, bool & has_break_line); - void writeTransferredRow(const Widths & max_widths, const std::vector & transferred_row); + void writeTransferredRow(const Widths & max_widths, const std::vector & transferred_row, const bool & space_block); void resetFormatterImpl() override { diff --git a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp index 16589e053c4..f5fe9496984 100644 --- a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp @@ -198,7 +198,7 @@ void PrettyCompactBlockOutputFormat::writeRow( writeCString("\n", out); if (has_transferred_row) - writeTransferredRow(max_widths, transferred_row); + writeTransferredRow(max_widths, transferred_row, false); } void PrettyCompactBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind) diff --git a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp index aab6f61eb51..30eb9b6417a 100644 --- a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp @@ -116,7 +116,7 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port writeChar('\n', out); if (has_transferred_row) - writeTransferredRow(max_widths, transferred_row); + writeTransferredRow(max_widths, transferred_row, true); } total_rows += num_rows; 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 0f87a0c3c22..38d7401abeb 100644 --- a/tests/queries/0_stateless/03132_pretty_format_break_line.reference +++ b/tests/queries/0_stateless/03132_pretty_format_break_line.reference @@ -1,8 +1,44 @@ -┌─id─┬─value─┐ -│ 0 │ hello…│ -│ │…world │ -└────┴───────┘ - ┌─id─┬─value─┐ -1. │ 0 │ hello…│ - │ │…world │ - └────┴───────┘ +┌─id─┬─value─┬─value1──────┐ +│ 0 │ hello…│ hello world │ +│ │…world │ │ +└────┴───────┴─────────────┘ + ┌─id─┬─value─┬─value1──────┐ +1. │ 0 │ hello…│ hello world │ + │ │…world │ │ + └────┴───────┴─────────────┘ + ┌─id─┬─value─┬─value1──────┐ +1. │ 0 │ hello…│ hello world │ + │ │…world │ │ + └────┴───────┴─────────────┘ + ┏━━━━┳━━━━━━━┳━━━━━━━━━━━━━┓ + ┃ id ┃ value ┃ value1 ┃ + ┡━━━━╇━━━━━━━╇━━━━━━━━━━━━━┩ +1. │ 0 │ hello…│ hello world │ + │ │…world │ │ + └────┴───────┴─────────────┘ + id value value1 + + 0 hello… hello world + …world + id value value1 + +1. 0 hello… hello world + …world +┏━━━━┳━━━━━━━━━━━━━┳━━━━━━━━━━━━━┓ +┃ id ┃ value ┃ value1 ┃ +┡━━━━╇━━━━━━━━━━━━━╇━━━━━━━━━━━━━┩ +│ 0 │ hello …│ hello world │ +│ │…world │ │ +├────┼─────────────┼─────────────┤ +│ 1 │ hello world │ hello …│ +│ │ │…world │ +└────┴─────────────┴─────────────┘ + ┏━━━━┳━━━━━━━━━━━━━┳━━━━━━━━━━━━━┓ + ┃ id ┃ value ┃ value1 ┃ + ┡━━━━╇━━━━━━━━━━━━━╇━━━━━━━━━━━━━┩ +1. │ 1 │ hello world │ hello …│ + │ │ │…world │ + ├────┼─────────────┼─────────────┤ +2. │ 0 │ hello …│ hello world │ + │ │…world │ │ + └────┴─────────────┴─────────────┘ 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 1c8b75eb64b..7c5fa3f1179 100644 --- a/tests/queries/0_stateless/03132_pretty_format_break_line.sql +++ b/tests/queries/0_stateless/03132_pretty_format_break_line.sql @@ -1,10 +1,18 @@ DROP TABLE IF EXISTS t_break_line; -CREATE TABLE t_break_line (id UInt64, value String) ENGINE=MergeTree ORDER BY id; +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'); +INSERT INTO t_break_line VALUES(0, 'hello\nworld', 'hello world'); SELECT * FROM t_break_line FORMAT PrettyCompactNoEscapes SETTINGS output_format_pretty_row_numbers = 0; SELECT * FROM t_break_line FORMAT PrettyCompactNoEscapes; +SELECT * FROM t_break_line FORMAT PrettyCompact; +SELECT * FROM t_break_line FORMAT Pretty; +SELECT * FROM t_break_line FORMAT PrettySpace SETTINGS output_format_pretty_row_numbers = 0; +SELECT * FROM t_break_line FORMAT PrettySpace; + +INSERT INTO t_break_line VALUES(1, 'hello world', 'hello\nworld'); +SELECT * FROM t_break_line FORMAT PrettyMonoBlock SETTINGS output_format_pretty_row_numbers = 0; +SELECT * FROM t_break_line FORMAT PrettyMonoBlock; DROP TABLE t_break_line; \ No newline at end of file From 43b9ba08995c48f36f7881b2f80ecf0900ba6b0f Mon Sep 17 00:00:00 2001 From: Volodya Date: Mon, 29 Apr 2024 19:21:56 +0000 Subject: [PATCH 223/624] order by --- .../0_stateless/03132_pretty_format_break_line.reference | 8 ++++---- .../0_stateless/03132_pretty_format_break_line.sql | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) 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 38d7401abeb..bf82c39f633 100644 --- a/tests/queries/0_stateless/03132_pretty_format_break_line.reference +++ b/tests/queries/0_stateless/03132_pretty_format_break_line.reference @@ -36,9 +36,9 @@ ┏━━━━┳━━━━━━━━━━━━━┳━━━━━━━━━━━━━┓ ┃ id ┃ value ┃ value1 ┃ ┡━━━━╇━━━━━━━━━━━━━╇━━━━━━━━━━━━━┩ -1. │ 1 │ hello world │ hello …│ - │ │ │…world │ - ├────┼─────────────┼─────────────┤ -2. │ 0 │ hello …│ hello world │ +1. │ 0 │ hello …│ hello world │ │ │…world │ │ + ├────┼─────────────┼─────────────┤ +2. │ 1 │ hello world │ hello …│ + │ │ │…world │ └────┴─────────────┴─────────────┘ 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 7c5fa3f1179..54ab54e47ff 100644 --- a/tests/queries/0_stateless/03132_pretty_format_break_line.sql +++ b/tests/queries/0_stateless/03132_pretty_format_break_line.sql @@ -12,7 +12,7 @@ SELECT * FROM t_break_line FORMAT PrettySpace SETTINGS output_format_pretty_row_ SELECT * FROM t_break_line FORMAT PrettySpace; INSERT INTO t_break_line VALUES(1, 'hello world', 'hello\nworld'); -SELECT * FROM t_break_line FORMAT PrettyMonoBlock SETTINGS output_format_pretty_row_numbers = 0; -SELECT * FROM t_break_line FORMAT PrettyMonoBlock; +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 6c9143a61e766159b66475d170088da66784302d Mon Sep 17 00:00:00 2001 From: Volodya Date: Tue, 30 Apr 2024 06:14:59 +0000 Subject: [PATCH 224/624] computeWidth and tests --- .../Formats/Impl/PrettyBlockOutputFormat.cpp | 12 ++--- .../03132_pretty_format_break_line.reference | 44 +++++++++++++++++++ .../03132_pretty_format_break_line.sql | 15 +++++++ 3 files changed, 65 insertions(+), 6 deletions(-) diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index f2ca63de2f2..7338ae18310 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -78,11 +78,11 @@ void PrettyBlockOutputFormat::calculateWidths( { if (serialized_value[k] == '\n') { - row_width = std::max(row_width, k - row_start); + row_width = std::max(row_width, UTF8::computeWidth(reinterpret_cast(serialized_value.data() + row_start), k - row_start, prefix)); row_start = k + 1; } } - widths[i][j] = std::max(row_width, serialized_value.size() - row_start); + widths[i][j] = std::max(row_width, UTF8::computeWidth(reinterpret_cast(serialized_value.data() + row_start), serialized_value.size() - row_start, prefix)); } max_padded_widths[i] = std::max(max_padded_widths[i], std::min(format_settings.pretty.max_column_pad_width, @@ -447,7 +447,7 @@ void PrettyBlockOutputFormat::writeValueWithPadding( { has_break_line = true; serialized_value = serialized_value.substr(0, break_line_pos); - value_width = serialized_value.size() - 1; + value_width = UTF8::computeWidth(reinterpret_cast(serialized_value.data() + 1), serialized_value.size()); } if (cut_to_width && value_width > cut_to_width) @@ -492,7 +492,7 @@ void PrettyBlockOutputFormat::writeValueWithPadding( write_padding(); } - if (has_break_line) + if (has_break_line && value_width != format_settings.pretty.max_value_width) writeString("…", out); } @@ -523,7 +523,7 @@ void PrettyBlockOutputFormat::writeTransferredRow(const Widths & max_widths, con writeCString(" ", out); String value = transferred_row[j]; - cur_width = value.size(); + cur_width = UTF8::computeWidth(reinterpret_cast(value.data()), value.size()); bool has_break_line = false; if (size_t break_line_pos = value.find_first_of('\n'); break_line_pos != String::npos) @@ -531,7 +531,7 @@ void PrettyBlockOutputFormat::writeTransferredRow(const Widths & max_widths, con has_transferred_row = true; new_transferred_row[j] = value.substr(break_line_pos + 1); value = value.substr(0, break_line_pos); - cur_width = value.size(); + cur_width = UTF8::computeWidth(reinterpret_cast(value.data()), value.size()); has_break_line = 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 bf82c39f633..a5282f89327 100644 --- a/tests/queries/0_stateless/03132_pretty_format_break_line.reference +++ b/tests/queries/0_stateless/03132_pretty_format_break_line.reference @@ -42,3 +42,47 @@ 2. │ 1 │ hello world │ hello …│ │ │ │…world │ └────┴─────────────┴─────────────┘ +┌─id─┬─value──┬─value1──────┐ +│ 0 │ привет…│ hello world │ +│ │…world │ │ +└────┴────────┴─────────────┘ + ┌─id─┬─value──┬─value1──────┐ +1. │ 0 │ привет…│ hello world │ + │ │…world │ │ + └────┴────────┴─────────────┘ + ┌─id─┬─value──┬─value1──────┐ +1. │ 0 │ привет…│ hello world │ + │ │…world │ │ + └────┴────────┴─────────────┘ + ┏━━━━┳━━━━━━━━┳━━━━━━━━━━━━━┓ + ┃ id ┃ value ┃ value1 ┃ + ┡━━━━╇━━━━━━━━╇━━━━━━━━━━━━━┩ +1. │ 0 │ привет…│ hello world │ + │ │…world │ │ + └────┴────────┴─────────────┘ + id value value1 + + 0 привет… hello world + …world + id value value1 + +1. 0 привет… hello world + …world +┏━━━━┳━━━━━━━━━━━━━┳━━━━━━━━━━━━━┓ +┃ id ┃ value ┃ value1 ┃ +┡━━━━╇━━━━━━━━━━━━━╇━━━━━━━━━━━━━┩ +│ 0 │ привет …│ hello world │ +│ │…world │ │ +├────┼─────────────┼─────────────┤ +│ 1 │ hello world │ hellow …│ +│ │ │…мир │ +└────┴─────────────┴─────────────┘ + ┏━━━━┳━━━━━━━━━━━━━┳━━━━━━━━━━━━━┓ + ┃ id ┃ value ┃ value1 ┃ + ┡━━━━╇━━━━━━━━━━━━━╇━━━━━━━━━━━━━┩ +1. │ 0 │ привет …│ hello world │ + │ │…world │ │ + ├────┼─────────────┼─────────────┤ +2. │ 1 │ hello world │ hellow …│ + │ │ │…мир │ + └────┴─────────────┴─────────────┘ 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 54ab54e47ff..ecf967c1836 100644 --- a/tests/queries/0_stateless/03132_pretty_format_break_line.sql +++ b/tests/queries/0_stateless/03132_pretty_format_break_line.sql @@ -15,4 +15,19 @@ INSERT INTO t_break_line VALUES(1, 'hello world', 'hello\nworld'); 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; +TRUNCATE TABLE t_break_line; + +INSERT INTO t_break_line VALUES(0, 'привет\nworld', 'hello world'); + +SELECT * FROM t_break_line FORMAT PrettyCompactNoEscapes SETTINGS output_format_pretty_row_numbers = 0; +SELECT * FROM t_break_line FORMAT PrettyCompactNoEscapes; +SELECT * FROM t_break_line FORMAT PrettyCompact; +SELECT * FROM t_break_line FORMAT Pretty; +SELECT * FROM t_break_line FORMAT PrettySpace SETTINGS output_format_pretty_row_numbers = 0; +SELECT * FROM t_break_line FORMAT PrettySpace; + +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; + DROP TABLE t_break_line; \ No newline at end of file From 5e3db5b5038df1ef4262d08bcfc5ea7176320f25 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 1 May 2024 12:23:13 +0100 Subject: [PATCH 225/624] fix tests 5 --- .../1_stateful/00144_functions_of_aggregation_states.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/1_stateful/00144_functions_of_aggregation_states.sql b/tests/queries/1_stateful/00144_functions_of_aggregation_states.sql index ff05ff86799..c5cd45d68b3 100644 --- a/tests/queries/1_stateful/00144_functions_of_aggregation_states.sql +++ b/tests/queries/1_stateful/00144_functions_of_aggregation_states.sql @@ -1 +1,3 @@ +SET allow_deprecated_functions = 1; + SELECT EventDate, finalizeAggregation(state), runningAccumulate(state) FROM (SELECT EventDate, uniqState(UserID) AS state FROM test.hits GROUP BY EventDate ORDER BY EventDate); From 6f484734c2b7c8f1c2aaa365461772d560dd0c60 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 1 May 2024 11:34:59 +0000 Subject: [PATCH 226/624] fix reading in general case --- src/Storages/MergeTree/MergeTreeIOSettings.h | 2 ++ src/Storages/MergeTree/MergeTreeReaderWide.cpp | 10 ++++++---- src/Storages/MergeTree/MergeTreeReaderWide.h | 2 +- src/Storages/MergeTree/MergeTreeSequentialSource.cpp | 1 + 4 files changed, 10 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIOSettings.h b/src/Storages/MergeTree/MergeTreeIOSettings.h index feeb1808a6f..12a83703148 100644 --- a/src/Storages/MergeTree/MergeTreeIOSettings.h +++ b/src/Storages/MergeTree/MergeTreeIOSettings.h @@ -44,6 +44,8 @@ struct MergeTreeReaderSettings bool enable_multiple_prewhere_read_steps = false; /// If true, try to lower size of read buffer according to granule size and compressed block size. bool adjust_read_buffer_size = true; + /// If true, it's allowed to read the whole part without reading marks. + bool can_read_part_without_marks = false; }; struct MergeTreeWriterSettings diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index d398668d5c8..59feb4dda19 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -43,7 +43,9 @@ MergeTreeReaderWide::MergeTreeReaderWide( mark_ranges_, settings_, avg_value_size_hints_) - , read_whole_part(all_mark_ranges.isOneRangeForWholePart(data_part_info_for_read->getMarksCount())) + , read_without_marks( + settings.can_read_part_without_marks + && all_mark_ranges.isOneRangeForWholePart(data_part_info_for_read->getMarksCount())) { try { @@ -254,7 +256,7 @@ void MergeTreeReaderWide::addStreams( std::move(marks_loader), profile_callback, clock_type); }; - if (read_whole_part) + if (read_without_marks) { streams.emplace(*stream_name, create_stream.operator()()); } @@ -340,7 +342,7 @@ void MergeTreeReaderWide::prefetchForColumn( if (stream_name && !prefetched_streams.contains(*stream_name)) { - bool seek_to_mark = !continue_reading && !read_whole_part; + bool seek_to_mark = !continue_reading && !read_without_marks; if (ReadBuffer * buf = getStream(false, substream_path, data_part_info_for_read->getChecksums(), streams, name_and_type, from_mark, seek_to_mark, current_task_last_mark, cache)) { @@ -365,7 +367,7 @@ void MergeTreeReaderWide::readData( deserialize_settings.getter = [&](const ISerialization::SubstreamPath & substream_path) { - bool seek_to_mark = !was_prefetched && !continue_reading && !read_whole_part; + bool seek_to_mark = !was_prefetched && !continue_reading && !read_without_marks; return getStream( /* seek_to_start = */false, substream_path, diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.h b/src/Storages/MergeTree/MergeTreeReaderWide.h index 7ffe565d262..9f6bdd79b00 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.h +++ b/src/Storages/MergeTree/MergeTreeReaderWide.h @@ -73,7 +73,7 @@ private: std::unordered_map caches; std::unordered_set prefetched_streams; ssize_t prefetched_from_mark = -1; - bool read_whole_part = false; + bool read_without_marks = false; }; } diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 47661a3ff93..0939378932b 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -174,6 +174,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( .read_settings = read_settings, .save_marks_in_cache = false, .apply_deleted_mask = apply_deleted_mask, + .can_read_part_without_marks = true, }; if (!mark_ranges) From fa7c8d59a2c37f154e3baf8c11f48c01473e765a Mon Sep 17 00:00:00 2001 From: Volodya Date: Wed, 1 May 2024 11:47:59 +0000 Subject: [PATCH 227/624] without transfer at max column width --- .../Formats/Impl/PrettyBlockOutputFormat.cpp | 24 +++++++++++++++++-- 1 file changed, 22 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index 7338ae18310..9f453be8827 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -466,6 +466,7 @@ void PrettyBlockOutputFormat::writeValueWithPadding( serialized_value += ellipsis; value_width = format_settings.pretty.max_value_width; + has_break_line = false; } else if (!has_break_line) serialized_value += ' '; @@ -492,7 +493,7 @@ void PrettyBlockOutputFormat::writeValueWithPadding( write_padding(); } - if (has_break_line && value_width != format_settings.pretty.max_value_width) + if (has_break_line) writeString("…", out); } @@ -535,6 +536,25 @@ void PrettyBlockOutputFormat::writeTransferredRow(const Widths & max_widths, con has_break_line = true; } + if (cur_width > format_settings.pretty.max_value_width) + { + value.resize(UTF8::computeBytesBeforeWidth( + reinterpret_cast(value.data()), value.size(), 0, 1 + format_settings.pretty.max_value_width)); + + const char * ellipsis = format_settings.pretty.charset == FormatSettings::Pretty::Charset::UTF8 ? "⋯" : "~"; + if (color) + { + value += "\033[31;1m"; + value += ellipsis; + value += "\033[0m"; + } + else + value += ellipsis; + + cur_width = format_settings.pretty.max_value_width; + has_break_line = false; + } + if (!value.empty()) writeString("…", out); else @@ -552,7 +572,7 @@ void PrettyBlockOutputFormat::writeTransferredRow(const Widths & max_widths, con if (has_break_line) writeString("…", out); - else + else if (cur_width != format_settings.pretty.max_value_width) writeChar(' ', out); } From f0fc9f23c405a23d1de59182b39e2c60c7815609 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 1 May 2024 15:29:45 +0000 Subject: [PATCH 228/624] init --- src/Parsers/ASTDropQuery.cpp | 6 +++++- src/Parsers/ParserDropQuery.cpp | 13 +++++++++---- src/Parsers/ParserQuery.cpp | 2 ++ .../0_stateless/00623_truncate_all_tables.sql | 2 +- 4 files changed, 17 insertions(+), 6 deletions(-) diff --git a/src/Parsers/ASTDropQuery.cpp b/src/Parsers/ASTDropQuery.cpp index ecb9ad8169b..6a91e866452 100644 --- a/src/Parsers/ASTDropQuery.cpp +++ b/src/Parsers/ASTDropQuery.cpp @@ -50,8 +50,12 @@ void ASTDropQuery::formatQueryImpl(const FormatSettings & settings, FormatState settings.ostr << "TEMPORARY "; if (has_all_tables) + { settings.ostr << "ALL TABLES "; - else if (!table && !database_and_tables && database) + settings.ostr << "FROM "; /// When using `TRUNCATE ALL TABLES` we need to use the `FROM DATABASE` keywords along with database name + } + + if (!table && !database_and_tables && database) settings.ostr << "DATABASE "; else if (is_dictionary) settings.ostr << "DICTIONARY "; diff --git a/src/Parsers/ParserDropQuery.cpp b/src/Parsers/ParserDropQuery.cpp index 09f15e9649f..79939a981db 100644 --- a/src/Parsers/ParserDropQuery.cpp +++ b/src/Parsers/ParserDropQuery.cpp @@ -17,6 +17,7 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, cons ParserKeyword s_dictionary(Keyword::DICTIONARY); ParserKeyword s_view(Keyword::VIEW); ParserKeyword s_database(Keyword::DATABASE); + ParserKeyword s_from(Keyword::FROM); ParserKeyword s_all(Keyword::ALL); ParserKeyword s_tables(Keyword::TABLES); ParserToken s_dot(TokenType::Dot); @@ -54,11 +55,15 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, cons else if (s_all.ignore(pos, expected) && s_tables.ignore(pos, expected) && kind == ASTDropQuery::Kind::Truncate) { has_all_tables = true; + if (s_from.ignore(pos, expected) && s_database.ignore(pos, expected)) + { + if (s_if_exists.ignore(pos, expected)) + if_exists = true; - if (s_if_exists.ignore(pos, expected)) - if_exists = true; - - if (!name_p.parse(pos, database, expected)) + if (!name_p.parse(pos, database, expected)) + return false; + } + else return false; } else diff --git a/src/Parsers/ParserQuery.cpp b/src/Parsers/ParserQuery.cpp index 22ddc25019f..06c4200a51b 100644 --- a/src/Parsers/ParserQuery.cpp +++ b/src/Parsers/ParserQuery.cpp @@ -40,6 +40,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserInsertQuery insert_p(end, allow_settings_after_format_in_insert); ParserUseQuery use_p; ParserSetQuery set_p; + ParserDropQuery drop_p; ParserSystemQuery system_p; ParserCreateUserQuery create_user_p; ParserCreateRoleQuery create_role_p; @@ -66,6 +67,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) || use_p.parse(pos, node, expected) || set_role_p.parse(pos, node, expected) || set_p.parse(pos, node, expected) + || drop_p.parse(pos, node, expected) || system_p.parse(pos, node, expected) || create_user_p.parse(pos, node, expected) || create_role_p.parse(pos, node, expected) diff --git a/tests/queries/0_stateless/00623_truncate_all_tables.sql b/tests/queries/0_stateless/00623_truncate_all_tables.sql index 3a6f94d2907..babec49bb08 100644 --- a/tests/queries/0_stateless/00623_truncate_all_tables.sql +++ b/tests/queries/0_stateless/00623_truncate_all_tables.sql @@ -25,7 +25,7 @@ SELECT * FROM truncate_test.truncate_test_stripe_log; SELECT * FROM truncate_test.truncate_test_merge_tree; SELECT '======After Truncate And Empty======'; -TRUNCATE ALL TABLES IF EXISTS truncate_test; +TRUNCATE ALL TABLES FROM DATABASE IF EXISTS truncate_test; SELECT * FROM system.numbers WHERE number NOT IN truncate_test.truncate_test_set LIMIT 1; SELECT * FROM truncate_test.truncate_test_log; SELECT * FROM truncate_test.truncate_test_memory; From a40e8a15c5a4082c87236a584055f7f563ffd083 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 1 May 2024 17:37:41 +0200 Subject: [PATCH 229/624] remove 'DATABASE' keyword --- src/Parsers/ASTDropQuery.cpp | 5 ++--- src/Parsers/ParserDropQuery.cpp | 2 +- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Parsers/ASTDropQuery.cpp b/src/Parsers/ASTDropQuery.cpp index 6a91e866452..7a90eff73ae 100644 --- a/src/Parsers/ASTDropQuery.cpp +++ b/src/Parsers/ASTDropQuery.cpp @@ -52,10 +52,9 @@ void ASTDropQuery::formatQueryImpl(const FormatSettings & settings, FormatState if (has_all_tables) { settings.ostr << "ALL TABLES "; - settings.ostr << "FROM "; /// When using `TRUNCATE ALL TABLES` we need to use the `FROM DATABASE` keywords along with database name + settings.ostr << "FROM "; /// When using `TRUNCATE ALL TABLES` we need to use the `FROM` keyword along with database name } - - if (!table && !database_and_tables && database) + else if (!table && !database_and_tables && database) settings.ostr << "DATABASE "; else if (is_dictionary) settings.ostr << "DICTIONARY "; diff --git a/src/Parsers/ParserDropQuery.cpp b/src/Parsers/ParserDropQuery.cpp index 79939a981db..0428c8509b7 100644 --- a/src/Parsers/ParserDropQuery.cpp +++ b/src/Parsers/ParserDropQuery.cpp @@ -55,7 +55,7 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, cons else if (s_all.ignore(pos, expected) && s_tables.ignore(pos, expected) && kind == ASTDropQuery::Kind::Truncate) { has_all_tables = true; - if (s_from.ignore(pos, expected) && s_database.ignore(pos, expected)) + if (s_from.ignore(pos, expected)) { if (s_if_exists.ignore(pos, expected)) if_exists = true; From fd2716c5c426aac276f6ac2c0a1a934247940268 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 1 May 2024 17:42:42 +0200 Subject: [PATCH 230/624] fix docs --- docs/en/sql-reference/statements/truncate.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/truncate.md b/docs/en/sql-reference/statements/truncate.md index 8cd5a6a1424..fb5d4a3055a 100644 --- a/docs/en/sql-reference/statements/truncate.md +++ b/docs/en/sql-reference/statements/truncate.md @@ -25,7 +25,7 @@ If the `alter_sync` is set to `2` and some replicas are not active for more than ## TRUNCATE ALL TABLES ``` sql -TRUNCATE ALL TABLES [IF EXISTS] db [ON CLUSTER cluster] +TRUNCATE ALL TABLES FROM [IF EXISTS] db [ON CLUSTER cluster] ``` Removes all data from all tables in a database. From e7b96d9ec0a4616704d480f0956b4d999f5ba1c2 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 1 May 2024 18:01:38 +0200 Subject: [PATCH 231/624] Update 00623_truncate_all_tables.sql --- tests/queries/0_stateless/00623_truncate_all_tables.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00623_truncate_all_tables.sql b/tests/queries/0_stateless/00623_truncate_all_tables.sql index babec49bb08..2d5e9d48f59 100644 --- a/tests/queries/0_stateless/00623_truncate_all_tables.sql +++ b/tests/queries/0_stateless/00623_truncate_all_tables.sql @@ -25,7 +25,7 @@ SELECT * FROM truncate_test.truncate_test_stripe_log; SELECT * FROM truncate_test.truncate_test_merge_tree; SELECT '======After Truncate And Empty======'; -TRUNCATE ALL TABLES FROM DATABASE IF EXISTS truncate_test; +TRUNCATE ALL TABLES FROM IF EXISTS truncate_test; SELECT * FROM system.numbers WHERE number NOT IN truncate_test.truncate_test_set LIMIT 1; SELECT * FROM truncate_test.truncate_test_log; SELECT * FROM truncate_test.truncate_test_memory; From 637eb754a90d27b92145ee3c3c5fa83660be735c Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 1 May 2024 18:16:37 +0200 Subject: [PATCH 232/624] fix due to review --- src/Parsers/ASTDropQuery.cpp | 5 +---- src/Parsers/ParserDropQuery.cpp | 12 +++++------- src/Parsers/ParserQuery.cpp | 1 - 3 files changed, 6 insertions(+), 12 deletions(-) diff --git a/src/Parsers/ASTDropQuery.cpp b/src/Parsers/ASTDropQuery.cpp index 7a90eff73ae..f94c2d05a15 100644 --- a/src/Parsers/ASTDropQuery.cpp +++ b/src/Parsers/ASTDropQuery.cpp @@ -50,10 +50,7 @@ void ASTDropQuery::formatQueryImpl(const FormatSettings & settings, FormatState settings.ostr << "TEMPORARY "; if (has_all_tables) - { - settings.ostr << "ALL TABLES "; - settings.ostr << "FROM "; /// When using `TRUNCATE ALL TABLES` we need to use the `FROM` keyword along with database name - } + settings.ostr << "ALL TABLES FROM "; else if (!table && !database_and_tables && database) settings.ostr << "DATABASE "; else if (is_dictionary) diff --git a/src/Parsers/ParserDropQuery.cpp b/src/Parsers/ParserDropQuery.cpp index 0428c8509b7..8d6656d972e 100644 --- a/src/Parsers/ParserDropQuery.cpp +++ b/src/Parsers/ParserDropQuery.cpp @@ -56,14 +56,12 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, cons { has_all_tables = true; if (s_from.ignore(pos, expected)) - { - if (s_if_exists.ignore(pos, expected)) - if_exists = true; + return false; - if (!name_p.parse(pos, database, expected)) - return false; - } - else + if (s_if_exists.ignore(pos, expected)) + if_exists = true; + + if (!name_p.parse(pos, database, expected)) return false; } else diff --git a/src/Parsers/ParserQuery.cpp b/src/Parsers/ParserQuery.cpp index 06c4200a51b..746fa202735 100644 --- a/src/Parsers/ParserQuery.cpp +++ b/src/Parsers/ParserQuery.cpp @@ -40,7 +40,6 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserInsertQuery insert_p(end, allow_settings_after_format_in_insert); ParserUseQuery use_p; ParserSetQuery set_p; - ParserDropQuery drop_p; ParserSystemQuery system_p; ParserCreateUserQuery create_user_p; ParserCreateRoleQuery create_role_p; From 2281945d72c9caa46947ec2f1b7e59d3d0b25392 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 1 May 2024 18:31:17 +0200 Subject: [PATCH 233/624] remove whitespaces --- src/Parsers/ASTDropQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ASTDropQuery.cpp b/src/Parsers/ASTDropQuery.cpp index f94c2d05a15..cf912d66b44 100644 --- a/src/Parsers/ASTDropQuery.cpp +++ b/src/Parsers/ASTDropQuery.cpp @@ -50,7 +50,7 @@ void ASTDropQuery::formatQueryImpl(const FormatSettings & settings, FormatState settings.ostr << "TEMPORARY "; if (has_all_tables) - settings.ostr << "ALL TABLES FROM "; + settings.ostr << "ALL TABLES FROM "; else if (!table && !database_and_tables && database) settings.ostr << "DATABASE "; else if (is_dictionary) From 1c361d0daa236e648afcecd449d83068215cc632 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 1 May 2024 18:36:42 +0200 Subject: [PATCH 234/624] Update ParserQuery.cpp --- src/Parsers/ParserQuery.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Parsers/ParserQuery.cpp b/src/Parsers/ParserQuery.cpp index 746fa202735..22ddc25019f 100644 --- a/src/Parsers/ParserQuery.cpp +++ b/src/Parsers/ParserQuery.cpp @@ -66,7 +66,6 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) || use_p.parse(pos, node, expected) || set_role_p.parse(pos, node, expected) || set_p.parse(pos, node, expected) - || drop_p.parse(pos, node, expected) || system_p.parse(pos, node, expected) || create_user_p.parse(pos, node, expected) || create_role_p.parse(pos, node, expected) From 642e225fdbee5f7d3b7b14f368c72d8feb20a8d9 Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 1 May 2024 12:36:48 -0400 Subject: [PATCH 235/624] Fix review --- src/Databases/DatabaseOnDisk.cpp | 6 ------ src/Storages/StorageMaterializedView.cpp | 10 +++++++++- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 93bc119009f..674e9afa8ac 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -68,11 +67,6 @@ std::pair createTableFromAST( ast_create_query.attach = true; ast_create_query.setDatabase(database_name); - if (!ast_create_query.sql_security && ast_create_query.supportSQLSecurity() && !context->getServerSettings().ignore_empty_sql_security_in_create_view_query) - { - ast_create_query.sql_security = std::make_shared(); - InterpreterCreateQuery::processSQLSecurityOption(context, ast_create_query.sql_security->as(), true, ast_create_query.is_materialized_view); - } if (ast_create_query.select && ast_create_query.isView()) ApplyWithSubqueryVisitor::visit(*ast_create_query.select); diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 696865dfa2f..8025cae512f 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -97,7 +97,15 @@ StorageMaterializedView::StorageMaterializedView( storage_metadata.columns, local_context->getGlobalContext()); - if (query.sql_security) + ASTPtr sql_security = query.sql_security; + if (!sql_security && query.supportSQLSecurity() && !getContext()->getServerSettings().ignore_empty_sql_security_in_create_view_query) + { + /// This is hack which allows to load materialized views during startup with default SQL security NONE for backward compatability. + sql_security = std::make_shared(); + InterpreterCreateQuery::processSQLSecurityOption(getContext(), sql_security->as(), true, query.is_materialized_view); + } + + if (sql_security) storage_metadata.setSQLSecurity(query.sql_security->as()); if (storage_metadata.sql_security_type == SQLSecurityType::INVOKER) From e52508b9128fc25ed128bafdc74922728c103a8d Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 1 May 2024 19:03:28 +0200 Subject: [PATCH 236/624] Update ParserDropQuery.cpp --- src/Parsers/ParserDropQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ParserDropQuery.cpp b/src/Parsers/ParserDropQuery.cpp index 8d6656d972e..bb7a98b9750 100644 --- a/src/Parsers/ParserDropQuery.cpp +++ b/src/Parsers/ParserDropQuery.cpp @@ -55,7 +55,7 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, cons else if (s_all.ignore(pos, expected) && s_tables.ignore(pos, expected) && kind == ASTDropQuery::Kind::Truncate) { has_all_tables = true; - if (s_from.ignore(pos, expected)) + if (!s_from.ignore(pos, expected)) return false; if (s_if_exists.ignore(pos, expected)) From 8f95ca23f6636a9bb3a0c9517741eab422adebe3 Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 1 May 2024 14:00:12 -0400 Subject: [PATCH 237/624] Fix typo --- src/Storages/StorageMaterializedView.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 8025cae512f..f1564d93c01 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -100,7 +100,7 @@ StorageMaterializedView::StorageMaterializedView( ASTPtr sql_security = query.sql_security; if (!sql_security && query.supportSQLSecurity() && !getContext()->getServerSettings().ignore_empty_sql_security_in_create_view_query) { - /// This is hack which allows to load materialized views during startup with default SQL security NONE for backward compatability. + /// This is hack which allows to load materialized views during startup with default SQL security NONE for backward compatibility. sql_security = std::make_shared(); InterpreterCreateQuery::processSQLSecurityOption(getContext(), sql_security->as(), true, query.is_materialized_view); } From 92dee331f157f05338b1a5d543a8af574e19eb25 Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 1 May 2024 15:54:41 -0400 Subject: [PATCH 238/624] fix --- src/Storages/StorageMaterializedView.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index f1564d93c01..3ee461a2185 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -98,15 +98,15 @@ StorageMaterializedView::StorageMaterializedView( local_context->getGlobalContext()); ASTPtr sql_security = query.sql_security; - if (!sql_security && query.supportSQLSecurity() && !getContext()->getServerSettings().ignore_empty_sql_security_in_create_view_query) + if (!sql_security && !getContext()->getServerSettings().ignore_empty_sql_security_in_create_view_query) { /// This is hack which allows to load materialized views during startup with default SQL security NONE for backward compatibility. sql_security = std::make_shared(); - InterpreterCreateQuery::processSQLSecurityOption(getContext(), sql_security->as(), true, query.is_materialized_view); + InterpreterCreateQuery::processSQLSecurityOption(getContext(), sql_security->as(), true, true); } if (sql_security) - storage_metadata.setSQLSecurity(query.sql_security->as()); + storage_metadata.setSQLSecurity(sql_security->as()); if (storage_metadata.sql_security_type == SQLSecurityType::INVOKER) throw Exception(ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW, "SQL SECURITY INVOKER can't be specified for MATERIALIZED VIEW"); From 08d6fa03b69fb3b8bcd53b6847c2723a1275cb81 Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 1 May 2024 16:00:55 -0400 Subject: [PATCH 239/624] fix comment --- src/Storages/StorageMaterializedView.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 3ee461a2185..24c3778670a 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -100,7 +100,8 @@ StorageMaterializedView::StorageMaterializedView( ASTPtr sql_security = query.sql_security; if (!sql_security && !getContext()->getServerSettings().ignore_empty_sql_security_in_create_view_query) { - /// This is hack which allows to load materialized views during startup with default SQL security NONE for backward compatibility. + /// This allows materialized views to be loaded during startup with default SQL security for backward compatibility. + /// If ClickHouse loads an old materialized view created without SQL security, it will use the default `SQL SECURITY NONE` sql_security = std::make_shared(); InterpreterCreateQuery::processSQLSecurityOption(getContext(), sql_security->as(), true, true); } From 33ec14758de2c452ec7ef38bbf4152d55843ece5 Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 1 May 2024 22:12:43 +0200 Subject: [PATCH 240/624] Document subtractXYZ --- .../functions/date-time-functions.md | 468 +++++++++++++++++- 1 file changed, 463 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 51f841657d9..1994e933b3b 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -2580,13 +2580,27 @@ SELECT └─────────────────────┴──────────────────────────┴─────────────────────────────────┘ ``` -## subtractYears, subtractQuarters, subtractMonths, subtractWeeks, subtractDays, subtractHours, subtractMinutes, subtractSeconds, subtractMilliseconds, subtractMicroseconds, subtractNanoseconds +## subtractYears -These functions subtract units of the interval specified by the function name from a date, a date with time or a string-encoded date / date with time. A date or date with time is returned. +Subtracts a specified number of years from a date, a date with time or a string-encoded date / date with time. -Example: +**Syntax** -``` sql +```sql +subtractYears(date, x) +``` + +**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). + +**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). + +**Example** + +```sql WITH toDate('2024-01-01') AS date, toDateTime('2024-01-01 00:00:00') AS date_time, @@ -2597,12 +2611,456 @@ SELECT subtractYears(date_time_string, 1) AS subtract_years_with_date_time_string ``` -``` text +```response ┌─subtract_years_with_date─┬─subtract_years_with_date_time─┬─subtract_years_with_date_time_string─┐ │ 2023-01-01 │ 2023-01-01 00:00:00 │ 2023-01-01 00:00:00.000 │ └──────────────────────────┴───────────────────────────────┴──────────────────────────────────────┘ ``` +## subtractQuarters + +Subtracts a specified number of quarters from a date, a date with time or a string-encoded date / date with time. + +**Syntax** + +```sql +subtractQuarters(date, x) +``` + +**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). + +**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). + +**Example** + +```sql +WITH + toDate('2024-01-01') AS date, + toDateTime('2024-01-01 00:00:00') AS date_time, + '2024-01-01 00:00:00' AS date_time_string +SELECT + subtractQuarters(date, 1) AS subtract_quarters_with_date, + subtractQuarters(date_time, 1) AS subtract_quarters_with_date_time, + subtractQuarters(date_time_string, 1) AS subtract_quarters_with_date_time_string +``` + +```response +┌─subtract_quarters_with_date─┬─subtract_quarters_with_date_time─┬─subtract_quarters_with_date_time_string─┐ +│ 2023-10-01 │ 2023-10-01 00:00:00 │ 2023-10-01 00:00:00.000 │ +└─────────────────────────────┴──────────────────────────────────┴─────────────────────────────────────────┘ +``` + +## subtractMonths + +Subtracts a specified number of months from a date, a date with time or a string-encoded date / date with time. + +**Syntax** + +```sql +subtractMonths(date, x) +``` + +**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). + +**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). + +**Example** + +```sql +WITH + toDate('2024-01-01') AS date, + toDateTime('2024-01-01 00:00:00') AS date_time, + '2024-01-01 00:00:00' AS date_time_string +SELECT + subtractMonths(date, 1) AS subtract_months_with_date, + subtractMonths(date_time, 1) AS subtract_months_with_date_time, + subtractMonths(date_time_string, 1) AS subtract_months_with_date_time_string +``` + +```response +┌─subtract_months_with_date─┬─subtract_months_with_date_time─┬─subtract_months_with_date_time_string─┐ +│ 2023-12-01 │ 2023-12-01 00:00:00 │ 2023-12-01 00:00:00.000 │ +└───────────────────────────┴────────────────────────────────┴───────────────────────────────────────┘ +``` + +## subtractWeeks + +Subtracts a specified number of weeks from a date, a date with time or a string-encoded date / date with time. + +**Syntax** + +```sql +subtractWeeks(date, x) +``` + +**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). + +**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). + +**Example** + +```sql +WITH + toDate('2024-01-01') AS date, + toDateTime('2024-01-01 00:00:00') AS date_time, + '2024-01-01 00:00:00' AS date_time_string +SELECT + subtractWeeks(date, 1) AS subtract_weeks_with_date, + subtractWeeks(date_time, 1) AS subtract_weeks_with_date_time, + subtractWeeks(date_time_string, 1) AS subtract_weeks_with_date_time_string +``` + +```response + ┌─subtract_weeks_with_date─┬─subtract_weeks_with_date_time─┬─subtract_weeks_with_date_time_string─┐ + │ 2023-12-25 │ 2023-12-25 00:00:00 │ 2023-12-25 00:00:00.000 │ + └──────────────────────────┴───────────────────────────────┴──────────────────────────────────────┘ +``` + +## subtractDays + +Subtracts a specified number of days from a date, a date with time or a string-encoded date / date with time. + +**Syntax** + +```sql +subtractDays(date, x) +``` + +**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). + +**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). + +**Example** + +```sql +WITH + toDate('2024-01-01') AS date, + toDateTime('2024-01-01 00:00:00') AS date_time, + '2024-01-01 00:00:00' AS date_time_string +SELECT + subtractDays(date, 31) AS subtract_days_with_date, + subtractDays(date_time, 31) AS subtract_days_with_date_time, + subtractDays(date_time_string, 31) AS subtract_days_with_date_time_string +``` + +```response +┌─subtract_days_with_date─┬─subtract_days_with_date_time─┬─subtract_days_with_date_time_string─┐ +│ 2023-12-01 │ 2023-12-01 00:00:00 │ 2023-12-01 00:00:00.000 │ +└─────────────────────────┴──────────────────────────────┴─────────────────────────────────────┘ +``` + +## subtractHours + +Subtracts a specified number of hours from a date, a date with time or a string-encoded date / date with time. + +**Syntax** + +```sql +subtractHours(date, x) +``` + +**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). + +**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). + +**Example** + +```sql +WITH + toDate('2024-01-01') AS date, + toDateTime('2024-01-01 00:00:00') AS date_time, + '2024-01-01 00:00:00' AS date_time_string +SELECT + subtractHours(date, 12) AS subtract_hours_with_date, + subtractHours(date_time, 12) AS subtract_hours_with_date_time, + subtractHours(date_time_string, 12) AS subtract_hours_with_date_time_string +``` + +```response +┌─subtract_hours_with_date─┬─subtract_hours_with_date_time─┬─subtract_hours_with_date_time_string─┐ +│ 2023-12-31 12:00:00 │ 2023-12-31 12:00:00 │ 2023-12-31 12:00:00.000 │ +└──────────────────────────┴───────────────────────────────┴──────────────────────────────────────┘ +``` + +## subtractMinutes + +Subtracts a specified number of minutes from a date, a date with time or a string-encoded date / date with time. + +**Syntax** + +```sql +subtractMinutes(date, x) +``` + +**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). + +**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). + +**Example** + +```sql +WITH + toDate('2024-01-01') AS date, + toDateTime('2024-01-01 00:00:00') AS date_time, + '2024-01-01 00:00:00' AS date_time_string +SELECT + subtractMinutes(date, 30) AS subtract_minutes_with_date, + subtractMinutes(date_time, 30) AS subtract_minutes_with_date_time, + subtractMinutes(date_time_string, 30) AS subtract_minutes_with_date_time_string +``` + +```response +┌─subtract_minutes_with_date─┬─subtract_minutes_with_date_time─┬─subtract_minutes_with_date_time_string─┐ +│ 2023-12-31 23:30:00 │ 2023-12-31 23:30:00 │ 2023-12-31 23:30:00.000 │ +└────────────────────────────┴─────────────────────────────────┴────────────────────────────────────────┘ +``` + +## subtractSeconds + +Subtracts a specified number of seconds from a date, a date with time or a string-encoded date / date with time. + +**Syntax** + +```sql +subtractSeconds(date, x) +``` + +**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). + +**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). + +**Example** + +```sql +WITH + toDate('2024-01-01') AS date, + toDateTime('2024-01-01 00:00:00') AS date_time, + '2024-01-01 00:00:00' AS date_time_string +SELECT + subtractSeconds(date, 60) AS subtract_seconds_with_date, + subtractSeconds(date_time, 60) AS subtract_seconds_with_date_time, + subtractSeconds(date_time_string, 60) AS subtract_seconds_with_date_time_string +``` + +```response +┌─subtract_seconds_with_date─┬─subtract_seconds_with_date_time─┬─subtract_seconds_with_date_time_string─┐ +│ 2023-12-31 23:59:00 │ 2023-12-31 23:59:00 │ 2023-12-31 23:59:00.000 │ +└────────────────────────────┴─────────────────────────────────┴────────────────────────────────────────┘ +``` + +## subtractMilliseconds + +Subtracts a specified number of milliseconds from a date with time or a string-encoded date with time. + +**Syntax** + +```sql +subtractMilliseconds(date_time, x) +``` + +**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). + +**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). + +**Example** + +```sql +WITH + toDateTime('2024-01-01 00:00:00') AS date_time, + '2024-01-01 00:00:00' AS date_time_string +SELECT + subtractMilliseconds(date_time, 1000) AS subtract_milliseconds_with_date_time, + subtractMilliseconds(date_time_string, 1000) AS subtract_milliseconds_with_date_time_string +``` + +```response +┌─subtract_milliseconds_with_date_time─┬─subtract_milliseconds_with_date_time_string─┐ +│ 2023-12-31 23:59:59.000 │ 2023-12-31 23:59:59.000 │ +└──────────────────────────────────────┴─────────────────────────────────────────────┘ +``` + +## subtractMicroseconds + +Subtracts a specified number of microseconds from a date with time or a string-encoded date with time. + +**Syntax** + +```sql +subtractMicroseconds(date_time, x) +``` + +**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). + +**Returned value** +- Returns `date_time` minus `x` microseconds. [datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md). + +**Example** + +```sql +WITH + toDateTime('2024-01-01 00:00:00') AS date_time, + '2024-01-01 00:00:00' AS date_time_string +SELECT + subtractMicroseconds(date_time, 1000000) AS subtract_microseconds_with_date_time, + subtractMicroseconds(date_time_string, 1000000) AS subtract_microseconds_with_date_time_string +``` + +```response +┌─subtract_microseconds_with_date_time─┬─subtract_microseconds_with_date_time_string─┐ +│ 2023-12-31 23:59:59.000000 │ 2023-12-31 23:59:59.000000 │ +└──────────────────────────────────────┴─────────────────────────────────────────────┘ +``` + +## subtractNanoseconds + +Subtracts a specified number of nanoseconds from a date with time or a string-encoded date with time. + +**Syntax** + +```sql +subtractNanoseconds(date_time, x) +``` + +**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). + +**Returned value** +- Returns `date_time` minus `x` nanoseconds. [datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md). + +**Example** + +```sql +WITH + toDateTime('2024-01-01 00:00:00') AS date_time, + '2024-01-01 00:00:00' AS date_time_string +SELECT + subtractNanoseconds(date_time, 1000) AS subtract_nanoseconds_with_date_time, + subtractNanoseconds(date_time_string, 1000) AS subtract_nanoseconds_with_date_time_string +``` + +```response +┌─subtract_nanoseconds_with_date_time─┬─subtract_nanoseconds_with_date_time_string─┐ +│ 2023-12-31 23:59:59.999999000 │ 2023-12-31 23:59:59.999999000 │ +└─────────────────────────────────────┴────────────────────────────────────────────┘ +``` + +## subtractInterval + +Adds a negated interval to another interval or tuple of intervals. + +**Syntax** + +```sql +subtractInterval(interval_1, interval_2) +``` + +**Parameters** + +- `interval_1`: First interval or interval of tuples. [interval](../data-types/special-data-types/interval.md), [tuple](../data-types/tuple.md)([interval](../data-types/special-data-types/interval.md)). +- `interval_2`: Second interval to be negated. [interval](../data-types/special-data-types/interval.md). + +**Returned value** +- Returns a tuple of intervals. [tuple](../data-types/tuple.md)([interval](../data-types/special-data-types/interval.md)). + +:::note +If the types of the first interval (or the interval in the tuple) and the second interval are the same they will be merged into one interval. +::: + +**Example** + +Query: + +```sql +SELECT subtractInterval(INTERVAL 1 DAY, INTERVAL 1 MONTH); +SELECT subtractInterval((INTERVAL 1 DAY, INTERVAL 1 YEAR), INTERVAL 1 MONTH); +SELECT subtractInterval(INTERVAL 2 DAY, INTERVAL 1 DAY); +``` + +Result: + +```response +┌─subtractInterval(toIntervalDay(1), toIntervalMonth(1))─┐ +│ (1,-1) │ +└────────────────────────────────────────────────────────┘ +┌─subtractInterval((toIntervalDay(1), toIntervalYear(1)), toIntervalMonth(1))─┐ +│ (1,1,-1) │ +└─────────────────────────────────────────────────────────────────────────────┘ +┌─subtractInterval(toIntervalDay(2), toIntervalDay(1))─┐ +│ (1) │ +└──────────────────────────────────────────────────────┘ +``` + +## subtractTupleOfIntervals + +Consecutively subtracts a tuple of intervals from a Date or a DateTime. + +**Syntax** + +```sql +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). +- `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). + +**Example** + +Query: + +```sql +WITH toDate('2018-01-01') AS date SELECT subtractTupleOfIntervals(date, (INTERVAL 1 DAY, INTERVAL 1 YEAR)) +``` + +Result: + +```response +┌─subtractTupleOfIntervals(date, (toIntervalDay(1), toIntervalYear(1)))─┐ +│ 2016-12-31 │ +└───────────────────────────────────────────────────────────────────────┘ +``` + ## timeSlots(StartTime, Duration,\[, Size\]) For a time interval starting at ‘StartTime’ and continuing for ‘Duration’ seconds, it returns an array of moments in time, consisting of points from this interval rounded down to the ‘Size’ in seconds. ‘Size’ is an optional parameter set to 1800 (30 minutes) by default. From 7aab5855b68247d57aef5e1d119ebe0d7d7971e8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Thu, 2 May 2024 07:00:40 +0000 Subject: [PATCH 241/624] Fix links --- .../settings.md | 2 +- .../operations/settings/settings-formats.md | 2 +- docs/en/operations/system-tables/grants.md | 2 +- .../functions/other-functions.md | 6 +-- .../en/sql-reference/statements/alter/user.md | 4 +- .../sql-reference/statements/create/user.md | 2 +- docs/en/sql-reference/statements/grant.md | 48 +++++++++---------- docs/en/sql-reference/statements/show.md | 4 +- 8 files changed, 35 insertions(+), 35 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index c839ea3ae5a..28831404a1f 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -2860,7 +2860,7 @@ 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. +[`displaySecretsInShowAndSelect`](../../sql-reference/statements/grant#display-secrets) privilege. Possible values: diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index f455fcba840..6666f68c177 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -15,7 +15,7 @@ table functions, and dictionaries. User wishing to see secrets must also have [`display_secrets_in_show_and_select` server setting](../server-configuration-parameters/settings#display_secrets_in_show_and_select) turned on and a -[`displaySecretsInShowAndSelect`](../../sql-reference/statements/grant#grant-display-secrets) privilege. +[`displaySecretsInShowAndSelect`](../../sql-reference/statements/grant#display-secrets) privilege. Possible values: diff --git a/docs/en/operations/system-tables/grants.md b/docs/en/operations/system-tables/grants.md index b12f656cb75..262a53a87a5 100644 --- a/docs/en/operations/system-tables/grants.md +++ b/docs/en/operations/system-tables/grants.md @@ -22,4 +22,4 @@ Columns: - `0` — The row describes a partial revoke. - `1` — The row describes a grant. -- `grant_option` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Permission is granted `WITH GRANT OPTION`, see [GRANT](../../sql-reference/statements/grant.md#grant-privigele-syntax). +- `grant_option` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Permission is granted `WITH GRANT OPTION`, see [GRANT](../../sql-reference/statements/grant.md#granting-privilege-syntax). diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 694a69b64cc..ef20cf808f1 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -556,7 +556,7 @@ hasColumnInTable(\[‘hostname’\[, ‘username’\[, ‘password’\]\],\] ‘ **Parameters** - `database` : name of the database. [String literal](../syntax#syntax-string-literal) -- `table` : name of the table. [String literal](../syntax#syntax-string-literal) +- `table` : name of the table. [String literal](../syntax#syntax-string-literal) - `column` : name of the column. [String literal](../syntax#syntax-string-literal) - `hostname` : remote server name to perform the check on. [String literal](../syntax#syntax-string-literal) - `username` : username for remote server. [String literal](../syntax#syntax-string-literal) @@ -565,7 +565,7 @@ hasColumnInTable(\[‘hostname’\[, ‘username’\[, ‘password’\]\],\] ‘ **Returned value** - `1` if the given column exists. -- `0`, otherwise. +- `0`, otherwise. **Implementation details** @@ -2444,7 +2444,7 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere ## defaultRoles -Returns the roles which are enabled by default for the current user when he logs in. Initially these are all roles granted to the current user (see [GRANT](../../sql-reference/statements/grant.md#grant-select)), but that can be changed with the [SET DEFAULT ROLE](../../sql-reference/statements/set-role.md#set-default-role-statement) statement. +Returns the roles which are enabled by default for the current user when he logs in. Initially these are all roles granted to the current user (see [GRANT](../../sql-reference/statements/grant.md#select)), but that can be changed with the [SET DEFAULT ROLE](../../sql-reference/statements/set-role.md#set-default-role-statement) statement. **Syntax** diff --git a/docs/en/sql-reference/statements/alter/user.md b/docs/en/sql-reference/statements/alter/user.md index fd7da05167c..b5c156f56a9 100644 --- a/docs/en/sql-reference/statements/alter/user.md +++ b/docs/en/sql-reference/statements/alter/user.md @@ -20,11 +20,11 @@ ALTER USER [IF EXISTS] name1 [ON CLUSTER cluster_name1] [RENAME TO new_name1] [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY | WRITABLE] | PROFILE 'profile_name'] [,...] ``` -To use `ALTER USER` you must have the [ALTER USER](../../../sql-reference/statements/grant.md#grant-access-management) privilege. +To use `ALTER USER` you must have the [ALTER USER](../../../sql-reference/statements/grant.md#access-management) privilege. ## GRANTEES Clause -Specifies users or roles which are allowed to receive [privileges](../../../sql-reference/statements/grant.md#grant-privileges) from this user on the condition this user has also all required access granted with [GRANT OPTION](../../../sql-reference/statements/grant.md#grant-privigele-syntax). Options of the `GRANTEES` clause: +Specifies users or roles which are allowed to receive [privileges](../../../sql-reference/statements/grant.md#privileges) from this user on the condition this user has also all required access granted with [GRANT OPTION](../../../sql-reference/statements/grant.md#granting-privilege-syntax). Options of the `GRANTEES` clause: - `user` — Specifies a user this user can grant privileges to. - `role` — Specifies a role this user can grant privileges to. diff --git a/docs/en/sql-reference/statements/create/user.md b/docs/en/sql-reference/statements/create/user.md index 96f75908c35..aee98cfcd10 100644 --- a/docs/en/sql-reference/statements/create/user.md +++ b/docs/en/sql-reference/statements/create/user.md @@ -177,7 +177,7 @@ Examples: ## GRANTEES Clause -Specifies users or roles which are allowed to receive [privileges](../../../sql-reference/statements/grant.md#grant-privileges) from this user on the condition this user has also all required access granted with [GRANT OPTION](../../../sql-reference/statements/grant.md#grant-privigele-syntax). Options of the `GRANTEES` clause: +Specifies users or roles which are allowed to receive [privileges](../../../sql-reference/statements/grant.md#privileges) from this user on the condition this user has also all required access granted with [GRANT OPTION](../../../sql-reference/statements/grant.md#granting-privilege-syntax). Options of the `GRANTEES` clause: - `user` — Specifies a user this user can grant privileges to. - `role` — Specifies a role this user can grant privileges to. diff --git a/docs/en/sql-reference/statements/grant.md b/docs/en/sql-reference/statements/grant.md index ccef5066e89..2850ce71781 100644 --- a/docs/en/sql-reference/statements/grant.md +++ b/docs/en/sql-reference/statements/grant.md @@ -6,7 +6,7 @@ sidebar_label: GRANT # GRANT Statement -- Grants [privileges](#grant-privileges) to ClickHouse user accounts or roles. +- Grants [privileges](#privileges) to ClickHouse user accounts or roles. - Assigns roles to user accounts or to the other roles. To revoke privileges, use the [REVOKE](../../sql-reference/statements/revoke.md) statement. Also you can list granted privileges with the [SHOW GRANTS](../../sql-reference/statements/show.md#show-grants) statement. @@ -82,9 +82,9 @@ Privileges have a hierarchical structure. A set of permitted queries depends on Hierarchy of privileges: -- [SELECT](#grant-select) -- [INSERT](#grant-insert) -- [ALTER](#grant-alter) +- [SELECT](#select) +- [INSERT](#insert) +- [ALTER](#alter) - `ALTER TABLE` - `ALTER UPDATE` - `ALTER DELETE` @@ -115,7 +115,7 @@ Hierarchy of privileges: - `ALTER VIEW REFRESH` - `ALTER VIEW MODIFY QUERY` - `ALTER VIEW MODIFY SQL SECURITY` -- [CREATE](#grant-create) +- [CREATE](#create) - `CREATE DATABASE` - `CREATE TABLE` - `CREATE ARBITRARY TEMPORARY TABLE` @@ -123,21 +123,21 @@ Hierarchy of privileges: - `CREATE VIEW` - `CREATE DICTIONARY` - `CREATE FUNCTION` -- [DROP](#grant-drop) +- [DROP](#drop) - `DROP DATABASE` - `DROP TABLE` - `DROP VIEW` - `DROP DICTIONARY` - `DROP FUNCTION` -- [TRUNCATE](#grant-truncate) -- [OPTIMIZE](#grant-optimize) -- [SHOW](#grant-show) +- [TRUNCATE](#truncate) +- [OPTIMIZE](#optimize) +- [SHOW](#show) - `SHOW DATABASES` - `SHOW TABLES` - `SHOW COLUMNS` - `SHOW DICTIONARIES` -- [KILL QUERY](#grant-kill-query) -- [ACCESS MANAGEMENT](#grant-access-management) +- [KILL QUERY](#kill-query) +- [ACCESS MANAGEMENT](#access-management) - `CREATE USER` - `ALTER USER` - `DROP USER` @@ -160,7 +160,7 @@ Hierarchy of privileges: - `SHOW_QUOTAS` - `SHOW_SETTINGS_PROFILES` - `ROLE ADMIN` -- [SYSTEM](#grant-system) +- [SYSTEM](#system) - `SYSTEM SHUTDOWN` - `SYSTEM DROP CACHE` - `SYSTEM DROP DNS CACHE` @@ -186,12 +186,12 @@ Hierarchy of privileges: - `SYSTEM FLUSH DISTRIBUTED` - `SYSTEM FLUSH LOGS` - `CLUSTER` (see also `access_control_improvements.on_cluster_queries_require_cluster_grant` configuration directive) -- [INTROSPECTION](#grant-introspection) +- [INTROSPECTION](#introspection) - `addressToLine` - `addressToLineWithInlines` - `addressToSymbol` - `demangle` -- [SOURCES](#grant-sources) +- [SOURCES](#sources) - `FILE` - `URL` - `REMOTE` @@ -200,16 +200,16 @@ Hierarchy of privileges: - `JDBC` - `HDFS` - `S3` -- [dictGet](#grant-dictget) -- [displaySecretsInShowAndSelect](#grant-display-secrets) -- [NAMED COLLECTION ADMIN](#grant-named-collection-admin) +- [dictGet](#dictget) +- [displaySecretsInShowAndSelect](#display-secrets) +- [NAMED COLLECTION ADMIN](#named-collection-admin) - `CREATE NAMED COLLECTION` - `DROP NAMED COLLECTION` - `ALTER NAMED COLLECTION` - `SHOW NAMED COLLECTIONS` - `SHOW NAMED COLLECTIONS SECRETS` - `NAMED COLLECTION` -- [TABLE ENGINE](#grant-table-engine) +- [TABLE ENGINE](#table-engine) Examples of how this hierarchy is treated: @@ -238,11 +238,11 @@ Examples of disallowed syntax: - `GRANT CREATE USER(x) ON db.table TO user` - `GRANT CREATE USER ON db.* TO user` -The special privilege [ALL](#grant-all) grants all the privileges to a user account or a role. +The special privilege [ALL](#all) grants all the privileges to a user account or a role. By default, a user account or a role has no privileges. -If a user or a role has no privileges, it is displayed as [NONE](#grant-none) privilege. +If a user or a role has no privileges, it is displayed as [NONE](#none) privilege. Some queries by their implementation require a set of privileges. For example, to execute the [RENAME](../../sql-reference/statements/optimize.md) query you need the following privileges: `SELECT`, `CREATE TABLE`, `INSERT` and `DROP TABLE`. @@ -326,8 +326,8 @@ Examples of how this hierarchy is treated: **Notes** - The `MODIFY SETTING` privilege allows modifying table engine settings. It does not affect settings or server configuration parameters. -- The `ATTACH` operation needs the [CREATE](#grant-create) privilege. -- The `DETACH` operation needs the [DROP](#grant-drop) privilege. +- The `ATTACH` operation needs the [CREATE](#create) privilege. +- The `DETACH` operation needs the [DROP](#drop) privilege. - To stop mutation by the [KILL MUTATION](../../sql-reference/statements/kill.md#kill-mutation) query, you need to have a privilege to start this mutation. For example, if you want to stop the `ALTER UPDATE` query, you need the `ALTER UPDATE`, `ALTER TABLE`, or `ALTER` privilege. ### CREATE @@ -344,7 +344,7 @@ Allows executing [CREATE](../../sql-reference/statements/create/index.md) and [A **Notes** -- To delete the created table, a user needs [DROP](#grant-drop). +- To delete the created table, a user needs [DROP](#drop). ### DROP @@ -498,7 +498,7 @@ Privilege level: `DICTIONARY`. - `GRANT dictGet ON mydictionary TO john` -### displaySecretsInShowAndSelect {#grant-display-secrets} +### displaySecretsInShowAndSelect {#display-secrets} Allows a user to view secrets in `SHOW` and `SELECT` queries if both [`display_secrets_in_show_and_select` server setting](../../operations/server-configuration-parameters/settings#display_secrets_in_show_and_select) diff --git a/docs/en/sql-reference/statements/show.md b/docs/en/sql-reference/statements/show.md index 029ca4adf3b..627f17b897d 100644 --- a/docs/en/sql-reference/statements/show.md +++ b/docs/en/sql-reference/statements/show.md @@ -11,7 +11,7 @@ N.B. `SHOW CREATE (TABLE|DATABASE|USER)` hides secrets unless is turned on, [`format_display_secrets_in_show_and_select` format setting](../../operations/settings/formats#format_display_secrets_in_show_and_select) is turned on and user has -[`displaySecretsInShowAndSelect`](grant.md#grant-display-secrets) privilege. +[`displaySecretsInShowAndSelect`](grant.md#display-secrets) privilege. ## SHOW CREATE TABLE | DICTIONARY | VIEW | DATABASE @@ -466,7 +466,7 @@ SHOW [CURRENT] QUOTA ``` ## SHOW ACCESS -Shows all [users](../../guides/sre/user-management/index.md#user-account-management), [roles](../../guides/sre/user-management/index.md#role-management), [profiles](../../guides/sre/user-management/index.md#settings-profiles-management), etc. and all their [grants](../../sql-reference/statements/grant.md#grant-privileges). +Shows all [users](../../guides/sre/user-management/index.md#user-account-management), [roles](../../guides/sre/user-management/index.md#role-management), [profiles](../../guides/sre/user-management/index.md#settings-profiles-management), etc. and all their [grants](../../sql-reference/statements/grant.md#privileges). **Syntax** From 5076fe47057167ecbde7c42b3aaa431b5c33344f Mon Sep 17 00:00:00 2001 From: Evgeniy Leko Date: Thu, 2 May 2024 10:02:06 +0200 Subject: [PATCH 242/624] Do not clear terminal after skim suggestions --- rust/workspace/skim/src/lib.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/rust/workspace/skim/src/lib.rs b/rust/workspace/skim/src/lib.rs index 58d5be51baa..d03059fae5c 100644 --- a/rust/workspace/skim/src/lib.rs +++ b/rust/workspace/skim/src/lib.rs @@ -47,6 +47,11 @@ fn skim_impl(prefix: &CxxString, words: &CxxVector) -> Result Date: Thu, 2 May 2024 11:23:47 +0200 Subject: [PATCH 243/624] Add subtractInterval and subtractTupleOfIntervals to aspell-dict.txt to 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 e7477ffc5e1..be97ef4927d 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2511,6 +2511,8 @@ subtractQuarters subtractSeconds subtractWeeks subtractYears +subtractInterval +subtractTupleOfIntervals subtree subtrees subtype From 8de80954809435e15aa65ab29fba59230bc1eae5 Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 2 May 2024 11:56:14 +0200 Subject: [PATCH 244/624] Update addXYZ documentation --- .../functions/date-time-functions.md | 468 +++++++++++++++++- .../aspell-ignore/en/aspell-dict.txt | 2 + 2 files changed, 465 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 51f841657d9..629b3ca9305 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -2557,13 +2557,27 @@ Like function `YYYYMMDDhhmmssToDate()` but produces a [DateTime64](../../sql-ref Accepts an additional, optional `precision` parameter after the `timezone` parameter. -## addYears, addQuarters, addMonths, addWeeks, addDays, addHours, addMinutes, addSeconds, addMilliseconds, addMicroseconds, addNanoseconds +## addYears -These functions add units of the interval specified by the function name to a date, a date with time or a string-encoded date / date with time. A date or date with time is returned. +Adds a specified number of years to a date, a date with time or a string-encoded date / date with time. -Example: +**Syntax** -``` sql +```sql +addYears(date, x) +``` + +**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). + +**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). + +**Example** + +```sql WITH toDate('2024-01-01') AS date, toDateTime('2024-01-01 00:00:00') AS date_time, @@ -2574,12 +2588,456 @@ SELECT addYears(date_time_string, 1) AS add_years_with_date_time_string ``` -``` text +```response ┌─add_years_with_date─┬─add_years_with_date_time─┬─add_years_with_date_time_string─┐ │ 2025-01-01 │ 2025-01-01 00:00:00 │ 2025-01-01 00:00:00.000 │ └─────────────────────┴──────────────────────────┴─────────────────────────────────┘ ``` +## addQuarters + +Adds a specified number of quarters to a date, a date with time or a string-encoded date / date with time. + +**Syntax** + +```sql +addQuarters(date, x) +``` + +**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). + +**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). + +**Example** + +```sql +WITH + toDate('2024-01-01') AS date, + toDateTime('2024-01-01 00:00:00') AS date_time, + '2024-01-01 00:00:00' AS date_time_string +SELECT + addQuarters(date, 1) AS add_quarters_with_date, + addQuarters(date_time, 1) AS add_quarters_with_date_time, + addQuarters(date_time_string, 1) AS add_quarters_with_date_time_string +``` + +```response +┌─add_quarters_with_date─┬─add_quarters_with_date_time─┬─add_quarters_with_date_time_string─┐ +│ 2024-04-01 │ 2024-04-01 00:00:00 │ 2024-04-01 00:00:00.000 │ +└────────────────────────┴─────────────────────────────┴────────────────────────────────────┘ +``` + +## addMonths + +Adds a specified number of months to a date, a date with time or a string-encoded date / date with time. + +**Syntax** + +```sql +addMonths(date, x) +``` + +**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). + +**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). + +**Example** + +```sql +WITH + toDate('2024-01-01') AS date, + toDateTime('2024-01-01 00:00:00') AS date_time, + '2024-01-01 00:00:00' AS date_time_string +SELECT + addMonths(date, 6) AS add_months_with_date, + addMonths(date_time, 6) AS add_months_with_date_time, + addMonths(date_time_string, 6) AS add_months_with_date_time_string +``` + +```response +┌─add_months_with_date─┬─add_months_with_date_time─┬─add_months_with_date_time_string─┐ +│ 2024-07-01 │ 2024-07-01 00:00:00 │ 2024-07-01 00:00:00.000 │ +└──────────────────────┴───────────────────────────┴──────────────────────────────────┘ +``` + +## addWeeks + +Adds a specified number of weeks to a date, a date with time or a string-encoded date / date with time. + +**Syntax** + +```sql +addWeeks(date, x) +``` + +**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). + +**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). + +**Example** + +```sql +WITH + toDate('2024-01-01') AS date, + toDateTime('2024-01-01 00:00:00') AS date_time, + '2024-01-01 00:00:00' AS date_time_string +SELECT + addWeeks(date, 5) AS add_weeks_with_date, + addWeeks(date_time, 5) AS add_weeks_with_date_time, + addWeeks(date_time_string, 5) AS add_weeks_with_date_time_string +``` + +```response +┌─add_weeks_with_date─┬─add_weeks_with_date_time─┬─add_weeks_with_date_time_string─┐ +│ 2024-02-05 │ 2024-02-05 00:00:00 │ 2024-02-05 00:00:00.000 │ +└─────────────────────┴──────────────────────────┴─────────────────────────────────┘ +``` + +## addDays + +Adds a specified number of days to a date, a date with time or a string-encoded date / date with time. + +**Syntax** + +```sql +addDays(date, x) +``` + +**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). + +**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). + +**Example** + +```sql +WITH + toDate('2024-01-01') AS date, + toDateTime('2024-01-01 00:00:00') AS date_time, + '2024-01-01 00:00:00' AS date_time_string +SELECT + addDays(date, 5) AS add_days_with_date, + addDays(date_time, 5) AS add_days_with_date_time, + addDays(date_time_string, 5) AS add_days_with_date_time_string +``` + +```response +┌─add_days_with_date─┬─add_days_with_date_time─┬─add_days_with_date_time_string─┐ +│ 2024-01-06 │ 2024-01-06 00:00:00 │ 2024-01-06 00:00:00.000 │ +└────────────────────┴─────────────────────────┴────────────────────────────────┘ +``` + +## addHours + +Adds a specified number of days to a date, a date with time or a string-encoded date / date with time. + +**Syntax** + +```sql +addHours(date, x) +``` + +**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). + +**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). + +**Example** + +```sql +WITH + toDate('2024-01-01') AS date, + toDateTime('2024-01-01 00:00:00') AS date_time, + '2024-01-01 00:00:00' AS date_time_string +SELECT + addHours(date, 12) AS add_hours_with_date, + addHours(date_time, 12) AS add_hours_with_date_time, + addHours(date_time_string, 12) AS add_hours_with_date_time_string +``` + +```response +┌─add_hours_with_date─┬─add_hours_with_date_time─┬─add_hours_with_date_time_string─┐ +│ 2024-01-01 12:00:00 │ 2024-01-01 12:00:00 │ 2024-01-01 12:00:00.000 │ +└─────────────────────┴──────────────────────────┴─────────────────────────────────┘ +``` + +## addMinutes + +Adds a specified number of minutes to a date, a date with time or a string-encoded date / date with time. + +**Syntax** + +```sql +addMinutes(date, x) +``` + +**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). + +**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). + +**Example** + +```sql +WITH + toDate('2024-01-01') AS date, + toDateTime('2024-01-01 00:00:00') AS date_time, + '2024-01-01 00:00:00' AS date_time_string +SELECT + addMinutes(date, 20) AS add_minutes_with_date, + addMinutes(date_time, 20) AS add_minutes_with_date_time, + addMinutes(date_time_string, 20) AS add_minutes_with_date_time_string +``` + +```response +┌─add_minutes_with_date─┬─add_minutes_with_date_time─┬─add_minutes_with_date_time_string─┐ +│ 2024-01-01 00:20:00 │ 2024-01-01 00:20:00 │ 2024-01-01 00:20:00.000 │ +└───────────────────────┴────────────────────────────┴───────────────────────────────────┘ +``` + +## addSeconds + +Adds a specified number of seconds to a date, a date with time or a string-encoded date / date with time. + +**Syntax** + +```sql +addSeconds(date, x) +``` + +**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). + +**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). + +**Example** + +```sql +WITH + toDate('2024-01-01') AS date, + toDateTime('2024-01-01 00:00:00') AS date_time, + '2024-01-01 00:00:00' AS date_time_string +SELECT + addSeconds(date, 30) AS add_seconds_with_date, + addSeconds(date_time, 30) AS add_seconds_with_date_time, + addSeconds(date_time_string, 30) AS add_seconds_with_date_time_string +``` + +```response +┌─add_seconds_with_date─┬─add_seconds_with_date_time─┬─add_seconds_with_date_time_string─┐ +│ 2024-01-01 00:00:30 │ 2024-01-01 00:00:30 │ 2024-01-01 00:00:30.000 │ +└───────────────────────┴────────────────────────────┴───────────────────────────────────┘ +``` + +## addMilliseconds + +Adds a specified number of milliseconds to a date with time or a string-encoded date with time. + +**Syntax** + +```sql +addMilliseconds(date_time, x) +``` + +**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). + +**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). + +**Example** + +```sql +WITH + toDateTime('2024-01-01 00:00:00') AS date_time, + '2024-01-01 00:00:00' AS date_time_string +SELECT + addMilliseconds(date_time, 1000) AS add_milliseconds_with_date_time, + addMilliseconds(date_time_string, 1000) AS add_milliseconds_with_date_time_string +``` + +```response +┌─add_milliseconds_with_date_time─┬─add_milliseconds_with_date_time_string─┐ +│ 2024-01-01 00:00:01.000 │ 2024-01-01 00:00:01.000 │ +└─────────────────────────────────┴────────────────────────────────────────┘ +``` + +## addMicroseconds + +Adds a specified number of microseconds to a date with time or a string-encoded date with time. + +**Syntax** + +```sql +addMicroseconds(date_time, x) +``` + +**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). + +**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). + +**Example** + +```sql +WITH + toDateTime('2024-01-01 00:00:00') AS date_time, + '2024-01-01 00:00:00' AS date_time_string +SELECT + addMicroseconds(date_time, 1000000) AS add_microseconds_with_date_time, + addMicroseconds(date_time_string, 1000000) AS add_microseconds_with_date_time_string +``` + +```response +┌─add_microseconds_with_date_time─┬─add_microseconds_with_date_time_string─┐ +│ 2024-01-01 00:00:01.000000 │ 2024-01-01 00:00:01.000000 │ +└─────────────────────────────────┴────────────────────────────────────────┘ +``` + +## addNanoseconds + +Adds a specified number of microseconds to a date with time or a string-encoded date with time. + +**Syntax** + +```sql +addNanoseconds(date_time, x) +``` + +**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). + +**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). + +**Example** + +```sql +WITH + toDateTime('2024-01-01 00:00:00') AS date_time, + '2024-01-01 00:00:00' AS date_time_string +SELECT + addNanoseconds(date_time, 1000) AS add_nanoseconds_with_date_time, + addNanoseconds(date_time_string, 1000) AS add_nanoseconds_with_date_time_string +``` + +```response +┌─add_nanoseconds_with_date_time─┬─add_nanoseconds_with_date_time_string─┐ +│ 2024-01-01 00:00:00.000001000 │ 2024-01-01 00:00:00.000001000 │ +└────────────────────────────────┴───────────────────────────────────────┘ +``` + +## addInterval + +Adds an interval to another interval or tuple of intervals. + +**Syntax** + +```sql +addInterval(interval_1, interval_2) +``` + +**Parameters** + +- `interval_1`: First interval or interval of tuples. [interval](../data-types/special-data-types/interval.md), [tuple](../data-types/tuple.md)([interval](../data-types/special-data-types/interval.md)). +- `interval_2`: Second interval to be added. [interval](../data-types/special-data-types/interval.md). + +**Returned value** +- Returns a tuple of intervals. [tuple](../data-types/tuple.md)([interval](../data-types/special-data-types/interval.md)). + +:::note +If the types of the first interval (or the interval in the tuple) and the second interval are the same they will be merged into one interval. +::: + +**Example** + +Query: + +```sql +SELECT addInterval(INTERVAL 1 DAY, INTERVAL 1 MONTH); +SELECT addInterval((INTERVAL 1 DAY, INTERVAL 1 YEAR), INTERVAL 1 MONTH); +SELECT addInterval(INTERVAL 2 DAY, INTERVAL 1 DAY); +``` + +Result: + +```response +┌─addInterval(toIntervalDay(1), toIntervalMonth(1))─┐ +│ (1,1) │ +└───────────────────────────────────────────────────┘ +┌─addInterval((toIntervalDay(1), toIntervalYear(1)), toIntervalMonth(1))─┐ +│ (1,1,1) │ +└────────────────────────────────────────────────────────────────────────┘ +┌─addInterval(toIntervalDay(2), toIntervalDay(1))─┐ +│ (3) │ +└─────────────────────────────────────────────────┘ +``` + +## addTupleOfIntervals + +Consecutively adds a tuple of intervals to a Date or a DateTime. + +**Syntax** + +```sql +addTupleOfIntervals(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). +- `intervals`: Tuple of intervals to add to `date`. [tuple](../data-types/tuple.md)([interval](../data-types/special-data-types/interval.md)). + +**Returned value** +- Returns `date` with added `intervals`. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md). + +**Example** + +Query: + +```sql +WITH toDate('2018-01-01') AS date SELECT addTupleOfIntervals(date, (INTERVAL 1 DAY, INTERVAL 1 MONTH, INTERVAL 1 YEAR)) +``` + +Result: + +```response +┌─addTupleOfIntervals(date, (toIntervalDay(1), toIntervalMonth(1), toIntervalYear(1)))─┐ +│ 2019-02-02 │ +└──────────────────────────────────────────────────────────────────────────────────────┘ +``` + ## subtractYears, subtractQuarters, subtractMonths, subtractWeeks, subtractDays, subtractHours, subtractMinutes, subtractSeconds, subtractMilliseconds, subtractMicroseconds, subtractNanoseconds These functions subtract units of the interval specified by the function name from a date, a date with time or a string-encoded date / date with time. A date or date with time is returned. diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index e7477ffc5e1..441df44dd3f 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1088,6 +1088,8 @@ addQuarters addSeconds addWeeks addYears +addInterval +addTupleOfIntervals addr addressToLine addressToLineWithInlines From a50c41c61703126fcef38615b1612bdff9a6a336 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 30 Apr 2024 13:19:10 +0300 Subject: [PATCH 245/624] QueryAnalysisPass improve QUALIFY validation --- src/Analyzer/ValidationUtils.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Analyzer/ValidationUtils.cpp b/src/Analyzer/ValidationUtils.cpp index 60cc1dd521f..e17639367eb 100644 --- a/src/Analyzer/ValidationUtils.cpp +++ b/src/Analyzer/ValidationUtils.cpp @@ -266,6 +266,9 @@ void validateAggregates(const QueryTreeNodePtr & query_node, AggregatesValidatio if (query_node_typed.hasHaving()) validate_group_by_columns_visitor.visit(query_node_typed.getHaving()); + if (query_node_typed.hasQualify()) + validate_group_by_columns_visitor.visit(query_node_typed.getQualify()); + if (query_node_typed.hasOrderBy()) validate_group_by_columns_visitor.visit(query_node_typed.getOrderByNode()); From 15eac2513ba4d716618256d85488c7ae4f1dd2b1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 2 May 2024 10:09:17 +0000 Subject: [PATCH 246/624] Docs: Update install instructions --- docs/en/getting-started/install.md | 56 ++++++++++++++++++------------ 1 file changed, 34 insertions(+), 22 deletions(-) diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index e3be30bde8c..d028301f08e 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -11,7 +11,7 @@ import CodeBlock from '@theme/CodeBlock'; # Install ClickHouse -You have three options for getting up and running with ClickHouse: +You have four options for getting up and running with ClickHouse: - **[ClickHouse Cloud](https://clickhouse.com/cloud/):** The official ClickHouse as a service, - built by, maintained and supported by the creators of ClickHouse - **[Quick Install](#quick-install):** an easy-to-download binary for testing and developing with ClickHouse @@ -32,37 +32,49 @@ On Linux, macOS and FreeBSD: 1. If you are just getting started and want to see what ClickHouse can do, the simplest way to download ClickHouse locally is to run the following command. It downloads a single binary for your operating system that can be used to run the ClickHouse server, - clickhouse-client, clickhouse-local, ClickHouse Keeper, and other tools: + `clickhouse-client`, `clickhouse-local`, ClickHouse Keeper, and other tools: - ```bash - curl https://clickhouse.com/ | sh - ``` + ```bash + curl https://clickhouse.com/ | sh + ``` -1. Run the following command to start the ClickHouse server: +2. Run the following command to start [clickhouse-local](../operations/utilities/clickhouse-local.md): + + ```bash + ./clickhouse + ``` + + `clickhouse-local` allows you to process local and remote files using ClickHouse's powerful SQL and without a need for configuration. Table + data is stored in a temporary location, meaning that after a restart of `clickhouse-local` all previously created tables are no longer + available. + + As an alternative, you can start the ClickHouse server with this command ... ```bash ./clickhouse server ``` - The first time you run this script, the necessary files and folders are created in the current directory, then the server starts. + ... and open a new terminal and use `clickhouse-client` to connect to the server: -1. Open a new terminal and use the **./clickhouse client** to connect to your service: + ```bash + ./clickhouse client + ``` - ```bash - ./clickhouse client - ``` + ```response + ./clickhouse client + ClickHouse client version 24.5.1.117 (official build). + Connecting to localhost:9000 as user default. + Connected to ClickHouse server version 24.5.1. - ```response - ./clickhouse client - ClickHouse client version 23.2.1.1501 (official build). - Connecting to localhost:9000 as user default. - Connected to ClickHouse server version 23.2.1. + local-host :) + ``` - local-host :) - ``` - - You are ready to start sending DDL and SQL commands to ClickHouse! + Table data is stored in the current directory and still available after a restart of ClickHouse server. If necessary, you can pass + `-C config.xml` as an additional command line argument to `./clickhouse server` and provide further configuration in a configuration + file. All availabe configuration settings are documented [here](../operations/settings.md) and in an [example configuration file + template](https://github.com/ClickHouse/ClickHouse/blob/master/programs/server/config.xml). + You are ready to start sending SQL commands to ClickHouse! :::tip The [Quick Start](/docs/en/quick-start.mdx) walks through the steps for creating tables and inserting data. @@ -377,14 +389,14 @@ build. ### macOS-only: Install with Homebrew -To install ClickHouse using [homebrew](https://brew.sh/), see [here](https://formulae.brew.sh/cask/clickhouse). +To install ClickHouse on macOS using [homebrew](https://brew.sh/), please see the ClickHouse [community homebrew formula](https://formulae.brew.sh/cask/clickhouse). ## Launch {#launch} To start the server as a daemon, run: ``` bash -$ sudo clickhouse start +$ clickhouse start ``` There are also other ways to run ClickHouse: From c2dd365e3724aff2837ce70509b714194ffe1577 Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 2 May 2024 12:41:06 +0200 Subject: [PATCH 247/624] Update test and reference file --- ..._skip_ANSI_in_UTF8_compute_width.reference | 23 ++++--------------- .../03142_skip_ANSI_in_UTF8_compute_width.sql | 19 +-------------- 2 files changed, 6 insertions(+), 36 deletions(-) diff --git a/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.reference b/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.reference index 4e1caa85ac7..864f62d3113 100644 --- a/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.reference +++ b/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.reference @@ -1,18 +1,5 @@ -█ -█ -█ -█ -█ -█ -█ -█ -█ -█ -█ -█ -█ -█ -█ -█ -█ -█ + ┏━━━┓ + ┃ x ┃ + ┡━━━┩ +1. │ █ │ + └───┘ diff --git a/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql b/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql index 6487e9d5140..048f1821113 100644 --- a/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql +++ b/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql @@ -1,18 +1 @@ -SELECT format('\x1b[38;2;{0};{1};{2}m█\x1b[0m', 255, 128, 0) AS x; -SELECT '\x1b[1A█' AS x; -SELECT '\x1b[1C█' AS x; -SELECT '\x1b[1D█' AS x; -SELECT '\x1b[1E█' AS x; -SELECT '\x1b[1F█' AS x; -SELECT '\x1b[1G█' AS x; -SELECT '\x1b[1;1H█' AS x; -SELECT '\x1b[1J█' AS x; -SELECT '\x1b[1J█' AS x; -SELECT '\x1b[1K█' AS x; -SELECT '\x1b[1S█' AS x; -SELECT '\x1b[1K█' AS x; -SELECT '\x1b[1;1f█' AS x; -SELECT '\x1b[1;1f█' AS x; -SELECT '\x1b[5i█' AS x; -SELECT '\x1b[4i█' AS x; -SELECT '\x1b[6n█' AS x; +SELECT format('\x1b[38;2;{0};{1};{2}m█\x1b[0m', 255, 128, 0) AS x FORMAT Pretty; \ No newline at end of file From 81e4a754d951df5753b88d819f22ec1f21217a81 Mon Sep 17 00:00:00 2001 From: Jordi Villar Date: Thu, 2 May 2024 12:51:53 +0200 Subject: [PATCH 248/624] Document start of week differences --- .../functions/date-time-functions.md | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 2cbcc193dd6..ed98f986139 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1413,9 +1413,10 @@ toStartOfFifteenMinutes(toDateTime('2023-04-21 10:20:00')): 2023-04-21 10:15:00 toStartOfFifteenMinutes(toDateTime('2023-04-21 10:23:00')): 2023-04-21 10:15:00 ``` -## toStartOfInterval(date_or_date_with_time, INTERVAL x unit \[, time_zone\]) +## toStartOfInterval -This function generalizes other `toStartOf*()` functions. For example, +This function generalizes other `toStartOf*()` functions with `toStartOfInterval(date_or_date_with_time, INTERVAL x unit [, time_zone])` syntax. +For example, - `toStartOfInterval(t, INTERVAL 1 year)` returns the same as `toStartOfYear(t)`, - `toStartOfInterval(t, INTERVAL 1 month)` returns the same as `toStartOfMonth(t)`, - `toStartOfInterval(t, INTERVAL 1 day)` returns the same as `toStartOfDay(t)`, @@ -1440,6 +1441,8 @@ The calculation is performed relative to specific points in time: (*) hour intervals are special: the calculation is always performed relative to 00:00:00 (midnight) of the current day. As a result, only hour values between 1 and 23 are useful. +If unit `week` was specified, `toStartOfInterval` assumes that weeks start on Monday. Note that this behavior is different from that of function `toStartOfWeek` in which weeks start by default on Sunday. + **See Also** - [date_trunc](#date_trunc) @@ -1673,7 +1676,7 @@ Like [fromDaysSinceYearZero](#fromDaysSinceYearZero) but returns a [Date32](../. Returns the `unit` component of the difference between `startdate` and `enddate`. The difference is calculated using a precision of 1 nanosecond. E.g. the difference between `2021-12-29` and `2022-01-01` is 3 days for `day` unit, 0 months for `month` unit, 0 years for `year` unit. -For an alternative to `age`, see function `date\_diff`. +For an alternative to `age`, see function `date_diff`. **Syntax** @@ -1742,14 +1745,14 @@ Result: ``` -## date\_diff +## date_diff Returns the count of the specified `unit` boundaries crossed between the `startdate` and the `enddate`. The difference is calculated using relative units, e.g. the difference between `2021-12-29` and `2022-01-01` is 3 days for unit `day` (see [toRelativeDayNum](#torelativedaynum)), 1 month for unit `month` (see [toRelativeMonthNum](#torelativemonthnum)) and 1 year for unit `year` (see [toRelativeYearNum](#torelativeyearnum)). -If unit `week` was specified, `date\_diff` assumes that weeks start on Monday. Note that this behavior is different from that of function `toWeek()` in which weeks start by default on Sunday. +If unit `week` was specified, `date_diff` assumes that weeks start on Monday. Note that this behavior is different from that of function `toWeek()` in which weeks start by default on Sunday. -For an alternative to `date\_diff`, see function `age`. +For an alternative to `date_diff`, see function `age`. **Syntax** @@ -2883,7 +2886,7 @@ Result: ## fromUnixTimestamp -This function converts a Unix timestamp to a calendar date and a time of a day. +This function converts a Unix timestamp to a calendar date and a time of a day. It can be called in two ways: From d2a3ffbe57c5c2b06800878e0c3ce57572cee317 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 2 May 2024 11:08:17 +0000 Subject: [PATCH 249/624] Typo --- docs/en/getting-started/install.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index d028301f08e..dc4d73cd13b 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -71,7 +71,7 @@ On Linux, macOS and FreeBSD: Table data is stored in the current directory and still available after a restart of ClickHouse server. If necessary, you can pass `-C config.xml` as an additional command line argument to `./clickhouse server` and provide further configuration in a configuration - file. All availabe configuration settings are documented [here](../operations/settings.md) and in an [example configuration file + file. All available configuration settings are documented [here](../operations/settings.md) and in an [example configuration file template](https://github.com/ClickHouse/ClickHouse/blob/master/programs/server/config.xml). You are ready to start sending SQL commands to ClickHouse! From 083bcfc1cef06f700e0ca027ac3ee49a575116c7 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 2 May 2024 13:07:35 +0200 Subject: [PATCH 250/624] distinct message for s3 error 'no key' for cases disk and storage --- src/Backups/BackupIO_AzureBlobStorage.cpp | 11 +-- .../IO/ReadBufferFromAzureBlobStorage.cpp | 4 +- .../AzureBlobStorage/AzureBlobStorageAuth.cpp | 3 + .../AzureBlobStorage/AzureObjectStorage.cpp | 22 +++--- src/Disks/ObjectStorages/IObjectStorage.cpp | 8 +-- .../ObjectStorages/S3/S3ObjectStorage.cpp | 21 +++--- .../copyAzureBlobStorageFile.cpp | 17 ++--- .../copyAzureBlobStorageFile.h | 6 +- src/IO/ReadBufferFromS3.cpp | 4 +- src/IO/ReadSettings.h | 3 - src/IO/S3/Client.cpp | 27 ++++++-- src/IO/S3/Client.h | 8 +++ src/IO/S3/copyS3File.cpp | 14 ++-- src/IO/S3/getObjectInfo.cpp | 22 +++--- src/IO/S3/getObjectInfo.h | 6 +- src/IO/WriteBufferFromS3.cpp | 14 ++-- src/IO/WriteSettings.h | 3 - src/Storages/StorageAzureBlob.h | 2 - src/Storages/StorageS3.cpp | 1 - .../test_checking_s3_blobs_paranoid/test.py | 67 +++++++++++++++++++ 20 files changed, 171 insertions(+), 92 deletions(-) diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index 9cf5c433826..a3998431674 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -14,6 +14,7 @@ #include #include +#include #include @@ -38,6 +39,8 @@ BackupReaderAzureBlobStorage::BackupReaderAzureBlobStorage( , configuration(configuration_) { auto client_ptr = StorageAzureBlob::createClient(configuration, /* is_read_only */ false); + client_ptr->SetClickhouseOptions(Azure::Storage::Blobs::ClickhouseClientOptions{.IsClientForDisk=true}); + object_storage = std::make_unique("BackupReaderAzureBlobStorage", std::move(client_ptr), StorageAzureBlob::createSettings(context_), @@ -97,8 +100,7 @@ void BackupReaderAzureBlobStorage::copyFileToDisk(const String & path_in_backup, /* dest_path */ blob_path[0], settings, read_settings, - threadPoolCallbackRunnerUnsafe(getBackupsIOThreadPool().get(), "BackupRDAzure"), - /* for_disk_azure_blob_storage= */ true); + threadPoolCallbackRunnerUnsafe(getBackupsIOThreadPool().get(), "BackupRDAzure")); return file_size; }; @@ -123,6 +125,8 @@ BackupWriterAzureBlobStorage::BackupWriterAzureBlobStorage( , configuration(configuration_) { auto client_ptr = StorageAzureBlob::createClient(configuration, /* is_read_only */ false, attempt_to_create_container); + client_ptr->SetClickhouseOptions(Azure::Storage::Blobs::ClickhouseClientOptions{.IsClientForDisk=true}); + object_storage = std::make_unique("BackupWriterAzureBlobStorage", std::move(client_ptr), StorageAzureBlob::createSettings(context_), @@ -177,8 +181,7 @@ void BackupWriterAzureBlobStorage::copyFile(const String & destination, const St /* dest_path */ destination, settings, read_settings, - threadPoolCallbackRunnerUnsafe(getBackupsIOThreadPool().get(), "BackupWRAzure"), - /* for_disk_azure_blob_storage= */ true); + threadPoolCallbackRunnerUnsafe(getBackupsIOThreadPool().get(), "BackupWRAzure")); } void BackupWriterAzureBlobStorage::copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp index 48b4ed23af0..da1ea65f2ea 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp @@ -225,7 +225,7 @@ void ReadBufferFromAzureBlobStorage::initialize() try { ProfileEvents::increment(ProfileEvents::AzureGetObject); - if (read_settings.for_object_storage) + if (blob_container_client->GetClickhouseOptions().IsClientForDisk) ProfileEvents::increment(ProfileEvents::DiskAzureGetObject); auto download_response = blob_client->Download(download_options); @@ -279,7 +279,7 @@ size_t ReadBufferFromAzureBlobStorage::readBigAt(char * to, size_t n, size_t ran try { ProfileEvents::increment(ProfileEvents::AzureGetObject); - if (read_settings.for_object_storage) + if (blob_container_client->GetClickhouseOptions().IsClientForDisk) ProfileEvents::increment(ProfileEvents::DiskAzureGetObject); Azure::Storage::Blobs::DownloadBlobOptions download_options; diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp index cf84fe46579..43bbb5cad4b 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -206,6 +207,8 @@ Azure::Storage::Blobs::BlobClientOptions getAzureBlobClientOptions(const Poco::U client_options.Retry = retry_options; client_options.Transport.Transport = std::make_shared(curl_options); + client_options.ClickhouseOptions = Azure::Storage::Blobs::ClickhouseClientOptions{.IsClientForDisk=true}; + return client_options; } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 0f45f40288e..38a7db5702c 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -69,7 +69,8 @@ private: bool getBatchAndCheckNext(RelativePathsWithMetadata & batch) override { ProfileEvents::increment(ProfileEvents::AzureListObjects); - ProfileEvents::increment(ProfileEvents::DiskAzureListObjects); + if (client->GetClickhouseOptions().IsClientForDisk) + ProfileEvents::increment(ProfileEvents::DiskAzureListObjects); batch.clear(); auto outcome = client->ListBlobs(options); @@ -130,7 +131,8 @@ bool AzureObjectStorage::exists(const StoredObject & object) const options.PageSizeHint = 1; ProfileEvents::increment(ProfileEvents::AzureListObjects); - ProfileEvents::increment(ProfileEvents::DiskAzureListObjects); + if (client_ptr->GetClickhouseOptions().IsClientForDisk) + ProfileEvents::increment(ProfileEvents::DiskAzureListObjects); auto blobs_list_response = client_ptr->ListBlobs(options); auto blobs_list = blobs_list_response.Blobs; @@ -169,7 +171,8 @@ void AzureObjectStorage::listObjects(const std::string & path, RelativePathsWith while (true) { ProfileEvents::increment(ProfileEvents::AzureListObjects); - ProfileEvents::increment(ProfileEvents::DiskAzureListObjects); + if (client_ptr->GetClickhouseOptions().IsClientForDisk) + ProfileEvents::increment(ProfileEvents::DiskAzureListObjects); blob_list_response = client_ptr->ListBlobs(options); auto blobs_list = blob_list_response.Blobs; @@ -298,7 +301,8 @@ std::unique_ptr AzureObjectStorage::writeObject( /// NO void AzureObjectStorage::removeObjectImpl(const StoredObject & object, const SharedAzureClientPtr & client_ptr, bool if_exists) { ProfileEvents::increment(ProfileEvents::AzureDeleteObjects); - ProfileEvents::increment(ProfileEvents::DiskAzureDeleteObjects); + if (client_ptr->GetClickhouseOptions().IsClientForDisk) + ProfileEvents::increment(ProfileEvents::DiskAzureDeleteObjects); const auto & path = object.remote_path; LOG_TEST(log, "Removing single object: {}", path); @@ -353,13 +357,14 @@ void AzureObjectStorage::removeObjectsIfExist(const StoredObjects & objects) ObjectMetadata AzureObjectStorage::getObjectMetadata(const std::string & path) const { - ProfileEvents::increment(ProfileEvents::AzureGetProperties); - ProfileEvents::increment(ProfileEvents::DiskAzureGetProperties); - auto client_ptr = client.get(); auto blob_client = client_ptr->GetBlobClient(path); auto properties = blob_client.GetProperties().Value; + ProfileEvents::increment(ProfileEvents::AzureGetProperties); + if (client_ptr->GetClickhouseOptions().IsClientForDisk) + ProfileEvents::increment(ProfileEvents::DiskAzureGetProperties); + ObjectMetadata result; result.size_bytes = properties.BlobSize; if (!properties.Metadata.empty()) @@ -391,7 +396,8 @@ void AzureObjectStorage::copyObject( /// NOLINT } ProfileEvents::increment(ProfileEvents::AzureCopyObject); - ProfileEvents::increment(ProfileEvents::DiskAzureCopyObject); + if (client_ptr->GetClickhouseOptions().IsClientForDisk) + ProfileEvents::increment(ProfileEvents::DiskAzureCopyObject); dest_blob_client.CopyFromUri(source_blob_client.GetUrl(), copy_options); } diff --git a/src/Disks/ObjectStorages/IObjectStorage.cpp b/src/Disks/ObjectStorages/IObjectStorage.cpp index 5fd9695ec9e..accef9a08ab 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.cpp +++ b/src/Disks/ObjectStorages/IObjectStorage.cpp @@ -84,16 +84,12 @@ const std::string & IObjectStorage::getCacheName() const ReadSettings IObjectStorage::patchSettings(const ReadSettings & read_settings) const { - ReadSettings settings{read_settings}; - settings.for_object_storage = true; - return settings; + return read_settings; } WriteSettings IObjectStorage::patchSettings(const WriteSettings & write_settings) const { - WriteSettings settings{write_settings}; - settings.for_object_storage = true; - return settings; + return write_settings; } } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 2f0d93907ae..2eae8877f87 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -158,7 +158,7 @@ private: bool S3ObjectStorage::exists(const StoredObject & object) const { auto settings_ptr = s3_settings.get(); - return S3::objectExists(*client.get(), uri.bucket, object.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); + return S3::objectExists(*client.get(), uri.bucket, object.remote_path, {}, settings_ptr->request_settings); } std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT @@ -425,7 +425,7 @@ void S3ObjectStorage::removeObjectsIfExist(const StoredObjects & objects) std::optional S3ObjectStorage::tryGetObjectMetadata(const std::string & path) const { auto settings_ptr = s3_settings.get(); - auto object_info = S3::getObjectInfo(*client.get(), uri.bucket, path, {}, settings_ptr->request_settings, /* with_metadata= */ true, /* for_disk_s3= */ true, /* throw_on_error= */ false); + auto object_info = S3::getObjectInfo(*client.get(), uri.bucket, path, {}, settings_ptr->request_settings, /* with_metadata= */ true, /* throw_on_error= */ false); if (object_info.size == 0 && object_info.last_modification_time == 0 && object_info.metadata.empty()) return {}; @@ -441,7 +441,7 @@ std::optional S3ObjectStorage::tryGetObjectMetadata(const std::s ObjectMetadata S3ObjectStorage::getObjectMetadata(const std::string & path) const { auto settings_ptr = s3_settings.get(); - auto object_info = S3::getObjectInfo(*client.get(), uri.bucket, path, {}, settings_ptr->request_settings, /* with_metadata= */ true, /* for_disk_s3= */ true); + auto object_info = S3::getObjectInfo(*client.get(), uri.bucket, path, {}, settings_ptr->request_settings, /* with_metadata= */ true); ObjectMetadata result; result.size_bytes = object_info.size; @@ -464,9 +464,11 @@ void S3ObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT { auto current_client = dest_s3->client.get(); auto settings_ptr = s3_settings.get(); - auto size = S3::getObjectSize(*current_client, uri.bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); + auto size = S3::getObjectSize(*current_client, uri.bucket, object_from.remote_path, {}, settings_ptr->request_settings); auto scheduler = threadPoolCallbackRunnerUnsafe(getThreadPoolWriter(), "S3ObjStor_copy"); - try { + + try + { copyS3File( current_client, uri.bucket, @@ -479,8 +481,7 @@ void S3ObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT patchSettings(read_settings), BlobStorageLogWriter::create(disk_name), object_to_attributes, - scheduler, - /* for_disk_s3= */ true); + scheduler); return; } catch (S3Exception & exc) @@ -506,8 +507,9 @@ void S3ObjectStorage::copyObject( // NOLINT { auto current_client = client.get(); auto settings_ptr = s3_settings.get(); - auto size = S3::getObjectSize(*current_client, uri.bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); + auto size = S3::getObjectSize(*current_client, uri.bucket, object_from.remote_path, {}, settings_ptr->request_settings); auto scheduler = threadPoolCallbackRunnerUnsafe(getThreadPoolWriter(), "S3ObjStor_copy"); + copyS3File(current_client, uri.bucket, object_from.remote_path, @@ -519,8 +521,7 @@ void S3ObjectStorage::copyObject( // NOLINT patchSettings(read_settings), BlobStorageLogWriter::create(disk_name), object_to_attributes, - scheduler, - /* for_disk_s3= */ true); + scheduler); } void S3ObjectStorage::setNewSettings(std::unique_ptr && s3_settings_) diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp index ef8c01f4b5e..769f1a184f6 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp @@ -46,7 +46,6 @@ namespace const String & dest_blob_, std::shared_ptr settings_, ThreadPoolCallbackRunnerUnsafe schedule_, - bool for_disk_azure_blob_storage_, const Poco::Logger * log_) : create_read_buffer(create_read_buffer_) , client(client_) @@ -56,7 +55,6 @@ namespace , dest_blob(dest_blob_) , settings(settings_) , schedule(schedule_) - , for_disk_azure_blob_storage(for_disk_azure_blob_storage_) , log(log_) , max_single_part_upload_size(settings_->max_single_part_upload_size) { @@ -73,7 +71,6 @@ namespace const String & dest_blob; std::shared_ptr settings; ThreadPoolCallbackRunnerUnsafe schedule; - bool for_disk_azure_blob_storage; const Poco::Logger * log; size_t max_single_part_upload_size; @@ -217,7 +214,7 @@ namespace void processUploadPartRequest(UploadPartTask & task) { ProfileEvents::increment(ProfileEvents::AzureUploadPart); - if (for_disk_azure_blob_storage) + if (client->GetClickhouseOptions().IsClientForDisk) ProfileEvents::increment(ProfileEvents::DiskAzureUploadPart); auto block_blob_client = client->GetBlockBlobClient(dest_blob); @@ -269,10 +266,9 @@ void copyDataToAzureBlobStorageFile( const String & dest_container_for_logging, const String & dest_blob, std::shared_ptr settings, - ThreadPoolCallbackRunnerUnsafe schedule, - bool for_disk_azure_blob_storage) + ThreadPoolCallbackRunnerUnsafe schedule) { - UploadHelper helper{create_read_buffer, dest_client, offset, size, dest_container_for_logging, dest_blob, settings, schedule, for_disk_azure_blob_storage, &Poco::Logger::get("copyDataToAzureBlobStorageFile")}; + UploadHelper helper{create_read_buffer, dest_client, offset, size, dest_container_for_logging, dest_blob, settings, schedule, &Poco::Logger::get("copyDataToAzureBlobStorageFile")}; helper.performCopy(); } @@ -288,14 +284,13 @@ void copyAzureBlobStorageFile( const String & dest_blob, std::shared_ptr settings, const ReadSettings & read_settings, - ThreadPoolCallbackRunnerUnsafe schedule, - bool for_disk_azure_blob_storage) + ThreadPoolCallbackRunnerUnsafe schedule) { if (settings->use_native_copy) { ProfileEvents::increment(ProfileEvents::AzureCopyObject); - if (for_disk_azure_blob_storage) + if (dest_client->GetClickhouseOptions().IsClientForDisk) ProfileEvents::increment(ProfileEvents::DiskAzureCopyObject); auto block_blob_client_src = src_client->GetBlockBlobClient(src_blob); @@ -330,7 +325,7 @@ void copyAzureBlobStorageFile( settings->max_single_download_retries); }; - UploadHelper helper{create_read_buffer, dest_client, offset, size, dest_container_for_logging, dest_blob, settings, schedule, for_disk_azure_blob_storage, &Poco::Logger::get("copyAzureBlobStorageFile")}; + UploadHelper helper{create_read_buffer, dest_client, offset, size, dest_container_for_logging, dest_blob, settings, schedule, &Poco::Logger::get("copyAzureBlobStorageFile")}; helper.performCopy(); } } diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h index 170a3d7f6aa..6ad54923ab5 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h @@ -31,8 +31,7 @@ void copyAzureBlobStorageFile( const String & dest_blob, std::shared_ptr settings, const ReadSettings & read_settings, - ThreadPoolCallbackRunnerUnsafe schedule_ = {}, - bool for_disk_azure_blob_storage = false); + ThreadPoolCallbackRunnerUnsafe schedule_ = {}); /// Copies data from any seekable source to AzureBlobStorage. @@ -48,8 +47,7 @@ void copyDataToAzureBlobStorageFile( const String & dest_container_for_logging, const String & dest_blob, std::shared_ptr settings, - ThreadPoolCallbackRunnerUnsafe schedule_ = {}, - bool for_disk_azure_blob_storage = false); + ThreadPoolCallbackRunnerUnsafe schedule_ = {}); } diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 491ff253066..813546aa052 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -314,7 +314,7 @@ size_t ReadBufferFromS3::getFileSize() if (file_size) return *file_size; - auto object_size = S3::getObjectSize(*client_ptr, bucket, key, version_id, request_settings, /* for_disk_s3= */ read_settings.for_object_storage); + auto object_size = S3::getObjectSize(*client_ptr, bucket, key, version_id, request_settings); file_size = object_size; return *file_size; @@ -415,7 +415,7 @@ Aws::S3::Model::GetObjectResult ReadBufferFromS3::sendRequest(size_t attempt, si } ProfileEvents::increment(ProfileEvents::S3GetObject); - if (read_settings.for_object_storage) + if (client_ptr->isClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskS3GetObject); ProfileEventTimeIncrement watch(ProfileEvents::ReadBufferFromS3InitMicroseconds); diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index 6a0cac35878..6c44861eae3 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -127,9 +127,6 @@ struct ReadSettings bool http_skip_not_found_url_for_globs = true; bool http_make_head_request = true; - /// Monitoring - bool for_object_storage = false; // to choose which profile events should be incremented - ReadSettings adjustBufferSize(size_t file_size) const { ReadSettings res = *this; diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 6c138eb3bff..9229342b8c1 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -384,7 +384,8 @@ Model::HeadObjectOutcome Client::HeadObject(HeadObjectRequest & request) const /// The next call is NOT a recurcive call /// This is a virtuall call Aws::S3::S3Client::HeadObject(const Model::HeadObjectRequest&) - return HeadObject(static_cast(request)); + return enrichErrorMessage( + HeadObject(static_cast(request))); } /// For each request, we wrap the request functions from Aws::S3::Client with doRequest @@ -404,7 +405,8 @@ Model::ListObjectsOutcome Client::ListObjects(ListObjectsRequest & request) cons Model::GetObjectOutcome Client::GetObject(GetObjectRequest & request) const { - return doRequest(request, [this](const Model::GetObjectRequest & req) { return GetObject(req); }); + return enrichErrorMessage( + doRequest(request, [this](const Model::GetObjectRequest & req) { return GetObject(req); })); } Model::AbortMultipartUploadOutcome Client::AbortMultipartUpload(AbortMultipartUploadRequest & request) const @@ -652,14 +654,14 @@ Client::doRequestWithRetryNetworkErrors(RequestType & request, RequestFn request if constexpr (IsReadMethod) { - if (client_configuration.for_disk_s3) + if (isClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskS3ReadRequestsErrors); else ProfileEvents::increment(ProfileEvents::S3ReadRequestsErrors); } else { - if (client_configuration.for_disk_s3) + if (isClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskS3WriteRequestsErrors); else ProfileEvents::increment(ProfileEvents::S3WriteRequestsErrors); @@ -689,6 +691,23 @@ Client::doRequestWithRetryNetworkErrors(RequestType & request, RequestFn request return doRequest(request, with_retries); } +template +RequestResult Client::enrichErrorMessage(RequestResult && outcome) const +{ + if (outcome.IsSuccess() || !isClientForDisk()) + return std::forward(outcome); + + String enriched_message = fmt::format( + "{} {}", + outcome.GetError().GetMessage(), + "This error happened for S3 disk."); + + auto error = outcome.GetError(); + error.SetMessage(enriched_message); + + return RequestResult(error); +} + bool Client::supportsMultiPartCopy() const { return provider_type != ProviderType::GCS; diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index c79ec05c8c6..bd281846343 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -214,6 +214,11 @@ public: bool isS3ExpressBucket() const { return client_settings.is_s3express_bucket; } + bool isClientForDisk() const + { + return client_configuration.for_disk_s3; + } + private: friend struct ::MockS3::Client; @@ -265,6 +270,9 @@ private: bool checkIfWrongRegionDefined(const std::string & bucket, const Aws::S3::S3Error & error, std::string & region) const; void insertRegionOverride(const std::string & bucket, const std::string & region) const; + template + RequestResult enrichErrorMessage(RequestResult && outcome) const; + String initial_endpoint; std::shared_ptr credentials_provider; PocoHTTPClientConfiguration client_configuration; diff --git a/src/IO/S3/copyS3File.cpp b/src/IO/S3/copyS3File.cpp index 3b1f25ed994..549d0a569c6 100644 --- a/src/IO/S3/copyS3File.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -140,7 +140,7 @@ namespace fillCreateMultipartRequest(request); ProfileEvents::increment(ProfileEvents::S3CreateMultipartUpload); - if (for_disk_s3) + if (client_ptr->isClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskS3CreateMultipartUpload); auto outcome = client_ptr->CreateMultipartUpload(request); @@ -189,7 +189,7 @@ namespace for (size_t retries = 1;; ++retries) { ProfileEvents::increment(ProfileEvents::S3CompleteMultipartUpload); - if (for_disk_s3) + if (client_ptr->isClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskS3CompleteMultipartUpload); auto outcome = client_ptr->CompleteMultipartUpload(request); @@ -239,7 +239,7 @@ namespace void checkObjectAfterUpload() { LOG_TRACE(log, "Checking object {} exists after upload", dest_key); - S3::checkObjectExists(*client_ptr, dest_bucket, dest_key, {}, request_settings, {}, "Immediately after upload"); + S3::checkObjectExists(*client_ptr, dest_bucket, dest_key, {}, request_settings, "Immediately after upload"); LOG_TRACE(log, "Object {} exists after upload", dest_key); } @@ -528,7 +528,7 @@ namespace for (size_t retries = 1;; ++retries) { ProfileEvents::increment(ProfileEvents::S3PutObject); - if (for_disk_s3) + if (client_ptr->isClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskS3PutObject); Stopwatch watch; @@ -615,7 +615,7 @@ namespace auto & req = typeid_cast(request); ProfileEvents::increment(ProfileEvents::S3UploadPart); - if (for_disk_s3) + if (client_ptr->isClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskS3UploadPart); auto outcome = client_ptr->UploadPart(req); @@ -726,7 +726,7 @@ namespace for (size_t retries = 1;; ++retries) { ProfileEvents::increment(ProfileEvents::S3CopyObject); - if (for_disk_s3) + if (client_ptr->isClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskS3CopyObject); auto outcome = client_ptr->CopyObject(request); @@ -830,7 +830,7 @@ namespace auto & req = typeid_cast(request); ProfileEvents::increment(ProfileEvents::S3UploadPartCopy); - if (for_disk_s3) + if (client_ptr->isClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskS3UploadPartCopy); auto outcome = client_ptr->UploadPartCopy(req); diff --git a/src/IO/S3/getObjectInfo.cpp b/src/IO/S3/getObjectInfo.cpp index 88f79f8d8d5..eee3da9fb74 100644 --- a/src/IO/S3/getObjectInfo.cpp +++ b/src/IO/S3/getObjectInfo.cpp @@ -25,10 +25,10 @@ namespace DB::S3 namespace { Aws::S3::Model::HeadObjectOutcome headObject( - const S3::Client & client, const String & bucket, const String & key, const String & version_id, bool for_disk_s3) + const S3::Client & client, const String & bucket, const String & key, const String & version_id) { ProfileEvents::increment(ProfileEvents::S3HeadObject); - if (for_disk_s3) + if (client.isClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskS3HeadObject); S3::HeadObjectRequest req; @@ -44,9 +44,9 @@ namespace /// Performs a request to get the size and last modification time of an object. std::pair, Aws::S3::S3Error> tryGetObjectInfo( const S3::Client & client, const String & bucket, const String & key, const String & version_id, - const S3Settings::RequestSettings & /*request_settings*/, bool with_metadata, bool for_disk_s3) + const S3Settings::RequestSettings & /*request_settings*/, bool with_metadata) { - auto outcome = headObject(client, bucket, key, version_id, for_disk_s3); + auto outcome = headObject(client, bucket, key, version_id); if (!outcome.IsSuccess()) return {std::nullopt, outcome.GetError()}; @@ -75,10 +75,9 @@ ObjectInfo getObjectInfo( const String & version_id, const S3Settings::RequestSettings & request_settings, bool with_metadata, - bool for_disk_s3, bool throw_on_error) { - auto [object_info, error] = tryGetObjectInfo(client, bucket, key, version_id, request_settings, with_metadata, for_disk_s3); + auto [object_info, error] = tryGetObjectInfo(client, bucket, key, version_id, request_settings, with_metadata); if (object_info) { return *object_info; @@ -98,10 +97,9 @@ size_t getObjectSize( const String & key, const String & version_id, const S3Settings::RequestSettings & request_settings, - bool for_disk_s3, bool throw_on_error) { - return getObjectInfo(client, bucket, key, version_id, request_settings, {}, for_disk_s3, throw_on_error).size; + return getObjectInfo(client, bucket, key, version_id, request_settings, {}, throw_on_error).size; } bool objectExists( @@ -109,10 +107,9 @@ bool objectExists( const String & bucket, const String & key, const String & version_id, - const S3Settings::RequestSettings & request_settings, - bool for_disk_s3) + const S3Settings::RequestSettings & request_settings) { - auto [object_info, error] = tryGetObjectInfo(client, bucket, key, version_id, request_settings, {}, for_disk_s3); + auto [object_info, error] = tryGetObjectInfo(client, bucket, key, version_id, request_settings, {}); if (object_info) return true; @@ -130,10 +127,9 @@ void checkObjectExists( const String & key, const String & version_id, const S3Settings::RequestSettings & request_settings, - bool for_disk_s3, std::string_view description) { - auto [object_info, error] = tryGetObjectInfo(client, bucket, key, version_id, request_settings, {}, for_disk_s3); + auto [object_info, error] = tryGetObjectInfo(client, bucket, key, version_id, request_settings, {}); if (object_info) return; throw S3Exception(error.GetErrorType(), "{}Object {} in bucket {} suddenly disappeared: {}", diff --git a/src/IO/S3/getObjectInfo.h b/src/IO/S3/getObjectInfo.h index a57d807644b..ac8072a4338 100644 --- a/src/IO/S3/getObjectInfo.h +++ b/src/IO/S3/getObjectInfo.h @@ -26,7 +26,6 @@ ObjectInfo getObjectInfo( const String & version_id = {}, const S3Settings::RequestSettings & request_settings = {}, bool with_metadata = false, - bool for_disk_s3 = false, bool throw_on_error = true); size_t getObjectSize( @@ -35,7 +34,6 @@ size_t getObjectSize( const String & key, const String & version_id = {}, const S3Settings::RequestSettings & request_settings = {}, - bool for_disk_s3 = false, bool throw_on_error = true); bool objectExists( @@ -43,8 +41,7 @@ bool objectExists( const String & bucket, const String & key, const String & version_id = {}, - const S3Settings::RequestSettings & request_settings = {}, - bool for_disk_s3 = false); + const S3Settings::RequestSettings & request_settings = {}); /// Throws an exception if a specified object doesn't exist. `description` is used as a part of the error message. void checkObjectExists( @@ -53,7 +50,6 @@ void checkObjectExists( const String & key, const String & version_id = {}, const S3Settings::RequestSettings & request_settings = {}, - bool for_disk_s3 = false, std::string_view description = {}); bool isNotFoundError(Aws::S3::S3Errors error); diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index e41867ce225..3ea372f75d8 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -214,9 +214,9 @@ void WriteBufferFromS3::finalizeImpl() if (request_settings.check_objects_after_upload) { - S3::checkObjectExists(*client_ptr, bucket, key, {}, request_settings, /* for_disk_s3= */ write_settings.for_object_storage, "Immediately after upload"); + S3::checkObjectExists(*client_ptr, bucket, key, {}, request_settings, "Immediately after upload"); - size_t actual_size = S3::getObjectSize(*client_ptr, bucket, key, {}, request_settings, /* for_disk_s3= */ write_settings.for_object_storage); + size_t actual_size = S3::getObjectSize(*client_ptr, bucket, key, {}, request_settings); if (actual_size != total_size) throw Exception( ErrorCodes::S3_ERROR, @@ -390,7 +390,7 @@ void WriteBufferFromS3::createMultipartUpload() client_ptr->setKMSHeaders(req); ProfileEvents::increment(ProfileEvents::S3CreateMultipartUpload); - if (write_settings.for_object_storage) + if (client_ptr->isClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskS3CreateMultipartUpload); Stopwatch watch; @@ -429,7 +429,7 @@ void WriteBufferFromS3::abortMultipartUpload() req.SetUploadId(multipart_upload_id); ProfileEvents::increment(ProfileEvents::S3AbortMultipartUpload); - if (write_settings.for_object_storage) + if (client_ptr->isClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskS3AbortMultipartUpload); Stopwatch watch; @@ -530,7 +530,7 @@ void WriteBufferFromS3::writePart(WriteBufferFromS3::PartData && data) getShortLogDetails(), data_size, part_number); ProfileEvents::increment(ProfileEvents::S3UploadPart); - if (write_settings.for_object_storage) + if (client_ptr->isClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskS3UploadPart); auto & request = std::get<0>(*worker_data); @@ -606,7 +606,7 @@ void WriteBufferFromS3::completeMultipartUpload() for (size_t i = 0; i < max_retry; ++i) { ProfileEvents::increment(ProfileEvents::S3CompleteMultipartUpload); - if (write_settings.for_object_storage) + if (client_ptr->isClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskS3CompleteMultipartUpload); Stopwatch watch; @@ -689,7 +689,7 @@ void WriteBufferFromS3::makeSinglepartUpload(WriteBufferFromS3::PartData && data for (size_t i = 0; i < max_retry; ++i) { ProfileEvents::increment(ProfileEvents::S3PutObject); - if (write_settings.for_object_storage) + if (client_ptr->isClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskS3PutObject); ResourceCost cost = request.GetContentLength(); diff --git a/src/IO/WriteSettings.h b/src/IO/WriteSettings.h index 4da3806e51d..84bb25439b5 100644 --- a/src/IO/WriteSettings.h +++ b/src/IO/WriteSettings.h @@ -25,9 +25,6 @@ struct WriteSettings 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 - bool operator==(const WriteSettings & other) const = default; }; diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h index 5b0d8802657..20e7f4a6c90 100644 --- a/src/Storages/StorageAzureBlob.h +++ b/src/Storages/StorageAzureBlob.h @@ -33,8 +33,6 @@ public: bool update(const ContextPtr & context); - void connect(const ContextPtr & context); - bool withGlobs() const { return blob_path.find_first_of("*?{") != std::string::npos; } bool withWildcard() const diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index c2039c2dd79..3fe0b66a453 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1866,7 +1866,6 @@ namespace configuration.url.version_id, configuration.request_settings, /*with_metadata=*/ false, - /*for_disk_s3=*/ false, /*throw_on_error= */ false).last_modification_time; } diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index 1391f1af6f1..22d6d263d23 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -2,6 +2,8 @@ import logging import pytest +import os +import minio from helpers.cluster import ClickHouseCluster from helpers.mock_servers import start_s3_mock @@ -608,3 +610,68 @@ def test_adaptive_timeouts(cluster, broken_s3, node_name): else: assert s3_use_adaptive_timeouts == "0" assert s3_errors == 0 + + +def test_no_key_found_disk(cluster, broken_s3): + node = cluster.instances["node"] + + node.query( + """ + CREATE TABLE no_key_found_disk ( + id Int64 + ) ENGINE=MergeTree() + ORDER BY id + SETTINGS + storage_policy='s3' + """ + ) + + uuid = node.query( + """ + SELECT uuid + FROM system.tables + WHERE name = 'no_key_found_disk' + """ + ).strip() + assert uuid + + node.query("INSERT INTO no_key_found_disk VALUES (1)") + + data = node.query("SELECT * FROM no_key_found_disk").strip() + + assert data == "1" + + remote_pathes = ( + node.query( + f""" + SELECT remote_path + FROM system.remote_data_paths + WHERE + local_path LIKE '%{uuid}%' + AND local_path LIKE '%.bin%' + ORDER BY ALL + """ + ) + .strip() + .split() + ) + + assert len(remote_pathes) > 0 + + # path_prefix = os.path.join('/', cluster.minio_bucket) + for path in remote_pathes: + # name = os.path.relpath(path, path_prefix) + # assert False, f"deleting full {path} prefix {path_prefix} name {name}" + assert cluster.minio_client.stat_object(cluster.minio_bucket, path).size > 0 + cluster.minio_client.remove_object(cluster.minio_bucket, path) + with pytest.raises(Exception) as exc_info: + size = cluster.minio_client.stat_object(cluster.minio_bucket, path).size + assert size == 0 + assert "code: NoSuchKey" in str(exc_info.value) + + error = node.query_and_get_error("SELECT * FROM no_key_found_disk").strip() + + assert ( + "DB::Exception: The specified key does not exist. This error happened for S3 disk." + in error + ) From 6d31638a69c86f5a6ced107c9c8c7b921f7c1400 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 2 May 2024 11:49:23 +0000 Subject: [PATCH 251/624] CROSS join ignore join_algorithm setting --- src/Planner/PlannerJoins.cpp | 8 ++++++++ .../01010_partial_merge_join_negative.reference | 1 - .../0_stateless/01010_partial_merge_join_negative.sql | 4 ---- .../0_stateless/02815_join_algorithm_setting.sql | 10 ++++++++++ 4 files changed, 18 insertions(+), 5 deletions(-) diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 94ee249106a..10a785c6d25 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -878,6 +878,14 @@ std::shared_ptr chooseJoinAlgorithm(std::shared_ptr & table_jo return std::make_shared(table_join, right_table_expression_header); } + /** We have only one way to execute a CROSS JOIN - with a hash join. + * Therefore, for a query with an explicit CROSS JOIN, it should not fail because of the `join_algorithm` setting. + * If the user expects CROSS JOIN + WHERE to be rewritten to INNER join and to be executed with a specific algorithm, + * then the setting `cross_to_inner_join_rewrite` may be used, and unsupported cases will fail earlier. + */ + if (table_join->kind() == JoinKind::Cross) + return std::make_shared(table_join, right_table_expression_header); + if (!table_join->oneDisjunct() && !table_join->isEnabledAlgorithm(JoinAlgorithm::HASH) && !table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO)) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Only `hash` join supports multiple ORs for keys in JOIN ON section"); diff --git a/tests/queries/0_stateless/01010_partial_merge_join_negative.reference b/tests/queries/0_stateless/01010_partial_merge_join_negative.reference index 26049d060cc..93d0255563a 100644 --- a/tests/queries/0_stateless/01010_partial_merge_join_negative.reference +++ b/tests/queries/0_stateless/01010_partial_merge_join_negative.reference @@ -1,5 +1,4 @@ all -cross any semi anti diff --git a/tests/queries/0_stateless/01010_partial_merge_join_negative.sql b/tests/queries/0_stateless/01010_partial_merge_join_negative.sql index 6b4790064b0..3ae0eee869c 100644 --- a/tests/queries/0_stateless/01010_partial_merge_join_negative.sql +++ b/tests/queries/0_stateless/01010_partial_merge_join_negative.sql @@ -18,10 +18,6 @@ SELECT * FROM t0 INNER JOIN t1 USING x; SELECT * FROM t0 RIGHT JOIN t1 USING x; SELECT * FROM t0 FULL JOIN t1 USING x; -SELECT 'cross'; - -SELECT * FROM t0 CROSS JOIN t1; -- { serverError 48 } - SELECT 'any'; SELECT * FROM t0 ANY LEFT JOIN t1 ON t1.x = t0.x; diff --git a/tests/queries/0_stateless/02815_join_algorithm_setting.sql b/tests/queries/0_stateless/02815_join_algorithm_setting.sql index a4c24bb60f9..c26f82d63bc 100644 --- a/tests/queries/0_stateless/02815_join_algorithm_setting.sql +++ b/tests/queries/0_stateless/02815_join_algorithm_setting.sql @@ -94,3 +94,13 @@ SELECT countIf(explain like '%Algorithm: DirectKeyValueJoin%'), countIf(explain EXPLAIN actions = 1 SELECT s FROM (SELECT toUInt64(9911) id) t1 INNER JOIN dict t2 USING (id) ); + +SET join_algorithm = 'grace_hash'; + +-- Cannot execute the grace hash with OR condition +SELECT * FROM ( SELECT number AS key, number * 10 AS key2 FROM numbers_mt(10) ) AS t1 +JOIN ( SELECT k AS key, k + 100 AS key2 FROM t2 ) AS t2 ON t1.key = t2.key OR t1.key2 = t2.key2; -- { serverError NOT_IMPLEMENTED } + +-- But for CROSS choose `hash` algorithm even though it's not enabled +SELECT * FROM ( SELECT number AS key, number * 10 AS key2 FROM numbers_mt(10) ) AS t1 +CROSS JOIN ( SELECT k AS key, k + 100 AS key2 FROM t2 ) AS t2 FORMAT Null; From 84bed102f149e52da22b4b6ea6416ac6dc005869 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Thu, 2 May 2024 14:39:59 +0200 Subject: [PATCH 252/624] More checks for concurrently deleted files and dirs --- .../System/StorageSystemRemoteDataPaths.cpp | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/src/Storages/System/StorageSystemRemoteDataPaths.cpp b/src/Storages/System/StorageSystemRemoteDataPaths.cpp index 7924ccb35ba..0a6e692f7e2 100644 --- a/src/Storages/System/StorageSystemRemoteDataPaths.cpp +++ b/src/Storages/System/StorageSystemRemoteDataPaths.cpp @@ -245,13 +245,20 @@ bool SystemRemoteDataPathsSource::nextFile() try { const auto & disk = disks[current_disk].second; + + const auto current_path = getCurrentPath(); + + /// Files or directories can disappear due to concurrent operations + if (!disk->exists(current_path)) + continue; + /// Stop if current path is a file - if (disk->isFile(getCurrentPath())) + if (disk->isFile(current_path)) return true; /// If current path is a directory list its contents and step into it std::vector children; - disk->listFiles(getCurrentPath(), children); + disk->listFiles(current_path, children); /// Use current predicate for all children const auto & skip_predicate = getCurrentSkipPredicate(); @@ -271,6 +278,14 @@ bool SystemRemoteDataPathsSource::nextFile() throw; } + catch (const fs::filesystem_error & e) + { + /// Files or directories can disappear due to concurrent operations + if (e.code() == std::errc::no_such_file_or_directory) + continue; + + throw; + } } } From 2c34e4b65051df15e93a74c07e03d48e585587ce Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 2 May 2024 14:42:58 +0200 Subject: [PATCH 253/624] Fix links --- docs/en/getting-started/install.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index dc4d73cd13b..6525c29306a 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -45,7 +45,7 @@ On Linux, macOS and FreeBSD: ``` `clickhouse-local` allows you to process local and remote files using ClickHouse's powerful SQL and without a need for configuration. Table - data is stored in a temporary location, meaning that after a restart of `clickhouse-local` all previously created tables are no longer + data is stored in a temporary location, meaning that after a restart of `clickhouse-local` previously created tables are no longer available. As an alternative, you can start the ClickHouse server with this command ... @@ -54,7 +54,7 @@ On Linux, macOS and FreeBSD: ./clickhouse server ``` - ... and open a new terminal and use `clickhouse-client` to connect to the server: + ... and open a new terminal to connect to the server with `clickhouse-client`: ```bash ./clickhouse client @@ -71,7 +71,7 @@ On Linux, macOS and FreeBSD: Table data is stored in the current directory and still available after a restart of ClickHouse server. If necessary, you can pass `-C config.xml` as an additional command line argument to `./clickhouse server` and provide further configuration in a configuration - file. All available configuration settings are documented [here](../operations/settings.md) and in an [example configuration file + file. All available configuration settings are documented [here](../operations/settings/settings.md) and in an [example configuration file template](https://github.com/ClickHouse/ClickHouse/blob/master/programs/server/config.xml). You are ready to start sending SQL commands to ClickHouse! From f2c916cfe7accfe2246f557342bb48ceb1dd0247 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 2 May 2024 14:59:06 +0200 Subject: [PATCH 254/624] Add more logs --- src/Storages/RabbitMQ/RabbitMQConsumer.cpp | 32 +++++++++++++++------- src/Storages/RabbitMQ/RabbitMQConsumer.h | 2 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 7 +++-- 3 files changed, 28 insertions(+), 13 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQConsumer.cpp b/src/Storages/RabbitMQ/RabbitMQConsumer.cpp index 945e0124420..54fc9b2088a 100644 --- a/src/Storages/RabbitMQ/RabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/RabbitMQConsumer.cpp @@ -94,17 +94,19 @@ void RabbitMQConsumer::subscribe() bool RabbitMQConsumer::ackMessages(const CommitInfo & commit_info) { if (state != State::OK) + { + LOG_TEST(log, "State is {}, will not ack messages", magic_enum::enum_name(state.load(std::memory_order_relaxed))); return false; - - /// Nothing to ack. - if (!commit_info.delivery_tag) - return false; + } /// Do not send ack to server if message's channel is not the same as /// current running channel because delivery tags are scoped per channel, /// so if channel fails, all previous delivery tags become invalid. if (commit_info.channel_id != channel_id) + { + LOG_TEST(log, "Channel ID changed {} -> {}, will not ack messages", commit_info.channel_id, channel_id); return false; + } for (const auto & delivery_tag : commit_info.failed_delivery_tags) { @@ -121,7 +123,7 @@ 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 (consumer_channel->ack(commit_info.delivery_tag, AMQP::multiple)) + if (commit_info.delivery_tag && consumer_channel->ack(commit_info.delivery_tag, AMQP::multiple)) { last_commited_delivery_tag = commit_info.delivery_tag; @@ -132,11 +134,14 @@ bool RabbitMQConsumer::ackMessages(const CommitInfo & commit_info) return true; } - LOG_ERROR( - log, - "Did not commit messages for {}:{}, (current commit point {}:{})", - commit_info.channel_id, commit_info.delivery_tag, - channel_id, last_commited_delivery_tag); + if (commit_info.delivery_tag) + { + LOG_ERROR( + log, + "Did not commit messages for {}:{}, (current commit point {}:{})", + commit_info.channel_id, commit_info.delivery_tag, + channel_id, last_commited_delivery_tag); + } return false; } @@ -144,11 +149,18 @@ bool RabbitMQConsumer::ackMessages(const CommitInfo & commit_info) bool RabbitMQConsumer::nackMessages(const CommitInfo & commit_info) { if (state != State::OK) + { + LOG_TEST(log, "State is {}, will not ack messages", magic_enum::enum_name(state.load(std::memory_order_relaxed))); return false; + } /// Nothing to nack. if (!commit_info.delivery_tag || commit_info.delivery_tag <= last_commited_delivery_tag) + { + LOG_TEST(log, "Delivery tag is {}, last committed delivery tag: {}, Will not ack messages", + commit_info.delivery_tag, last_commited_delivery_tag); return false; + } if (consumer_channel->reject(commit_info.delivery_tag, AMQP::multiple)) { diff --git a/src/Storages/RabbitMQ/RabbitMQConsumer.h b/src/Storages/RabbitMQ/RabbitMQConsumer.h index 470cb0a2458..33d7173c840 100644 --- a/src/Storages/RabbitMQ/RabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/RabbitMQConsumer.h @@ -111,7 +111,7 @@ private: ConcurrentBoundedQueue received; MessageData current; - UInt64 last_commited_delivery_tag; + UInt64 last_commited_delivery_tag = 0; std::condition_variable cv; std::mutex mutex; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 66a216f1808..096fd44a68c 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -1135,7 +1135,10 @@ bool StorageRabbitMQ::tryStreamToViews() if (!connection->isConnected()) { if (shutdown_called) + { + LOG_DEBUG(log, "Shutdown called, quitting"); return false; + } if (connection->reconnect()) { @@ -1192,7 +1195,7 @@ bool StorageRabbitMQ::tryStreamToViews() if (write_failed) { LOG_TRACE(log, "Write failed, reschedule"); - return false; + return true; } if (!hasDependencies(getStorageID())) @@ -1214,7 +1217,7 @@ bool StorageRabbitMQ::tryStreamToViews() startLoop(); } - /// Do not reschedule, do not stop event loop. + /// Reschedule. return true; } From be02a16599367cafdfa912aaf64c6b8992f0d2cc Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 2 May 2024 09:30:46 -0400 Subject: [PATCH 255/624] better --- src/Core/ServerSettings.h | 1 + src/Storages/StorageMaterializedView.cpp | 2 +- .../attach_materialized_views_with_sql_security_none.xml | 3 +++ .../ignore_empty_sql_security_in_create_view_query.xml | 3 --- .../test_sql_security/test_ignore_empty_sql_security.py | 4 ++-- 5 files changed, 7 insertions(+), 6 deletions(-) create mode 100644 tests/integration/test_sql_security/configs/attach_materialized_views_with_sql_security_none.xml delete mode 100644 tests/integration/test_sql_security/configs/ignore_empty_sql_security_in_create_view_query.xml diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index f41c596282f..cd616414016 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -61,6 +61,7 @@ namespace DB M(UInt64, async_insert_threads, 16, "Maximum number of threads to actually parse and insert data in background. Zero means asynchronous mode is disabled", 0) \ M(Bool, async_insert_queue_flush_on_shutdown, true, "If true queue of asynchronous inserts is flushed on graceful shutdown", 0) \ M(Bool, ignore_empty_sql_security_in_create_view_query, true, "If true, ClickHouse doesn't write defaults for empty SQL security statement in CREATE VIEW queries. This setting is only necessary for the migration period and will become obsolete in 24.4", 0) \ + M(Bool, attach_materialized_views_with_sql_security_none, true, "If true, all materialized views loaded without SQL security statement (e.g. `ignore_empty_sql_security_in_create_view_query = false` or when loading old created views) will use SQL security NONE.", 0) \ \ M(UInt64, max_concurrent_queries, 0, "Maximum number of concurrently executed queries. Zero means unlimited.", 0) \ M(UInt64, max_concurrent_insert_queries, 0, "Maximum number of concurrently INSERT queries. Zero means unlimited.", 0) \ diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 24c3778670a..97e2cf6ef4d 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -98,7 +98,7 @@ StorageMaterializedView::StorageMaterializedView( local_context->getGlobalContext()); ASTPtr sql_security = query.sql_security; - if (!sql_security && !getContext()->getServerSettings().ignore_empty_sql_security_in_create_view_query) + if (!sql_security && query.attach && getContext()->getServerSettings().attach_materialized_views_with_sql_security_none) { /// This allows materialized views to be loaded during startup with default SQL security for backward compatibility. /// If ClickHouse loads an old materialized view created without SQL security, it will use the default `SQL SECURITY NONE` diff --git a/tests/integration/test_sql_security/configs/attach_materialized_views_with_sql_security_none.xml b/tests/integration/test_sql_security/configs/attach_materialized_views_with_sql_security_none.xml new file mode 100644 index 00000000000..556d845d43e --- /dev/null +++ b/tests/integration/test_sql_security/configs/attach_materialized_views_with_sql_security_none.xml @@ -0,0 +1,3 @@ + + 0 + diff --git a/tests/integration/test_sql_security/configs/ignore_empty_sql_security_in_create_view_query.xml b/tests/integration/test_sql_security/configs/ignore_empty_sql_security_in_create_view_query.xml deleted file mode 100644 index 99819f58630..00000000000 --- a/tests/integration/test_sql_security/configs/ignore_empty_sql_security_in_create_view_query.xml +++ /dev/null @@ -1,3 +0,0 @@ - - 1 - diff --git a/tests/integration/test_sql_security/test_ignore_empty_sql_security.py b/tests/integration/test_sql_security/test_ignore_empty_sql_security.py index d6349242cfd..c2e167700a2 100644 --- a/tests/integration/test_sql_security/test_ignore_empty_sql_security.py +++ b/tests/integration/test_sql_security/test_ignore_empty_sql_security.py @@ -5,7 +5,7 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node1", - main_configs=["configs/ignore_empty_sql_security_in_create_view_query.xml"], + main_configs=["configs/attach_materialized_views_with_sql_security_none.xml"], stay_alive=True, ) @@ -36,8 +36,8 @@ def test_load_mv_with_security_none(started_cluster: ClickHouseCluster): node.replace_in_config( "/etc/clickhouse-server/config.d/ignore_empty_sql_security_in_create_view_query.xml", - "1", "0", + "1", ) node.restart_clickhouse() From 8f7ccdffcb8e8344e88b7ec941011e7ca9bc93e3 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 2 May 2024 13:49:25 +0000 Subject: [PATCH 256/624] Docs: Add workaround for UUID sorting --- docs/en/sql-reference/data-types/uuid.md | 43 +++++++++++++++++------- 1 file changed, 30 insertions(+), 13 deletions(-) diff --git a/docs/en/sql-reference/data-types/uuid.md b/docs/en/sql-reference/data-types/uuid.md index 59d7b0d7549..175c82f6c09 100644 --- a/docs/en/sql-reference/data-types/uuid.md +++ b/docs/en/sql-reference/data-types/uuid.md @@ -23,8 +23,8 @@ The default UUID is all-zero. It is used, for example, when a new record is inse 00000000-0000-0000-0000-000000000000 ``` -Due to historical reasons, UUIDs are sorted by their second half (which is unintuitive). -UUIDs should therefore not be used in an primary key (or sorting key) of a table, or as partition key. +Due to historical reasons, UUIDs are sorted by their second half. +UUIDs should therefore not be used directly in a primary key, sorting key, or partition key of a table. Example: @@ -43,18 +43,7 @@ Result: │ 3fda7c49-282e-421a-85ab-c5684ef1d350 │ │ 16ab55a7-45f6-44a8-873c-7a0b44346b3e │ │ e3776711-6359-4f22-878d-bf290d052c85 │ -│ 1be30226-57b2-4739-88ec-5e3d490090f2 │ -│ f65853a9-4375-4f0e-8b96-906ff622ed3c │ -│ d5a0c7a6-79c6-4107-8bb8-df85915edcb7 │ -│ 258e6068-17d1-4a1a-8be3-ed2ceb21815c │ -│ 04b0f6a9-1f7b-4a42-8bfc-62f37b8a32b8 │ -│ 9924f0d9-9c16-43a9-8f08-0944ab495aed │ -│ 6720dc14-4eab-4e3e-8f0c-10c4ae8d2673 │ -│ 5ddadb52-0452-4f5d-9030-c3f969af93a4 │ │ [...] │ -│ 2dde30e6-59a1-48f8-b260-eb37921185b6 │ -│ d5402a1b-77b3-4897-b288-29edf5c3ed12 │ -│ 01843939-3ba7-4fea-b2aa-45f9a6f1e057 │ │ 9eceda2f-6946-40e3-b725-16f2709ca41a │ │ 03644f74-47ba-4020-b865-be5fd4c8c7ff │ │ ce3bc93d-ab19-4c74-b8cc-737cb9212099 │ @@ -63,6 +52,34 @@ Result: └──────────────────────────────────────┘ ``` +As a workaround, the UUID can be converted to a type with an intuitive sort order. + +Example using conversion to UInt128: + +``` sql +CREATE TABLE tab (uuid UUID) ENGINE = Memory; +INSERT INTO tab SELECT generateUUIDv4() FROM numbers(50); +SELECT * FROM tab ORDER BY toUInt128(uuid); +``` + +Result: + +```sql +┌─uuid─────────────────────────────────┐ +│ 018b81cd-aca1-4e9c-9e56-a84a074dc1a8 │ +│ 02380033-c96a-438e-913f-a2c67e341def │ +│ 057cf435-7044-456a-893b-9183a4475cea │ +│ 0a3c1d4c-f57d-44cc-8567-60cb0c46f76e │ +│ 0c15bf1c-8633-4414-a084-7017eead9e41 │ +│ [...] │ +│ f808cf05-ea57-4e81-8add-29a195bde63d │ +│ f859fb5d-764b-4a33-81e6-9e4239dae083 │ +│ fb1b7e37-ab7b-421a-910b-80e60e2bf9eb │ +│ fc3174ff-517b-49b5-bfe2-9b369a5c506d │ +│ fece9bf6-3832-449a-b058-cd1d70a02c8b │ +└──────────────────────────────────────┘ +``` + ## Generating UUIDs ClickHouse provides the [generateUUIDv4](../../sql-reference/functions/uuid-functions.md) function to generate random UUID version 4 values. From f685eea7c545b00af1249e62e38d1aa5e03ae13b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 2 May 2024 16:59:19 +0200 Subject: [PATCH 257/624] Get rid of fallbacks, `all` is too far away in 2022 --- tests/ci/download_release_packages.py | 24 ++++-------------------- 1 file changed, 4 insertions(+), 20 deletions(-) diff --git a/tests/ci/download_release_packages.py b/tests/ci/download_release_packages.py index 550301e8fa2..b9376497258 100755 --- a/tests/ci/download_release_packages.py +++ b/tests/ci/download_release_packages.py @@ -1,10 +1,9 @@ #!/usr/bin/env python3 -import os import logging +import os import requests - from requests.adapters import HTTPAdapter # type: ignore from urllib3.util.retry import Retry # type: ignore @@ -19,10 +18,8 @@ CLICKHOUSE_COMMON_STATIC_PACKAGE_NAME = "clickhouse-common-static_{version}_amd6 CLICKHOUSE_COMMON_STATIC_DBG_PACKAGE_NAME = ( "clickhouse-common-static-dbg_{version}_amd64.deb" ) -CLICKHOUSE_SERVER_PACKAGE_NAME = "clickhouse-server_{version}_amd64.deb" -CLICKHOUSE_SERVER_PACKAGE_FALLBACK = "clickhouse-server_{version}_all.deb" CLICKHOUSE_CLIENT_PACKAGE_NAME = "clickhouse-client_{version}_amd64.deb" -CLICKHOUSE_CLIENT_PACKAGE_FALLBACK = "clickhouse-client_{version}_all.deb" +CLICKHOUSE_SERVER_PACKAGE_NAME = "clickhouse-server_{version}_amd64.deb" PACKAGES_DIR = "previous_release_package_folder/" VERSION_PATTERN = r"((?:\d+\.)?(?:\d+\.)?(?:\d+\.)?\d+-[a-zA-Z]*)" @@ -59,26 +56,13 @@ def download_packages(release, dest_path=PACKAGES_DIR): for pkg in ( CLICKHOUSE_COMMON_STATIC_PACKAGE_NAME, CLICKHOUSE_COMMON_STATIC_DBG_PACKAGE_NAME, + CLICKHOUSE_SERVER_PACKAGE_NAME, + CLICKHOUSE_CLIENT_PACKAGE_NAME, ): url = (DOWNLOAD_PREFIX + pkg).format(version=release.version, type=release.type) pkg_name = get_dest_path(pkg.format(version=release.version)) download_package(url, pkg_name) - for pkg, fallback in ( - (CLICKHOUSE_SERVER_PACKAGE_NAME, CLICKHOUSE_SERVER_PACKAGE_FALLBACK), - (CLICKHOUSE_CLIENT_PACKAGE_NAME, CLICKHOUSE_CLIENT_PACKAGE_FALLBACK), - ): - url = (DOWNLOAD_PREFIX + pkg).format(version=release.version, type=release.type) - pkg_name = get_dest_path(pkg.format(version=release.version)) - try: - download_package(url, pkg_name) - except Exception: - url = (DOWNLOAD_PREFIX + fallback).format( - version=release.version, type=release.type - ) - pkg_name = get_dest_path(fallback.format(version=release.version)) - download_package(url, pkg_name) - def download_last_release(dest_path): current_release = get_previous_release(None) From fb9b2be2f9692776e6ad9b57c1f1177d50c41aaa Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 2 May 2024 17:00:48 +0200 Subject: [PATCH 258/624] Leftover to fix after #62114 --- tests/ci/download_release_packages.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/ci/download_release_packages.py b/tests/ci/download_release_packages.py index b9376497258..1ba4ff8ff2e 100755 --- a/tests/ci/download_release_packages.py +++ b/tests/ci/download_release_packages.py @@ -19,6 +19,8 @@ CLICKHOUSE_COMMON_STATIC_DBG_PACKAGE_NAME = ( "clickhouse-common-static-dbg_{version}_amd64.deb" ) CLICKHOUSE_CLIENT_PACKAGE_NAME = "clickhouse-client_{version}_amd64.deb" +CLICKHOUSE_LIBRARY_BRIDGE_PACKAGE_NAME = "clickhouse-library-bridge_{version}_amd64.deb" +CLICKHOUSE_ODBC_BRIDGE_PACKAGE_NAME = "clickhouse-odbc-bridge_{version}_amd64.deb" CLICKHOUSE_SERVER_PACKAGE_NAME = "clickhouse-server_{version}_amd64.deb" PACKAGES_DIR = "previous_release_package_folder/" @@ -56,8 +58,10 @@ def download_packages(release, dest_path=PACKAGES_DIR): for pkg in ( CLICKHOUSE_COMMON_STATIC_PACKAGE_NAME, CLICKHOUSE_COMMON_STATIC_DBG_PACKAGE_NAME, - CLICKHOUSE_SERVER_PACKAGE_NAME, CLICKHOUSE_CLIENT_PACKAGE_NAME, + CLICKHOUSE_LIBRARY_BRIDGE_PACKAGE_NAME, + CLICKHOUSE_ODBC_BRIDGE_PACKAGE_NAME, + CLICKHOUSE_SERVER_PACKAGE_NAME, ): url = (DOWNLOAD_PREFIX + pkg).format(version=release.version, type=release.type) pkg_name = get_dest_path(pkg.format(version=release.version)) From 55e57421fb2791e34dedeaba4c8192c9ecc9a955 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 2 May 2024 17:09:10 +0200 Subject: [PATCH 259/624] Fix SettingsChangesHistory.h for allow_experimental_join_condition --- src/Core/SettingsChangesHistory.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 4df60330c93..64098b20cc6 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -85,6 +85,8 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { + {"24.5", {{"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, + }}, {"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"}, {"ignore_drop_queries_probability", 0, 0, "Allow to ignore drop queries in server with specified probability for testing purposes"}, @@ -96,7 +98,6 @@ static std::map sett {"temporary_data_in_cache_reserve_space_wait_lock_timeout_milliseconds", (10 * 60 * 1000), (10 * 60 * 1000), "Wait time to lock cache for sapce reservation in temporary data in filesystem cache"}, {"optimize_rewrite_sum_if_to_count_if", false, true, "Only available for the analyzer, where it works correctly"}, {"azure_allow_parallel_part_upload", "true", "true", "Use multiple threads for azure multipart upload."}, - {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, {"max_recursive_cte_evaluation_depth", DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, "Maximum limit on recursive CTE evaluation depth"}, {"query_plan_convert_outer_join_to_inner_join", false, true, "Allow to convert OUTER JOIN to INNER JOIN if filter after JOIN always filters default values"}, }}, From 5f68eb847c7b88a118fb40599a2f51f9782a97fd Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 2 May 2024 18:40:31 +0300 Subject: [PATCH 260/624] Added tests --- ...dow_functions_qualify_validation.reference | 0 ...43_window_functions_qualify_validation.sql | 26 +++++++++++++++++++ 2 files changed, 26 insertions(+) create mode 100644 tests/queries/0_stateless/03143_window_functions_qualify_validation.reference create mode 100644 tests/queries/0_stateless/03143_window_functions_qualify_validation.sql diff --git a/tests/queries/0_stateless/03143_window_functions_qualify_validation.reference b/tests/queries/0_stateless/03143_window_functions_qualify_validation.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03143_window_functions_qualify_validation.sql b/tests/queries/0_stateless/03143_window_functions_qualify_validation.sql new file mode 100644 index 00000000000..2b6d1820b00 --- /dev/null +++ b/tests/queries/0_stateless/03143_window_functions_qualify_validation.sql @@ -0,0 +1,26 @@ +DROP TABLE IF EXISTS uk_price_paid; +CREATE TABLE uk_price_paid +( + `price` UInt32, + `date` Date, + `postcode1` LowCardinality(String), + `postcode2` LowCardinality(String), + `type` Enum8('terraced' = 1, 'semi-detached' = 2, 'detached' = 3, 'flat' = 4, 'other' = 0), + `is_new` UInt8, + `duration` Enum8('freehold' = 1, 'leasehold' = 2, 'unknown' = 0), + `addr1` String, + `addr2` String, + `street` LowCardinality(String), + `locality` LowCardinality(String), + `town` LowCardinality(String), + `district` LowCardinality(String), + `county` LowCardinality(String) +) +ENGINE = MergeTree +ORDER BY (postcode1, postcode2, addr1, addr2); + +SELECT count(), (quantile(0.9)(price) OVER ()) AS price_quantile FROM uk_price_paid WHERE toYear(date) = 2023 QUALIFY price > price_quantile; -- { serverError 215 } + +SELECT count() FROM uk_price_paid WHERE toYear(date) = 2023 QUALIFY price > (quantile(0.9)(price) OVER ()); -- { serverError 215 } + +DROP TABLE uk_price_paid; From e013b9970990c358b6f080cf0362d5d6aa2dfd87 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 2 May 2024 16:21:26 +0000 Subject: [PATCH 261/624] Always try to backport to the cloud (v24.3-must-backport) --- tests/ci/cherry_pick.py | 24 ++++++++++++++++-------- 1 file changed, 16 insertions(+), 8 deletions(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index 96906d55899..4a037398909 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -402,11 +402,18 @@ class Backport: logging.info("Getting release PRs") self.release_prs = self.gh.get_release_pulls(self._repo_name) self.release_branches = [pr.head.ref for pr in self.release_prs] - self.labels_to_backport = [ - f"v{branch}-must-backport" for branch in self.release_branches - ] - if self._fetch_from: + if not self._fetch_from: + self.labels_to_backport = [ + f"v{branch}-must-backport" for branch in self.release_branches + ] + else: + fetch_release_prs = self.gh.get_release_pulls(self._fetch_from) + fetch_release_branches = [pr.head.ref for pr in fetch_release_prs] + self.labels_to_backport = [ + f"v{branch}-must-backport" for branch in fetch_release_branches + ] + logging.info("Fetching from %s", self._fetch_from) fetch_from_repo = self.gh.get_repo(self._fetch_from) git_runner( @@ -445,7 +452,7 @@ class Backport: ) - timedelta(days=reserve_search_days) # To not have a possible TZ issues tomorrow = date.today() + timedelta(days=1) - logging.info("Receive PRs suppose to be backported") + logging.info("Receive PRs supposed to be backported") query_args = { "query": f"type:pr repo:{self._fetch_from} -label:{self.backport_created_label}", @@ -473,7 +480,8 @@ class Backport: def process_pr(self, pr: PullRequest) -> None: pr_labels = [label.name for label in pr.labels] - if self.must_create_backport_label in pr_labels: + # We backport any vXXX-must-backport to all branches of the fetch repo (better than no backport) + if self.must_create_backport_label in pr_labels or self._fetch_from: branches = [ ReleaseBranch(br, pr, self.repo, self.backport_created_label) for br in self.release_branches @@ -496,7 +504,7 @@ class Backport: raise self.error logging.info( - " PR #%s is suppose to be backported to %s", + " PR #%s is supposed to be backported to %s", pr.number, ", ".join(map(str, branches)), ) @@ -657,7 +665,7 @@ def main(): bp.receive_prs_for_backport(args.reserve_search_days) bp.process_backports() if bp.error is not None: - logging.error("Finished successfully, but errors occured!") + logging.error("Finished successfully, but errors occurred!") raise bp.error From 7baac14e3862ff316aecbac3822d771a759457d0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 2 May 2024 16:30:11 +0000 Subject: [PATCH 262/624] Create cloud backports for all pr-must-backport PRs Even if pr-must-backport-cloud is not declared --- tests/ci/cherry_pick.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index 4a037398909..9e49df2504c 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -414,6 +414,9 @@ class Backport: f"v{branch}-must-backport" for branch in fetch_release_branches ] + # Forcefully add also non cloud backports + self.labels_to_backport += [Labels.MUST_BACKPORT] + logging.info("Fetching from %s", self._fetch_from) fetch_from_repo = self.gh.get_repo(self._fetch_from) git_runner( From 245fac10aa75e3ffe80f95809e4a741d24525336 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 2 May 2024 17:10:56 +0000 Subject: [PATCH 263/624] Change must-create-backport-label to support a list python3 cherry_pick.py --repo ClickHouse/clickhouse-private --from-repo ClickHouse/ClickHouse --must-create-backport-label pr-must-backport-cloud pr-must-backport --backport-created-label pr-backports-created-cloud --- tests/ci/cherry_pick.py | 115 +++++++++++++++++++++------------------- 1 file changed, 60 insertions(+), 55 deletions(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index 9e49df2504c..71cb7ee6ba9 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -348,7 +348,7 @@ class Backport: repo: str, fetch_from: Optional[str], dry_run: bool, - must_create_backport_label: str, + must_create_backport_labels: List[str], backport_created_label: str, ): self.gh = gh @@ -356,7 +356,7 @@ class Backport: self._fetch_from = fetch_from self.dry_run = dry_run - self.must_create_backport_label = must_create_backport_label + self.must_create_backport_labels = must_create_backport_labels self.backport_created_label = backport_created_label self._remote = "" @@ -414,9 +414,6 @@ class Backport: f"v{branch}-must-backport" for branch in fetch_release_branches ] - # Forcefully add also non cloud backports - self.labels_to_backport += [Labels.MUST_BACKPORT] - logging.info("Fetching from %s", self._fetch_from) fetch_from_repo = self.gh.get_repo(self._fetch_from) git_runner( @@ -460,7 +457,7 @@ class Backport: query_args = { "query": f"type:pr repo:{self._fetch_from} -label:{self.backport_created_label}", "label": ",".join( - self.labels_to_backport + [self.must_create_backport_label] + self.labels_to_backport + self.must_create_backport_labels ), "merged": [since_date, tomorrow], } @@ -483,13 +480,18 @@ class Backport: def process_pr(self, pr: PullRequest) -> None: pr_labels = [label.name for label in pr.labels] - # We backport any vXXX-must-backport to all branches of the fetch repo (better than no backport) - if self.must_create_backport_label in pr_labels or self._fetch_from: - branches = [ - ReleaseBranch(br, pr, self.repo, self.backport_created_label) - for br in self.release_branches - ] # type: List[ReleaseBranch] - else: + + for label in self.must_create_backport_labels: + # We backport any vXXX-must-backport to all branches of the fetch repo (better than no backport) + if label in pr_labels or self._fetch_from: + branches = [ + ReleaseBranch(br, pr, self.repo, self.backport_created_label) + for br in self.release_branches + ] # type: List[ReleaseBranch] + found = True + break + + if not branches: branches = [ ReleaseBranch(br, pr, self.repo, self.backport_created_label) for br in [ @@ -498,13 +500,13 @@ class Backport: if label in self.labels_to_backport ] ] - if not branches: - # This is definitely some error. There must be at least one branch - # It also make the whole program exit code non-zero - self.error = Exception( - f"There are no branches to backport PR #{pr.number}, logical error" - ) - raise self.error + if not branches: + # This is definitely some error. There must be at least one branch + # It also make the whole program exit code non-zero + self.error = Exception( + f"There are no branches to backport PR #{pr.number}, logical error" + ) + raise self.error logging.info( " PR #%s is supposed to be backported to %s", @@ -512,40 +514,40 @@ class Backport: ", ".join(map(str, branches)), ) # All PRs for cherrypick and backport branches as heads - query_suffix = " ".join( - [ - f"head:{branch.backport_branch} head:{branch.cherrypick_branch}" - for branch in branches - ] - ) - bp_cp_prs = self.gh.get_pulls_from_search( - query=f"type:pr repo:{self._repo_name} {query_suffix}", - label=f"{Labels.PR_BACKPORT},{Labels.PR_CHERRYPICK}", - ) - for br in branches: - br.pop_prs(bp_cp_prs) - - if bp_cp_prs: - # This is definitely some error. All prs must be consumed by - # branches with ReleaseBranch.pop_prs. It also makes the whole - # program exit code non-zero - self.error = Exception( - "The following PRs are not filtered by release branches:\n" - "\n".join(map(str, bp_cp_prs)) - ) - raise self.error - - if all(br.backported for br in branches): - # Let's check if the PR is already backported - self.mark_pr_backported(pr) - return - - for br in branches: - br.process(self.dry_run) - - if all(br.backported for br in branches): - # And check it after the running - self.mark_pr_backported(pr) + # query_suffix = " ".join( + # [ + # f"head:{branch.backport_branch} head:{branch.cherrypick_branch}" + # for branch in branches + # ] + # ) + # bp_cp_prs = self.gh.get_pulls_from_search( + # query=f"type:pr repo:{self._repo_name} {query_suffix}", + # label=f"{Labels.PR_BACKPORT},{Labels.PR_CHERRYPICK}", + # ) + # for br in branches: + # br.pop_prs(bp_cp_prs) + # + # if bp_cp_prs: + # # This is definitely some error. All prs must be consumed by + # # branches with ReleaseBranch.pop_prs. It also makes the whole + # # program exit code non-zero + # self.error = Exception( + # "The following PRs are not filtered by release branches:\n" + # "\n".join(map(str, bp_cp_prs)) + # ) + # raise self.error + # + # if all(br.backported for br in branches): + # # Let's check if the PR is already backported + # self.mark_pr_backported(pr) + # return + # + # for br in branches: + # br.process(self.dry_run) + # + # if all(br.backported for br in branches): + # # And check it after the running + # self.mark_pr_backported(pr) def mark_pr_backported(self, pr: PullRequest) -> None: if self.dry_run: @@ -587,6 +589,7 @@ def parse_args(): default=Labels.MUST_BACKPORT, choices=(Labels.MUST_BACKPORT, Labels.MUST_BACKPORT_CLOUD), help="label to filter PRs to backport", + nargs="+", ) parser.add_argument( "--backport-created-label", @@ -658,7 +661,9 @@ def main(): args.repo, args.from_repo, args.dry_run, - args.must_create_backport_label, + args.must_create_backport_label + if isinstance(args.must_create_backport_label, list) + else [args.must_create_backport_label], args.backport_created_label, ) # https://github.com/python/mypy/issues/3004 From 713467e812d50eee0f80a679e5a00d0621b63f68 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 2 May 2024 17:20:58 +0000 Subject: [PATCH 264/624] Fix dev --- tests/ci/cherry_pick.py | 68 ++++++++++++++++++++--------------------- 1 file changed, 34 insertions(+), 34 deletions(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index 71cb7ee6ba9..74eb983b820 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -514,40 +514,40 @@ class Backport: ", ".join(map(str, branches)), ) # All PRs for cherrypick and backport branches as heads - # query_suffix = " ".join( - # [ - # f"head:{branch.backport_branch} head:{branch.cherrypick_branch}" - # for branch in branches - # ] - # ) - # bp_cp_prs = self.gh.get_pulls_from_search( - # query=f"type:pr repo:{self._repo_name} {query_suffix}", - # label=f"{Labels.PR_BACKPORT},{Labels.PR_CHERRYPICK}", - # ) - # for br in branches: - # br.pop_prs(bp_cp_prs) - # - # if bp_cp_prs: - # # This is definitely some error. All prs must be consumed by - # # branches with ReleaseBranch.pop_prs. It also makes the whole - # # program exit code non-zero - # self.error = Exception( - # "The following PRs are not filtered by release branches:\n" - # "\n".join(map(str, bp_cp_prs)) - # ) - # raise self.error - # - # if all(br.backported for br in branches): - # # Let's check if the PR is already backported - # self.mark_pr_backported(pr) - # return - # - # for br in branches: - # br.process(self.dry_run) - # - # if all(br.backported for br in branches): - # # And check it after the running - # self.mark_pr_backported(pr) + query_suffix = " ".join( + [ + f"head:{branch.backport_branch} head:{branch.cherrypick_branch}" + for branch in branches + ] + ) + bp_cp_prs = self.gh.get_pulls_from_search( + query=f"type:pr repo:{self._repo_name} {query_suffix}", + label=f"{Labels.PR_BACKPORT},{Labels.PR_CHERRYPICK}", + ) + for br in branches: + br.pop_prs(bp_cp_prs) + + if bp_cp_prs: + # This is definitely some error. All prs must be consumed by + # branches with ReleaseBranch.pop_prs. It also makes the whole + # program exit code non-zero + self.error = Exception( + "The following PRs are not filtered by release branches:\n" + "\n".join(map(str, bp_cp_prs)) + ) + raise self.error + + if all(br.backported for br in branches): + # Let's check if the PR is already backported + self.mark_pr_backported(pr) + return + + for br in branches: + br.process(self.dry_run) + + if all(br.backported for br in branches): + # And check it after the running + self.mark_pr_backported(pr) def mark_pr_backported(self, pr: PullRequest) -> None: if self.dry_run: From e412d161a6fce182c78375053334eadf4d1f3ea9 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 2 May 2024 19:26:02 +0200 Subject: [PATCH 265/624] More logging --- src/Storages/RabbitMQ/RabbitMQSource.cpp | 7 +++++-- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 8 ++++++-- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQSource.cpp b/src/Storages/RabbitMQ/RabbitMQSource.cpp index e41edb60a63..bb077e2ea50 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSource.cpp @@ -123,7 +123,10 @@ Chunk RabbitMQSource::generate() { auto chunk = generateImpl(); if (!chunk && ack_in_suffix) + { + LOG_TEST(log, "Will send ack on select"); sendAck(); + } return chunk; } @@ -204,9 +207,9 @@ Chunk RabbitMQSource::generateImpl() const auto exchange_name = storage.getExchange(); const auto & message = consumer->currentMessage(); - LOG_TEST(log, "Pulled {} rows, message delivery tag: {}, " + LOG_TEST(log, "Pulled {} rows, message delivery tag: {} on channel {}, " "previous delivery tag: {}, redelivered: {}, failed delivery tags by this moment: {}, exception message: {}", - new_rows, message.delivery_tag, commit_info.delivery_tag, message.redelivered, + new_rows, message.delivery_tag, consumer->getChannelID(), commit_info.delivery_tag, message.redelivered, commit_info.failed_delivery_tags.size(), exception_message.has_value() ? exception_message.value() : "None"); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 096fd44a68c..7277d31ae1a 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -745,8 +745,9 @@ void StorageRabbitMQ::read( for (size_t i = 0; i < num_created_consumers; ++i) { auto rabbit_source = std::make_shared( - *this, storage_snapshot, modified_context, column_names, 1, - max_execution_time_ms, rabbitmq_settings->rabbitmq_handle_error_mode, reject_unhandled_messages, rabbitmq_settings->rabbitmq_commit_on_select); + *this, storage_snapshot, modified_context, column_names, /* max_block_size */1, + 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(), @@ -1154,6 +1155,8 @@ bool StorageRabbitMQ::tryStreamToViews() } else { + LOG_TEST(log, "Will {} messages for {} channels", write_failed ? "nack" : "ack", sources.size()); + /// Commit for (auto & source : sources) { @@ -1214,6 +1217,7 @@ bool StorageRabbitMQ::tryStreamToViews() } else { + LOG_TEST(log, "Will start background loop to let messages be pushed to channel"); startLoop(); } From 5cc9a77a896552fee0d5d68b92d79f6656dc4f40 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 2 May 2024 20:07:43 +0200 Subject: [PATCH 266/624] Remove delete_all_files to fix flaky test --- .../test.py | 25 +------------------ 1 file changed, 1 insertion(+), 24 deletions(-) diff --git a/tests/integration/test_backup_restore_azure_blob_storage/test.py b/tests/integration/test_backup_restore_azure_blob_storage/test.py index bfc7044e41a..ca1c0e907c3 100644 --- a/tests/integration/test_backup_restore_azure_blob_storage/test.py +++ b/tests/integration/test_backup_restore_azure_blob_storage/test.py @@ -162,30 +162,7 @@ def put_azure_file_content(filename, port, data): blob_client = container_client.get_blob_client(filename) buf = io.BytesIO(data) - blob_client.upload_blob(buf) - - -@pytest.fixture(autouse=True, scope="function") -def delete_all_files(cluster): - 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) - blob_list = container_client.list_blobs() - for blob in blob_list: - print(blob) - blob_client = container_client.get_blob_client(blob) - blob_client.delete_blob() - - assert len(list(container_client.list_blobs())) == 0 - - yield + blob_client.upload_blob(buf)g def test_backup_restore(cluster): From bb26856dd4699c4b510f1ecd08904d50fe9caaa3 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 2 May 2024 18:07:48 +0000 Subject: [PATCH 267/624] Update version_date.tsv and changelogs after v23.8.14.6-lts --- docs/changelogs/v23.8.14.6-lts.md | 14 ++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 15 insertions(+) create mode 100644 docs/changelogs/v23.8.14.6-lts.md diff --git a/docs/changelogs/v23.8.14.6-lts.md b/docs/changelogs/v23.8.14.6-lts.md new file mode 100644 index 00000000000..0053502a9dc --- /dev/null +++ b/docs/changelogs/v23.8.14.6-lts.md @@ -0,0 +1,14 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v23.8.14.6-lts (967e51c1d6b) FIXME as compared to v23.8.13.25-lts (37e034f903e) + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Set server name for SSL handshake in MongoDB engine [#63122](https://github.com/ClickHouse/ClickHouse/pull/63122) ([Alexander Gololobov](https://github.com/davenger)). +* Use user specified db instead of "config" for MongoDB wire protocol version check [#63126](https://github.com/ClickHouse/ClickHouse/pull/63126) ([Alexander Gololobov](https://github.com/davenger)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 2389d661786..1f47a999162 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -36,6 +36,7 @@ v23.9.4.11-stable 2023-11-08 v23.9.3.12-stable 2023-10-31 v23.9.2.56-stable 2023-10-19 v23.9.1.1854-stable 2023-09-29 +v23.8.14.6-lts 2024-05-02 v23.8.13.25-lts 2024-04-26 v23.8.12.13-lts 2024-03-26 v23.8.11.28-lts 2024-03-15 From 752c31722a516166ebe529e1089b7ab7384ba505 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 2 May 2024 20:10:37 +0200 Subject: [PATCH 268/624] Fix typo --- .../integration/test_backup_restore_azure_blob_storage/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_backup_restore_azure_blob_storage/test.py b/tests/integration/test_backup_restore_azure_blob_storage/test.py index ca1c0e907c3..1a1458cb68e 100644 --- a/tests/integration/test_backup_restore_azure_blob_storage/test.py +++ b/tests/integration/test_backup_restore_azure_blob_storage/test.py @@ -162,7 +162,7 @@ def put_azure_file_content(filename, port, data): blob_client = container_client.get_blob_client(filename) buf = io.BytesIO(data) - blob_client.upload_blob(buf)g + blob_client.upload_blob(buf) def test_backup_restore(cluster): From a7935021f7355454a34ced2c234e26d66a1455b1 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 2 May 2024 19:45:39 +0100 Subject: [PATCH 269/624] impl --- src/Dictionaries/CacheDictionaryUpdateQueue.cpp | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/src/Dictionaries/CacheDictionaryUpdateQueue.cpp b/src/Dictionaries/CacheDictionaryUpdateQueue.cpp index 1e9b1da390a..15c3ae8e345 100644 --- a/src/Dictionaries/CacheDictionaryUpdateQueue.cpp +++ b/src/Dictionaries/CacheDictionaryUpdateQueue.cpp @@ -36,8 +36,16 @@ CacheDictionaryUpdateQueue::CacheDictionaryUpdateQueue( , update_queue(configuration.max_update_queue_size) , update_pool(CurrentMetrics::CacheDictionaryThreads, CurrentMetrics::CacheDictionaryThreadsActive, CurrentMetrics::CacheDictionaryThreadsScheduled, configuration.max_threads_for_updates) { - for (size_t i = 0; i < configuration.max_threads_for_updates; ++i) - update_pool.scheduleOrThrowOnError([this] { updateThreadFunction(); }); + try + { + for (size_t i = 0; i < configuration.max_threads_for_updates; ++i) + update_pool.scheduleOrThrowOnError([this] { updateThreadFunction(); }); + } + catch (...) + { + stopAndWait(); + throw; + } } template From 2ca0251de43838d67241c987089b9bde72babadc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 2 May 2024 21:09:53 +0200 Subject: [PATCH 270/624] 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, 2 insertions(+), 10 deletions(-) delete mode 100644 tests/queries/0_stateless/03095_group_by_server_constants_bug.reference delete 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 d7fa270a643..ad8db83d66c 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -85,8 +85,6 @@ 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()) @@ -102,7 +100,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_secondary_query && is_constant_key && !aggregates_descriptions.empty()) + if (is_constant_key && !aggregates_descriptions.empty()) continue; auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions, grouping_set_key_node); @@ -154,7 +152,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_secondary_query && is_constant_key && !aggregates_descriptions.empty()) + if (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 deleted file mode 100644 index 80ab3c879bb..00000000000 --- a/tests/queries/0_stateless/03095_group_by_server_constants_bug.reference +++ /dev/null @@ -1 +0,0 @@ -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 deleted file mode 100644 index 9f9fda1ef62..00000000000 --- a/tests/queries/0_stateless/03095_group_by_server_constants_bug.sql +++ /dev/null @@ -1,5 +0,0 @@ -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 54cd5487bfb56f57df71ff1a753028eb74beb3d1 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 2 May 2024 20:17:06 +0100 Subject: [PATCH 271/624] impl --- .../MergeTree/MergeTreeDeduplicationLog.cpp | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp index 548b61ce422..22ff9b7194f 100644 --- a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp +++ b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp @@ -341,21 +341,21 @@ void MergeTreeDeduplicationLog::shutdown() stopped = true; if (current_writer) { - current_writer->finalize(); + try + { + current_writer->finalize(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } current_writer.reset(); } } MergeTreeDeduplicationLog::~MergeTreeDeduplicationLog() { - try - { - shutdown(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } + shutdown(); } } From a30d92367272756325b6747ddad78037f775a7bd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 2 May 2024 21:42:49 +0200 Subject: [PATCH 272/624] Update cherry_pick.py --- tests/ci/cherry_pick.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index 74eb983b820..7f267d5ed1a 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -488,7 +488,6 @@ class Backport: ReleaseBranch(br, pr, self.repo, self.backport_created_label) for br in self.release_branches ] # type: List[ReleaseBranch] - found = True break if not branches: From e8a156a39118f0d33254308a4067d752e1c8ff6b Mon Sep 17 00:00:00 2001 From: Xiaofei Hu <10776938+litlig@users.noreply.github.com> Date: Thu, 2 May 2024 14:45:24 -0700 Subject: [PATCH 273/624] remove whitespace --- src/Storages/System/StorageSystemPartsColumns.cpp | 2 +- .../0_stateless/02117_show_create_table_system.reference | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/System/StorageSystemPartsColumns.cpp b/src/Storages/System/StorageSystemPartsColumns.cpp index e09ce02faea..0a8fce68a7f 100644 --- a/src/Storages/System/StorageSystemPartsColumns.cpp +++ b/src/Storages/System/StorageSystemPartsColumns.cpp @@ -254,7 +254,7 @@ void StorageSystemPartsColumns::processNextStorage( } bool column_has_ttl = part->ttl_infos.columns_ttl.contains(column.name); if (columns_mask[src_index++]) - { + { if (column_has_ttl) columns[res_index++]->insert(static_cast(part->ttl_infos.columns_ttl[column.name].min)); else diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 8ba7dbe58e5..8b3e0428546 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -566,7 +566,7 @@ CREATE TABLE system.parts_columns `column_marks_bytes` UInt64, `column_modification_time` Nullable(DateTime), `column_ttl_info.min` Nullable(DateTime), - `column_ttl_info.max` Nullable(DateTime), + `column_ttl_info.max` Nullable(DateTime), `serialization_kind` String, `substreams` Array(String), `filenames` Array(String), From 426f5a0434d8f042747a6daca80a58a25ece6ed2 Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 2 May 2024 18:56:54 -0400 Subject: [PATCH 274/624] Use INVOKER as a default for old MV --- src/Core/ServerSettings.h | 1 - .../Transforms/buildPushingToViewsChain.cpp | 3 +- src/Storages/StorageMaterializedView.cpp | 13 +++--- .../integration/test_sql_security/__init__.py | 0 ...erialized_views_with_sql_security_none.xml | 3 -- .../test_ignore_empty_sql_security.py | 45 ------------------- ...te_view_with_sql_security_option.reference | 2 + ...84_create_view_with_sql_security_option.sh | 39 ++++++++++++++++ 8 files changed, 48 insertions(+), 58 deletions(-) delete mode 100644 tests/integration/test_sql_security/__init__.py delete mode 100644 tests/integration/test_sql_security/configs/attach_materialized_views_with_sql_security_none.xml delete mode 100644 tests/integration/test_sql_security/test_ignore_empty_sql_security.py diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index cd616414016..f41c596282f 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -61,7 +61,6 @@ namespace DB M(UInt64, async_insert_threads, 16, "Maximum number of threads to actually parse and insert data in background. Zero means asynchronous mode is disabled", 0) \ M(Bool, async_insert_queue_flush_on_shutdown, true, "If true queue of asynchronous inserts is flushed on graceful shutdown", 0) \ M(Bool, ignore_empty_sql_security_in_create_view_query, true, "If true, ClickHouse doesn't write defaults for empty SQL security statement in CREATE VIEW queries. This setting is only necessary for the migration period and will become obsolete in 24.4", 0) \ - M(Bool, attach_materialized_views_with_sql_security_none, true, "If true, all materialized views loaded without SQL security statement (e.g. `ignore_empty_sql_security_in_create_view_query = false` or when loading old created views) will use SQL security NONE.", 0) \ \ M(UInt64, max_concurrent_queries, 0, "Maximum number of concurrently executed queries. Zero means unlimited.", 0) \ M(UInt64, max_concurrent_insert_queries, 0, "Maximum number of concurrently INSERT queries. Zero means unlimited.", 0) \ diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index aa5a1c0cc1a..91e43dc8134 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -361,7 +361,8 @@ std::optional generateViewChain( } InterpreterInsertQuery interpreter(nullptr, insert_context, false, false, false); - out = interpreter.buildChain(inner_table, inner_metadata_snapshot, insert_columns, thread_status_holder, view_counter_ms, !materialized_view->hasInnerTable()); + bool check_access = !materialized_view->hasInnerTable() && materialized_view->getInMemoryMetadataPtr()->sql_security_type != SQLSecurityType::INVOKER; + out = interpreter.buildChain(inner_table, inner_metadata_snapshot, insert_columns, thread_status_holder, view_counter_ms, check_access); if (interpreter.shouldAddSquashingFroStorage(inner_table)) { diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 97e2cf6ef4d..4c3f90e23ed 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -98,20 +98,17 @@ StorageMaterializedView::StorageMaterializedView( local_context->getGlobalContext()); ASTPtr sql_security = query.sql_security; - if (!sql_security && query.attach && getContext()->getServerSettings().attach_materialized_views_with_sql_security_none) + if (!sql_security) { /// This allows materialized views to be loaded during startup with default SQL security for backward compatibility. - /// If ClickHouse loads an old materialized view created without SQL security, it will use the default `SQL SECURITY NONE` + /// If ClickHouse loads an old materialized view created without SQL security, it will use the default `SQL SECURITY INVOKER` sql_security = std::make_shared(); - InterpreterCreateQuery::processSQLSecurityOption(getContext(), sql_security->as(), true, true); + sql_security->as().type = SQLSecurityType::INVOKER; } if (sql_security) storage_metadata.setSQLSecurity(sql_security->as()); - if (storage_metadata.sql_security_type == SQLSecurityType::INVOKER) - throw Exception(ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW, "SQL SECURITY INVOKER can't be specified for MATERIALIZED VIEW"); - if (!query.select) throw Exception(ErrorCodes::INCORRECT_QUERY, "SELECT query is not specified for {}", getName()); @@ -229,7 +226,7 @@ void StorageMaterializedView::read( auto storage_id = storage->getStorageID(); /// We don't need to check access if the inner table was created automatically. - if (!has_inner_table && !storage_id.empty()) + if (!has_inner_table && !storage_id.empty() && getInMemoryMetadataPtr()->sql_security_type != SQLSecurityType::INVOKER) context->checkAccess(AccessType::SELECT, storage_id, column_names); storage->read(query_plan, column_names, target_storage_snapshot, query_info, context, processed_stage, max_block_size, num_streams); @@ -278,7 +275,7 @@ SinkToStoragePtr StorageMaterializedView::write(const ASTPtr & query, const Stor auto storage_id = storage->getStorageID(); /// We don't need to check access if the inner table was created automatically. - if (!has_inner_table && !storage_id.empty()) + if (!has_inner_table && !storage_id.empty() && getInMemoryMetadataPtr()->sql_security_type != SQLSecurityType::INVOKER) { auto query_sample_block = InterpreterInsertQuery::getSampleBlock(query->as(), storage, metadata_snapshot, context); context->checkAccess(AccessType::INSERT, storage_id, query_sample_block.getNames()); diff --git a/tests/integration/test_sql_security/__init__.py b/tests/integration/test_sql_security/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_sql_security/configs/attach_materialized_views_with_sql_security_none.xml b/tests/integration/test_sql_security/configs/attach_materialized_views_with_sql_security_none.xml deleted file mode 100644 index 556d845d43e..00000000000 --- a/tests/integration/test_sql_security/configs/attach_materialized_views_with_sql_security_none.xml +++ /dev/null @@ -1,3 +0,0 @@ - - 0 - diff --git a/tests/integration/test_sql_security/test_ignore_empty_sql_security.py b/tests/integration/test_sql_security/test_ignore_empty_sql_security.py deleted file mode 100644 index c2e167700a2..00000000000 --- a/tests/integration/test_sql_security/test_ignore_empty_sql_security.py +++ /dev/null @@ -1,45 +0,0 @@ -#!/usr/bin/env python3 -import pytest -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) -node = cluster.add_instance( - "node1", - main_configs=["configs/attach_materialized_views_with_sql_security_none.xml"], - stay_alive=True, -) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - - yield cluster - - finally: - cluster.shutdown() - - -def test_load_mv_with_security_none(started_cluster: ClickHouseCluster): - node.query("CREATE TABLE test_table (s String) ENGINE = MergeTree ORDER BY s") - node.query( - "CREATE MATERIALIZED VIEW test_mv_1 (s String) ENGINE = MergeTree ORDER BY s AS SELECT * FROM test_table" - ) - node.query("INSERT INTO test_table VALUES ('foo'), ('bar')") - - node.query("CREATE USER test_user") - node.query("GRANT SELECT ON test_mv_1 TO test_user") - - with pytest.raises(Exception, match="Not enough privileges"): - node.query("SELECT count() FROM test_mv_1", user="test_user") - - node.replace_in_config( - "/etc/clickhouse-server/config.d/ignore_empty_sql_security_in_create_view_query.xml", - "0", - "1", - ) - - node.restart_clickhouse() - - assert node.query("SELECT count() FROM test_mv_1", user="test_user") == "2\n" diff --git a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference index 6d9d1f07ec2..9ba927fa201 100644 --- a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference +++ b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference @@ -32,3 +32,5 @@ OK 2 2 6 6 9 9 +1000 +1000 diff --git a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh index bead7db8450..48d9a3e220b 100755 --- a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh +++ b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh @@ -222,4 +222,43 @@ EOF ${CLICKHOUSE_CLIENT} --user $user2 --query "SELECT * FROM $db.test_mv_row_2" +${CLICKHOUSE_CLIENT} --multiquery < Date: Fri, 3 May 2024 02:28:21 +0200 Subject: [PATCH 275/624] Fix a test --- tests/queries/0_stateless/02784_connection_string.reference | 2 +- tests/queries/0_stateless/02784_connection_string.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02784_connection_string.reference b/tests/queries/0_stateless/02784_connection_string.reference index 9d58d485a14..2f4e5efa53f 100644 --- a/tests/queries/0_stateless/02784_connection_string.reference +++ b/tests/queries/0_stateless/02784_connection_string.reference @@ -101,7 +101,7 @@ BAD_ARGUMENTS BAD_ARGUMENTS BAD_ARGUMENTS BAD_ARGUMENTS -BAD_ARGUMENTS +SYNTAX_ERROR BAD_ARGUMENTS BAD_ARGUMENTS BAD_ARGUMENTS diff --git a/tests/queries/0_stateless/02784_connection_string.sh b/tests/queries/0_stateless/02784_connection_string.sh index fd736e6838c..3519a0debdf 100755 --- a/tests/queries/0_stateless/02784_connection_string.sh +++ b/tests/queries/0_stateless/02784_connection_string.sh @@ -120,7 +120,7 @@ runClient "clickhouse:" --database "$CLICKHOUSE_DATABASE" --port "$CLICKHOUSE_PO runClient "clickhouse:" --connection "connection" 2>&1 | grep -o 'BAD_ARGUMENTS' # Space is used in connection string (This is prohibited). -runClient " clickhouse:" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient " clickhouse:" 2>&1 | grep -o 'SYNTAX_ERROR' runClient "clickhouse: " 2>&1 | grep -o 'BAD_ARGUMENTS' runClient "clickhouse://host1 /" 2>&1 | grep -o 'BAD_ARGUMENTS' runClient "clickhouse://host1, host2/" 2>&1 | grep -o 'BAD_ARGUMENTS' From ecf524175ab0a407f2e1743961840b49221909bf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 3 May 2024 03:20:11 +0200 Subject: [PATCH 276/624] Less directories --- 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 29c6c101910..fed08b9d3c5 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -140,7 +140,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) /// Will write file with database metadata, if needed. String database_name_escaped = escapeForFileName(database_name); - fs::path metadata_path = fs::canonical(getContext()->getPath()); + fs::path metadata_path = fs::weakly_canonical(getContext()->getPath()); fs::path metadata_file_tmp_path = metadata_path / "metadata" / (database_name_escaped + ".sql.tmp"); fs::path metadata_file_path = metadata_path / "metadata" / (database_name_escaped + ".sql"); From aea309007454f41798f1139a162f2e7bdb19ec92 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 3 May 2024 03:24:03 +0200 Subject: [PATCH 277/624] Less directories --- src/Interpreters/InterpreterCreateQuery.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index fed08b9d3c5..82d2c3fd5f7 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -141,6 +141,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) /// Will write file with database metadata, if needed. String database_name_escaped = escapeForFileName(database_name); fs::path metadata_path = fs::weakly_canonical(getContext()->getPath()); + fs::create_directories(metadata_path / "metadata"); fs::path metadata_file_tmp_path = metadata_path / "metadata" / (database_name_escaped + ".sql.tmp"); fs::path metadata_file_path = metadata_path / "metadata" / (database_name_escaped + ".sql"); From b5f454fb461c6dd6727a68f3f5c96b88a38730f3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 3 May 2024 03:30:28 +0200 Subject: [PATCH 278/624] Less directories --- programs/local/LocalServer.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index ac63090f007..553b48584bc 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -693,6 +693,7 @@ void LocalServer::processConfig() if (config().has("path")) { String path = global_context->getPath(); + fs::create_directories(fs::path(path)); /// Lock path directory before read status.emplace(fs::path(path) / "status", StatusFile::write_full_info); From 249afa7b29872035ef10ad7740fcbedf4655d48e Mon Sep 17 00:00:00 2001 From: "wangtao.2077" Date: Thu, 29 Feb 2024 11:58:57 +0800 Subject: [PATCH 279/624] fix boundRatio incorrect merge --- src/AggregateFunctions/AggregateFunctionBoundingRatio.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/AggregateFunctions/AggregateFunctionBoundingRatio.cpp b/src/AggregateFunctions/AggregateFunctionBoundingRatio.cpp index 62adb74924d..77d3d35d97b 100644 --- a/src/AggregateFunctions/AggregateFunctionBoundingRatio.cpp +++ b/src/AggregateFunctions/AggregateFunctionBoundingRatio.cpp @@ -63,6 +63,9 @@ struct AggregateFunctionBoundingRatioData { *this = other; } + else if (other.empty) + { + } else { if (other.left.x < left.x) From 347c24e6f3da9499fbfc82c6466696dbe3c52c4a Mon Sep 17 00:00:00 2001 From: "wangtao.2077" Date: Mon, 4 Mar 2024 22:22:06 +0800 Subject: [PATCH 280/624] add tests --- tests/queries/0_stateless/00715_bounding_ratio.reference | 1 + tests/queries/0_stateless/00715_bounding_ratio.sql | 9 +++++++++ 2 files changed, 10 insertions(+) diff --git a/tests/queries/0_stateless/00715_bounding_ratio.reference b/tests/queries/0_stateless/00715_bounding_ratio.reference index f1e96af83a9..1e20b164437 100644 --- a/tests/queries/0_stateless/00715_bounding_ratio.reference +++ b/tests/queries/0_stateless/00715_bounding_ratio.reference @@ -1,5 +1,6 @@ 1 1 +1 1.5 1.5 1.5 diff --git a/tests/queries/0_stateless/00715_bounding_ratio.sql b/tests/queries/0_stateless/00715_bounding_ratio.sql index b790aedf741..cad67722f35 100644 --- a/tests/queries/0_stateless/00715_bounding_ratio.sql +++ b/tests/queries/0_stateless/00715_bounding_ratio.sql @@ -1,10 +1,19 @@ drop table if exists rate_test; +drop table if exists rate_test2; create table rate_test (timestamp UInt32, event UInt32) engine=Memory; insert into rate_test values (0,1000),(1,1001),(2,1002),(3,1003),(4,1004),(5,1005),(6,1006),(7,1007),(8,1008); select 1.0 = boundingRatio(timestamp, event) from rate_test; +create table rate_test2 (timestamp UInt32, event UInt32) engine=Memory; + +SELECT boundingRatioMerge(state) FROM ( + select boundingRatioState(timestamp, event) as state from rate_test + UNION ALL + SELECT boundingRatioState(timestamp, event) FROM rate_test2 WHERE 1=0 +); + drop table if exists rate_test2; create table rate_test2 (uid UInt32 default 1,timestamp DateTime, event UInt32) engine=Memory; insert into rate_test2(timestamp, event) values ('2018-01-01 01:01:01',1001),('2018-01-01 01:01:02',1002),('2018-01-01 01:01:03',1003),('2018-01-01 01:01:04',1004),('2018-01-01 01:01:05',1005),('2018-01-01 01:01:06',1006),('2018-01-01 01:01:07',1007),('2018-01-01 01:01:08',1008); From c4062de0474e962ad6c0cf854152f3da147ea4c9 Mon Sep 17 00:00:00 2001 From: "wangtao.2077" Date: Mon, 8 Apr 2024 14:38:32 +0800 Subject: [PATCH 281/624] fix as comment --- .../AggregateFunctionBoundingRatio.cpp | 2 ++ .../0_stateless/00715_bounding_ratio.reference | 1 - .../queries/0_stateless/00715_bounding_ratio.sql | 9 --------- .../00715_bounding_ratio_merge_empty.reference | 1 + .../00715_bounding_ratio_merge_empty.sql | 16 ++++++++++++++++ 5 files changed, 19 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/00715_bounding_ratio_merge_empty.reference create mode 100644 tests/queries/0_stateless/00715_bounding_ratio_merge_empty.sql diff --git a/src/AggregateFunctions/AggregateFunctionBoundingRatio.cpp b/src/AggregateFunctions/AggregateFunctionBoundingRatio.cpp index 77d3d35d97b..1e716d5255f 100644 --- a/src/AggregateFunctions/AggregateFunctionBoundingRatio.cpp +++ b/src/AggregateFunctions/AggregateFunctionBoundingRatio.cpp @@ -65,6 +65,8 @@ struct AggregateFunctionBoundingRatioData } else if (other.empty) { + // if other.empty = true, other.x/other.y may be uninitialized values, + // so don't use them to update this->state } else { diff --git a/tests/queries/0_stateless/00715_bounding_ratio.reference b/tests/queries/0_stateless/00715_bounding_ratio.reference index 1e20b164437..f1e96af83a9 100644 --- a/tests/queries/0_stateless/00715_bounding_ratio.reference +++ b/tests/queries/0_stateless/00715_bounding_ratio.reference @@ -1,6 +1,5 @@ 1 1 -1 1.5 1.5 1.5 diff --git a/tests/queries/0_stateless/00715_bounding_ratio.sql b/tests/queries/0_stateless/00715_bounding_ratio.sql index cad67722f35..b790aedf741 100644 --- a/tests/queries/0_stateless/00715_bounding_ratio.sql +++ b/tests/queries/0_stateless/00715_bounding_ratio.sql @@ -1,19 +1,10 @@ drop table if exists rate_test; -drop table if exists rate_test2; create table rate_test (timestamp UInt32, event UInt32) engine=Memory; insert into rate_test values (0,1000),(1,1001),(2,1002),(3,1003),(4,1004),(5,1005),(6,1006),(7,1007),(8,1008); select 1.0 = boundingRatio(timestamp, event) from rate_test; -create table rate_test2 (timestamp UInt32, event UInt32) engine=Memory; - -SELECT boundingRatioMerge(state) FROM ( - select boundingRatioState(timestamp, event) as state from rate_test - UNION ALL - SELECT boundingRatioState(timestamp, event) FROM rate_test2 WHERE 1=0 -); - drop table if exists rate_test2; create table rate_test2 (uid UInt32 default 1,timestamp DateTime, event UInt32) engine=Memory; insert into rate_test2(timestamp, event) values ('2018-01-01 01:01:01',1001),('2018-01-01 01:01:02',1002),('2018-01-01 01:01:03',1003),('2018-01-01 01:01:04',1004),('2018-01-01 01:01:05',1005),('2018-01-01 01:01:06',1006),('2018-01-01 01:01:07',1007),('2018-01-01 01:01:08',1008); diff --git a/tests/queries/0_stateless/00715_bounding_ratio_merge_empty.reference b/tests/queries/0_stateless/00715_bounding_ratio_merge_empty.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/00715_bounding_ratio_merge_empty.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/00715_bounding_ratio_merge_empty.sql b/tests/queries/0_stateless/00715_bounding_ratio_merge_empty.sql new file mode 100644 index 00000000000..8d2332758b5 --- /dev/null +++ b/tests/queries/0_stateless/00715_bounding_ratio_merge_empty.sql @@ -0,0 +1,16 @@ +drop table if exists rate_test; +drop table if exists rate_test2; + +create table rate_test (timestamp UInt32, event UInt32) engine=Memory; +insert into rate_test values (0,1000),(1,1001),(2,1002),(3,1003),(4,1004),(5,1005),(6,1006),(7,1007),(8,1008); + +create table rate_test2 (timestamp UInt32, event UInt32) engine=Memory; + +SELECT boundingRatioMerge(state) FROM ( + select boundingRatioState(timestamp, event) as state from rate_test + UNION ALL + SELECT boundingRatioState(timestamp, event) FROM rate_test2 WHERE 1=0 +); + +drop table if exists rate_test; +drop table if exists rate_test2; From dd7e88b37eb33ad962b108edc87da493cf24b142 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Thu, 2 May 2024 05:52:11 +0000 Subject: [PATCH 282/624] fix infinite loop in AzureObjectStorage::listObjects While the exit condition is correct for `AzureIteratorAsync`, listObject may never exit. This is because BlobContainerClient::LstBlobs sets a page, thus making a sequential HasPage() condition true. This issue reproduces when passing an empty path. Tested with the following integration test: https://pastila.nl/?01691b54/9ceaf103c91ab9e38ea3541abfcd1ae6#hUtrrz5sFHcBSHCLS9AlVA== (will be added when plain_rewritable is enabled for all disk types). --- .../AzureBlobStorage/AzureObjectStorage.cpp | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 0f45f40288e..4c5cd57615b 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -164,15 +164,13 @@ void AzureObjectStorage::listObjects(const std::string & path, RelativePathsWith else options.PageSizeHint = settings.get()->list_object_keys_size; - Azure::Storage::Blobs::ListBlobsPagedResponse blob_list_response; - - while (true) + for (auto blob_list_response = client_ptr->ListBlobs(options); blob_list_response.HasPage(); blob_list_response.MoveToNextPage()) { ProfileEvents::increment(ProfileEvents::AzureListObjects); ProfileEvents::increment(ProfileEvents::DiskAzureListObjects); blob_list_response = client_ptr->ListBlobs(options); - auto blobs_list = blob_list_response.Blobs; + const auto & blobs_list = blob_list_response.Blobs; for (const auto & blob : blobs_list) { @@ -193,11 +191,6 @@ void AzureObjectStorage::listObjects(const std::string & path, RelativePathsWith break; options.PageSizeHint = keys_left; } - - if (blob_list_response.HasPage()) - options.ContinuationToken = blob_list_response.NextPageToken; - else - break; } } From 8ab8fab83741d10d3f2c80059adf6e0337a538b1 Mon Sep 17 00:00:00 2001 From: Xiaofei Hu <10776938+litlig@users.noreply.github.com> Date: Thu, 2 May 2024 19:22:35 -0700 Subject: [PATCH 283/624] tests --- .../03143_ttl_in_system_parts_columns_table.reference | 4 ++-- .../0_stateless/03143_ttl_in_system_parts_columns_table.sql | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03143_ttl_in_system_parts_columns_table.reference b/tests/queries/0_stateless/03143_ttl_in_system_parts_columns_table.reference index b67f6764969..99e5bf7620a 100644 --- a/tests/queries/0_stateless/03143_ttl_in_system_parts_columns_table.reference +++ b/tests/queries/0_stateless/03143_ttl_in_system_parts_columns_table.reference @@ -1,4 +1,4 @@ all_1_1_0 timestamp DateTime \N \N -all_1_1_0 x UInt32 2024-05-23 00:00:00 2024-05-23 00:00:00 -all_1_1_0 y String 2024-04-24 00:00:00 2024-04-24 00:00:00 +all_1_1_0 x UInt32 2023-05-23 00:00:00 2023-05-23 00:00:00 +all_1_1_0 y String 2023-04-24 00:00:00 2023-04-24 00:00:00 all_1_1_0 z String \N \N diff --git a/tests/queries/0_stateless/03143_ttl_in_system_parts_columns_table.sql b/tests/queries/0_stateless/03143_ttl_in_system_parts_columns_table.sql index d3b372b137d..486759a65ea 100644 --- a/tests/queries/0_stateless/03143_ttl_in_system_parts_columns_table.sql +++ b/tests/queries/0_stateless/03143_ttl_in_system_parts_columns_table.sql @@ -10,7 +10,7 @@ ENGINE = MergeTree ORDER BY tuple(); -INSERT INTO test_03143 VALUES ('2024-04-23', 123, 'Hello, world!', 'xxx yyy'); +INSERT INTO test_03143 VALUES ('2023-04-23', 123, 'Hello, world!', 'xxx yyy'); SELECT name, From c9fd1df4fce3910eddf28eec665e087a6c623a01 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 3 May 2024 05:27:03 +0300 Subject: [PATCH 284/624] Revert "Introduce bulk loading to StorageEmbeddedRocksDB" --- .../integrations/embedded-rocksdb.md | 6 - src/Interpreters/IKeyValueEntity.h | 1 - .../RocksDB/EmbeddedRocksDBBulkSink.cpp | 240 ------------------ .../RocksDB/EmbeddedRocksDBBulkSink.h | 69 ----- src/Storages/RocksDB/RocksDBSettings.cpp | 41 --- src/Storages/RocksDB/RocksDBSettings.h | 39 --- .../RocksDB/StorageEmbeddedRocksDB.cpp | 81 ++---- src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 17 +- tests/queries/0_stateless/01686_rocksdb.sql | 2 +- .../02956_rocksdb_bulk_sink.reference | 10 - .../0_stateless/02956_rocksdb_bulk_sink.sh | 48 ---- .../aspell-ignore/en/aspell-dict.txt | 2 - 12 files changed, 20 insertions(+), 536 deletions(-) delete mode 100644 src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp delete mode 100644 src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h delete mode 100644 src/Storages/RocksDB/RocksDBSettings.cpp delete mode 100644 src/Storages/RocksDB/RocksDBSettings.h delete mode 100644 tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference delete mode 100755 tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh diff --git a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md index 1958250ed73..44febe78c77 100644 --- a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md +++ b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md @@ -17,7 +17,6 @@ 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 name=value, ... ] ``` Engine parameters: @@ -30,11 +29,6 @@ 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); default value: `1`. -- `bulk_insert_block_size` - Minimum size of SST files (in term of rows) created by bulk insertion; default value: `1048449`. - Example: ``` sql diff --git a/src/Interpreters/IKeyValueEntity.h b/src/Interpreters/IKeyValueEntity.h index f9287e4793c..856ce28bae7 100644 --- a/src/Interpreters/IKeyValueEntity.h +++ b/src/Interpreters/IKeyValueEntity.h @@ -3,7 +3,6 @@ #include #include #include -#include namespace DB { diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp deleted file mode 100644 index 1aca0edc223..00000000000 --- a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp +++ /dev/null @@ -1,240 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int 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; -} - -/// 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 - 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(); - for (size_t idx = 0; idx < rows;) - { - /// We will write the last row of the same key - size_t next_idx = idx + 1; - while (next_idx < rows && keys.compareAt(perm[idx], perm[next_idx], keys, 1) == 0) - ++next_idx; - - auto row = perm[next_idx - 1]; - status = sst_file_writer.Put(keys.getDataAt(row).toView(), values.getDataAt(row).toView()); - if (!status.ok()) - return status; - - idx = next_idx; - } - - return sst_file_writer.Finish(); -} - -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(); - 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 - auto base_directory_name = TMP_INSERT_PREFIX + sipHash128String(getContext()->getCurrentQueryId()); - insert_directory_queue = fs::path(storage.getDataPaths()[0]) / (base_directory_name + "-" + getRandomASCIIString(8)); - fs::create_directory(insert_directory_queue); -} - -EmbeddedRocksDBBulkSink::~EmbeddedRocksDBBulkSink() -{ - try - { - if (fs::exists(insert_directory_queue)) - fs::remove_all(insert_directory_queue); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("Error while removing temporary directory {}:", insert_directory_queue)); - } -} - -std::vector EmbeddedRocksDBBulkSink::squash(Chunk chunk) -{ - /// End of input stream - if (chunk.getNumRows() == 0) - { - return std::move(chunks); - } - - /// 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 {}; - } - - /// 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; - } - - /// 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(); - - { - 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(); - WriteBufferFromVector writer_key(serialized_key_data); - WriteBufferFromVector writer_value(serialized_value_data); - - for (const auto & chunk : input_chunks) - { - 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, {}); - /// String in ColumnString must be null-terminated - 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(); - } - - 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 sst_file_path = getTemporarySSTFilePath(); - if (auto status = buildSSTFile(sst_file_path, *serialized_key_column, *serialized_value_column); !status.ok()) - throw Exception(ErrorCodes::ROCKSDB_ERROR, "RocksDB write error: {}", status.ToString()); - - /// Ingest the SST file - static 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({sst_file_path}, ingest_options); !status.ok()) - throw Exception(ErrorCodes::ROCKSDB_ERROR, "RocksDB write error: {}", status.ToString()); - - if (fs::exists(sst_file_path)) - fs::remove(sst_file_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 deleted file mode 100644 index 19ce1e3b83e..00000000000 --- a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h +++ /dev/null @@ -1,69 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ -namespace fs = std::filesystem; - -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. Squash chunks to reduce the number of SST files -class EmbeddedRocksDBBulkSink : public SinkToStorage, public WithContext -{ -public: - EmbeddedRocksDBBulkSink( - ContextPtr context_, - StorageEmbeddedRocksDB & storage_, - const StorageMetadataPtr & metadata_snapshot_); - - ~EmbeddedRocksDBBulkSink() override; - - 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(); - - /// 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 - size_t file_counter = 0; - static constexpr auto TMP_INSERT_PREFIX = "tmp_insert_"; - String insert_directory_queue; -}; - -} diff --git a/src/Storages/RocksDB/RocksDBSettings.cpp b/src/Storages/RocksDB/RocksDBSettings.cpp deleted file mode 100644 index 7de2077eb47..00000000000 --- a/src/Storages/RocksDB/RocksDBSettings.cpp +++ /dev/null @@ -1,41 +0,0 @@ -#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 deleted file mode 100644 index 1b168c56d89..00000000000 --- a/src/Storages/RocksDB/RocksDBSettings.h +++ /dev/null @@ -1,39 +0,0 @@ -#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) \ - 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) - -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 ad185f8ec2b..7c4581025e5 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -27,15 +28,8 @@ #include #include #include -#include -#include -#include -#include #include -#include -#include -#include #include #include #include @@ -45,6 +39,8 @@ #include +namespace fs = std::filesystem; + namespace DB { @@ -178,7 +174,6 @@ StorageEmbeddedRocksDB::StorageEmbeddedRocksDB(const StorageID & table_id_, const StorageInMemoryMetadata & metadata_, LoadingStrictnessLevel mode, ContextPtr context_, - std::unique_ptr settings_, const String & primary_key_, Int32 ttl_, String rocksdb_dir_, @@ -191,7 +186,6 @@ 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_; @@ -242,20 +236,22 @@ void StorageEmbeddedRocksDB::mutate(const MutationCommands & commands, ContextPt if (commands.front().type == MutationCommand::Type::DELETE) { - MutationsInterpreter::Settings mutation_settings(true); - mutation_settings.return_all_columns = true; - mutation_settings.return_mutated_rows = true; + MutationsInterpreter::Settings settings(true); + settings.return_all_columns = true; + settings.return_mutated_rows = true; auto interpreter = std::make_unique( storage_ptr, metadata_snapshot, commands, context_, - mutation_settings); + 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); @@ -291,16 +287,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 mutation_settings(true); - mutation_settings.return_all_columns = true; - mutation_settings.return_mutated_rows = true; + MutationsInterpreter::Settings settings(true); + settings.return_all_columns = true; + settings.return_mutated_rows = true; auto interpreter = std::make_unique( storage_ptr, metadata_snapshot, commands, context_, - mutation_settings); + settings); auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute()); PullingPipelineExecutor executor(pipeline); @@ -356,6 +352,7 @@ 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; @@ -585,11 +582,8 @@ void ReadFromEmbeddedRocksDB::applyFilters(ActionDAGNodes added_filter_nodes) } SinkToStoragePtr StorageEmbeddedRocksDB::write( - const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context, bool /*async_insert*/) + const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*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); } @@ -628,21 +622,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "StorageEmbeddedRocksDB must require one column in primary key"); } - auto settings = std::make_unique(); - settings->loadFromQuery(*args.storage_def, args.getContext()); - if (args.storage_def->settings) - metadata.settings_changes = args.storage_def->settings->ptr(); - else - { - /// A workaround because embedded rocksdb doesn't have default immutable settings - /// But InterpreterAlterQuery requires settings_changes to be set to run ALTER MODIFY - /// SETTING queries. So we just add a setting with its default value. - auto settings_changes = std::make_shared(); - settings_changes->is_standalone = false; - settings_changes->changes.insertSetting("optimize_for_bulk_insert", settings->optimize_for_bulk_insert.value); - metadata.settings_changes = settings_changes; - } - return std::make_shared(args.table_id, args.relative_data_path, metadata, args.mode, args.getContext(), std::move(settings), primary_key_names[0], ttl, std::move(rocksdb_dir), read_only); + return std::make_shared(args.table_id, args.relative_data_path, metadata, args.mode, args.getContext(), primary_key_names[0], ttl, std::move(rocksdb_dir), read_only); } std::shared_ptr StorageEmbeddedRocksDB::getRocksDBStatistics() const @@ -733,9 +713,9 @@ Chunk StorageEmbeddedRocksDB::getBySerializedKeys( return Chunk(std::move(columns), num_rows); } -std::optional StorageEmbeddedRocksDB::totalRows(const Settings & query_settings) const +std::optional StorageEmbeddedRocksDB::totalRows(const Settings & settings) const { - if (!query_settings.optimize_trivial_approximate_count_query) + if (!settings.optimize_trivial_approximate_count_query) return {}; std::shared_lock lock(rocksdb_ptr_mx); if (!rocksdb_ptr) @@ -757,26 +737,9 @@ std::optional StorageEmbeddedRocksDB::totalBytes(const Settings & /*sett return estimated_bytes; } -void StorageEmbeddedRocksDB::alter( - const AlterCommands & params, - ContextPtr query_context, - AlterLockHolder & holder) -{ - IStorage::alter(params, query_context, holder); - auto new_metadata = getInMemoryMetadataPtr(); - if (new_metadata->settings_changes) - { - const auto & settings_changes = new_metadata->settings_changes->as(); - auto new_settings = std::make_unique(); - new_settings->applyChanges(settings_changes.changes); - setSettings(std::move(new_settings)); - } -} - void registerStorageEmbeddedRocksDB(StorageFactory & factory) { StorageFactory::StorageFeatures features{ - .supports_settings = true, .supports_sort_order = true, .supports_ttl = true, .supports_parallel_insert = true, @@ -784,12 +747,4 @@ void registerStorageEmbeddedRocksDB(StorageFactory & factory) factory.registerStorage("EmbeddedRocksDB", create, features); } - -void StorageEmbeddedRocksDB::checkAlterIsPossible(const AlterCommands & commands, ContextPtr /* context */) const -{ - for (const auto & command : commands) - if (!command.isCommentAlter() && !command.isSettingsAlter()) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Alter of type '{}' is not supported by storage {}", command.type, getName()); -} - } diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index 9fc58ea6b38..230464a161f 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -1,14 +1,11 @@ #pragma once #include -#include #include +#include #include #include -#include #include -#include -#include namespace rocksdb @@ -30,7 +27,6 @@ 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_, @@ -38,7 +34,6 @@ public: const StorageInMemoryMetadata & metadata, LoadingStrictnessLevel mode, ContextPtr context_, - std::unique_ptr settings_, const String & primary_key_, Int32 ttl_ = 0, String rocksdb_dir_ = "", @@ -64,7 +59,6 @@ public: void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override; void mutate(const MutationCommands &, ContextPtr) override; void drop() override; - void alter(const AlterCommands & params, ContextPtr query_context, AlterLockHolder &) override; bool optimize( const ASTPtr & query, @@ -105,16 +99,7 @@ public: std::optional totalBytes(const Settings & settings) const override; - void checkAlterIsPossible(const AlterCommands & commands, ContextPtr /* context */) const override; - - const RocksDBSettings & getSettings() const { return *storage_settings.get(); } - - void setSettings(std::unique_ptr && settings_) { storage_settings.set(std::move(settings_)); } - private: - SinkToStoragePtr getSink(ContextPtr context, const StorageMetadataPtr & metadata_snapshot); - - MultiVersion storage_settings; const String primary_key; using RocksDBPtr = std::unique_ptr; RocksDBPtr rocksdb_ptr; diff --git a/tests/queries/0_stateless/01686_rocksdb.sql b/tests/queries/0_stateless/01686_rocksdb.sql index 3ff218bf398..f3177ce140e 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) SETTINGS optimize_for_bulk_insert = 0; +CREATE TABLE 01686_test (key UInt64, value String) Engine=EmbeddedRocksDB PRIMARY KEY(key); 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.reference b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference deleted file mode 100644 index 74c71827e6e..00000000000 --- a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference +++ /dev/null @@ -1,10 +0,0 @@ -0 -1000 -1000 -1 -1000 -2 -1000000 -1000 -0 999001 -1000000 diff --git a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh deleted file mode 100755 index 8acc83fc86c..00000000000 --- a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh +++ /dev/null @@ -1,48 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-ordinary-database, use-rocksdb - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -# Normal importing, as we only insert 1000 rows, so it should be in memtable -${CLICKHOUSE_CLIENT} --query "CREATE TABLE IF NOT EXISTS rocksdb_worm (key UInt64, value UInt64) ENGINE = EmbeddedRocksDB() PRIMARY KEY key SETTINGS optimize_for_bulk_insert = 0;" -${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers(1000);" -${CLICKHOUSE_CLIENT} --query "SELECT sum(value) FROM system.rocksdb WHERE database = currentDatabase() AND table = 'rocksdb_worm' AND name = 'no.file.opens';" # should be 0 because all data is still in memtable -${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm;" - -# Enabling bulk insertion -${CLICKHOUSE_CLIENT} --query "ALTER TABLE rocksdb_worm MODIFY SETTING optimize_for_bulk_insert = 1;" - -# Testing that key serialization is identical w. and w/o bulk sink -${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE rocksdb_worm;" -${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+2 FROM numbers(1000);" # should override previous keys -${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm WHERE value = key + 2;" - -# With bulk insertion, there is no memtable, so a small insert should create a new file -${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE rocksdb_worm;" -${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers(1000);" -${CLICKHOUSE_CLIENT} --query "SELECT sum(value) FROM system.rocksdb WHERE database = currentDatabase() AND table = 'rocksdb_worm' AND name = 'no.file.opens';" # should be 1 -${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 -${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm;" - -# Testing insert with duplicated keys -${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE rocksdb_worm;" -${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number % 1000, number+1 FROM numbers_mt(1000000) SETTINGS max_block_size = 100000, max_insert_threads = 1;" -${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm;" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM rocksdb_worm WHERE key = 0;" # should be the latest value - 999001 - - -# Testing insert with multiple threads -${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE rocksdb_worm;" -${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers_mt(1000000)" & -${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers_mt(1000000)" & -wait -${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm;" - diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 2b260566816..1457eacbc92 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1949,8 +1949,6 @@ mdadm meanZTest meanztest mebibytes -memtable -memtables mergeTreeIndex mergeable mergetree From 21145e4fb7607de236eca1b3466655eea8997ca8 Mon Sep 17 00:00:00 2001 From: Xiaofei Hu <10776938+litlig@users.noreply.github.com> Date: Thu, 2 May 2024 19:59:37 -0700 Subject: [PATCH 285/624] test --- .../03143_ttl_in_system_parts_columns_table.reference | 4 ++-- .../0_stateless/03143_ttl_in_system_parts_columns_table.sql | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03143_ttl_in_system_parts_columns_table.reference b/tests/queries/0_stateless/03143_ttl_in_system_parts_columns_table.reference index 99e5bf7620a..f358d128f8a 100644 --- a/tests/queries/0_stateless/03143_ttl_in_system_parts_columns_table.reference +++ b/tests/queries/0_stateless/03143_ttl_in_system_parts_columns_table.reference @@ -1,4 +1,4 @@ all_1_1_0 timestamp DateTime \N \N -all_1_1_0 x UInt32 2023-05-23 00:00:00 2023-05-23 00:00:00 -all_1_1_0 y String 2023-04-24 00:00:00 2023-04-24 00:00:00 +all_1_1_0 x UInt32 2100-02-01 00:00:00 2100-02-01 00:00:00 +all_1_1_0 y String 2100-01-02 00:00:00 2100-01-02 00:00:00 all_1_1_0 z String \N \N diff --git a/tests/queries/0_stateless/03143_ttl_in_system_parts_columns_table.sql b/tests/queries/0_stateless/03143_ttl_in_system_parts_columns_table.sql index 486759a65ea..81a88ef45c4 100644 --- a/tests/queries/0_stateless/03143_ttl_in_system_parts_columns_table.sql +++ b/tests/queries/0_stateless/03143_ttl_in_system_parts_columns_table.sql @@ -10,7 +10,7 @@ ENGINE = MergeTree ORDER BY tuple(); -INSERT INTO test_03143 VALUES ('2023-04-23', 123, 'Hello, world!', 'xxx yyy'); +INSERT INTO test_03143 VALUES ('2100-01-01', 123, 'Hello, world!', 'xxx yyy'); SELECT name, From 9018136faf957f7e3382dea86710c853768df9aa Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 3 May 2024 10:00:18 +0200 Subject: [PATCH 286/624] empty commit From 51f5c8b90105ad8bb6fd8d71a873c10c6fe487f7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 3 May 2024 11:35:23 +0200 Subject: [PATCH 287/624] Add tests for #63264 --- .../03143_group_by_constant_secondary.reference | 1 + .../0_stateless/03143_group_by_constant_secondary.sql | 8 ++++++++ 2 files changed, 9 insertions(+) create mode 100644 tests/queries/0_stateless/03143_group_by_constant_secondary.reference create mode 100644 tests/queries/0_stateless/03143_group_by_constant_secondary.sql diff --git a/tests/queries/0_stateless/03143_group_by_constant_secondary.reference b/tests/queries/0_stateless/03143_group_by_constant_secondary.reference new file mode 100644 index 00000000000..0cfbf08886f --- /dev/null +++ b/tests/queries/0_stateless/03143_group_by_constant_secondary.reference @@ -0,0 +1 @@ +2 diff --git a/tests/queries/0_stateless/03143_group_by_constant_secondary.sql b/tests/queries/0_stateless/03143_group_by_constant_secondary.sql new file mode 100644 index 00000000000..030e3212748 --- /dev/null +++ b/tests/queries/0_stateless/03143_group_by_constant_secondary.sql @@ -0,0 +1,8 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/63264 +SELECT count() +FROM remote(test_cluster_two_shards, system, one) +GROUP BY 'hi' +SETTINGS + allow_experimental_analyzer = 1, + group_by_two_level_threshold = 1, + group_by_two_level_threshold_bytes = 33950592; From 97a411365de85e1d5370de6de857e33b25f0ad5f Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 1 May 2024 14:40:50 +0300 Subject: [PATCH 288/624] JOIN filter push down filled join fix --- .../Optimizations/filterPushDown.cpp | 2 +- ...filter_push_down_filled_join_fix.reference | 1 + ..._join_filter_push_down_filled_join_fix.sql | 45 +++++++++++++++++++ 3 files changed, 47 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03143_join_filter_push_down_filled_join_fix.reference create mode 100644 tests/queries/0_stateless/03143_join_filter_push_down_filled_join_fix.sql diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 2031b64325b..7dd526cbe95 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -253,7 +253,7 @@ static size_t tryPushDownOverJoinStep(QueryPlan::Node * parent_node, QueryPlan:: bool has_single_clause = table_join.getClauses().size() == 1; - if (has_single_clause) + if (has_single_clause && !filled_join) { const auto & join_clause = table_join.getClauses()[0]; size_t key_names_size = join_clause.key_names_left.size(); diff --git a/tests/queries/0_stateless/03143_join_filter_push_down_filled_join_fix.reference b/tests/queries/0_stateless/03143_join_filter_push_down_filled_join_fix.reference new file mode 100644 index 00000000000..4f89085d5ff --- /dev/null +++ b/tests/queries/0_stateless/03143_join_filter_push_down_filled_join_fix.reference @@ -0,0 +1 @@ +1 1 1 test diff --git a/tests/queries/0_stateless/03143_join_filter_push_down_filled_join_fix.sql b/tests/queries/0_stateless/03143_join_filter_push_down_filled_join_fix.sql new file mode 100644 index 00000000000..fc816623bd4 --- /dev/null +++ b/tests/queries/0_stateless/03143_join_filter_push_down_filled_join_fix.sql @@ -0,0 +1,45 @@ +DROP TABLE IF EXISTS t1; +CREATE TABLE t1 +( + id UInt64, + external_id UInt64 +) +ENGINE = MergeTree +ORDER BY id; + +DROP TABLE IF EXISTS t2; +CREATE TABLE t2 +( + id UInt64, + name String +) +ENGINE = MergeTree +ORDER BY id; + +INSERT INTO t1 VALUES (1, 1); + +INSERT INTO t2 VALUES (1, 'test'); + +DROP DICTIONARY IF EXISTS d2; +CREATE DICTIONARY d2 +( + id UInt64, + name String, +) +PRIMARY KEY id +SOURCE(CLICKHOUSE( + table t2)) +LIFETIME(MIN 600 MAX 900) +LAYOUT(HASHED()); + +SELECT + * +FROM + t1 + LEFT JOIN d2 ON d2.id = t1.external_id + WHERE t1.id = 1 +LIMIT 1; + +DROP DICTIONARY d2; +DROP TABLE t2; +DROP TABLE t1; From 4e6e234d24eb46d62c7e7f0e99a1ecccda5cc0f4 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 3 May 2024 12:19:57 +0200 Subject: [PATCH 289/624] Safer parent part access --- src/Storages/MergeTree/MergeTreeReadPoolBase.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp index 0cbb0a86b2f..3e10285d6b0 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp @@ -113,9 +113,9 @@ MergeTreeReadTaskPtr MergeTreeReadPoolBase::createTask( ? std::make_unique(*read_info->shared_size_predictor) : nullptr; /// make a copy - auto get_part_name = [](const auto & task_info) -> const String & + auto get_part_name = [](const auto & task_info) -> String { - return task_info.data_part->isProjectionPart() ? task_info.data_part->getParentPart()->name : task_info.data_part->name; + return task_info.data_part->isProjectionPart() ? task_info.data_part->getParentPartName() : task_info.data_part->name; }; auto extras = getExtras(); From effcc09476365845c512aaaeac823c5c6f52551c Mon Sep 17 00:00:00 2001 From: Blargian Date: Fri, 3 May 2024 12:19:41 +0200 Subject: [PATCH 290/624] Review changes --- src/Common/UTF8Helpers.cpp | 4 ++-- .../0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Common/UTF8Helpers.cpp b/src/Common/UTF8Helpers.cpp index 9b9c08afbaa..9425d02278e 100644 --- a/src/Common/UTF8Helpers.cpp +++ b/src/Common/UTF8Helpers.cpp @@ -97,7 +97,7 @@ namespace enum ComputeWidthMode { Width, /// Calculate and return visible width - BytesBeforeLimit /// Calculate and return the maximum number of bytes when substring fits in visible width. + BytesBeforeLimit /// Calculate and return the maximum number of bytes when substring fits in visible width. }; template @@ -180,7 +180,7 @@ size_t computeWidthImpl(const UInt8 * data, size_t size, size_t prefix, size_t l size_t next_width = width; if (decoder.codepoint == '\x1b') isEscapeSequence = true; - if (decoder.codepoint == '\t') + else if (decoder.codepoint == '\t') next_width += 8 - (prefix + width) % 8; else next_width += wcwidth(decoder.codepoint); diff --git a/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql b/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql index 048f1821113..e37b0db08e9 100644 --- a/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql +++ b/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql @@ -1 +1 @@ -SELECT format('\x1b[38;2;{0};{1};{2}m█\x1b[0m', 255, 128, 0) AS x FORMAT Pretty; \ No newline at end of file +SELECT format('\x1b[38;2;{0};{1};{2}m█\x1b[0m', 255, 128, 0) AS x FORMAT Pretty; From 0507be3ac38d3a65fc3e63e9fd868cfa95f7cf26 Mon Sep 17 00:00:00 2001 From: Blargian Date: Fri, 3 May 2024 12:23:56 +0200 Subject: [PATCH 291/624] Review change --- 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 9425d02278e..b68665f3277 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; - 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 5d2feb41e745129f8dcb5018f69dbf298f389a6a Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Fri, 3 May 2024 18:29:36 +0800 Subject: [PATCH 292/624] Revert "Revert "Introduce bulk loading to StorageEmbeddedRocksDB"" --- .../integrations/embedded-rocksdb.md | 6 + src/Interpreters/IKeyValueEntity.h | 1 + .../RocksDB/EmbeddedRocksDBBulkSink.cpp | 240 ++++++++++++++++++ .../RocksDB/EmbeddedRocksDBBulkSink.h | 69 +++++ src/Storages/RocksDB/RocksDBSettings.cpp | 41 +++ src/Storages/RocksDB/RocksDBSettings.h | 39 +++ .../RocksDB/StorageEmbeddedRocksDB.cpp | 81 ++++-- src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 17 +- tests/queries/0_stateless/01686_rocksdb.sql | 2 +- .../02956_rocksdb_bulk_sink.reference | 10 + .../0_stateless/02956_rocksdb_bulk_sink.sh | 48 ++++ .../aspell-ignore/en/aspell-dict.txt | 2 + 12 files changed, 536 insertions(+), 20 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 100755 tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh diff --git a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md index 44febe78c77..1958250ed73 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 name=value, ... ] ``` Engine parameters: @@ -29,6 +30,11 @@ 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); default value: `1`. +- `bulk_insert_block_size` - Minimum size of SST files (in term of rows) created by bulk insertion; default value: `1048449`. + Example: ``` sql diff --git a/src/Interpreters/IKeyValueEntity.h b/src/Interpreters/IKeyValueEntity.h index 856ce28bae7..f9287e4793c 100644 --- a/src/Interpreters/IKeyValueEntity.h +++ b/src/Interpreters/IKeyValueEntity.h @@ -3,6 +3,7 @@ #include #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..1aca0edc223 --- /dev/null +++ b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp @@ -0,0 +1,240 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int 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; +} + +/// 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 + 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(); + for (size_t idx = 0; idx < rows;) + { + /// We will write the last row of the same key + size_t next_idx = idx + 1; + while (next_idx < rows && keys.compareAt(perm[idx], perm[next_idx], keys, 1) == 0) + ++next_idx; + + auto row = perm[next_idx - 1]; + status = sst_file_writer.Put(keys.getDataAt(row).toView(), values.getDataAt(row).toView()); + if (!status.ok()) + return status; + + idx = next_idx; + } + + return sst_file_writer.Finish(); +} + +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(); + 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 + auto base_directory_name = TMP_INSERT_PREFIX + sipHash128String(getContext()->getCurrentQueryId()); + insert_directory_queue = fs::path(storage.getDataPaths()[0]) / (base_directory_name + "-" + getRandomASCIIString(8)); + fs::create_directory(insert_directory_queue); +} + +EmbeddedRocksDBBulkSink::~EmbeddedRocksDBBulkSink() +{ + try + { + if (fs::exists(insert_directory_queue)) + fs::remove_all(insert_directory_queue); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("Error while removing temporary directory {}:", insert_directory_queue)); + } +} + +std::vector EmbeddedRocksDBBulkSink::squash(Chunk chunk) +{ + /// End of input stream + if (chunk.getNumRows() == 0) + { + return std::move(chunks); + } + + /// 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 {}; + } + + /// 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; + } + + /// 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(); + + { + 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(); + WriteBufferFromVector writer_key(serialized_key_data); + WriteBufferFromVector writer_value(serialized_value_data); + + for (const auto & chunk : input_chunks) + { + 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, {}); + /// String in ColumnString must be null-terminated + 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(); + } + + 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 sst_file_path = getTemporarySSTFilePath(); + if (auto status = buildSSTFile(sst_file_path, *serialized_key_column, *serialized_value_column); !status.ok()) + throw Exception(ErrorCodes::ROCKSDB_ERROR, "RocksDB write error: {}", status.ToString()); + + /// Ingest the SST file + static 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({sst_file_path}, ingest_options); !status.ok()) + throw Exception(ErrorCodes::ROCKSDB_ERROR, "RocksDB write error: {}", status.ToString()); + + if (fs::exists(sst_file_path)) + fs::remove(sst_file_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 new file mode 100644 index 00000000000..19ce1e3b83e --- /dev/null +++ b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h @@ -0,0 +1,69 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace fs = std::filesystem; + +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. Squash chunks to reduce the number of SST files +class EmbeddedRocksDBBulkSink : public SinkToStorage, public WithContext +{ +public: + EmbeddedRocksDBBulkSink( + ContextPtr context_, + StorageEmbeddedRocksDB & storage_, + const StorageMetadataPtr & metadata_snapshot_); + + ~EmbeddedRocksDBBulkSink() override; + + 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(); + + /// 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 + size_t file_counter = 0; + static constexpr auto TMP_INSERT_PREFIX = "tmp_insert_"; + String insert_directory_queue; +}; + +} 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..1b168c56d89 --- /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) \ + 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) + +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 7c4581025e5..ad185f8ec2b 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,15 @@ #include #include #include +#include +#include +#include +#include #include +#include +#include +#include #include #include #include @@ -39,8 +45,6 @@ #include -namespace fs = std::filesystem; - namespace DB { @@ -174,6 +178,7 @@ StorageEmbeddedRocksDB::StorageEmbeddedRocksDB(const StorageID & table_id_, const StorageInMemoryMetadata & metadata_, LoadingStrictnessLevel mode, ContextPtr context_, + std::unique_ptr settings_, const String & primary_key_, Int32 ttl_, String rocksdb_dir_, @@ -186,6 +191,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_; @@ -236,22 +242,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); @@ -287,16 +291,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); @@ -352,7 +356,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; @@ -582,8 +585,11 @@ void ReadFromEmbeddedRocksDB::applyFilters(ActionDAGNodes added_filter_nodes) } 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); } @@ -622,7 +628,21 @@ 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.mode, args.getContext(), primary_key_names[0], ttl, std::move(rocksdb_dir), read_only); + auto settings = std::make_unique(); + settings->loadFromQuery(*args.storage_def, args.getContext()); + if (args.storage_def->settings) + metadata.settings_changes = args.storage_def->settings->ptr(); + else + { + /// A workaround because embedded rocksdb doesn't have default immutable settings + /// But InterpreterAlterQuery requires settings_changes to be set to run ALTER MODIFY + /// SETTING queries. So we just add a setting with its default value. + auto settings_changes = std::make_shared(); + settings_changes->is_standalone = false; + settings_changes->changes.insertSetting("optimize_for_bulk_insert", settings->optimize_for_bulk_insert.value); + metadata.settings_changes = settings_changes; + } + return std::make_shared(args.table_id, args.relative_data_path, metadata, args.mode, args.getContext(), std::move(settings), primary_key_names[0], ttl, std::move(rocksdb_dir), read_only); } std::shared_ptr StorageEmbeddedRocksDB::getRocksDBStatistics() const @@ -713,9 +733,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) @@ -737,9 +757,26 @@ std::optional StorageEmbeddedRocksDB::totalBytes(const Settings & /*sett return estimated_bytes; } +void StorageEmbeddedRocksDB::alter( + const AlterCommands & params, + ContextPtr query_context, + AlterLockHolder & holder) +{ + IStorage::alter(params, query_context, holder); + auto new_metadata = getInMemoryMetadataPtr(); + if (new_metadata->settings_changes) + { + const auto & settings_changes = new_metadata->settings_changes->as(); + auto new_settings = std::make_unique(); + new_settings->applyChanges(settings_changes.changes); + setSettings(std::move(new_settings)); + } +} + void registerStorageEmbeddedRocksDB(StorageFactory & factory) { StorageFactory::StorageFeatures features{ + .supports_settings = true, .supports_sort_order = true, .supports_ttl = true, .supports_parallel_insert = true, @@ -747,4 +784,12 @@ void registerStorageEmbeddedRocksDB(StorageFactory & factory) factory.registerStorage("EmbeddedRocksDB", create, features); } + +void StorageEmbeddedRocksDB::checkAlterIsPossible(const AlterCommands & commands, ContextPtr /* context */) const +{ + for (const auto & command : commands) + if (!command.isCommentAlter() && !command.isSettingsAlter()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Alter of type '{}' is not supported by storage {}", command.type, getName()); +} + } diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index 230464a161f..9fc58ea6b38 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -1,11 +1,14 @@ #pragma once #include +#include #include -#include #include #include +#include #include +#include +#include namespace rocksdb @@ -27,6 +30,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 +38,7 @@ public: const StorageInMemoryMetadata & metadata, LoadingStrictnessLevel mode, ContextPtr context_, + std::unique_ptr settings_, const String & primary_key_, Int32 ttl_ = 0, String rocksdb_dir_ = "", @@ -59,6 +64,7 @@ public: void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override; void mutate(const MutationCommands &, ContextPtr) override; void drop() override; + void alter(const AlterCommands & params, ContextPtr query_context, AlterLockHolder &) override; bool optimize( const ASTPtr & query, @@ -99,7 +105,16 @@ public: std::optional totalBytes(const Settings & settings) const override; + void checkAlterIsPossible(const AlterCommands & commands, ContextPtr /* context */) const override; + + const RocksDBSettings & getSettings() const { return *storage_settings.get(); } + + void setSettings(std::unique_ptr && settings_) { storage_settings.set(std::move(settings_)); } + private: + SinkToStoragePtr getSink(ContextPtr context, const StorageMetadataPtr & metadata_snapshot); + + MultiVersion storage_settings; const String primary_key; using RocksDBPtr = std::unique_ptr; RocksDBPtr rocksdb_ptr; 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.reference b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference new file mode 100644 index 00000000000..74c71827e6e --- /dev/null +++ b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference @@ -0,0 +1,10 @@ +0 +1000 +1000 +1 +1000 +2 +1000000 +1000 +0 999001 +1000000 diff --git a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh new file mode 100755 index 00000000000..8acc83fc86c --- /dev/null +++ b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh @@ -0,0 +1,48 @@ +#!/usr/bin/env bash +# Tags: no-ordinary-database, use-rocksdb + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# Normal importing, as we only insert 1000 rows, so it should be in memtable +${CLICKHOUSE_CLIENT} --query "CREATE TABLE IF NOT EXISTS rocksdb_worm (key UInt64, value UInt64) ENGINE = EmbeddedRocksDB() PRIMARY KEY key SETTINGS optimize_for_bulk_insert = 0;" +${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers(1000);" +${CLICKHOUSE_CLIENT} --query "SELECT sum(value) FROM system.rocksdb WHERE database = currentDatabase() AND table = 'rocksdb_worm' AND name = 'no.file.opens';" # should be 0 because all data is still in memtable +${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm;" + +# Enabling bulk insertion +${CLICKHOUSE_CLIENT} --query "ALTER TABLE rocksdb_worm MODIFY SETTING optimize_for_bulk_insert = 1;" + +# Testing that key serialization is identical w. and w/o bulk sink +${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE rocksdb_worm;" +${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+2 FROM numbers(1000);" # should override previous keys +${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm WHERE value = key + 2;" + +# With bulk insertion, there is no memtable, so a small insert should create a new file +${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE rocksdb_worm;" +${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers(1000);" +${CLICKHOUSE_CLIENT} --query "SELECT sum(value) FROM system.rocksdb WHERE database = currentDatabase() AND table = 'rocksdb_worm' AND name = 'no.file.opens';" # should be 1 +${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 +${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm;" + +# Testing insert with duplicated keys +${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE rocksdb_worm;" +${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number % 1000, number+1 FROM numbers_mt(1000000) SETTINGS max_block_size = 100000, max_insert_threads = 1;" +${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm;" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM rocksdb_worm WHERE key = 0;" # should be the latest value - 999001 + + +# Testing insert with multiple threads +${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE rocksdb_worm;" +${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers_mt(1000000)" & +${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers_mt(1000000)" & +wait +${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm;" + diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 1457eacbc92..2b260566816 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1949,6 +1949,8 @@ mdadm meanZTest meanztest mebibytes +memtable +memtables mergeTreeIndex mergeable mergetree From 44b7ec46c0f815b3365ff6f65c2013e534b77ed2 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Fri, 3 May 2024 10:32:55 +0000 Subject: [PATCH 293/624] fix data race Signed-off-by: Duc Canh Le --- src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp index 1aca0edc223..9909bf34d7a 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp @@ -204,7 +204,7 @@ void EmbeddedRocksDBBulkSink::consume(Chunk chunk_) throw Exception(ErrorCodes::ROCKSDB_ERROR, "RocksDB write error: {}", status.ToString()); /// Ingest the SST file - static rocksdb::IngestExternalFileOptions ingest_options; + 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({sst_file_path}, ingest_options); !status.ok()) throw Exception(ErrorCodes::ROCKSDB_ERROR, "RocksDB write error: {}", status.ToString()); From 56afdcb2e3afcb78497c3f5f58db79de38a0cd55 Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Fri, 3 May 2024 12:37:06 +0200 Subject: [PATCH 294/624] Update README.md Will add upcoming release webinars and meetups as a separate exercise. --- README.md | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/README.md b/README.md index 0d50813e30a..43a4d8bb362 100644 --- a/README.md +++ b/README.md @@ -30,23 +30,15 @@ curl https://clickhouse.com/ | sh * [Code Browser (github.dev)](https://github.dev/ClickHouse/ClickHouse) with syntax highlighting, powered by github.dev. * [Contacts](https://clickhouse.com/company/contact) can help to get your questions answered if there are any. -## Monthly Release & Community Call - -Every month we get together with the community (users, contributors, customers, those interested in learning more about ClickHouse) to discuss what is coming in the latest release. If you are interested in sharing what you've built on ClickHouse, let us know. - -* [v24.3 Community Call](https://clickhouse.com/company/events/v24-3-community-release-call) - Mar 26 -* [v24.4 Community Call](https://clickhouse.com/company/events/v24-4-community-release-call) - Apr 30 - ## Upcoming Events 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 Dubai](https://www.meetup.com/clickhouse-dubai-meetup-group/events/299629189/) - May 28 - +* More Coming Soon ## Recent Recordings * **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments" -* **Recording available**: [**v24.2 Release Call**](https://www.youtube.com/watch?v=iN2y-TK8f3A) All the features of 24.2, one convenient video! Watch it now! +* **Recording available**: [**v24.4 Release Call**](https://www.youtube.com/watch?v=dtUqgcfOGmE) All the features of 24.4, one convenient video! Watch it now! ## Interested in joining ClickHouse and making it your full-time job? From 8976116f44633b702e50e2dfb0b7b2adfc9c8785 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 3 May 2024 12:40:01 +0200 Subject: [PATCH 295/624] Fix --- src/Storages/RabbitMQ/RabbitMQConsumer.cpp | 5 ++--- src/Storages/RabbitMQ/RabbitMQSource.cpp | 4 ++-- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 2 +- 3 files changed, 5 insertions(+), 6 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQConsumer.cpp b/src/Storages/RabbitMQ/RabbitMQConsumer.cpp index 54fc9b2088a..ae69ea3ede7 100644 --- a/src/Storages/RabbitMQ/RabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/RabbitMQConsumer.cpp @@ -121,9 +121,8 @@ 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 && 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)) { last_commited_delivery_tag = commit_info.delivery_tag; diff --git a/src/Storages/RabbitMQ/RabbitMQSource.cpp b/src/Storages/RabbitMQ/RabbitMQSource.cpp index bb077e2ea50..eb44e9e2ef3 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSource.cpp @@ -186,8 +186,6 @@ Chunk RabbitMQSource::generateImpl() StreamingFormatExecutor executor(non_virtual_header, input_format, on_error); /// Channel id will not change during read. - commit_info.channel_id = consumer->getChannelID(); - while (true) { exception_message.reset(); @@ -213,6 +211,8 @@ Chunk RabbitMQSource::generateImpl() commit_info.failed_delivery_tags.size(), exception_message.has_value() ? exception_message.value() : "None"); + commit_info.channel_id = message.channel_id; + if (exception_message.has_value() && nack_broken_messages) { commit_info.failed_delivery_tags.push_back(message.delivery_tag); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 7277d31ae1a..5d8fd2c8502 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -982,7 +982,7 @@ void StorageRabbitMQ::streamingToViewsFunc() } catch (...) { - tryLogCurrentException(__PRETTY_FUNCTION__); + LOG_ERROR(log, "Error while streaming to views: {}", getCurrentExceptionMessage(true)); } mv_attached.store(false); From 314573b7a66a2379aecb0b8e680d24561f00e03e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 3 May 2024 12:40:05 +0200 Subject: [PATCH 296/624] Add debug check --- .../MergeTree/MergeTreeReadPoolBase.cpp | 23 ++++++++++++++++++- 1 file changed, 22 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp index 3e10285d6b0..c759a12e151 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp @@ -6,6 +6,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + MergeTreeReadPoolBase::MergeTreeReadPoolBase( RangesInDataParts && parts_, VirtualFields shared_virtual_fields_, @@ -115,7 +120,23 @@ MergeTreeReadTaskPtr MergeTreeReadPoolBase::createTask( auto get_part_name = [](const auto & task_info) -> String { - return task_info.data_part->isProjectionPart() ? task_info.data_part->getParentPartName() : task_info.data_part->name; + const auto & data_part = task_info.data_part; + + if (data_part->isProjectionPart()) + { + auto parent_part_name = data_part->getParentPartName(); + + auto parent_part = data_part->storage.getPartIfExists( + 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 {}", + parent_part_name, data_part->getDataPartStorage().getFullPath()); + + return parent_part_name; + } + + return data_part->name; }; auto extras = getExtras(); From 49741b640dee1e214455f0f9878126a7ae8f5ce0 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 3 May 2024 13:37:40 +0200 Subject: [PATCH 297/624] Skip unaccessible table dirs --- .../System/StorageSystemRemoteDataPaths.cpp | 25 +++++++++++++++++-- 1 file changed, 23 insertions(+), 2 deletions(-) diff --git a/src/Storages/System/StorageSystemRemoteDataPaths.cpp b/src/Storages/System/StorageSystemRemoteDataPaths.cpp index 0a6e692f7e2..a5c496db7e7 100644 --- a/src/Storages/System/StorageSystemRemoteDataPaths.cpp +++ b/src/Storages/System/StorageSystemRemoteDataPaths.cpp @@ -62,6 +62,9 @@ private: /// Moves to the next disk in the list, if no more disks returns false bool nextDisk(); + /// Check if the path is a table path like "store/364/3643ff83-0996-4a4a-a90b-a96e66a10c74" + static bool isTablePath(const fs::path & path); + /// Returns full local path of the current file fs::path getCurrentPath() const { @@ -223,6 +226,19 @@ bool SystemRemoteDataPathsSource::nextDisk() return false; } +/// Check if the path is a table path like "store/364/3643ff83-0996-4a4a-a90b-a96e66a10c74" +bool SystemRemoteDataPathsSource::isTablePath(const fs::path & path) +{ + std::vector components; + for (auto it = path.begin(); it != path.end(); ++it) + components.push_back(it->string()); + + return components.size() == 3 + && components[0] == "store" + && components[1].size() == 3 /// "364" + && components[2].size() == 36; /// "3643ff83-0996-4a4a-a90b-a96e66a10c74" +} + bool SystemRemoteDataPathsSource::nextFile() { while (true) @@ -242,12 +258,12 @@ bool SystemRemoteDataPathsSource::nextFile() if (paths_stack.empty()) return false; + const auto current_path = getCurrentPath(); + try { const auto & disk = disks[current_disk].second; - const auto current_path = getCurrentPath(); - /// Files or directories can disappear due to concurrent operations if (!disk->exists(current_path)) continue; @@ -284,6 +300,11 @@ bool SystemRemoteDataPathsSource::nextFile() if (e.code() == std::errc::no_such_file_or_directory) continue; + /// Skip path if it's table path and we don't have permissions to read it + /// This can happen if the table is being dropped by first chmoding the directory to 000 + if (e.code() == std::errc::permission_denied && isTablePath(current_path)) + continue; + throw; } } From d35813d7d63e5ad188ed85f69954b98e23f06334 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 3 May 2024 14:24:18 +0200 Subject: [PATCH 298/624] The commit url has different pattern --- tests/ci/pr_info.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 975e560437a..9322ee198e4 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -158,7 +158,7 @@ class PRInfo: else: self.sha = github_event["pull_request"]["head"]["sha"] - self.commit_html_url = f"{repo_prefix}/commits/{self.sha}" + self.commit_html_url = f"{repo_prefix}/commit/{self.sha}" self.pr_html_url = f"{repo_prefix}/pull/{self.number}" # master or backport/xx.x/xxxxx - where the PR will be merged @@ -213,7 +213,7 @@ class PRInfo: .replace("{base}", base_sha) .replace("{head}", self.sha) ) - self.commit_html_url = f"{repo_prefix}/commits/{self.sha}" + self.commit_html_url = f"{repo_prefix}/commit/{self.sha}" elif "commits" in github_event: self.event_type = EventType.PUSH @@ -227,7 +227,7 @@ class PRInfo: logging.error("Failed to convert %s to integer", merged_pr) self.sha = github_event["after"] pull_request = get_pr_for_commit(self.sha, github_event["ref"]) - self.commit_html_url = f"{repo_prefix}/commits/{self.sha}" + self.commit_html_url = f"{repo_prefix}/commit/{self.sha}" if pull_request is None or pull_request["state"] == "closed": # it's merged PR to master @@ -296,7 +296,7 @@ class PRInfo: "GITHUB_SHA", "0000000000000000000000000000000000000000" ) self.number = 0 - self.commit_html_url = f"{repo_prefix}/commits/{self.sha}" + self.commit_html_url = f"{repo_prefix}/commit/{self.sha}" self.pr_html_url = f"{repo_prefix}/commits/{ref}" self.base_ref = ref self.base_name = self.repo_full_name From 03c7d1eebe3bd281d72bd1ba33d871fa231490ef Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 3 May 2024 15:32:25 +0200 Subject: [PATCH 299/624] Add test for #56287 --- .../0_stateless/03143_cte_scope.reference | 2 + tests/queries/0_stateless/03143_cte_scope.sql | 43 +++++++++++++++++++ 2 files changed, 45 insertions(+) create mode 100644 tests/queries/0_stateless/03143_cte_scope.reference create mode 100644 tests/queries/0_stateless/03143_cte_scope.sql diff --git a/tests/queries/0_stateless/03143_cte_scope.reference b/tests/queries/0_stateless/03143_cte_scope.reference new file mode 100644 index 00000000000..0e1e7dfa6be --- /dev/null +++ b/tests/queries/0_stateless/03143_cte_scope.reference @@ -0,0 +1,2 @@ +1 2 3 0.3 1 2 4 0.3 +5 6 7 0.4 5 6 8 0.4 diff --git a/tests/queries/0_stateless/03143_cte_scope.sql b/tests/queries/0_stateless/03143_cte_scope.sql new file mode 100644 index 00000000000..1b1d9444651 --- /dev/null +++ b/tests/queries/0_stateless/03143_cte_scope.sql @@ -0,0 +1,43 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/56287 +SET allow_experimental_analyzer = 1; +DROP TABLE IF EXISTS tmp_a; +DROP TABLE IF EXISTS tmp_b; + +CREATE TEMPORARY TABLE IF NOT EXISTS tmp_a +( + k1 Int32, + k2 Int32, + d1 Int32, + d2 Int32 +) ENGINE = Memory; +INSERT INTO tmp_a VALUES (1,2,3,4); +INSERT INTO tmp_a VALUES (5,6,7,8); + +CREATE TEMPORARY TABLE IF NOT EXISTS tmp_b ( + k1 Int32, + k2 Int32, + d0 Float64 +) ENGINE = Memory; +INSERT INTO tmp_b VALUES (1,2,0.3); +INSERT INTO tmp_b VALUES (5,6,0.4); + +SELECT tb1.*,tb2.* +FROM + ( + with tmp0 as (select k1,k2,d1 from tmp_a), + tmp_s as (select k1,k2,d0 from tmp_b), + tmp1 as (select tmp0.*,tmp_s.d0 from tmp0 left join tmp_s on tmp0.k1=tmp_s.k1 and tmp0.k2=tmp_s.k2) + select * from tmp1 + ) as tb1 + LEFT JOIN + ( + with tmp0 as (select k1,k2,d2 from tmp_a), + tmp_s as (select k1,k2,d0 from tmp_b), + tmp1 as (select tmp0.*,tmp_s.d0 from tmp0 left join tmp_s on tmp0.k1=tmp_s.k1 and tmp0.k2=tmp_s.k2) + select * from tmp1 + ) as tb2 + ON tb1.k1=tb2.k1 AND tb1.k2=tb2.k2 +ORDER BY k1; + +DROP TABLE IF EXISTS tmp_a; +DROP TABLE IF EXISTS tmp_b; From f48ef2dc33087c1b236c2d9796ab3d0293eba114 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 3 May 2024 15:32:43 +0200 Subject: [PATCH 300/624] Tweak 00002_log_and_exception_messages_formatting --- .../0_stateless/00002_log_and_exception_messages_formatting.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index 74608360b9c..16ef13d1eb8 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -129,6 +129,7 @@ create temporary table known_short_messages (s String) as select * from (select 'String size is too big ({}), maximum: {}', 'Substitution {} is not set', 'Table {} does not exist', + 'Table {} doesn\'t exist', 'Table {}.{} doesn\'t exist', 'There are duplicate id {}', 'There is no cache by name: {}', @@ -142,6 +143,7 @@ create temporary table known_short_messages (s String) as select * from (select 'Unknown identifier: \'{}\'', 'Unknown input format {}', 'Unknown setting {}', + 'Unknown setting \'{}\'', 'Unknown statistic column: {}', 'Unknown table function {}', 'User has been dropped', From 2732ce1acee4a367b8c57b14fb81a1a6cf0161ea Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 12 Apr 2024 13:59:17 +0200 Subject: [PATCH 301/624] Add a way to mask RUN_URL for Mergeable Check --- tests/ci/commit_status_helper.py | 21 ++++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index c678b195316..fae0a145451 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -425,12 +425,17 @@ def set_mergeable_check( commit: Commit, description: str = "", state: StatusType = SUCCESS, + hide_url: bool = False, ) -> None: - commit.create_status( - context=StatusNames.MERGEABLE, - description=format_description(description), - state=state, - target_url=GITHUB_RUN_URL, + report_url = GITHUB_RUN_URL + if hide_url: + report_url = "" + post_commit_status( + commit, + state, + report_url, + format_description(description), + StatusNames.MERGEABLE, ) @@ -453,7 +458,9 @@ def update_mergeable_check(commit: Commit, pr_info: PRInfo, check_name: str) -> trigger_mergeable_check(commit, statuses) -def trigger_mergeable_check(commit: Commit, statuses: CommitStatuses) -> None: +def trigger_mergeable_check( + commit: Commit, statuses: CommitStatuses, hide_url: bool = False +) -> None: """calculate and update StatusNames.MERGEABLE""" required_checks = [ status for status in statuses if status.context in REQUIRED_CHECKS @@ -486,4 +493,4 @@ def trigger_mergeable_check(commit: Commit, statuses: CommitStatuses) -> None: description = format_description(description) if mergeable_status is None or mergeable_status.description != description: - set_mergeable_check(commit, description, state) + set_mergeable_check(commit, description, state, hide_url) From ccfe3f8cbcbb180004ddceabc33b5339cec9785a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 3 May 2024 13:59:02 +0000 Subject: [PATCH 302/624] fix test --- tests/integration/test_merge_tree_load_marks/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_merge_tree_load_marks/test.py b/tests/integration/test_merge_tree_load_marks/test.py index b066b2a6ec0..a7078017ac9 100644 --- a/tests/integration/test_merge_tree_load_marks/test.py +++ b/tests/integration/test_merge_tree_load_marks/test.py @@ -43,7 +43,7 @@ def test_merge_load_marks(started_cluster, min_bytes_for_wide_part): ) uuid = node.query( - "SELECT uuid FROM system.tables WHERE table = 't_prewarm_merge'" + "SELECT uuid FROM system.tables WHERE table = 't_load_marks'" ).strip() result = node.query( From 1424e8a6de64198c1f7850613f77db56345e4e38 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 3 May 2024 14:19:39 +0000 Subject: [PATCH 303/624] Fix finalize WriteBufferToFileSegment and StatusFile to avoid exceptions in destructor --- src/Common/StatusFile.cpp | 2 ++ src/Interpreters/Cache/WriteBufferToFileSegment.cpp | 9 +++------ src/Interpreters/Cache/WriteBufferToFileSegment.h | 1 - 3 files changed, 5 insertions(+), 7 deletions(-) diff --git a/src/Common/StatusFile.cpp b/src/Common/StatusFile.cpp index 56eb1d4d0cb..ba7595ae6d7 100644 --- a/src/Common/StatusFile.cpp +++ b/src/Common/StatusFile.cpp @@ -86,6 +86,8 @@ StatusFile::StatusFile(std::string path_, FillFunction fill_) /// Write information about current server instance to the file. WriteBufferFromFileDescriptor out(fd, 1024); fill(out); + /// Finalize here to avoid throwing exceptions in destructor. + out.finalize(); } catch (...) { diff --git a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp index 2ac38aeeca7..a593ebfdab2 100644 --- a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp +++ b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp @@ -110,14 +110,11 @@ void WriteBufferToFileSegment::nextImpl() std::unique_ptr WriteBufferToFileSegment::getReadBufferImpl() { + /** Finalize here and we don't need to finalize in the destructor, + * because in case destructor called without `getReadBufferImpl` called, data won't be read. + */ finalize(); return std::make_unique(file_segment->getPath()); } -WriteBufferToFileSegment::~WriteBufferToFileSegment() -{ - /// To be sure that file exists before destructor of segment_holder is called - WriteBufferFromFileDecorator::finalize(); -} - } diff --git a/src/Interpreters/Cache/WriteBufferToFileSegment.h b/src/Interpreters/Cache/WriteBufferToFileSegment.h index 822488ceb48..c4b0491f8c0 100644 --- a/src/Interpreters/Cache/WriteBufferToFileSegment.h +++ b/src/Interpreters/Cache/WriteBufferToFileSegment.h @@ -16,7 +16,6 @@ public: explicit WriteBufferToFileSegment(FileSegmentsHolderPtr segment_holder); void nextImpl() override; - ~WriteBufferToFileSegment() override; private: From 58fb5a9dc4b53c7422c14ccd8d82dde05197d139 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 3 May 2024 10:23:45 -0400 Subject: [PATCH 304/624] Fix issues --- src/Databases/DatabasesCommon.cpp | 15 ++++++++++----- src/Storages/StorageMaterializedView.cpp | 13 +++++++++++-- .../02884_create_view_with_sql_security_option.sh | 2 +- 3 files changed, 22 insertions(+), 8 deletions(-) diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index fc75f8e44b9..d31581bc7fe 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -73,13 +73,18 @@ void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemo if (metadata.sql_security_type) { - auto new_sql_security = std::make_shared(); - new_sql_security->type = metadata.sql_security_type; + /// TODO: remove after we turn `ignore_empty_sql_security_in_create_view_query=false` + /// If materialized view has SQL SECURITY INVOKER we shouldn't dump it on disk. + if (!ast_create_query.is_materialized_view || metadata.sql_security_type != SQLSecurityType::INVOKER) + { + auto new_sql_security = std::make_shared(); + new_sql_security->type = metadata.sql_security_type; - if (metadata.definer) - new_sql_security->definer = std::make_shared(*metadata.definer); + if (metadata.definer) + new_sql_security->definer = std::make_shared(*metadata.definer); - ast_create_query.sql_security = std::move(new_sql_security); + ast_create_query.sql_security = std::move(new_sql_security); + } } /// MaterializedView, Dictionary are types of CREATE query without storage. diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 4c3f90e23ed..2bf48e759da 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -98,6 +98,12 @@ StorageMaterializedView::StorageMaterializedView( local_context->getGlobalContext()); ASTPtr sql_security = query.sql_security; + + /// Materialized view doesn't support SQL SECURITY INVOKER. It's reserved type for backward compatibility + if (sql_security && sql_security->as().type == SQLSecurityType::INVOKER) + throw Exception(ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW, "SQL SECURITY INVOKER can't be specified for MATERIALIZED VIEW"); + + // TODO: remove after we turn `ignore_empty_sql_security_in_create_view_query=false` if (!sql_security) { /// This allows materialized views to be loaded during startup with default SQL security for backward compatibility. @@ -106,8 +112,7 @@ StorageMaterializedView::StorageMaterializedView( sql_security->as().type = SQLSecurityType::INVOKER; } - if (sql_security) - storage_metadata.setSQLSecurity(sql_security->as()); + storage_metadata.setSQLSecurity(sql_security->as()); if (!query.select) throw Exception(ErrorCodes::INCORRECT_QUERY, "SELECT query is not specified for {}", getName()); @@ -225,6 +230,8 @@ void StorageMaterializedView::read( context->checkAccess(AccessType::SELECT, getInMemoryMetadataPtr()->select.select_table_id, column_names); auto storage_id = storage->getStorageID(); + + /// TODO: remove INVOKER check after we turn `ignore_empty_sql_security_in_create_view_query=false` /// We don't need to check access if the inner table was created automatically. if (!has_inner_table && !storage_id.empty() && getInMemoryMetadataPtr()->sql_security_type != SQLSecurityType::INVOKER) context->checkAccess(AccessType::SELECT, storage_id, column_names); @@ -274,6 +281,8 @@ SinkToStoragePtr StorageMaterializedView::write(const ASTPtr & query, const Stor auto metadata_snapshot = storage->getInMemoryMetadataPtr(); auto storage_id = storage->getStorageID(); + + /// TODO: remove INVOKER check after we turn `ignore_empty_sql_security_in_create_view_query=false` /// We don't need to check access if the inner table was created automatically. if (!has_inner_table && !storage_id.empty() && getInMemoryMetadataPtr()->sql_security_type != SQLSecurityType::INVOKER) { diff --git a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh index 48d9a3e220b..9c9df120298 100755 --- a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh +++ b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh @@ -259,6 +259,6 @@ EOF ${CLICKHOUSE_CLIENT} --user $user3 --query "INSERT INTO $db.session_events SELECT * FROM generateRandom('clientId UUID, sessionId UUID, pageId UUID, timestamp DateTime, type Enum(\'type1\', \'type2\')', 1, 10, 2) LIMIT 1000" ${CLICKHOUSE_CLIENT} --user $user3 --query "SELECT count(*) FROM session_events" -${CLICKHOUSE_CLIENT} --user $user3 --query "SELECT count(*) FROM materialized_events" +${CLICKHOUSE_CLIENT} --query "SELECT count(*) FROM materialized_events" ${CLICKHOUSE_CLIENT} --query "DROP USER IF EXISTS $user1, $user2, $user3"; From 8732855d27d577db11f80c046d66bf5805106d01 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 3 May 2024 10:24:57 -0400 Subject: [PATCH 305/624] additional todo --- src/Processors/Transforms/buildPushingToViewsChain.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 91e43dc8134..d5a3591eaa7 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -361,6 +361,8 @@ std::optional generateViewChain( } InterpreterInsertQuery interpreter(nullptr, insert_context, false, false, false); + + /// TODO: remove INVOKER check after we turn `ignore_empty_sql_security_in_create_view_query=false` bool check_access = !materialized_view->hasInnerTable() && materialized_view->getInMemoryMetadataPtr()->sql_security_type != SQLSecurityType::INVOKER; out = interpreter.buildChain(inner_table, inner_metadata_snapshot, insert_columns, thread_status_holder, view_counter_ms, check_access); From feed11efcdbd9e58cf6ab5b1d989dee433154b9c Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 3 May 2024 16:40:29 +0200 Subject: [PATCH 306/624] Removed commented code --- src/Storages/RabbitMQ/RabbitMQConsumer.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/RabbitMQ/RabbitMQConsumer.cpp b/src/Storages/RabbitMQ/RabbitMQConsumer.cpp index ae69ea3ede7..129a74f662a 100644 --- a/src/Storages/RabbitMQ/RabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/RabbitMQConsumer.cpp @@ -226,7 +226,6 @@ 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; } From d02c8da1f7c664c81f6f2f3dc098d6b3a63213b5 Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Fri, 3 May 2024 16:55:31 +0200 Subject: [PATCH 307/624] Update README.md Adding Meetups --- README.md | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index 43a4d8bb362..294c5a36b6a 100644 --- a/README.md +++ b/README.md @@ -30,11 +30,25 @@ curl https://clickhouse.com/ | sh * [Code Browser (github.dev)](https://github.dev/ClickHouse/ClickHouse) with syntax highlighting, powered by github.dev. * [Contacts](https://clickhouse.com/company/contact) can help to get your questions answered if there are any. +## Monthly Release & Community Call + +Every month we get together with the community (users, contributors, customers, those interested in learning more about ClickHouse) to discuss what is coming in the latest release. If you are interested in sharing what you've built on ClickHouse, let us know. + +* [v24.5 Community Call](https://clickhouse.com/company/events/v24-5-community-release-call) - May 30 + ## Upcoming Events 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. -* More Coming Soon +* [ClickHouse Meetup in Bengaluru](https://www.meetup.com/clickhouse-bangalore-user-group/events/300405581/) - May 4 +* [ClickHouse Happy Hour @ Tom's Watch Bar - Los Angelese](https://www.meetup.com/clickhouse-los-angeles-user-group/events/300740584/) - May 22 +* [ClickHouse & Confluent Meetup in Dubai](https://www.meetup.com/clickhouse-dubai-meetup-group/events/299629189/) - May 28 +* [ClickHouse Meetup in Stockholm](https://www.meetup.com/clickhouse-stockholm-user-group/events/299752651/) - Jun 3 +* [ClickHouse Meetup @ Cloudflare - San Francisco](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/300523061/) - Jun 4 +* [ClickHouse (クリックハウス) Meetup Tokyo](https://www.meetup.com/clickhouse-tokyo-user-group/events/300798053/) - Jun 5 +* [ClickHouse Meetup in Amsterdam](https://www.meetup.com/clickhouse-netherlands-user-group/events/300781068/) - Jun 27 +* [ClickHouse Meetup in Paris](https://www.meetup.com/clickhouse-france-user-group/events/300783448/) - Jul 9 +* [ClickHouse Meetup @ Ramp - New York City](https://www.meetup.com/clickhouse-new-york-user-group/events/300595845/) - Jul 9 ## Recent Recordings * **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments" From a55802fbaeeaa29b89f1da336ef7d54c48184ddf Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Fri, 3 May 2024 17:22:42 +0200 Subject: [PATCH 308/624] Update README.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit spelling is hard Co-authored-by: Raúl Marín --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 294c5a36b6a..d7202454332 100644 --- a/README.md +++ b/README.md @@ -41,7 +41,7 @@ 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 Bengaluru](https://www.meetup.com/clickhouse-bangalore-user-group/events/300405581/) - May 4 -* [ClickHouse Happy Hour @ Tom's Watch Bar - Los Angelese](https://www.meetup.com/clickhouse-los-angeles-user-group/events/300740584/) - May 22 +* [ClickHouse Happy Hour @ Tom's Watch Bar - Los Angeles](https://www.meetup.com/clickhouse-los-angeles-user-group/events/300740584/) - May 22 * [ClickHouse & Confluent Meetup in Dubai](https://www.meetup.com/clickhouse-dubai-meetup-group/events/299629189/) - May 28 * [ClickHouse Meetup in Stockholm](https://www.meetup.com/clickhouse-stockholm-user-group/events/299752651/) - Jun 3 * [ClickHouse Meetup @ Cloudflare - San Francisco](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/300523061/) - Jun 4 From d39f2fb2f94a529c135792d2abfd4440c8664c16 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 24 Apr 2024 17:29:59 +0200 Subject: [PATCH 309/624] Make mergeable-check related return CommitStatus --- tests/ci/commit_status_helper.py | 24 +++++++++++++++--------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index fae0a145451..6421ac6f0db 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -85,13 +85,13 @@ def post_commit_status( check_name: Optional[str] = None, pr_info: Optional[PRInfo] = None, dump_to_file: bool = False, -) -> None: +) -> CommitStatus: """The parameters are given in the same order as for commit.create_status, if an optional parameter `pr_info` is given, the `set_status_comment` functions is invoked to add or update the comment with statuses overview""" for i in range(RETRY): try: - commit.create_status( + commit_status = commit.create_status( state=state, target_url=report_url if report_url is not None else NotSet, description=description if description is not None else NotSet, @@ -128,6 +128,8 @@ def post_commit_status( pr_num=pr_info.number, ).dump_status() + return commit_status + STATUS_ICON_MAP = defaultdict( str, @@ -426,11 +428,11 @@ def set_mergeable_check( description: str = "", state: StatusType = SUCCESS, hide_url: bool = False, -) -> None: +) -> CommitStatus: report_url = GITHUB_RUN_URL if hide_url: report_url = "" - post_commit_status( + return post_commit_status( commit, state, report_url, @@ -439,7 +441,9 @@ def set_mergeable_check( ) -def update_mergeable_check(commit: Commit, pr_info: PRInfo, check_name: str) -> None: +def update_mergeable_check( + commit: Commit, pr_info: PRInfo, check_name: str +) -> Optional[CommitStatus]: "check if the check_name in REQUIRED_CHECKS and then trigger update" not_run = ( pr_info.labels.intersection({Labels.SKIP_MERGEABLE_CHECK, Labels.RELEASE}) @@ -450,17 +454,17 @@ def update_mergeable_check(commit: Commit, pr_info: PRInfo, check_name: str) -> if not_run: # Let's avoid unnecessary work - return + return None logging.info("Update Mergeable Check by %s", check_name) statuses = get_commit_filtered_statuses(commit) - trigger_mergeable_check(commit, statuses) + return trigger_mergeable_check(commit, statuses) def trigger_mergeable_check( commit: Commit, statuses: CommitStatuses, hide_url: bool = False -) -> None: +) -> CommitStatus: """calculate and update StatusNames.MERGEABLE""" required_checks = [ status for status in statuses if status.context in REQUIRED_CHECKS @@ -493,4 +497,6 @@ def trigger_mergeable_check( description = format_description(description) if mergeable_status is None or mergeable_status.description != description: - set_mergeable_check(commit, description, state, hide_url) + return set_mergeable_check(commit, description, state, hide_url) + + return mergeable_status From ba644d691e558f94c9ac00104ab6655436cc366a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 3 May 2024 17:41:08 +0200 Subject: [PATCH 310/624] Add a helper function to convert str into StatusType --- tests/ci/report.py | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/tests/ci/report.py b/tests/ci/report.py index a3c9b53637a..8676c998afb 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -49,6 +49,15 @@ def _state_rank(status: str) -> int: return 3 +def get_status(status: str) -> StatusType: + "function to get the StatusType for a status or ERROR" + try: + ind = STATUSES.index(status) # type: ignore + return STATUSES[ind] + except ValueError: + return ERROR + + def get_worst_status(statuses: Iterable[str]) -> StatusType: worst_status = SUCCESS # type: StatusType for status in statuses: From 60336681b287e91b3fbc2a0d3d0ad88d71ab9b18 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 3 May 2024 17:45:39 +0200 Subject: [PATCH 311/624] Add processing `A Sync` to ci.py --- tests/ci/ci.py | 53 ++++++++++++++++++++++++++++++---- tests/ci/env_helper.py | 3 ++ tests/ci/synchronizer_utils.py | 4 +++ 3 files changed, 54 insertions(+), 6 deletions(-) create mode 100644 tests/ci/synchronizer_utils.py diff --git a/tests/ci/ci.py b/tests/ci/ci.py index a95ada628ff..78f8bf182a3 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -17,7 +17,7 @@ from typing import Any, Dict, List, Optional, Sequence, Set, Tuple, Union import docker_images_helper import upload_result_helper from build_check import get_release_or_pr -from ci_config import CI_CONFIG, Build, CILabels, CIStages, JobNames +from ci_config import CI_CONFIG, Build, CILabels, CIStages, JobNames, StatusNames from ci_utils import GHActions, is_hex, normalize_string from clickhouse_helper import ( CiLogsCredentials, @@ -32,15 +32,19 @@ from commit_status_helper import ( RerunHelper, format_description, get_commit, + get_commit_filtered_statuses, post_commit_status, set_status_comment, + trigger_mergeable_check, update_mergeable_check, ) from digest_helper import DockerDigester, JobDigester from env_helper import ( CI, GITHUB_JOB_API_URL, + GITHUB_REPOSITORY, GITHUB_RUN_URL, + GITHUB_UPSTREAM_REPOSITORY, REPO_COPY, REPORT_PATH, S3_BUILDS_BUCKET, @@ -51,8 +55,9 @@ from git_helper import GIT_PREFIX, Git from git_helper import Runner as GitRunner from github_helper import GitHub from pr_info import PRInfo -from report import ERROR, SUCCESS, BuildResult, JobReport +from report import ERROR, SUCCESS, BuildResult, JobReport, get_status from s3_helper import S3Helper +from synchronizer_utils import SYNC_BRANCH_PREFIX from version_helper import get_version_from_repo # pylint: disable=too-many-lines @@ -2106,6 +2111,7 @@ def main() -> int: check_url = log_url else: # test job + gh = GitHub(get_best_robot_token(), per_page=100) additional_urls = [] s3_path_prefix = "/".join( ( @@ -2133,9 +2139,7 @@ def main() -> int: job_report.check_name or _get_ext_check_name(args.job_name), additional_urls=additional_urls or None, ) - commit = get_commit( - GitHub(get_best_robot_token(), per_page=100), pr_info.sha - ) + commit = get_commit(gh, pr_info.sha) post_commit_status( commit, job_report.status, @@ -2147,12 +2151,49 @@ def main() -> int: ) if not pr_info.is_merge_queue: # in the merge queue mergeable status must be set only in FinishCheck (last job in wf) - update_mergeable_check( + mergeable_status = update_mergeable_check( commit, pr_info, job_report.check_name or _get_ext_check_name(args.job_name), ) + # Process upstream StatusNames.SYNC + if ( + pr_info.head_ref.startswith(f"{SYNC_BRANCH_PREFIX}/pr/") + and mergeable_status + and GITHUB_REPOSITORY != GITHUB_UPSTREAM_REPOSITORY + ): + pr_number = int(pr_info.head_ref.split("/pr/", maxsplit=1)[1]) + upstream_repo = gh.get_repo(GITHUB_UPSTREAM_REPOSITORY) + head_sha = upstream_repo.get_pull(pr_number).head.sha + upstream_commit = upstream_repo.get_commit(head_sha) + post_commit_status( + upstream_commit, + get_status(mergeable_status.state), + mergeable_status.target_url, + mergeable_status.description, + StatusNames.SYNC, + ) + trigger_mergeable_check( + upstream_commit, + get_commit_filtered_statuses(upstream_commit), + True, + ) + + prepared_events = prepare_tests_results_for_clickhouse( + pr_info, + [], + job_report.status, + 0, + job_report.start_time, + f"https://github.com/ClickHouse/ClickHouse/pull/{pr_number}", + StatusNames.SYNC, + ) + prepared_events[0]["test_context_raw"] = args.job_name + ch_helper.insert_events_into( + db="default", table="checks", events=prepared_events + ) + print(f"Job report url: [{check_url}]") prepared_events = prepare_tests_results_for_clickhouse( pr_info, diff --git a/tests/ci/env_helper.py b/tests/ci/env_helper.py index 155a1acaca5..9b9652d5bd3 100644 --- a/tests/ci/env_helper.py +++ b/tests/ci/env_helper.py @@ -22,6 +22,9 @@ GITHUB_JOB = os.getenv("GITHUB_JOB_OVERRIDDEN", "") or os.getenv("GITHUB_JOB", " GITHUB_REPOSITORY = os.getenv("GITHUB_REPOSITORY", "ClickHouse/ClickHouse") GITHUB_RUN_ID = os.getenv("GITHUB_RUN_ID", "0") GITHUB_SERVER_URL = os.getenv("GITHUB_SERVER_URL", "https://github.com") +GITHUB_UPSTREAM_REPOSITORY = os.getenv( + "GITHUB_UPSTREAM_REPOSITORY", "ClickHouse/ClickHouse" +) GITHUB_WORKSPACE = os.getenv("GITHUB_WORKSPACE", git_root) GITHUB_RUN_URL = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/actions/runs/{GITHUB_RUN_ID}" IMAGES_PATH = os.getenv("IMAGES_PATH", TEMP_PATH) diff --git a/tests/ci/synchronizer_utils.py b/tests/ci/synchronizer_utils.py new file mode 100644 index 00000000000..b4aa507cce0 --- /dev/null +++ b/tests/ci/synchronizer_utils.py @@ -0,0 +1,4 @@ +#!/usr/bin/env python + +SYNC_BRANCH_PREFIX = "sync-upstream" +SYNC_MASTER_BRANCH = f"{SYNC_BRANCH_PREFIX}/master" From 160b6d5d041d183391b5e59dbd4cfbe4d8d09b15 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 3 May 2024 17:58:07 +0200 Subject: [PATCH 312/624] Add test for #48049 --- .../0_stateless/03144_invalid_filter.reference | 0 tests/queries/0_stateless/03144_invalid_filter.sql | 12 ++++++++++++ 2 files changed, 12 insertions(+) create mode 100644 tests/queries/0_stateless/03144_invalid_filter.reference create mode 100644 tests/queries/0_stateless/03144_invalid_filter.sql diff --git a/tests/queries/0_stateless/03144_invalid_filter.reference b/tests/queries/0_stateless/03144_invalid_filter.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03144_invalid_filter.sql b/tests/queries/0_stateless/03144_invalid_filter.sql new file mode 100644 index 00000000000..b79770a5e35 --- /dev/null +++ b/tests/queries/0_stateless/03144_invalid_filter.sql @@ -0,0 +1,12 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/48049 +SET allow_experimental_analyzer = 1; + +CREATE TABLE test_table (`id` UInt64, `value` String) ENGINE = TinyLog() AS Select number, number::String from numbers(10); + +WITH CAST(tuple(1), 'Tuple (value UInt64)') AS compound_value +SELECT id, test_table.* APPLY x -> compound_value.* +FROM test_table +WHERE arrayMap(x -> toString(x) AS lambda, [NULL, 256, 257, NULL, NULL]) +SETTINGS convert_query_to_cnf = true, optimize_using_constraints = true, optimize_substitute_columns = true; -- { serverError ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER } + +DESCRIBE TABLE (SELECT test_table.COLUMNS(id) FROM test_table WHERE '2147483647'); -- { serverError ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER } From 27db04dbcfdc3f8e5704994381efd815c929392c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 3 May 2024 18:21:17 +0200 Subject: [PATCH 313/624] Hide the mergeable_status.url from cloud CI --- tests/ci/ci.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 78f8bf182a3..d6b1866c563 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -2170,7 +2170,7 @@ def main() -> int: post_commit_status( upstream_commit, get_status(mergeable_status.state), - mergeable_status.target_url, + "", # let's won't expose any urls from cloud mergeable_status.description, StatusNames.SYNC, ) From 9e670fe67cd560638ffd731ba391333bb7b78ee8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 3 May 2024 18:38:09 +0200 Subject: [PATCH 314/624] Fix logical error during SELECT query after ALTER in rare case --- src/Storages/MergeTree/IMergeTreeReader.cpp | 9 +- ...rop_column_zookeeper_on_steroids.reference | 11 ++ ...r_add_drop_column_zookeeper_on_steroids.sh | 149 ++++++++++++++++++ 3 files changed, 168 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03144_parallel_alter_add_drop_column_zookeeper_on_steroids.reference create mode 100755 tests/queries/0_stateless/03144_parallel_alter_add_drop_column_zookeeper_on_steroids.sh diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index cf6b64aac85..54da03d1756 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -152,7 +152,14 @@ void IMergeTreeReader::evaluateMissingDefaults(Block additional_columns, Columns if (res_columns[pos] == nullptr) continue; - additional_columns.insert({res_columns[pos], name_and_type->type, name_and_type->name}); + /// We must take column type from part if it exists. Because at the end of defaults + /// calculations we will materialize ALL the columns, not only missing. + /// If column doesn't exist in part than it will be substituted with default expression + const auto * column_in_part = part_columns.tryGet(name_and_type->name); + if (column_in_part != nullptr) + additional_columns.insert({res_columns[pos], column_in_part->type, name_and_type->name}); + else + additional_columns.insert({res_columns[pos], name_and_type->type, name_and_type->name}); } auto dag = DB::evaluateMissingDefaults( diff --git a/tests/queries/0_stateless/03144_parallel_alter_add_drop_column_zookeeper_on_steroids.reference b/tests/queries/0_stateless/03144_parallel_alter_add_drop_column_zookeeper_on_steroids.reference new file mode 100644 index 00000000000..ed135e928a9 --- /dev/null +++ b/tests/queries/0_stateless/03144_parallel_alter_add_drop_column_zookeeper_on_steroids.reference @@ -0,0 +1,11 @@ +Starting alters +Finishing alters +Equal number of columns +Replication did not hang: synced all replicas of concurrent_alter_add_drop_steroids_ +Consistency: 1 +0 +0 +0 +0 +0 +0 diff --git a/tests/queries/0_stateless/03144_parallel_alter_add_drop_column_zookeeper_on_steroids.sh b/tests/queries/0_stateless/03144_parallel_alter_add_drop_column_zookeeper_on_steroids.sh new file mode 100755 index 00000000000..ea7bb8f7ad0 --- /dev/null +++ b/tests/queries/0_stateless/03144_parallel_alter_add_drop_column_zookeeper_on_steroids.sh @@ -0,0 +1,149 @@ +#!/usr/bin/env bash +# Tags: zookeeper, no-parallel, no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh +# shellcheck source=./replication.lib +. "$CURDIR"/replication.lib + +REPLICAS=3 + +for i in $(seq $REPLICAS); do + $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_alter_add_drop_steroids_$i" +done + + +for i in $(seq $REPLICAS); do + $CLICKHOUSE_CLIENT --query "CREATE TABLE concurrent_alter_add_drop_steroids_$i (key UInt64, value0 UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/concurrent_alter_add_drop_steroids_column', '$i') ORDER BY key SETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192, index_granularity_bytes = '10Mi'" +done + +$CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_alter_add_drop_steroids_1 SELECT number, number + 10 from numbers(100000)" + +for i in $(seq $REPLICAS); do + $CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA concurrent_alter_add_drop_steroids_$i" +done + + +function alter_thread() +{ + while true; do + REPLICA=$(($RANDOM % 3 + 1)) + ADD=$(($RANDOM % 5 + 1)) + $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_add_drop_steroids_$REPLICA ADD COLUMN value$ADD UInt32 DEFAULT 42 SETTINGS replication_alter_partitions_sync=0"; # additionaly we don't wait anything for more heavy concurrency + DROP=$(($RANDOM % 5 + 1)) + $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_add_drop_steroids_$REPLICA DROP COLUMN value$DROP SETTINGS replication_alter_partitions_sync=0"; # additionaly we don't wait anything for more heavy concurrency + sleep 0.$RANDOM + done +} + +function alter_thread_1() +{ + while true; do + REPLICA=$(($RANDOM % 3 + 1)) + ${CLICKHOUSE_CLIENT} --query "ALTER TABLE concurrent_alter_add_drop_steroids_1 MODIFY COLUMN value0 String SETTINGS mutations_sync = 0" + sleep 1.$RANDOM + ${CLICKHOUSE_CLIENT} --query "ALTER TABLE concurrent_alter_add_drop_steroids_1 MODIFY COLUMN value0 UInt8 SETTINGS mutations_sync = 0" + sleep 1.$RANDOM + done + +} + +function optimize_thread() +{ + while true; do + REPLICA=$(($RANDOM % 3 + 1)) + $CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE concurrent_alter_add_drop_steroids_$REPLICA FINAL SETTINGS replication_alter_partitions_sync=0"; + sleep 0.$RANDOM + done +} + +function insert_thread() +{ + while true; do + REPLICA=$(($RANDOM % 3 + 1)) + $CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_alter_add_drop_steroids_$REPLICA VALUES($RANDOM, 7)" + sleep 0.$RANDOM + done +} + +function select_thread() +{ + while true; do + REPLICA=$(($RANDOM % 3 + 1)) + $CLICKHOUSE_CLIENT --query "SELECT * FROM merge(currentDatabase(), 'concurrent_alter_add_drop_steroids_') FORMAT Null" + sleep 0.$RANDOM + done +} + + +echo "Starting alters" +export -f alter_thread; +export -f alter_thread_1; +export -f select_thread; +export -f optimize_thread; +export -f insert_thread; + + +TIMEOUT=30 + +# Sometimes we detach and attach tables +timeout $TIMEOUT bash -c alter_thread 2> /dev/null & +timeout $TIMEOUT bash -c alter_thread 2> /dev/null & +timeout $TIMEOUT bash -c alter_thread 2> /dev/null & + +timeout $TIMEOUT bash -c alter_thread_1 2> /dev/null & +timeout $TIMEOUT bash -c alter_thread_1 2> /dev/null & +timeout $TIMEOUT bash -c alter_thread_1 2> /dev/null & + +timeout $TIMEOUT bash -c select_thread 2> /dev/null & +timeout $TIMEOUT bash -c select_thread 2> /dev/null & +timeout $TIMEOUT bash -c select_thread 2> /dev/null & + +timeout $TIMEOUT bash -c optimize_thread 2> /dev/null & +timeout $TIMEOUT bash -c optimize_thread 2> /dev/null & +timeout $TIMEOUT bash -c optimize_thread 2> /dev/null & + +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & + +wait + +echo "Finishing alters" + +columns1=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_steroids_1' and database='$CLICKHOUSE_DATABASE'" 2> /dev/null) +columns2=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_steroids_2' and database='$CLICKHOUSE_DATABASE'" 2> /dev/null) +columns3=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_steroids_3' and database='$CLICKHOUSE_DATABASE'" 2> /dev/null) + +while [ "$columns1" != "$columns2" ] || [ "$columns2" != "$columns3" ]; do + columns1=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_steroids_1' and database='$CLICKHOUSE_DATABASE'" 2> /dev/null) + columns2=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_steroids_2' and database='$CLICKHOUSE_DATABASE'" 2> /dev/null) + columns3=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_steroids_3' and database='$CLICKHOUSE_DATABASE'" 2> /dev/null) + + sleep 1 +done + +echo "Equal number of columns" + +# This alter will finish all previous, but replica 1 maybe still not up-to-date +while [[ $(timeout 120 ${CLICKHOUSE_CLIENT} --query "ALTER TABLE concurrent_alter_add_drop_steroids_1 MODIFY COLUMN value0 String SETTINGS replication_alter_partitions_sync=2" 2>&1) ]]; do + sleep 1 +done + +check_replication_consistency "concurrent_alter_add_drop_steroids_" "count(), sum(key), sum(cityHash64(value0))" + +for i in $(seq $REPLICAS); do + $CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA concurrent_alter_add_drop_steroids_$i" + $CLICKHOUSE_CLIENT --query "SELECT COUNT() FROM system.mutations WHERE is_done = 0 and table = 'concurrent_alter_add_drop_steroids_$i'" + $CLICKHOUSE_CLIENT --query "SELECT * FROM system.mutations WHERE is_done = 0 and table = 'concurrent_alter_add_drop_steroids_$i'" + $CLICKHOUSE_CLIENT --query "SELECT COUNT() FROM system.replication_queue WHERE table = 'concurrent_alter_add_drop_steroids_$i'" + $CLICKHOUSE_CLIENT --query "SELECT * FROM system.replication_queue WHERE table = 'concurrent_alter_add_drop_steroids_$i' and (type = 'ALTER_METADATA' or type = 'MUTATE_PART')" + + $CLICKHOUSE_CLIENT --query "DETACH TABLE concurrent_alter_add_drop_steroids_$i" + $CLICKHOUSE_CLIENT --query "ATTACH TABLE concurrent_alter_add_drop_steroids_$i" + + $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_alter_add_drop_steroids_$i" +done From b1eb0a8e9c76d322b2719ff59bb2aa023340704d Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Thu, 2 May 2024 15:40:16 +0200 Subject: [PATCH 315/624] Try rollback azurite (cherry picked from commit 967e51c1d6b9478a119a492ffa25449162854be2) --- docker/test/stateless/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index f02a628d13e..f70837207d4 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -89,7 +89,7 @@ ENV MINIO_ROOT_USER="clickhouse" ENV MINIO_ROOT_PASSWORD="clickhouse" ENV EXPORT_S3_STORAGE_POLICIES=1 -RUN npm install -g azurite \ +RUN npm install -g azurite@3.29.0 \ && npm install -g tslib COPY run.sh / From 960440e59aec63c6ebc9f5e590d19bfd8c333a63 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 3 May 2024 16:20:03 +0000 Subject: [PATCH 316/624] add option query_id_prefix to clickhouse-benchmark --- programs/benchmark/Benchmark.cpp | 10 ++++++++++ .../03143_benchmark_query_id_prefix.reference | 1 + .../0_stateless/03143_benchmark_query_id_prefix.sh | 11 +++++++++++ 3 files changed, 22 insertions(+) create mode 100644 tests/queries/0_stateless/03143_benchmark_query_id_prefix.reference create mode 100755 tests/queries/0_stateless/03143_benchmark_query_id_prefix.sh diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp index eecc352d073..48dca82eb2b 100644 --- a/programs/benchmark/Benchmark.cpp +++ b/programs/benchmark/Benchmark.cpp @@ -7,6 +7,8 @@ #include #include #include +#include +#include #include #include #include @@ -80,6 +82,7 @@ public: double max_time_, size_t confidence_, const String & query_id_, + const String & query_id_prefix_, const String & query_to_execute_, size_t max_consecutive_errors_, bool continue_on_errors_, @@ -98,6 +101,7 @@ public: max_time(max_time_), confidence(confidence_), query_id(query_id_), + query_id_prefix(query_id_prefix_), query_to_execute(query_to_execute_), continue_on_errors(continue_on_errors_), max_consecutive_errors(max_consecutive_errors_), @@ -205,6 +209,7 @@ private: double max_time; size_t confidence; String query_id; + String query_id_prefix; String query_to_execute; bool continue_on_errors; size_t max_consecutive_errors; @@ -463,8 +468,11 @@ private: RemoteQueryExecutor executor( *entry, query, {}, global_context, nullptr, Scalars(), Tables(), query_processing_stage); + if (!query_id.empty()) executor.setQueryId(query_id); + else if (!query_id_prefix.empty()) + executor.setQueryId(query_id_prefix + "_" + Poco::UUIDGenerator().createRandom().toString()); Progress progress; executor.setProgressCallback([&progress](const Progress & value) { progress.incrementPiecewiseAtomically(value); }); @@ -617,6 +625,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) ("stacktrace", "print stack traces of exceptions") ("confidence", value()->default_value(5), "set the level of confidence for T-test [0=80%, 1=90%, 2=95%, 3=98%, 4=99%, 5=99.5%(default)") ("query_id", value()->default_value(""), "") + ("query_id_prefix", value()->default_value(""), "") ("max-consecutive-errors", value()->default_value(0), "set number of allowed consecutive errors") ("ignore-error,continue_on_errors", "continue testing even if a query fails") ("reconnect", "establish new connection for every query") @@ -671,6 +680,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) options["timelimit"].as(), options["confidence"].as(), options["query_id"].as(), + options["query_id_prefix"].as(), options["query"].as(), options["max-consecutive-errors"].as(), options.count("ignore-error"), diff --git a/tests/queries/0_stateless/03143_benchmark_query_id_prefix.reference b/tests/queries/0_stateless/03143_benchmark_query_id_prefix.reference new file mode 100644 index 00000000000..d7a23e16ea5 --- /dev/null +++ b/tests/queries/0_stateless/03143_benchmark_query_id_prefix.reference @@ -0,0 +1 @@ +0 100 diff --git a/tests/queries/0_stateless/03143_benchmark_query_id_prefix.sh b/tests/queries/0_stateless/03143_benchmark_query_id_prefix.sh new file mode 100755 index 00000000000..ed68b443c9a --- /dev/null +++ b/tests/queries/0_stateless/03143_benchmark_query_id_prefix.sh @@ -0,0 +1,11 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +query_id_prefix=${CLICKHOUSE_DATABASE}_test_benchmark +$CLICKHOUSE_BENCHMARK -i 100 -c 8 <<< "SELECT 1" --query_id_prefix $query_id_prefix 2>/dev/null + +$CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS" +$CLICKHOUSE_CLIENT --query "SELECT countIf(query_id = '$query_id_prefix'), countIf(query_id LIKE '$query_id_prefix%') FROM system.query_log WHERE current_database = currentDatabase() AND type = 'QueryFinish'" From b991f77d0475fba8bd9c7fd925dcf7f9e3bcb097 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 3 May 2024 17:22:37 +0000 Subject: [PATCH 317/624] randomize setting enable_block_offset_column in stress tests --- docker/test/stateless/stress_tests.lib | 3 ++- tests/config/config.d/block_number.xml | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/docker/test/stateless/stress_tests.lib b/docker/test/stateless/stress_tests.lib index 23f942a00a2..a0865f638a1 100644 --- a/docker/test/stateless/stress_tests.lib +++ b/docker/test/stateless/stress_tests.lib @@ -78,7 +78,8 @@ function configure() randomize_config_boolean_value use_compression zookeeper fi - randomize_config_boolean_value allow_experimental_block_number_column block_number + randomize_config_boolean_value enable_block_number_column block_number + randomize_config_boolean_value enable_block_offset_column block_number # for clickhouse-server (via service) echo "ASAN_OPTIONS='malloc_context_size=10 verbosity=1 allocator_release_to_os_interval_ms=10000'" >> /etc/environment diff --git a/tests/config/config.d/block_number.xml b/tests/config/config.d/block_number.xml index b56f1f1afc2..4b08c37d2f5 100644 --- a/tests/config/config.d/block_number.xml +++ b/tests/config/config.d/block_number.xml @@ -1,6 +1,7 @@ - 0 + 0 + 0 From 77f4b0206efba0cdf6508bfc9694ca9cf224bbe6 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Fri, 3 May 2024 03:21:37 +0000 Subject: [PATCH 318/624] Move s3_plain_rewritable unit test to shell - Rewrite to shell (to execute shell_config.sh) - Add no-shared-merge-tree tag --- .../0_stateless/03008_s3_plain_rewritable.sh | 53 +++++++++++++++++++ .../0_stateless/03008_s3_plain_rewritable.sql | 35 ------------ 2 files changed, 53 insertions(+), 35 deletions(-) create mode 100755 tests/queries/0_stateless/03008_s3_plain_rewritable.sh delete mode 100644 tests/queries/0_stateless/03008_s3_plain_rewritable.sql diff --git a/tests/queries/0_stateless/03008_s3_plain_rewritable.sh b/tests/queries/0_stateless/03008_s3_plain_rewritable.sh new file mode 100755 index 00000000000..d72fc47f689 --- /dev/null +++ b/tests/queries/0_stateless/03008_s3_plain_rewritable.sh @@ -0,0 +1,53 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-shared-merge-tree +# Tag no-fasttest: requires S3 +# Tag no-shared-merge-tree: does not support replication + +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" + +${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( + name = s3_plain_rewritable, + type = s3_plain_rewritable, + endpoint = 'http://localhost:11111/test/test_mt/', + access_key_id = clickhouse, + secret_access_key = clickhouse); +" + +${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); +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} -m --query " +alter table test_mt add projection test_mt_projection (select * order by b)" 2>&1 | grep -Fq "SUPPORT_IS_DISABLED" + +${CLICKHOUSE_CLIENT} -nm --query " +alter table test_mt update c = 0 where a % 2 = 1; +alter table test_mt add column d Int64 after c; +alter table test_mt drop column c; +" 2>&1 | grep -Fq "SUPPORT_IS_DISABLED" + +${CLICKHOUSE_CLIENT} -nm --query " +detach table test_mt; +attach table test_mt; +" + +${CLICKHOUSE_CLIENT} --query "drop table if exists test_mt_dst" + +${CLICKHOUSE_CLIENT} -m --query " +create table test_mt_dst (a Int32, b Int64, c Int64) engine = MergeTree() partition by intDiv(a, 1000) order by tuple(a, b) +settings disk = 's3_plain_rewritable' +" + +${CLICKHOUSE_CLIENT} -m --query " +alter table test_mt move partition 0 to table test_mt_dst" 2>&1 | grep -Fq "SUPPORT_IS_DISABLED" diff --git a/tests/queries/0_stateless/03008_s3_plain_rewritable.sql b/tests/queries/0_stateless/03008_s3_plain_rewritable.sql deleted file mode 100644 index af02ebbd874..00000000000 --- a/tests/queries/0_stateless/03008_s3_plain_rewritable.sql +++ /dev/null @@ -1,35 +0,0 @@ --- Tags: no-fasttest --- Tag: no-fasttest -- requires S3 - -drop table if exists test_mt; -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( - name = s3_plain_rewritable, - type = s3_plain_rewritable, - endpoint = 'http://localhost:11111/test/test_mt/', - access_key_id = clickhouse, - secret_access_key = clickhouse); - -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); - -select count(*) from test_mt; -select (*) from test_mt order by tuple(a, b) limit 10; - -optimize table test_mt final; - -alter table test_mt add projection test_mt_projection ( - select * order by b); -- { serverError SUPPORT_IS_DISABLED } - -alter table test_mt update c = 0 where a % 2 = 1; -- { serverError SUPPORT_IS_DISABLED } -alter table test_mt add column d Int64 after c; -- { serverError SUPPORT_IS_DISABLED } -alter table test_mt drop column c; -- { serverError SUPPORT_IS_DISABLED } - -detach table test_mt; -attach table test_mt; - -drop table if exists test_mt_dst; - -create table test_mt_dst (a Int32, b Int64, c Int64) engine = MergeTree() partition by intDiv(a, 1000) order by tuple(a, b) -settings disk = 's3_plain_rewritable'; -alter table test_mt move partition 0 to table test_mt_dst; -- { serverError SUPPORT_IS_DISABLED } From 031ea22e24527738ba3c8c64346086d4dba3d71e Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 30 Apr 2024 22:00:13 +0000 Subject: [PATCH 319/624] Enable custom parquet encoder by default, attempt 2 --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 55ea2d43713..b3e25d61034 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1123,7 +1123,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 f4a51bc64a7..4a651811a2a 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -101,7 +101,8 @@ static std::map sett {"azure_allow_parallel_part_upload", "true", "true", "Use multiple threads for azure multipart upload."}, {"max_recursive_cte_evaluation_depth", DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, "Maximum limit on recursive CTE evaluation depth"}, {"query_plan_convert_outer_join_to_inner_join", false, true, "Allow to convert OUTER JOIN to INNER JOIN if filter after JOIN always filters default values"}, - }}, + {"output_format_parquet_use_custom_encoder", false, true, "Enable custom Parquet encoder."}, + }}, {"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"}, {"use_page_cache_for_disks_without_file_cache", false, false, "Added userspace page cache"}, From 1af8066cdaf444f8f76fe36a87b46473e065ba24 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 2 May 2024 21:22:36 +0000 Subject: [PATCH 320/624] Fix test --- src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp | 7 +++++-- .../0_stateless/03036_parquet_arrow_nullable.reference | 6 +++--- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp b/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp index 9b51ca0c295..ce859b38b3c 100644 --- a/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp +++ b/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp @@ -33,7 +33,7 @@ /// * `def` and `rep` arrays can be longer than `primitive_column`, because they include nulls and /// empty arrays; the values in primitive_column correspond to positions where def[i] == max_def. /// -/// If you do want to learn it, dremel paper: https://research.google/pubs/pub36632/ +/// If you do want to learn it, see dremel paper: https://research.google/pubs/pub36632/ /// Instead of reading the whole paper, try staring at figures 2-3 for a while - it might be enough. /// (Why does Parquet do all this instead of just storing array lengths and null masks? I'm not /// really sure.) @@ -430,13 +430,16 @@ void prepareColumnNullable( if (schemas[child_schema_idx].repetition_type == parq::FieldRepetitionType::REQUIRED) { - /// Normal case: we just slap a FieldRepetitionType::OPTIONAL onto the nested column. + /// Normal case: the column inside Nullable is a primitive type (not Nullable/Array/Map). + /// Just slap a FieldRepetitionType::OPTIONAL onto it. schemas[child_schema_idx].repetition_type = parq::FieldRepetitionType::OPTIONAL; } else { /// Weird case: Nullable(Nullable(...)). Or Nullable(Tuple(Nullable(...))), etc. /// This is probably not allowed in ClickHouse, but let's support it just in case. + /// The nested column already has a nontrivial repetition type, so we have to wrap it in a + /// group and assign repetition type OPTIONAL to the group. auto & schema = *schemas.insert(schemas.begin() + child_schema_idx, {}); schema.__set_repetition_type(parq::FieldRepetitionType::OPTIONAL); schema.__set_name("nullable"); diff --git a/tests/queries/0_stateless/03036_parquet_arrow_nullable.reference b/tests/queries/0_stateless/03036_parquet_arrow_nullable.reference index 8820bb7cb9f..985f8192f26 100644 --- a/tests/queries/0_stateless/03036_parquet_arrow_nullable.reference +++ b/tests/queries/0_stateless/03036_parquet_arrow_nullable.reference @@ -5,7 +5,7 @@ Arrow a UInt64 a_nullable Nullable(UInt64) Parquet -b Array(Nullable(UInt64)) +b Array(UInt64) b_nullable Array(Nullable(UInt64)) Arrow b Array(Nullable(UInt64)) @@ -21,13 +21,13 @@ d Tuple(\n a UInt64,\n b Tuple(\n a UInt64,\n b String),\n Arrow d Tuple(\n a UInt64,\n b Tuple(\n a UInt64,\n b String),\n d_nullable Tuple(\n a UInt64,\n b Tuple(\n a Nullable(UInt64),\n b Nullable(String)))) Parquet -e Map(UInt64, Nullable(String)) +e Map(UInt64, String) e_nullable Map(UInt64, Nullable(String)) Arrow e Map(UInt64, Nullable(String)) e_nullable Map(UInt64, Nullable(String)) Parquet -f Map(UInt64, Map(UInt64, Nullable(String))) +f Map(UInt64, Map(UInt64, String)) f_nullables Map(UInt64, Map(UInt64, Nullable(String))) Arrow f Map(UInt64, Map(UInt64, Nullable(String))) From 6188338710cabf9c07a7cc5511d8f307459ee2c1 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 3 May 2024 20:02:42 +0000 Subject: [PATCH 321/624] May --- src/Core/SettingsChangesHistory.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 4a651811a2a..becd20bc66f 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -87,6 +87,7 @@ static std::map sett { {"24.5", {{"allow_deprecated_functions", true, false, "Allow usage of deprecated functions"}, {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, + {"output_format_parquet_use_custom_encoder", false, true, "Enable custom Parquet encoder."}, }}, {"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"}, @@ -101,8 +102,7 @@ static std::map sett {"azure_allow_parallel_part_upload", "true", "true", "Use multiple threads for azure multipart upload."}, {"max_recursive_cte_evaluation_depth", DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, "Maximum limit on recursive CTE evaluation depth"}, {"query_plan_convert_outer_join_to_inner_join", false, true, "Allow to convert OUTER JOIN to INNER JOIN if filter after JOIN always filters default values"}, - {"output_format_parquet_use_custom_encoder", false, true, "Enable custom Parquet encoder."}, - }}, + }}, {"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"}, {"use_page_cache_for_disks_without_file_cache", false, false, "Added userspace page cache"}, From bc81b084d7a3876b97b9b3c598d9323fc4ce436f Mon Sep 17 00:00:00 2001 From: Maxim Alexeev Date: Fri, 3 May 2024 23:15:16 +0300 Subject: [PATCH 322/624] changed settings --- src/Core/Settings.h | 4 ++-- src/Core/SettingsChangesHistory.h | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2715cb52035..9310bd8bdb2 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -456,8 +456,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(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, cross_join_min_rows_to_compress, 4000, "Minimal count of rows to compress block in CROSS JOIN", 0) \ + M(UInt64, cross_join_min_bytes_to_compress, 1000000, "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 ad9bdfb9e8d..50864c58bef 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -86,6 +86,8 @@ namespace SettingsChangesHistory static std::map settings_changes_history = { {"24.5", {{"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, + {"cross_join_min_rows_to_compress", 1000, 1000, "A new setting."}, + {"cross_join_min_bytes_to_compress", 10000, 10000, "A new setting."}, }}, {"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"}, @@ -122,8 +124,6 @@ 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_min_rows_to_compress", 1000, 1000, "A new setting."}, - {"cross_join_min_bytes_to_compress", 10000, 10000, "A new setting."}, {"allow_experimental_analyzer", false, true, "Enable analyzer and planner by default."}, {"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."}, From 05ba6fd54c2cff0851706b2cdc8e4c093ff3f27a Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 2 May 2024 21:32:52 +0000 Subject: [PATCH 323/624] Fix 00002_log_and_exception_messages_formatting (likely unrelated to this PR) --- .../00002_log_and_exception_messages_formatting.sql | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index 74608360b9c..41428736cf0 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -130,6 +130,8 @@ create temporary table known_short_messages (s String) as select * from (select 'Substitution {} is not set', 'Table {} does not exist', 'Table {}.{} doesn\'t exist', + 'Table {} doesn\'t exist', + 'Table {} is not empty', 'There are duplicate id {}', 'There is no cache by name: {}', 'Too large node state size', @@ -139,9 +141,11 @@ create temporary table known_short_messages (s String) as select * from (select 'Unknown BSON type: {}', 'Unknown explain kind \'{}\'', 'Unknown format {}', + 'Unknown geometry type {}', 'Unknown identifier: \'{}\'', 'Unknown input format {}', 'Unknown setting {}', + 'Unknown setting \'{}\'', 'Unknown statistic column: {}', 'Unknown table function {}', 'User has been dropped', From 835c6b1c3371116166df15f2366cab23f1360f46 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 3 May 2024 20:18:37 +0000 Subject: [PATCH 324/624] Fix another 00002_log_and_exception_messages_formatting flakiness --- .../0_stateless/00002_log_and_exception_messages_formatting.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index 41428736cf0..32d1bd26d2b 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -20,6 +20,7 @@ SELECT length(message_format_string) = 0 AND message not like '% Received from %clickhouse-staging.com:9440%' AND source_file not like '%/AWSLogger.cpp%' + AND logger_name not in ('RaftInstance') GROUP BY message ORDER BY c LIMIT 10 )) FROM logs From 22d695c9952e167b445b8fc96cf9e97177917db3 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 3 May 2024 20:22:09 +0000 Subject: [PATCH 325/624] Fix yet another 00002_log_and_exception_messages_formatting flakiness --- .../0_stateless/00002_log_and_exception_messages_formatting.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index 32d1bd26d2b..32576b0d48a 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -20,6 +20,7 @@ SELECT length(message_format_string) = 0 AND message not like '% Received from %clickhouse-staging.com:9440%' AND source_file not like '%/AWSLogger.cpp%' + AND source_file not like '%/BaseDaemon.cpp%' AND logger_name not in ('RaftInstance') GROUP BY message ORDER BY c LIMIT 10 )) From 3317234bbf550216c13e04dfef076bc0a97e8712 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 3 May 2024 20:27:50 +0000 Subject: [PATCH 326/624] And another one --- .../00002_log_and_exception_messages_formatting.sql | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index 32576b0d48a..18ec7a9952e 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -222,7 +222,10 @@ select 'noisy Debug messages', -- Same as above for Info WITH 0.05 as threshold select 'noisy Info messages', - greatest(coalesce(((select message_format_string, count() from logs where level = 'Information' group by message_format_string order by count() desc limit 1) as top_message).2, 0) / (select count() from logs), threshold) as r, + greatest(coalesce(((select message_format_string, count() from logs + where level = 'Information' + and message_format_string not in ('Sorting and writing part of data into temporary file {}', 'Done writing part of data into temporary file {}, compressed {}, uncompressed {}') + group by message_format_string order by count() desc limit 1) as top_message).2, 0) / (select count() from logs), threshold) as r, r <= threshold ? '' : top_message.1; -- Same as above for Warning From 37f6e331c6f172b72c8dd174e39cf5d5353f4f01 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 2 May 2024 19:12:38 +0000 Subject: [PATCH 327/624] DiskApp: fix 'list --recursive /' and crash on invalid arguments --- programs/disks/CommandList.cpp | 2 +- programs/disks/DisksApp.cpp | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/programs/disks/CommandList.cpp b/programs/disks/CommandList.cpp index ea84cd0682d..7213802ea86 100644 --- a/programs/disks/CommandList.cpp +++ b/programs/disks/CommandList.cpp @@ -84,7 +84,7 @@ private: for (const auto & file_name : file_names) { - auto path = relative_path + "/" + file_name; + auto path = relative_path.empty() ? file_name : (relative_path + "/" + file_name); if (disk->isDirectory(path)) listRecursive(disk, path); } diff --git a/programs/disks/DisksApp.cpp b/programs/disks/DisksApp.cpp index 6c768799221..5da5ab4bae9 100644 --- a/programs/disks/DisksApp.cpp +++ b/programs/disks/DisksApp.cpp @@ -97,7 +97,8 @@ void DisksApp::processOptions() DisksApp::~DisksApp() { - global_context->shutdown(); + if (global_context) + global_context->shutdown(); } void DisksApp::init(std::vector & common_arguments) From c63cc7eb0c4771be6c63f9e34d4183e2d7411cf7 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 3 May 2024 16:50:49 -0400 Subject: [PATCH 328/624] better --- src/Databases/DatabasesCommon.cpp | 15 ++++------- .../Transforms/buildPushingToViewsChain.cpp | 4 +-- src/Storages/StorageMaterializedView.cpp | 26 ++++++------------- 3 files changed, 15 insertions(+), 30 deletions(-) diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index d31581bc7fe..fc75f8e44b9 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -73,18 +73,13 @@ void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemo if (metadata.sql_security_type) { - /// TODO: remove after we turn `ignore_empty_sql_security_in_create_view_query=false` - /// If materialized view has SQL SECURITY INVOKER we shouldn't dump it on disk. - if (!ast_create_query.is_materialized_view || metadata.sql_security_type != SQLSecurityType::INVOKER) - { - auto new_sql_security = std::make_shared(); - new_sql_security->type = metadata.sql_security_type; + auto new_sql_security = std::make_shared(); + new_sql_security->type = metadata.sql_security_type; - if (metadata.definer) - new_sql_security->definer = std::make_shared(*metadata.definer); + if (metadata.definer) + new_sql_security->definer = std::make_shared(*metadata.definer); - ast_create_query.sql_security = std::move(new_sql_security); - } + ast_create_query.sql_security = std::move(new_sql_security); } /// MaterializedView, Dictionary are types of CREATE query without storage. diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index d5a3591eaa7..5e8ecdca95e 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -362,8 +362,8 @@ std::optional generateViewChain( InterpreterInsertQuery interpreter(nullptr, insert_context, false, false, false); - /// TODO: remove INVOKER check after we turn `ignore_empty_sql_security_in_create_view_query=false` - bool check_access = !materialized_view->hasInnerTable() && materialized_view->getInMemoryMetadataPtr()->sql_security_type != SQLSecurityType::INVOKER; + /// TODO: remove sql_security_type check after we turn `ignore_empty_sql_security_in_create_view_query=false` + bool check_access = !materialized_view->hasInnerTable() && materialized_view->getInMemoryMetadataPtr()->sql_security_type; out = interpreter.buildChain(inner_table, inner_metadata_snapshot, insert_columns, thread_status_holder, view_counter_ms, check_access); if (interpreter.shouldAddSquashingFroStorage(inner_table)) diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 2bf48e759da..7e27b1d5005 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -97,23 +97,13 @@ StorageMaterializedView::StorageMaterializedView( storage_metadata.columns, local_context->getGlobalContext()); - ASTPtr sql_security = query.sql_security; + if (query.sql_security) + storage_metadata.setSQLSecurity(query.sql_security->as()); - /// Materialized view doesn't support SQL SECURITY INVOKER. It's reserved type for backward compatibility - if (sql_security && sql_security->as().type == SQLSecurityType::INVOKER) + /// Materialized view doesn't support SQL SECURITY INVOKER. + if (storage_metadata.sql_security_type == SQLSecurityType::INVOKER) throw Exception(ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW, "SQL SECURITY INVOKER can't be specified for MATERIALIZED VIEW"); - // TODO: remove after we turn `ignore_empty_sql_security_in_create_view_query=false` - if (!sql_security) - { - /// This allows materialized views to be loaded during startup with default SQL security for backward compatibility. - /// If ClickHouse loads an old materialized view created without SQL security, it will use the default `SQL SECURITY INVOKER` - sql_security = std::make_shared(); - sql_security->as().type = SQLSecurityType::INVOKER; - } - - storage_metadata.setSQLSecurity(sql_security->as()); - if (!query.select) throw Exception(ErrorCodes::INCORRECT_QUERY, "SELECT query is not specified for {}", getName()); @@ -231,9 +221,9 @@ void StorageMaterializedView::read( auto storage_id = storage->getStorageID(); - /// TODO: remove INVOKER check after we turn `ignore_empty_sql_security_in_create_view_query=false` + /// TODO: remove sql_security_type check after we turn `ignore_empty_sql_security_in_create_view_query=false` /// We don't need to check access if the inner table was created automatically. - if (!has_inner_table && !storage_id.empty() && getInMemoryMetadataPtr()->sql_security_type != SQLSecurityType::INVOKER) + if (!has_inner_table && !storage_id.empty() && getInMemoryMetadataPtr()->sql_security_type) context->checkAccess(AccessType::SELECT, storage_id, column_names); storage->read(query_plan, column_names, target_storage_snapshot, query_info, context, processed_stage, max_block_size, num_streams); @@ -282,9 +272,9 @@ SinkToStoragePtr StorageMaterializedView::write(const ASTPtr & query, const Stor auto storage_id = storage->getStorageID(); - /// TODO: remove INVOKER check after we turn `ignore_empty_sql_security_in_create_view_query=false` + /// TODO: remove sql_security_type check after we turn `ignore_empty_sql_security_in_create_view_query=false` /// We don't need to check access if the inner table was created automatically. - if (!has_inner_table && !storage_id.empty() && getInMemoryMetadataPtr()->sql_security_type != SQLSecurityType::INVOKER) + if (!has_inner_table && !storage_id.empty() && getInMemoryMetadataPtr()->sql_security_type) { auto query_sample_block = InterpreterInsertQuery::getSampleBlock(query->as(), storage, metadata_snapshot, context); context->checkAccess(AccessType::INSERT, storage_id, query_sample_block.getNames()); From 7857d018697513cdbce319729b5ee7233bf22d47 Mon Sep 17 00:00:00 2001 From: Volodya Date: Fri, 3 May 2024 22:15:18 +0000 Subject: [PATCH 329/624] refactoring --- .../Formats/Impl/PrettyBlockOutputFormat.cpp | 146 ++++++------------ .../Formats/Impl/PrettyBlockOutputFormat.h | 8 +- .../Impl/PrettyCompactBlockOutputFormat.cpp | 25 ++- .../Impl/PrettySpaceBlockOutputFormat.cpp | 26 ++-- 4 files changed, 75 insertions(+), 130 deletions(-) diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index 9f453be8827..6339c26b5e3 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 no_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.max_value_width_apply_for_single_value || num_rows != 1 || num_columns != 1 || total_rows != 0; /// Calculate widths of all values. String serialized_value; @@ -70,7 +70,7 @@ void PrettyBlockOutputFormat::calculateWidths( } widths[i][j] = UTF8::computeWidth(reinterpret_cast(serialized_value.data()), serialized_value.size(), prefix); - if (!no_need_cut_to_width && serialized_value.contains('\n')) + if (need_cut_to_width && serialized_value.contains('\n')) { size_t row_width = 0; size_t row_start = 0; @@ -327,22 +327,18 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind if (j != 0) writeCString(grid_symbols.bar, out); const auto & type = *header.getByPosition(j).type; - bool has_break_line = false; - writeValueWithPadding(*columns[j], *serializations[j], i, - widths[j].empty() ? max_widths[j] : widths[j][i], - max_widths[j], cut_to_width, type.shouldAlignRightInPrettyFormats(), isNumber(type), has_break_line); - - if (has_break_line) + size_t cur_width = widths[j].empty() ? max_widths[j] : widths[j][i]; + String serialized_value; { - has_transferred_row = true; - String serialized_value = " "; - { - WriteBufferFromString out_serialize(serialized_value, AppendModeTag()); - serializations[j]->serializeText(*columns[j], i, out_serialize, format_settings); - } - size_t break_line_pos = serialized_value.find_first_of('\n'); - transferred_row[j] = serialized_value.substr(break_line_pos + 1); + WriteBufferFromString out_serialize(serialized_value, AppendModeTag()); + serializations[j]->serializeText(*columns[j], i, out_serialize, format_settings); } + if (cut_to_width) + splitValueAtBreakLine(serialized_value, transferred_row[j], cur_width); + has_transferred_row |= !transferred_row[j].empty() && cur_width <= cut_to_width; + + writeValueWithPadding(serialized_value, cur_width, max_widths[j], cut_to_width, + type.shouldAlignRightInPrettyFormats(), isNumber(type), !transferred_row[j].empty(), false); } writeCString(grid_symbols.bar, out); @@ -350,7 +346,7 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind writeCString("\n", out); if (has_transferred_row) - writeTransferredRow(max_widths, transferred_row, false); + writeTransferredRow(max_widths, header, transferred_row, cut_to_width, false); } if (format_settings.pretty.output_format_pretty_row_numbers) @@ -431,45 +427,32 @@ static String highlightDigitGroups(String source) void PrettyBlockOutputFormat::writeValueWithPadding( - const IColumn & column, const ISerialization & serialization, size_t row_num, - size_t value_width, size_t pad_to_width, size_t cut_to_width, bool align_right, bool is_number, bool & has_break_line) + String & value, size_t value_width, size_t pad_to_width, size_t cut_to_width, + bool align_right, bool is_number, bool has_break_line, bool is_transferred_value) { - String serialized_value = " "; - { - WriteBufferFromString out_serialize(serialized_value, AppendModeTag()); - serialization.serializeText(column, row_num, out_serialize, format_settings); - } - - size_t break_line_pos = String::npos; - if (cut_to_width) - break_line_pos = serialized_value.find_first_of('\n'); - if (break_line_pos != String::npos) - { - has_break_line = true; - serialized_value = serialized_value.substr(0, break_line_pos); - value_width = UTF8::computeWidth(reinterpret_cast(serialized_value.data() + 1), serialized_value.size()); - } + if (is_transferred_value) + writeString("…", out); + else + writeChar(' ', out); if (cut_to_width && value_width > cut_to_width) { - serialized_value.resize(UTF8::computeBytesBeforeWidth( - reinterpret_cast(serialized_value.data()), serialized_value.size(), 0, 1 + format_settings.pretty.max_value_width)); + value.resize(UTF8::computeBytesBeforeWidth( + reinterpret_cast(value.data()), value.size(), 0, format_settings.pretty.max_value_width)); const char * ellipsis = format_settings.pretty.charset == FormatSettings::Pretty::Charset::UTF8 ? "⋯" : "~"; if (color) { - serialized_value += "\033[31;1m"; - serialized_value += ellipsis; - serialized_value += "\033[0m"; + value += "\033[31;1m"; + value += ellipsis; + value += "\033[0m"; } else - serialized_value += ellipsis; + value += ellipsis; value_width = format_settings.pretty.max_value_width; has_break_line = false; } - else if (!has_break_line) - serialized_value += ' '; auto write_padding = [&]() { @@ -480,24 +463,26 @@ void PrettyBlockOutputFormat::writeValueWithPadding( /// Highlight groups of thousands. if (color && is_number && format_settings.pretty.highlight_digit_groups) - serialized_value = highlightDigitGroups(serialized_value); + value = highlightDigitGroups(value); if (align_right) { write_padding(); - out.write(serialized_value.data(), serialized_value.size()); + out.write(value.data(), value.size()); } else { - out.write(serialized_value.data(), serialized_value.size()); + out.write(value.data(), value.size()); write_padding(); } if (has_break_line) writeString("…", out); + else if (value_width != format_settings.pretty.max_value_width) + writeChar(' ', out); } -void PrettyBlockOutputFormat::writeTransferredRow(const Widths & max_widths, const std::vector & transferred_row, const bool & space_block) +void PrettyBlockOutputFormat::writeTransferredRow(const Widths & max_widths, const Block & header, std::vector & transferred_row, size_t cut_to_width, bool space_block) { const GridSymbols & grid_symbols = format_settings.pretty.charset == FormatSettings::Pretty::Charset::UTF8 ? utf8_grid_symbols : @@ -514,7 +499,6 @@ void PrettyBlockOutputFormat::writeTransferredRow(const Widths & max_widths, con std::vector new_transferred_row(num_columns); bool has_transferred_row = false; - size_t cur_width = 0; for (size_t j = 0; j < num_columns; ++j) { @@ -523,57 +507,14 @@ void PrettyBlockOutputFormat::writeTransferredRow(const Widths & max_widths, con else if (j != 0) writeCString(" ", out); - String value = transferred_row[j]; - cur_width = UTF8::computeWidth(reinterpret_cast(value.data()), value.size()); - bool has_break_line = false; + const auto & type = *header.getByPosition(j).type; + size_t cur_width = UTF8::computeWidth(reinterpret_cast(transferred_row[j].data()), transferred_row[j].size()); + if (cut_to_width) + splitValueAtBreakLine(transferred_row[j], new_transferred_row[j], cur_width); + has_transferred_row |= !new_transferred_row[j].empty() && cur_width <= cut_to_width; - if (size_t break_line_pos = value.find_first_of('\n'); break_line_pos != String::npos) - { - has_transferred_row = true; - new_transferred_row[j] = value.substr(break_line_pos + 1); - value = value.substr(0, break_line_pos); - cur_width = UTF8::computeWidth(reinterpret_cast(value.data()), value.size()); - has_break_line = true; - } - - if (cur_width > format_settings.pretty.max_value_width) - { - value.resize(UTF8::computeBytesBeforeWidth( - reinterpret_cast(value.data()), value.size(), 0, 1 + format_settings.pretty.max_value_width)); - - const char * ellipsis = format_settings.pretty.charset == FormatSettings::Pretty::Charset::UTF8 ? "⋯" : "~"; - if (color) - { - value += "\033[31;1m"; - value += ellipsis; - value += "\033[0m"; - } - else - value += ellipsis; - - cur_width = format_settings.pretty.max_value_width; - has_break_line = false; - } - - if (!value.empty()) - writeString("…", out); - else - writeChar(' ', out); - - auto write_padding = [&]() - { - if (max_widths[j] > cur_width) - for (size_t k = 0; k < max_widths[j] - cur_width; ++k) - writeChar(' ', out); - }; - - out.write(value.data(), value.size()); - write_padding(); - - if (has_break_line) - writeString("…", out); - else if (cur_width != format_settings.pretty.max_value_width) - writeChar(' ', out); + writeValueWithPadding(transferred_row[j], cur_width, max_widths[j], cut_to_width, + type.shouldAlignRightInPrettyFormats(), isNumber(type), !new_transferred_row[j].empty(), !transferred_row[j].empty()); } if (!space_block) @@ -581,7 +522,16 @@ void PrettyBlockOutputFormat::writeTransferredRow(const Widths & max_widths, con writeCString("\n", out); if (has_transferred_row) - writeTransferredRow(max_widths, new_transferred_row, space_block); + writeTransferredRow(max_widths, header, new_transferred_row, cut_to_width, space_block); +} + +void PrettyBlockOutputFormat::splitValueAtBreakLine(String & value, String & transferred_value, size_t & value_width) { + if (size_t break_line_pos = value.find_first_of('\n'); break_line_pos != String::npos) + { + transferred_value = value.substr(break_line_pos + 1); + value = value.substr(0, break_line_pos); + value_width = UTF8::computeWidth(reinterpret_cast(value.data()), value.size()); + } } diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h index 5173610024c..6673c61c61b 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h @@ -47,10 +47,12 @@ protected: WidthsPerColumn & widths, Widths & max_padded_widths, Widths & name_widths); void writeValueWithPadding( - const IColumn & column, const ISerialization & serialization, size_t row_num, - size_t value_width, size_t pad_to_width, size_t cut_to_width, bool align_right, bool is_number, bool & has_break_line); + String & value, size_t value_width, size_t pad_to_width, size_t cut_to_width, + bool align_right, bool is_number, bool has_break_line, bool is_transferred_value); - void writeTransferredRow(const Widths & max_widths, const std::vector & transferred_row, const bool & space_block); + void writeTransferredRow(const Widths & max_widths, const Block & header, std::vector & transferred_row, size_t cut_to_width, bool space_block); + + void splitValueAtBreakLine(String & value, String & transferred_value, size_t & value_width); void resetFormatterImpl() override { diff --git a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp index f5fe9496984..345b6c84ecd 100644 --- a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp @@ -176,21 +176,18 @@ void PrettyCompactBlockOutputFormat::writeRow( writeCString(grid_symbols.bar, out); const auto & type = *header.getByPosition(j).type; - const auto & cur_widths = widths[j].empty() ? max_widths[j] : widths[j][row_num]; - bool has_break_line = false; - writeValueWithPadding(*columns[j], *serializations[j], row_num, cur_widths, max_widths[j], cut_to_width, type.shouldAlignRightInPrettyFormats(), isNumber(type), has_break_line); - - if (has_break_line) + size_t cur_width = widths[j].empty() ? max_widths[j] : widths[j][row_num]; + String serialized_value; { - has_transferred_row = true; - String serialized_value = " "; - { - WriteBufferFromString out_serialize(serialized_value, AppendModeTag()); - serializations[j]->serializeText(*columns[j], row_num, out_serialize, format_settings); - } - size_t break_line_pos = serialized_value.find_first_of('\n'); - transferred_row[j] = serialized_value.substr(break_line_pos + 1); + WriteBufferFromString out_serialize(serialized_value, AppendModeTag()); + serializations[j]->serializeText(*columns[j], row_num, out_serialize, format_settings); } + if (cut_to_width) + splitValueAtBreakLine(serialized_value, transferred_row[j], cur_width); + has_transferred_row |= !transferred_row[j].empty() && cur_width <= cut_to_width; + + writeValueWithPadding(serialized_value, cur_width, max_widths[j], cut_to_width, + type.shouldAlignRightInPrettyFormats(), isNumber(type), !transferred_row[j].empty(), false); } writeCString(grid_symbols.bar, out); @@ -198,7 +195,7 @@ void PrettyCompactBlockOutputFormat::writeRow( writeCString("\n", out); if (has_transferred_row) - writeTransferredRow(max_widths, transferred_row, false); + writeTransferredRow(max_widths, header, transferred_row, cut_to_width, false); } void PrettyCompactBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind) diff --git a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp index 30eb9b6417a..6940c20e25b 100644 --- a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp @@ -94,29 +94,25 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port writeCString(" ", out); const auto & type = *header.getByPosition(column).type; - auto & cur_width = widths[column].empty() ? max_widths[column] : widths[column][row]; - bool has_break_line = false; - writeValueWithPadding(*columns[column], *serializations[column], - row, cur_width, max_widths[column], cut_to_width, type.shouldAlignRightInPrettyFormats(), isNumber(type), has_break_line); - - if (has_break_line) + size_t cur_width = widths[column].empty() ? max_widths[column] : widths[column][row]; + String serialized_value; { - has_transferred_row = true; - String serialized_value = " "; - { - WriteBufferFromString out_serialize(serialized_value, AppendModeTag()); - serializations[column]->serializeText(*columns[column], row, out_serialize, format_settings); - } - size_t break_line_pos = serialized_value.find_first_of('\n'); - transferred_row[column] = serialized_value.substr(break_line_pos + 1); + WriteBufferFromString out_serialize(serialized_value, AppendModeTag()); + serializations[column]->serializeText(*columns[column], row, out_serialize, format_settings); } + if (cut_to_width) + splitValueAtBreakLine(serialized_value, transferred_row[column], cur_width); + has_transferred_row |= !transferred_row[column].empty() && cur_width <= cut_to_width; + + writeValueWithPadding(serialized_value, cur_width, max_widths[column], cut_to_width, + type.shouldAlignRightInPrettyFormats(), isNumber(type), !transferred_row[column].empty(), false); } writeReadableNumberTip(chunk); writeChar('\n', out); if (has_transferred_row) - writeTransferredRow(max_widths, transferred_row, true); + writeTransferredRow(max_widths, header, transferred_row, cut_to_width, true); } total_rows += num_rows; From 081e6f7af1cb937d80cb6c1765fb09e9a2a37a50 Mon Sep 17 00:00:00 2001 From: Volodya Date: Fri, 3 May 2024 22:33:25 +0000 Subject: [PATCH 330/624] code style --- src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index 6339c26b5e3..24716529583 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -525,7 +525,8 @@ void PrettyBlockOutputFormat::writeTransferredRow(const Widths & max_widths, con writeTransferredRow(max_widths, header, new_transferred_row, cut_to_width, space_block); } -void PrettyBlockOutputFormat::splitValueAtBreakLine(String & value, String & transferred_value, size_t & value_width) { +void PrettyBlockOutputFormat::splitValueAtBreakLine(String & value, String & transferred_value, size_t & value_width) +{ if (size_t break_line_pos = value.find_first_of('\n'); break_line_pos != String::npos) { transferred_value = value.substr(break_line_pos + 1); From 300f353cd1762a6577795edc38eb5f85ff8cbe18 Mon Sep 17 00:00:00 2001 From: Volodya Date: Fri, 3 May 2024 23:14:54 +0000 Subject: [PATCH 331/624] if width == max_width need padding --- src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index 24716529583..178d0b912e1 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -453,6 +453,8 @@ void PrettyBlockOutputFormat::writeValueWithPadding( value_width = format_settings.pretty.max_value_width; has_break_line = false; } + else if (!has_break_line) + value += ' '; auto write_padding = [&]() { @@ -478,8 +480,6 @@ void PrettyBlockOutputFormat::writeValueWithPadding( if (has_break_line) writeString("…", out); - else if (value_width != format_settings.pretty.max_value_width) - writeChar(' ', out); } void PrettyBlockOutputFormat::writeTransferredRow(const Widths & max_widths, const Block & header, std::vector & transferred_row, size_t cut_to_width, bool space_block) From b51a54b8c1abe8328a3c3d8c9cb61bfee1d740a2 Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Sat, 4 May 2024 10:14:16 +0300 Subject: [PATCH 332/624] Add tags for the test --- .../0_stateless/03000_traverse_shadow_system_data_paths.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03000_traverse_shadow_system_data_paths.sql b/tests/queries/0_stateless/03000_traverse_shadow_system_data_paths.sql index 514b4227b71..be4b16f1264 100644 --- a/tests/queries/0_stateless/03000_traverse_shadow_system_data_paths.sql +++ b/tests/queries/0_stateless/03000_traverse_shadow_system_data_paths.sql @@ -1,4 +1,4 @@ --- Tags: no-replicated-database, no-fasttest +-- Tags: no-replicated-database, no-fasttest, no-parallel, no-random-settings, no-random-merge-tree-settings DROP TABLE IF EXISTS 03000_traverse_shadow_system_data_path_table; From d8972cae0f9d7e03650557cb180e8fa22cce294a Mon Sep 17 00:00:00 2001 From: Maxim Alexeev Date: Sat, 4 May 2024 11:41:27 +0300 Subject: [PATCH 333/624] changed settings --- src/Core/Settings.h | 4 ++-- src/Interpreters/HashJoin.cpp | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9310bd8bdb2..902ce987c35 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -456,8 +456,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(UInt64, cross_join_min_rows_to_compress, 4000, "Minimal count of rows to compress block in CROSS JOIN", 0) \ - M(UInt64, cross_join_min_bytes_to_compress, 1000000, "Minimal size of block to compress in CROSS JOIN", 0) \ + M(UInt64, cross_join_min_rows_to_compress, 10000000, "Minimal count of rows to compress block in CROSS JOIN", 0) \ + M(UInt64, cross_join_min_bytes_to_compress, 1000000000, "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/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 393b1e88e4f..578fedd4428 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -835,8 +835,8 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) assertBlocksHaveEqualStructure(data->sample_block, block_to_save, "joined block"); - if (kind == JoinKind::Cross && block_to_save.allocatedBytes() >= table_join->crossJoinMinBytesToCompress() - && block_to_save.rows() >= table_join->crossJoinMinRowsToCompress()) + if (kind == JoinKind::Cross && getTotalByteCount() >= table_join->crossJoinMinBytesToCompress() + && getTotalRowCount() >= table_join->crossJoinMinRowsToCompress()) block_to_save = block_to_save.compress(); data->blocks_allocated_size += block_to_save.allocatedBytes(); From b8b4a3a6dd082cb0b901a484c55ec9870984b236 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 4 May 2024 12:24:07 +0200 Subject: [PATCH 334/624] Fxi --- docker/test/stateless/Dockerfile | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index f70837207d4..c3d80a7334b 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -89,8 +89,8 @@ ENV MINIO_ROOT_USER="clickhouse" ENV MINIO_ROOT_PASSWORD="clickhouse" ENV EXPORT_S3_STORAGE_POLICIES=1 -RUN npm install -g azurite@3.29.0 \ - && npm install -g tslib +RUN npm install -g azurite@3.30.0 \ + && npm install -g tslib && npm install -g node COPY run.sh / COPY setup_minio.sh / From 1d9195b082a17c2e88e8df8d5a32265e4c63af21 Mon Sep 17 00:00:00 2001 From: Blargian Date: Sat, 4 May 2024 13:05:09 +0200 Subject: [PATCH 335/624] Add singleValueOrNull to documenttion --- .../aggregate-functions/reference/index.md | 1 + .../reference/singlevalueornull.md | 57 +++++++++++++++++++ 2 files changed, 58 insertions(+) create mode 100644 docs/en/sql-reference/aggregate-functions/reference/singlevalueornull.md diff --git a/docs/en/sql-reference/aggregate-functions/reference/index.md b/docs/en/sql-reference/aggregate-functions/reference/index.md index b99d4b06d55..e82cda0bd0d 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/index.md +++ b/docs/en/sql-reference/aggregate-functions/reference/index.md @@ -92,6 +92,7 @@ ClickHouse-specific aggregate functions: - [quantileBFloat16Weighted](/docs/en/sql-reference/aggregate-functions/reference/quantilebfloat16.md#quantilebfloat16weighted) - [quantileDD](/docs/en/sql-reference/aggregate-functions/reference/quantileddsketch.md#quantileddsketch) - [simpleLinearRegression](/docs/en/sql-reference/aggregate-functions/reference/simplelinearregression.md) +- [singleValueOrNull](/docs/en/sql-reference/aggregate-functions/reference/singlevalueornull.md) - [stochasticLinearRegression](/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md) - [stochasticLogisticRegression](/docs/en/sql-reference/aggregate-functions/reference/stochasticlogisticregression.md) - [categoricalInformationValue](/docs/en/sql-reference/aggregate-functions/reference/categoricalinformationvalue.md) diff --git a/docs/en/sql-reference/aggregate-functions/reference/singlevalueornull.md b/docs/en/sql-reference/aggregate-functions/reference/singlevalueornull.md new file mode 100644 index 00000000000..e39af77059a --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/singlevalueornull.md @@ -0,0 +1,57 @@ +--- +slug: /en/sql-reference/aggregate-functions/reference/singlevalueornull +sidebar_position: 220 +--- + +# singleValueOrNull + +The aggregate function `singleValueOrNull` is used to implement subquery operators, such as `x = ALL (SELECT ...)`. It checks if there is only one unique non-NULL value in the data. +If there is only one unique value, it returns it. If there are zero or at least two distinct values, it returns NULL. + +**Syntax** + +``` sql +singleValueOrNull(x) +``` + +**Parameters** + +- `x` — Column of any [data type](../../data-types/index.md). + +**Returned values** + +- The unique value, if there is only one unique non-NULL value in `x`. +- `NULL`, if there are zero or at least two distinct values. + +**Examples** + +Query: + +``` sql +CREATE TABLE test (x UInt8 NULL) ENGINE=Log; +INSERT INTO test (x) VALUES (NULL), (NULL), (5), (NULL), (NULL); +SELECT singleValueOrNull(x) FROM test; +``` + +Result: + +```response +┌─singleValueOrNull(x)─┐ +│ 5 │ +└──────────────────────┘ +``` + +Query: + +```sql +INSERT INTO test (x) VALUES (10); +SELECT singleValueOrNull(x) FROM test; +``` + +Result: + +```response +┌─singleValueOrNull(x)─┐ +│ ᴺᵁᴸᴸ │ +└──────────────────────┘ +``` From 0d86a20b14149bdead45e5afed75538c4efc60dc Mon Sep 17 00:00:00 2001 From: Blargian Date: Sat, 4 May 2024 16:20:58 +0200 Subject: [PATCH 336/624] Add anyXYZ functions to documentation --- .../aggregate-functions/reference/any.md | 43 ++++++++++++++++-- .../reference/any_respect_nulls.md | 44 +++++++++++++++++++ .../aggregate-functions/reference/anylast.md | 34 +++++++++++++- .../reference/anylast_respect_nulls.md | 39 ++++++++++++++++ .../aggregate-functions/reference/index.md | 2 + 5 files changed, 156 insertions(+), 6 deletions(-) create mode 100644 docs/en/sql-reference/aggregate-functions/reference/any_respect_nulls.md create mode 100644 docs/en/sql-reference/aggregate-functions/reference/anylast_respect_nulls.md diff --git a/docs/en/sql-reference/aggregate-functions/reference/any.md b/docs/en/sql-reference/aggregate-functions/reference/any.md index 4631060f33f..f1b5a6683e5 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/any.md +++ b/docs/en/sql-reference/aggregate-functions/reference/any.md @@ -7,15 +7,50 @@ sidebar_position: 6 Selects the first encountered value of a column. -By default, it ignores NULL values and returns the first NOT NULL value found in the column. As [`first_value`](../../../sql-reference/aggregate-functions/reference/first_value.md) if supports `RESPECT NULLS`, in which case it will select the first value passed, independently on whether it's NULL or not. +**Syntax** +```sql +any(column) +``` + +Aliases: `any_value`, [`first_value`](../reference/first_value.md). + +**Parameters** +- `column`: The column name. + +**Returned value** + +By default, it ignores NULL values and returns the first NOT NULL value found in the column. Like [`first_value`](../../../sql-reference/aggregate-functions/reference/first_value.md) it supports `RESPECT NULLS`, in which case it will select the first value passed, independently on whether it's NULL or not. + +:::note The return type of the function is the same as the input, except for LowCardinality which is discarded. This means that given no rows as input it will return the default value of that type (0 for integers, or Null for a Nullable() column). You might use the `-OrNull` [combinator](../../../sql-reference/aggregate-functions/combinators.md) ) to modify this behaviour. +::: +:::warning The query can be executed in any order and even in a different order each time, so the result of this function is indeterminate. -To get a determinate result, you can use the ‘min’ or ‘max’ function instead of ‘any’. +To get a determinate result, you can use the [`min`](../reference/min.md) or [`max`](../reference/max.md) function instead of `any`. +::: -In some cases, you can rely on the order of execution. This applies to cases when SELECT comes from a subquery that uses ORDER BY. +**Implementation details** + +In some cases, you can rely on the order of execution. This applies to cases when `SELECT` comes from a subquery that uses `ORDER BY`. When a `SELECT` query has the `GROUP BY` clause or at least one aggregate function, ClickHouse (in contrast to MySQL) requires that all expressions in the `SELECT`, `HAVING`, and `ORDER BY` clauses be calculated from keys or from aggregate functions. In other words, each column selected from the table must be used either in keys or inside aggregate functions. To get behavior like in MySQL, you can put the other columns in the `any` aggregate function. -- Alias: `any_value`, `first_value`. +**Example** + +Query: + +```sql +CREATE TABLE any_nulls (city Nullable(String)) ENGINE=Log; + +INSERT INTO any_nulls (city) VALUES (NULL), ('Amsterdam'), ('New York'), ('Tokyo'), ('Valencia'), (NULL); + +SELECT any(city) FROM any_nulls; +``` + +```response +┌─any(city)─┐ +│ Amsterdam │ +└───────────┘ +``` diff --git a/docs/en/sql-reference/aggregate-functions/reference/any_respect_nulls.md b/docs/en/sql-reference/aggregate-functions/reference/any_respect_nulls.md new file mode 100644 index 00000000000..99104a9b8c7 --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/any_respect_nulls.md @@ -0,0 +1,44 @@ +--- +slug: /en/sql-reference/aggregate-functions/reference/any_respect_nulls +sidebar_position: 103 +--- + +# any_respect_nulls + +Selects the first encountered value of a column, irregardless of whether it is a `NULL` value or not. + +Alias: `any_value_respect_nulls`, `first_value_repect_nulls`. + +**Syntax** + +```sql +any_respect_nulls(column) +``` + +**Parameters** +- `column`: The column name. + +**Returned value** + +- The last value encountered, irregardless of whether it is a `NULL` value or not. + +**Example** + +Query: + +```sql +CREATE TABLE any_nulls (city Nullable(String)) ENGINE=Log; + +INSERT INTO any_nulls (city) VALUES (NULL), ('Amsterdam'), ('New York'), ('Tokyo'), ('Valencia'), (NULL); + +SELECT any(city), any_respect_nulls(city) FROM any_nulls; +``` + +```response +┌─any(city)─┬─any_respect_nulls(city)─┐ +│ Amsterdam │ ᴺᵁᴸᴸ │ +└───────────┴─────────────────────────┘ +``` + +**See Also** +- [any](../reference/any.md) diff --git a/docs/en/sql-reference/aggregate-functions/reference/anylast.md b/docs/en/sql-reference/aggregate-functions/reference/anylast.md index 351c9fd8e2f..8fcee2cf8e6 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/anylast.md +++ b/docs/en/sql-reference/aggregate-functions/reference/anylast.md @@ -5,5 +5,35 @@ sidebar_position: 104 # anyLast -Selects the last value encountered. -The result is just as indeterminate as for the [any](../../../sql-reference/aggregate-functions/reference/any.md) function. +Selects the last value encountered. The result is just as indeterminate as for the [any](../../../sql-reference/aggregate-functions/reference/any.md) function. + +**Syntax** + +```sql +anyLast(column) +``` + +**Parameters** +- `column`: The column name. + +**Returned value** + +- The last value encountered. + +**Example** + +Query: + +```sql +CREATE TABLE any_last_nulls (city Nullable(String)) ENGINE=Log; + +INSERT INTO any_last_nulls (city) VALUES ('Amsterdam'),(NULL),('New York'),('Tokyo'),('Valencia'),(NULL); + +SELECT anyLast(city) FROM any_last_nulls; +``` + +```response +┌─anyLast(city)─┐ +│ Valencia │ +└───────────────┘ +``` \ No newline at end of file diff --git a/docs/en/sql-reference/aggregate-functions/reference/anylast_respect_nulls.md b/docs/en/sql-reference/aggregate-functions/reference/anylast_respect_nulls.md new file mode 100644 index 00000000000..b6d0806f35d --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/anylast_respect_nulls.md @@ -0,0 +1,39 @@ +--- +slug: /en/sql-reference/aggregate-functions/reference/anylast_respect_nulls +sidebar_position: 104 +--- + +# anyLast_respect_nulls + +Selects the last value encountered, irregardless of whether it is `NULL` or not. + +**Syntax** + +```sql +anyLast_respect_nulls(column) +``` + +**Parameters** +- `column`: The column name. + +**Returned value** + +- The last value encountered, irregardless of whether it is `NULL` or not. + +**Example** + +Query: + +```sql +CREATE TABLE any_last_nulls (city Nullable(String)) ENGINE=Log; + +INSERT INTO any_last_nulls (city) VALUES ('Amsterdam'),(NULL),('New York'),('Tokyo'),('Valencia'),(NULL); + +SELECT anyLast(city), anyLast_respect_nulls(city) FROM any_last_nulls; +``` + +```response +┌─anyLast(city)─┬─anyLast_respect_nulls(city)─┐ +│ Valencia │ ᴺᵁᴸᴸ │ +└───────────────┴─────────────────────────────┘ +``` \ No newline at end of file diff --git a/docs/en/sql-reference/aggregate-functions/reference/index.md b/docs/en/sql-reference/aggregate-functions/reference/index.md index b99d4b06d55..377897fa6b0 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/index.md +++ b/docs/en/sql-reference/aggregate-functions/reference/index.md @@ -35,8 +35,10 @@ Standard aggregate functions: ClickHouse-specific aggregate functions: +- [any](/docs/en/sql-reference/aggregate-functions/reference/any_respect_nulls.md) - [anyHeavy](/docs/en/sql-reference/aggregate-functions/reference/anyheavy.md) - [anyLast](/docs/en/sql-reference/aggregate-functions/reference/anylast.md) +- [anyLast](/docs/en/sql-reference/aggregate-functions/reference/anylast_respect_nulls.md) - [boundingRatio](/docs/en/sql-reference/aggregate-functions/reference/boundrat.md) - [first_value](/docs/en/sql-reference/aggregate-functions/reference/first_value.md) - [last_value](/docs/en/sql-reference/aggregate-functions/reference/last_value.md) From 32c2b08cf697a2d252e9bb1b316a6bfc959727e4 Mon Sep 17 00:00:00 2001 From: Maxim Alexeev Date: Sat, 4 May 2024 19:53:33 +0300 Subject: [PATCH 337/624] Fixed settings --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 902ce987c35..aee6ef05983 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -457,7 +457,7 @@ class IColumn; 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(UInt64, cross_join_min_rows_to_compress, 10000000, "Minimal count of rows to compress block in CROSS JOIN", 0) \ - M(UInt64, cross_join_min_bytes_to_compress, 1000000000, "Minimal size of block to compress in CROSS JOIN", 0) \ + M(UInt64, cross_join_min_bytes_to_compress, 1_GiB, "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 50864c58bef..783c701cfe6 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -86,8 +86,8 @@ namespace SettingsChangesHistory static std::map settings_changes_history = { {"24.5", {{"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, - {"cross_join_min_rows_to_compress", 1000, 1000, "A new setting."}, - {"cross_join_min_bytes_to_compress", 10000, 10000, "A new setting."}, + {"cross_join_min_rows_to_compress", 10000000, 10000000, "A new setting."}, + {"cross_join_min_bytes_to_compress", 1_GiB, 1_GiB, "A new setting."}, }}, {"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 2045c7f04076ab5b48f946a9900fc8a78b8145c4 Mon Sep 17 00:00:00 2001 From: Xiaofei Hu <10776938+litlig@users.noreply.github.com> Date: Sat, 4 May 2024 14:13:18 -0700 Subject: [PATCH 338/624] rename --- src/Storages/System/StorageSystemPartsColumns.cpp | 4 ++-- .../0_stateless/02117_show_create_table_system.reference | 4 ++-- .../0_stateless/03143_ttl_in_system_parts_columns_table.sql | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Storages/System/StorageSystemPartsColumns.cpp b/src/Storages/System/StorageSystemPartsColumns.cpp index 0a8fce68a7f..08cde25584b 100644 --- a/src/Storages/System/StorageSystemPartsColumns.cpp +++ b/src/Storages/System/StorageSystemPartsColumns.cpp @@ -65,8 +65,8 @@ StorageSystemPartsColumns::StorageSystemPartsColumns(const StorageID & table_id_ {"column_data_uncompressed_bytes", std::make_shared(), "Total size of the decompressed data in the column, in bytes."}, {"column_marks_bytes", std::make_shared(), "The size of the marks for column, in bytes."}, {"column_modification_time", std::make_shared(std::make_shared()), "The last time the column was modified."}, - {"column_ttl_info.min", std::make_shared(std::make_shared()), "The minimum value of the calculated TTL expression of the column."}, - {"column_ttl_info.max", std::make_shared(std::make_shared()), "The maximum value of the calculated TTL expression of the column."}, + {"column_ttl_min", std::make_shared(std::make_shared()), "The minimum value of the calculated TTL expression of the column."}, + {"column_ttl_max", std::make_shared(std::make_shared()), "The maximum value of the calculated TTL expression of the column."}, {"serialization_kind", std::make_shared(), "Kind of serialization of a column"}, {"substreams", std::make_shared(std::make_shared()), "Names of substreams to which column is serialized"}, diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 8b3e0428546..d625feb94d3 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -565,8 +565,8 @@ CREATE TABLE system.parts_columns `column_data_uncompressed_bytes` UInt64, `column_marks_bytes` UInt64, `column_modification_time` Nullable(DateTime), - `column_ttl_info.min` Nullable(DateTime), - `column_ttl_info.max` Nullable(DateTime), + `column_ttl_min` Nullable(DateTime), + `column_ttl_max` Nullable(DateTime), `serialization_kind` String, `substreams` Array(String), `filenames` Array(String), diff --git a/tests/queries/0_stateless/03143_ttl_in_system_parts_columns_table.sql b/tests/queries/0_stateless/03143_ttl_in_system_parts_columns_table.sql index 81a88ef45c4..50adab2e9b0 100644 --- a/tests/queries/0_stateless/03143_ttl_in_system_parts_columns_table.sql +++ b/tests/queries/0_stateless/03143_ttl_in_system_parts_columns_table.sql @@ -16,8 +16,8 @@ SELECT name, column, type, - column_ttl_info.min, - column_ttl_info.max + column_ttl_min, + column_ttl_max FROM system.parts_columns WHERE table = 'test_03143' and database = currentDatabase() ORDER BY name, column; From 010d158d877a7295e62bc96b6190db948593f6ca Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Sat, 4 May 2024 21:48:04 +0000 Subject: [PATCH 339/624] set result timezone after query execution --- src/Interpreters/executeQuery.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index c785de61a18..cd1953f4b09 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1525,6 +1525,8 @@ void executeQuery( throw; } + result_details.timezone = DateLUT::instance().getTimeZone(); + auto & pipeline = streams.pipeline; std::unique_ptr compressed_buffer; From d269e4eb9ff90723a0af20af97d1faccbe7c01ef Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Sat, 4 May 2024 22:29:53 +0000 Subject: [PATCH 340/624] check X-ClickHouse-Timezone with session_timezone setting --- .../00265_http_content_type_format_timezone.reference | 2 ++ .../0_stateless/00265_http_content_type_format_timezone.sh | 3 +++ ...zone_setting.reference => 02737_session_timezone.reference} | 0 .../{02737_timezone_setting.sql => 02737_session_timezone.sql} | 0 4 files changed, 5 insertions(+) rename tests/queries/0_stateless/{02737_timezone_setting.reference => 02737_session_timezone.reference} (100%) rename tests/queries/0_stateless/{02737_timezone_setting.sql => 02737_session_timezone.sql} (100%) 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 9de2c3d4ba4..4b992161128 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 @@ -19,3 +19,5 @@ < Content-Type: application/octet-stream < X-ClickHouse-Format: RowBinary < X-ClickHouse-Timezone: CLICKHOUSE_TIMEZONE +< X-ClickHouse-Timezone: Europe/Berlin +< X-ClickHouse-Timezone: Africa/Cairo 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 74cbbe7f71d..8b04bd74a96 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 @@ -13,3 +13,6 @@ ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}" --data-binary @- <<< "SELECT 1 FORMA ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}" --data-binary @- <<< "SELECT 1 FORMAT Vertical" 2>&1 | grep -e '< Content-Type' -e '< X-ClickHouse-Format' -e '< X-ClickHouse-Timezone' | sed "s|$CLICKHOUSE_TIMEZONE_ESCAPED|CLICKHOUSE_TIMEZONE|" | sed 's/\r$//' | sort; ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}" --data-binary @- <<< "SELECT 1 FORMAT Native" 2>&1 | grep -e '< Content-Type' -e '< X-ClickHouse-Format' -e '< X-ClickHouse-Timezone' | sed "s|$CLICKHOUSE_TIMEZONE_ESCAPED|CLICKHOUSE_TIMEZONE|" | sed 's/\r$//' | sort; ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}" --data-binary @- <<< "SELECT 1 FORMAT RowBinary" 2>&1 | grep -e '< Content-Type' -e '< X-ClickHouse-Format' -e '< X-ClickHouse-Timezone' | sed "s|$CLICKHOUSE_TIMEZONE_ESCAPED|CLICKHOUSE_TIMEZONE|" | sed 's/\r$//' | sort; + +${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}" --data-binary @- <<< "SELECT timezone() SETTINGS session_timezone='Europe/Berlin'" 2>&1 | grep '< X-ClickHouse-Timezone' | grep -v 'GET' | tr -d '\r'; +${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}" --data-binary @- <<< "SELECT timezone() SETTINGS session_timezone='Africa/Cairo'" 2>&1 | grep '< X-ClickHouse-Timezone' | grep -v 'GET' | tr -d '\r'; diff --git a/tests/queries/0_stateless/02737_timezone_setting.reference b/tests/queries/0_stateless/02737_session_timezone.reference similarity index 100% rename from tests/queries/0_stateless/02737_timezone_setting.reference rename to tests/queries/0_stateless/02737_session_timezone.reference diff --git a/tests/queries/0_stateless/02737_timezone_setting.sql b/tests/queries/0_stateless/02737_session_timezone.sql similarity index 100% rename from tests/queries/0_stateless/02737_timezone_setting.sql rename to tests/queries/0_stateless/02737_session_timezone.sql From 4e817c03d4a0dc5f9d2cde688ef34be787fd2aee Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 5 May 2024 02:55:39 +0200 Subject: [PATCH 341/624] Fix test --- tests/queries/0_stateless/03134_positional_arguments.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/03134_positional_arguments.sh b/tests/queries/0_stateless/03134_positional_arguments.sh index 4390a8d5250..437c8226010 100755 --- a/tests/queries/0_stateless/03134_positional_arguments.sh +++ b/tests/queries/0_stateless/03134_positional_arguments.sh @@ -33,7 +33,7 @@ ${CLICKHOUSE_LOCAL} -q "SELECT 2" ${CLICKHOUSE_LOCAL} "SELECT 3" ${CLICKHOUSE_LOCAL} --param_test Hello "SELECT {test:String}" -${CLICKHOUSE_CLIENT} --query "SELECT 1" -${CLICKHOUSE_CLIENT} -q "SELECT 2" -${CLICKHOUSE_CLIENT} "SELECT 3" -${CLICKHOUSE_CLIENT} --param_test Hello "SELECT {test:String}" +${CLICKHOUSE_CLIENT_BINARY} --query "SELECT 1" +${CLICKHOUSE_CLIENT_BINARY} -q "SELECT 2" +${CLICKHOUSE_CLIENT_BINARY} "SELECT 3" +${CLICKHOUSE_CLIENT_BINARY} --param_test Hello "SELECT {test:String}" From 25ef78413d7fe75672be9c0c3061991f5fb1b7e7 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Sun, 5 May 2024 03:38:18 +0200 Subject: [PATCH 342/624] Add missing sigmoid function Add missing sigmoid function --- .../sql-reference/functions/math-functions.md | 36 +++++++++++++++++++ 1 file changed, 36 insertions(+) diff --git a/docs/en/sql-reference/functions/math-functions.md b/docs/en/sql-reference/functions/math-functions.md index 5ebc6191010..f215b9151b6 100644 --- a/docs/en/sql-reference/functions/math-functions.md +++ b/docs/en/sql-reference/functions/math-functions.md @@ -914,3 +914,39 @@ 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) + +Type: [Float*](../../sql-reference/data-types/float.md#float32-float64). + +**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 984e24f069e16c4ff7ea6736876b847645a98248 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Sun, 5 May 2024 03:41:32 +0200 Subject: [PATCH 343/624] Correct typo --- docs/en/sql-reference/functions/math-functions.md | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/math-functions.md b/docs/en/sql-reference/functions/math-functions.md index f215b9151b6..ebf834aeb58 100644 --- a/docs/en/sql-reference/functions/math-functions.md +++ b/docs/en/sql-reference/functions/math-functions.md @@ -931,9 +931,7 @@ sigmoid(x) **Returned value** -- Corresponding value along the sigmoid curve between 0 and 1. [Float64](../../sql-reference/data-types/float.md) - -Type: [Float*](../../sql-reference/data-types/float.md#float32-float64). +- Corresponding value along the sigmoid curve between 0 and 1. [Float64](../../sql-reference/data-types/float.md). **Example** From ead13c52e4115e9e65c4bba657a172a9e2b68476 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Sun, 5 May 2024 03:51:42 +0200 Subject: [PATCH 344/624] Add `showCertificate` to other-functions --- docs/en/sql-reference/functions/other-functions.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 2b0215115cb..17c4e6662fa 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -3301,3 +3301,13 @@ The setting is not enabled by default for security reasons, because some headers HTTP headers are case sensitive for this function. If the function is used in the context of a distributed query, it returns non-empty result only on the initiator node. + +## showCertificate + +Shows the current server's SSL certificate if it has been configured. + +**Syntax** + +```sql +showCertificate() +``` From 5eaf944b996caba18ac42cff700fffa935a2a5a0 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Sun, 5 May 2024 03:54:01 +0200 Subject: [PATCH 345/624] Update other-functions.md --- docs/en/sql-reference/functions/other-functions.md | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 17c4e6662fa..ba613b619be 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -3311,3 +3311,11 @@ Shows the current server's SSL certificate if it has been configured. ```sql showCertificate() ``` + +**Parameters** + +- None. + +**Returned value** + +- SSL certificate. [String](../../sql-reference/data-types/string.md). From e8824c3d6c2900b990999564b70e6f181c847571 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 5 May 2024 03:57:29 +0200 Subject: [PATCH 346/624] Add a test for #55655 --- ...gate_states_with_different_types.reference | 2 ++ ..._aggregate_states_with_different_types.sql | 26 +++++++++++++++++++ 2 files changed, 28 insertions(+) create mode 100644 tests/queries/0_stateless/03144_aggregate_states_with_different_types.reference create mode 100644 tests/queries/0_stateless/03144_aggregate_states_with_different_types.sql diff --git a/tests/queries/0_stateless/03144_aggregate_states_with_different_types.reference b/tests/queries/0_stateless/03144_aggregate_states_with_different_types.reference new file mode 100644 index 00000000000..1e94d71681e --- /dev/null +++ b/tests/queries/0_stateless/03144_aggregate_states_with_different_types.reference @@ -0,0 +1,2 @@ +28A42640E39BB93F016A919E241536D33F020000000000000002000000000000000000000000000000000000000000000000 8014AE47E17AA43F01676666666666F6BF020000000000000002000000000000000000000000000000000000000000000000 +1 0 diff --git a/tests/queries/0_stateless/03144_aggregate_states_with_different_types.sql b/tests/queries/0_stateless/03144_aggregate_states_with_different_types.sql new file mode 100644 index 00000000000..731325f313f --- /dev/null +++ b/tests/queries/0_stateless/03144_aggregate_states_with_different_types.sql @@ -0,0 +1,26 @@ +select * APPLY hex +from ( + select ( + select stochasticLogisticRegressionState(0.1, 0., 5, 'SGD')(number, number) + from numbers(10) + ) as col1, + ( + select stochasticLinearRegressionState(0.1, 0., 5, 'SGD')(number, number) + from numbers(10) + ) as col2 +from numbers(1) +); + +SELECT * +FROM +( + SELECT + bitmapHasAny(bitmapBuild([toUInt8(1)]), + ( + SELECT groupBitmapState(toUInt8(1)) + )) has1, + bitmapHasAny(bitmapBuild([toUInt64(1)]), + ( + SELECT groupBitmapState(toUInt64(2)) + )) has2 +); From f5767113619a470e7abd022a076f7bd33cd138ae Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 5 May 2024 04:23:58 +0200 Subject: [PATCH 347/624] Finish implementation --- src/Parsers/ExpressionElementParsers.cpp | 22 ++++++++++++++++--- src/Parsers/Lexer.cpp | 17 +++++--------- .../03145_unicode_quotes.reference | 5 +++++ .../0_stateless/03145_unicode_quotes.sql | 9 ++++++++ 4 files changed, 39 insertions(+), 14 deletions(-) create mode 100644 tests/queries/0_stateless/03145_unicode_quotes.reference create mode 100644 tests/queries/0_stateless/03145_unicode_quotes.sql diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index de926e83024..59b586d46a0 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -170,9 +170,17 @@ bool ParserSubquery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) bool ParserIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - /// Identifier in backquotes or in double quotes + /// Identifier in backquotes or in double quotes or in English-style Unicode double quotes if (pos->type == TokenType::QuotedIdentifier) { + /// The case of Unicode quotes. No escaping is supported. Assuming UTF-8. + if (*pos->begin == '\xE2' && pos->size() > 6) /// Empty identifiers are not allowed. + { + node = std::make_shared(String(pos->begin + 3, pos->end - 3)); + ++pos; + return true; + } + ReadBufferFromMemory buf(pos->begin, pos->size()); String s; @@ -1140,18 +1148,26 @@ bool ParserStringLiteral::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte if (pos->type == TokenType::StringLiteral) { - if (*pos->begin == 'x' || *pos->begin == 'X') + char first_char = *pos->begin; + + if (first_char == 'x' || first_char == 'X') { constexpr size_t word_size = 2; return makeHexOrBinStringLiteral(pos, node, true, word_size); } - if (*pos->begin == 'b' || *pos->begin == 'B') + if (first_char == 'b' || first_char == 'B') { constexpr size_t word_size = 8; return makeHexOrBinStringLiteral(pos, node, false, word_size); } + /// The case of Unicode quotes. No escaping is supported. Assuming UTF-8. + if (first_char == '\xE2' && pos->size() >= 6) + { + return makeStringLiteral(pos, node, String(pos->begin + 3, pos->end - 3)); + } + ReadBufferFromMemory in(pos->begin, pos->size()); try diff --git a/src/Parsers/Lexer.cpp b/src/Parsers/Lexer.cpp index c3b503404a3..15f0c83c074 100644 --- a/src/Parsers/Lexer.cpp +++ b/src/Parsers/Lexer.cpp @@ -56,9 +56,12 @@ Token quotedStringWithUnicodeQuotes(const char *& pos, const char * const token_ while (true) { - pos = find_first_symbols<'\xE2', '\\'>(pos, end); + pos = find_first_symbols<'\xE2'>(pos, end); if (pos + 2 >= end) return Token(error_token, token_begin, end); + /// Empty identifiers are not allowed, while empty strings are. + if (success_token == TokenType::QuotedIdentifier && pos + 3 >= end) + return Token(error_token, token_begin, end); if (pos[0] == '\xE2' && pos[1] == '\x80' && pos[2] == expected_end_byte) { @@ -66,14 +69,7 @@ Token quotedStringWithUnicodeQuotes(const char *& pos, const char * const token_ return Token(success_token, token_begin, pos); } - if (*pos == '\\') - { - ++pos; - if (pos >= end) - return Token(error_token, token_begin, end); - ++pos; - continue; - } + ++pos; } } @@ -470,10 +466,9 @@ Token Lexer::nextTokenImpl() if (pos + 5 < end && pos[0] == '\xE2' && pos[1] == '\x80' && (pos[2] == '\x98' || pos[2] == '\x9C')) { const char expected_end_byte = pos[2] + 1; - pos += 3; - TokenType success_token = pos[2] == '\x98' ? TokenType::StringLiteral : TokenType::QuotedIdentifier; TokenType error_token = pos[2] == '\x98' ? TokenType::ErrorSingleQuoteIsNotClosed : TokenType::ErrorDoubleQuoteIsNotClosed; + pos += 3; return quotedStringWithUnicodeQuotes(pos, token_begin, end, expected_end_byte, success_token, error_token); } /// Other characters starting at E2 can be parsed, see skipWhitespacesUTF8 diff --git a/tests/queries/0_stateless/03145_unicode_quotes.reference b/tests/queries/0_stateless/03145_unicode_quotes.reference new file mode 100644 index 00000000000..8d33edf14e3 --- /dev/null +++ b/tests/queries/0_stateless/03145_unicode_quotes.reference @@ -0,0 +1,5 @@ +This is an example of using English-style Unicode single quotes. +Row 1: +────── +\c\\u\\\r\\\\l\\\\\y\\\\\\: This is \an \\example ‘of using English-style Unicode single quotes.\ +{"1":1} diff --git a/tests/queries/0_stateless/03145_unicode_quotes.sql b/tests/queries/0_stateless/03145_unicode_quotes.sql new file mode 100644 index 00000000000..34a465cd874 --- /dev/null +++ b/tests/queries/0_stateless/03145_unicode_quotes.sql @@ -0,0 +1,9 @@ +-- They work: +SELECT ‘This is an example of using English-style Unicode single quotes.’ AS “curly”; + +-- It is unspecified which escaping rules apply inside the literal in Unicode quotes, and currently none apply (similarly to heredocs) +-- This could be changed. + +SELECT ‘This is \an \\example ‘of using English-style Unicode single quotes.\’ AS “\c\\u\\\r\\\\l\\\\\y\\\\\\” FORMAT Vertical; + +SELECT ‘’ = '' AS “1” FORMAT JSONLines; From 1bcf035bbdc693c5b16e30bf40caaf903ed15f33 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 5 May 2024 04:59:12 +0200 Subject: [PATCH 348/624] Zero means unlimited; apply the thresholds by logical or; better documentation --- src/Core/Settings.h | 4 ++-- src/Core/SettingsChangesHistory.h | 4 ++-- src/Interpreters/HashJoin.cpp | 10 ++++++++-- 3 files changed, 12 insertions(+), 6 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 682d9f85d3c..7ed175d38f4 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -456,8 +456,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(UInt64, cross_join_min_rows_to_compress, 10000000, "Minimal count of rows to compress block in CROSS JOIN", 0) \ - M(UInt64, cross_join_min_bytes_to_compress, 1_GiB, "Minimal size of block to compress in CROSS JOIN", 0) \ + M(UInt64, cross_join_min_rows_to_compress, 10000000, "Minimal count of rows to compress block in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached.", 0) \ + M(UInt64, cross_join_min_bytes_to_compress, 1_GiB, "Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached.", 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 060677e86e1..18dcfd4c322 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -88,8 +88,8 @@ static std::map sett {"24.5", {{"allow_deprecated_functions", true, false, "Allow usage of deprecated functions"}, {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, {"output_format_parquet_use_custom_encoder", false, true, "Enable custom Parquet encoder."}, - {"cross_join_min_rows_to_compress", 10000000, 10000000, "A new setting."}, - {"cross_join_min_bytes_to_compress", 1_GiB, 1_GiB, "A new setting."}, + {"cross_join_min_rows_to_compress", 0, 10000000, "A new setting."}, + {"cross_join_min_bytes_to_compress", 0, 1_GiB, "A new setting."}, }}, {"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/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 578fedd4428..9b05edbce36 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -835,9 +835,15 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) assertBlocksHaveEqualStructure(data->sample_block, block_to_save, "joined block"); - if (kind == JoinKind::Cross && getTotalByteCount() >= table_join->crossJoinMinBytesToCompress() - && getTotalRowCount() >= table_join->crossJoinMinRowsToCompress()) + size_t min_bytes_to_compress = table_join->crossJoinMinBytesToCompress(); + size_t min_rows_to_compress = table_join->crossJoinMinRowsToCompress(); + + if (kind == JoinKind::Cross + && ((min_bytes_to_compress && getTotalByteCount() >= min_bytes_to_compress) + || (min_rows_to_compress && getTotalRowCount() >= min_rows_to_compress))) + { block_to_save = block_to_save.compress(); + } data->blocks_allocated_size += block_to_save.allocatedBytes(); data->blocks.emplace_back(std::move(block_to_save)); From 3dbc9b8549639817f98a9853793b670e48b235d0 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Sun, 5 May 2024 05:28:49 +0200 Subject: [PATCH 349/624] Add singleValueOrNull to aspell-dict --- 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 e7477ffc5e1..4d5d2ae1392 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2396,6 +2396,7 @@ simpleaggregatefunction simplelinearregression simpod singlepart +singleValueOrNull sinh sipHash siphash From 26e3e5daa83e3d3a16c74483a3eb55cad20dd8ac Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 5 May 2024 20:19:03 +0300 Subject: [PATCH 350/624] Update math-functions.md --- 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 ebf834aeb58..2feb91cfce4 100644 --- a/docs/en/sql-reference/functions/math-functions.md +++ b/docs/en/sql-reference/functions/math-functions.md @@ -938,7 +938,7 @@ sigmoid(x) Query: ``` sql -SELECT Round(sigmoid(x),5) FROM (SELECT arrayJoin([-1, 0, 1]) AS x); +SELECT round(sigmoid(x), 5) FROM (SELECT arrayJoin([-1, 0, 1]) AS x); ``` Result: From 2441ed9081d2c96661f4104452bf592e49b47f81 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Sun, 5 May 2024 22:04:31 +0200 Subject: [PATCH 351/624] add sigmoid 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 2b260566816..9c08acd2ef4 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -879,6 +879,7 @@ Simhash SimpleAggregateFunction SimpleState SipHash +sigmoid Smirnov's Smirnov'test Soundex From eeaf46ceaa129eb6159b5f94249c7c0974e4abc6 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Sun, 5 May 2024 22:06:45 +0200 Subject: [PATCH 352/624] Update 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 4d5d2ae1392..5d2c901635f 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2397,6 +2397,7 @@ simplelinearregression simpod singlepart singleValueOrNull +singlevalueornull sinh sipHash siphash From 6bfc9c17fbb454a65c3db77a6c095b15e4345bcb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 6 May 2024 00:08:38 +0300 Subject: [PATCH 353/624] Update 03144_aggregate_states_with_different_types.sql --- .../0_stateless/03144_aggregate_states_with_different_types.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03144_aggregate_states_with_different_types.sql b/tests/queries/0_stateless/03144_aggregate_states_with_different_types.sql index 731325f313f..5bb4a8c9c9c 100644 --- a/tests/queries/0_stateless/03144_aggregate_states_with_different_types.sql +++ b/tests/queries/0_stateless/03144_aggregate_states_with_different_types.sql @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 1; + select * APPLY hex from ( select ( From ec15298b824d6e51a7549ec79260a0c349174d98 Mon Sep 17 00:00:00 2001 From: Maxim Alexeev Date: Sun, 28 Apr 2024 14:20:59 +0300 Subject: [PATCH 354/624] First working prototype --- src/Core/Settings.h | 1 + src/Interpreters/GraceHashJoin.cpp | 1 + src/Interpreters/HashJoin.cpp | 37 +++++++++-- src/Interpreters/HashJoin.h | 14 ++++ src/Interpreters/JoinedTables.cpp | 2 +- src/Interpreters/TableJoin.cpp | 4 +- src/Interpreters/TableJoin.h | 10 ++- src/Interpreters/TemporaryDataOnDisk.cpp | 85 ++++++++++++++++-------- src/Interpreters/TemporaryDataOnDisk.h | 27 +++++++- src/Planner/PlannerJoinTree.cpp | 2 +- 10 files changed, 147 insertions(+), 36 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 7ed175d38f4..5dc80912ebb 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -463,6 +463,7 @@ class IColumn; 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) \ M(UInt64, join_on_disk_max_files_to_merge, 64, "For MergeJoin on disk set how much files it's allowed to sort simultaneously. Then this value bigger then more memory used and then less disk I/O needed. Minimum is 2.", 0) \ M(UInt64, max_rows_in_set_to_optimize_join, 0, "Maximal size of the set to filter joined tables by each other row sets before joining. 0 - disable.", 0) \ + M(UInt64, cross_join_max_bytes_inmemory, 1000000, "Maximal bytes to complete cross join inmemory.", 0) \ \ M(Bool, compatibility_ignore_collation_in_create_table, true, "Compatibility ignore collation in create table", 0) \ \ diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 53d1f48c291..ba51953e979 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -657,6 +657,7 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() GraceHashJoin::InMemoryJoinPtr GraceHashJoin::makeInMemoryJoin(const String & bucket_id, size_t reserve_num) { + LOG_INFO(log, "GreaceHashJoin\n"); return std::make_unique(table_join, right_sample_block, any_take_last_row, reserve_num, bucket_id); } diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 9b05edbce36..e194df4e660 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -35,6 +35,7 @@ #include #include #include +#include "Core/Joins.h" #include #include @@ -249,11 +250,13 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s , instance_id(instance_id_) , asof_inequality(table_join->getAsofInequality()) , data(std::make_shared()) + , tmp_data(std::make_unique(table_join_->getTempDataOnDisk())) , right_sample_block(right_sample_block_) , max_joined_block_rows(table_join->maxJoinedBlockRows()) , instance_log_id(!instance_id_.empty() ? "(" + instance_id_ + ") " : "") , log(getLogger("HashJoin")) { + LOG_INFO(log, "KEK CONSTRUCTOR {}\n", reserve_num); LOG_TRACE(log, "{}Keys: {}, datatype: {}, kind: {}, strictness: {}, right header: {}", instance_log_id, TableJoin::formatClauses(table_join->getClauses(), true), data->type, kind, strictness, right_sample_block.dumpStructure()); @@ -827,6 +830,16 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) if (shrink_blocks) block_to_save = block_to_save.shrinkToFit(); + + if (kind == JoinKind::Cross) + { + if (tmp_stream == nullptr) + { + tmp_stream = &tmp_data->createStream(right_sample_block); + } + tmp_stream->write(block_to_save); + } + size_t total_rows = 0; size_t total_bytes = 0; { @@ -928,6 +941,7 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) if (!flag_per_row && !is_inserted) { + LOG_INFO(log, "LOL\n\n\n\n\n\n"); LOG_TRACE(log, "Skipping inserting block with {} rows", rows); data->blocks_allocated_size -= stored_block->allocatedBytes(); data->blocks.pop_back(); @@ -944,7 +958,6 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) shrinkStoredBlocksToFit(total_bytes); - return table_join->sizeLimits().check(total_rows, total_bytes, "JOIN", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED); } @@ -2275,13 +2288,13 @@ 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 & compressed_block_right : data->blocks) + + auto process_right_block = [&](const Block & block_right) { ++block_number; if (block_number < start_right_block) - continue; + return; - auto block_right = compressed_block_right.decompress(); size_t rows_right = block_right.rows(); rows_added += rows_right; @@ -2294,6 +2307,22 @@ void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) const IColumn & column_right = *block_right.getByPosition(col_num).column; dst_columns[num_existing_columns + col_num]->insertRangeFrom(column_right, 0, rows_right); } + }; + + if (tmp_stream) + { + tmp_stream->finishWritingAsyncSafe(); + auto reader = tmp_stream->getReadStream(); + while (auto block_right = reader->read()) + { + process_right_block(block_right); + } + } + + for (const Block & compressed_block_right : data->blocks) + { + auto block_right = compressed_block_right.decompress(); + process_right_block(block_right); } start_right_block = 0; diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index b7f41a7eb6b..1eb4d0f8030 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -26,6 +26,7 @@ #include #include +#include namespace DB { @@ -414,6 +415,15 @@ public: void setMaxJoinedBlockRows(size_t value) { max_joined_block_rows = value; } + TemporaryFileStream* getStreamForCrossJoin() + { + auto streams = tmp_data->getStreams(); + assert(streams.size() <= 1); + if (streams.empty()) + return nullptr; + return streams[0]; + } + private: friend class NotJoinedHash; @@ -442,6 +452,10 @@ private: RightTableDataPtr data; std::vector key_sizes; + /// Needed to do external cross join + TemporaryDataOnDiskPtr tmp_data; + TemporaryFileStream* tmp_stream{nullptr}; + /// Block with columns from the right-side table. Block right_sample_block; /// Block with columns from the right-side table except key columns. diff --git a/src/Interpreters/JoinedTables.cpp b/src/Interpreters/JoinedTables.cpp index 5b549a19083..457ed3ef4a6 100644 --- a/src/Interpreters/JoinedTables.cpp +++ b/src/Interpreters/JoinedTables.cpp @@ -310,7 +310,7 @@ std::shared_ptr JoinedTables::makeTableJoin(const ASTSelectQuery & se auto settings = context->getSettingsRef(); MultiEnum join_algorithm = settings.join_algorithm; bool try_use_direct_join = join_algorithm.isSet(JoinAlgorithm::DIRECT) || join_algorithm.isSet(JoinAlgorithm::DEFAULT); - auto table_join = std::make_shared(settings, context->getGlobalTemporaryVolume()); + auto table_join = std::make_shared(settings, context->getGlobalTemporaryVolume(), context->getTempDataOnDisk()); const ASTTablesInSelectQueryElement * ast_join = select_query_.join(); const auto & table_to_join = ast_join->table_expression->as(); diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 48d59dd3b24..7ceb90704f3 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -103,7 +103,7 @@ bool forAllKeys(OnExpr & expressions, Func callback) } -TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_) +TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_, TemporaryDataOnDiskScopePtr tmp_data_) : 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) @@ -111,12 +111,14 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_) , 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) + , cross_join_max_bytes_inmemory(settings.cross_join_max_bytes_inmemory) , partial_merge_join_rows_in_right_blocks(settings.partial_merge_join_rows_in_right_blocks) , partial_merge_join_left_table_buffer_bytes(settings.partial_merge_join_left_table_buffer_bytes) , max_files_to_merge(settings.join_on_disk_max_files_to_merge) , temporary_files_codec(settings.temporary_files_codec) , max_memory_usage(settings.max_memory_usage) , tmp_volume(tmp_volume_) + , tmp_data(tmp_data_) { } diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 88905edd3e8..d6920d6afbf 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -9,6 +9,7 @@ #include #include #include +#include #include #include @@ -144,6 +145,7 @@ private: const UInt64 cross_join_min_bytes_to_compress = 10000; const size_t max_joined_block_rows = 0; std::vector join_algorithm; + const UInt64 cross_join_max_bytes_inmemory = 1000000; const size_t partial_merge_join_rows_in_right_blocks = 0; const size_t partial_merge_join_left_table_buffer_bytes = 0; const size_t max_files_to_merge = 0; @@ -188,6 +190,8 @@ private: VolumePtr tmp_volume; + TemporaryDataOnDiskScopePtr tmp_data; + std::shared_ptr right_storage_join; std::shared_ptr right_kv_storage; @@ -233,7 +237,7 @@ private: public: TableJoin() = default; - TableJoin(const Settings & settings, VolumePtr tmp_volume_); + TableJoin(const Settings & settings, VolumePtr tmp_volume_, TemporaryDataOnDiskScopePtr tmp_data_); /// for StorageJoin TableJoin(SizeLimits limits, bool use_nulls, JoinKind kind, JoinStrictness strictness, @@ -259,6 +263,8 @@ public: VolumePtr getGlobalTemporaryVolume() { return tmp_volume; } + TemporaryDataOnDiskScopePtr getTempDataOnDisk() { return tmp_data; } + ActionsDAGPtr createJoinedBlockActions(ContextPtr context) const; const std::vector & getEnabledJoinAlgorithms() const { return join_algorithm; } @@ -275,6 +281,8 @@ public: bool allowParallelHashJoin() const; + UInt64 crossJoinMaxBytesInmemory() const { return cross_join_max_bytes_inmemory; } + bool joinUseNulls() const { return join_use_nulls; } UInt64 crossJoinMinRowsToCompress() const { return cross_join_min_rows_to_compress; } diff --git a/src/Interpreters/TemporaryDataOnDisk.cpp b/src/Interpreters/TemporaryDataOnDisk.cpp index 9a237738b3e..e50d501f6d3 100644 --- a/src/Interpreters/TemporaryDataOnDisk.cpp +++ b/src/Interpreters/TemporaryDataOnDisk.cpp @@ -1,12 +1,11 @@ +#include +#include #include #include -#include #include -#include #include #include -#include #include #include #include @@ -14,6 +13,7 @@ #include #include +#include "Common/Exception.h" namespace ProfileEvents { @@ -224,33 +224,26 @@ struct TemporaryFileStream::OutputWriter bool finalized = false; }; -struct TemporaryFileStream::InputReader +InputReader::InputReader(const String & path, const Block & header_, size_t size) + : in_file_buf(path, size ? std::min(DBMS_DEFAULT_BUFFER_SIZE, size) : DBMS_DEFAULT_BUFFER_SIZE) + , in_compressed_buf(in_file_buf) + , in_reader(in_compressed_buf, header_, DBMS_TCP_PROTOCOL_VERSION) { - InputReader(const String & path, const Block & header_, size_t size = 0) - : in_file_buf(path, size ? std::min(DBMS_DEFAULT_BUFFER_SIZE, size) : DBMS_DEFAULT_BUFFER_SIZE) - , in_compressed_buf(in_file_buf) - , in_reader(in_compressed_buf, header_, DBMS_TCP_PROTOCOL_VERSION) - { - LOG_TEST(getLogger("TemporaryFileStream"), "Reading {} from {}", header_.dumpStructure(), path); - } + LOG_TEST(getLogger("TemporaryFileStream"), "Reading {} from {}", header_.dumpStructure(), path); +} - explicit InputReader(const String & path, size_t size = 0) - : in_file_buf(path, size ? std::min(DBMS_DEFAULT_BUFFER_SIZE, size) : DBMS_DEFAULT_BUFFER_SIZE) - , in_compressed_buf(in_file_buf) - , in_reader(in_compressed_buf, DBMS_TCP_PROTOCOL_VERSION) - { - LOG_TEST(getLogger("TemporaryFileStream"), "Reading from {}", path); - } +InputReader::InputReader(const String & path, size_t size) + : in_file_buf(path, size ? std::min(DBMS_DEFAULT_BUFFER_SIZE, size) : DBMS_DEFAULT_BUFFER_SIZE) + , in_compressed_buf(in_file_buf) + , in_reader(in_compressed_buf, DBMS_TCP_PROTOCOL_VERSION) +{ + LOG_TEST(getLogger("TemporaryFileStream"), "Reading from {}", path); +} - Block read() - { - return in_reader.read(); - } - - ReadBufferFromFile in_file_buf; - CompressedReadBuffer in_compressed_buf; - NativeReader in_reader; -}; +Block InputReader::read() +{ + return in_reader.read(); +} TemporaryFileStream::TemporaryFileStream(TemporaryFileOnDiskHolder file_, const Block & header_, TemporaryDataOnDisk * parent_) : parent(parent_) @@ -310,6 +303,20 @@ TemporaryFileStream::Stat TemporaryFileStream::finishWriting() return stat; } +TemporaryFileStream::Stat TemporaryFileStream::finishWritingAsyncSafe() +{ + if (!writing_finished.load(std::memory_order_relaxed)) + { + std::lock_guard lock(finish_writing); + if (!writing_finished.load()) + { + return finishWriting(); + } + writing_finished.store(true); + } + return stat; +} + bool TemporaryFileStream::isWriteFinished() const { assert(in_reader == nullptr || out_writer == nullptr); @@ -324,6 +331,12 @@ Block TemporaryFileStream::read() if (isEof()) return {}; + if (auto type = read_type.exchange(1); type == 2) + { + read_type.store(2); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Different type of reading was requested earlier"); + } + if (!in_reader) { in_reader = std::make_unique(getPath(), header, getSize()); @@ -334,10 +347,28 @@ Block TemporaryFileStream::read() { /// finalize earlier to release resources, do not wait for the destructor this->release(); + in_reader.reset(); } return block; } +std::unique_ptr TemporaryFileStream::getReadStream() +{ + if (!isWriteFinished()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Writing has been not finished"); + + if (isEof()) + return nullptr; + + if (auto type = read_type.exchange(2); type == 1) + { + read_type.store(1); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Different type of reading was requested earlier"); + } + + return std::make_unique(getPath(), header, getSize()); +} + void TemporaryFileStream::updateAllocAndCheck() { assert(out_writer); diff --git a/src/Interpreters/TemporaryDataOnDisk.h b/src/Interpreters/TemporaryDataOnDisk.h index 40100a62b44..792988e94bd 100644 --- a/src/Interpreters/TemporaryDataOnDisk.h +++ b/src/Interpreters/TemporaryDataOnDisk.h @@ -1,7 +1,11 @@ #pragma once +#include #include +#include +#include +#include #include #include #include @@ -130,6 +134,19 @@ private: typename CurrentMetrics::Metric current_metric_scope = CurrentMetrics::TemporaryFilesUnknown; }; +struct InputReader +{ + InputReader(const String & path, const Block & header_, size_t size = 0); + + explicit InputReader(const String & path, size_t size = 0); + + Block read(); + + ReadBufferFromFile in_file_buf; + CompressedReadBuffer in_compressed_buf; + NativeReader in_reader; +}; + /* * Data can be written into this stream and then read. * After finish writing, call `finishWriting` and then `read` to read the data. @@ -154,8 +171,11 @@ public: void flush(); Stat finishWriting(); + Stat finishWritingAsyncSafe(); bool isWriteFinished() const; + std::unique_ptr getReadStream(); + Block read(); String getPath() const; @@ -184,10 +204,15 @@ private: Stat stat; + /// 0 - means that we haven't requested any read, 1 - read from function TemporaryFileStream::read, 2 - + std::atomic_char read_type{0}; + + mutable std::mutex finish_writing; + std::atomic_bool writing_finished{false}; + struct OutputWriter; std::unique_ptr out_writer; - struct InputReader; std::unique_ptr in_reader; }; diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 514c19b0f89..3467299812a 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1196,7 +1196,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ } } - auto table_join = std::make_shared(settings, query_context->getGlobalTemporaryVolume()); + auto table_join = std::make_shared(settings, query_context->getGlobalTemporaryVolume(), query_context->getTempDataOnDisk()); table_join->getTableJoin() = join_node.toASTTableJoin()->as(); if (join_constant) From 0c1b07d5270393a5137088af567740c2d354545c Mon Sep 17 00:00:00 2001 From: liuneng1994 Date: Mon, 6 May 2024 10:49:37 +0800 Subject: [PATCH 355/624] remove perf test --- .../performance/external_sort_wide_table.xml | 354 ------------------ 1 file changed, 354 deletions(-) delete mode 100644 tests/performance/external_sort_wide_table.xml diff --git a/tests/performance/external_sort_wide_table.xml b/tests/performance/external_sort_wide_table.xml deleted file mode 100644 index 4c6220a4503..00000000000 --- a/tests/performance/external_sort_wide_table.xml +++ /dev/null @@ -1,354 +0,0 @@ - - - create table test_sort - ( - val00 Int64, - val01 Int64, - val02 Int64, - val03 Int64, - val04 Int64, - val05 Int64, - val06 Int64, - val07 Int64, - val08 Int64, - val09 Int64, - val10 Int64, - val11 Int64, - val12 Int64, - val13 Int64, - val14 Int64, - val15 Int64, - val16 Int64, - val17 Int64, - val18 Int64, - val19 Int64, - val20 Int64, - val21 Int64, - val22 Int64, - val23 Int64, - val24 Int64, - val25 Int64, - val26 Int64, - val27 Int64, - val28 Int64, - val29 Int64, - val30 Int64, - val31 Int64, - val32 Int64, - val33 Int64, - val34 Int64, - val35 Int64, - val36 Int64, - val37 Int64, - val38 Int64, - val39 Int64, - val40 Int64, - val41 Int64, - val42 Int64, - val43 Int64, - val44 Int64, - val45 Int64, - val46 Int64, - val47 Int64, - val48 Int64, - val49 Int64, - val50 Int64, - val51 Int64, - val52 Int64, - val53 Int64, - val54 Int64, - val55 Int64, - val56 Int64, - val57 Int64, - val58 Int64, - val59 Int64, - val60 Int64, - val61 Int64, - val62 Int64, - val63 Int64, - val64 Int64, - val65 Int64, - val66 Int64, - val67 Int64, - val68 Int64, - val69 Int64, - val70 Int64, - val71 Int64, - val72 Int64, - val73 Int64, - val74 Int64, - val75 Int64, - val76 Int64, - val77 Int64, - val78 Int64, - val79 Int64, - val80 Int64, - val81 Int64, - val82 Int64, - val83 Int64, - val84 Int64, - val85 Int64, - val86 Int64, - val87 Int64, - val88 Int64, - val89 Int64, - val90 Int64, - val91 Int64, - val92 Int64, - val93 Int64, - val94 Int64, - val95 Int64, - val96 Int64, - val97 Int64, - val98 Int64, - val99 Int64, - val100 Int64, - val101 Int64, - val102 Int64, - val103 Int64, - val104 Int64, - val105 Int64, - val106 Int64, - val107 Int64, - val108 Int64, - val109 Int64, - val110 Int64, - val111 Int64, - val112 Int64, - val113 Int64, - val114 Int64, - val115 Int64, - val116 Int64, - val117 Int64, - val118 Int64, - val119 Int64, - val120 Int64, - val121 Int64, - val122 Int64, - val123 Int64, - val124 Int64, - val125 Int64, - val126 Int64, - val127 Int64, - val128 Int64, - val129 Int64, - val130 Int64, - val131 Int64, - val132 Int64, - val133 Int64, - val134 Int64, - val135 Int64, - val136 Int64, - val137 Int64, - val138 Int64, - val139 Int64, - val140 Int64, - val141 Int64, - val142 Int64, - val143 Int64, - val144 Int64, - val145 Int64, - val146 Int64, - val147 Int64, - val148 Int64, - val149 Int64, - val150 Int64, - val151 Int64, - val152 Int64, - val153 Int64, - val154 Int64, - val155 Int64, - val156 Int64, - val157 Int64, - val158 Int64, - val159 Int64, - val160 Int64, - val161 Int64, - val162 Int64, - val163 Int64, - val164 Int64, - val165 Int64, - val166 Int64, - val167 Int64, - val168 Int64, - val169 Int64, - val170 Int64, - val171 Int64, - val172 Int64, - val173 Int64, - val174 Int64, - val175 Int64, - val176 Int64, - val177 Int64, - val178 Int64, - val179 Int64, - val180 Int64, - val181 Int64, - val182 Int64, - val183 Int64, - val184 Int64, - val185 Int64, - val186 Int64, - val187 Int64, - val188 Int64, - val189 Int64, - val190 Int64, - val191 Int64, - val192 Int64, - val193 Int64, - val194 Int64, - val195 Int64, - val196 Int64, - val197 Int64, - val198 Int64, - val199 Int64, - val200 Int64, - val201 Int64, - val202 Int64, - val203 Int64, - val204 Int64, - val205 Int64, - val206 Int64, - val207 Int64, - val208 Int64, - val209 Int64, - val210 Int64, - val211 Int64, - val212 Int64, - val213 Int64, - val214 Int64, - val215 Int64, - val216 Int64, - val217 Int64, - val218 Int64, - val219 Int64, - val220 Int64, - val221 Int64, - val222 Int64, - val223 Int64, - val224 Int64, - val225 Int64, - val226 Int64, - val227 Int64, - val228 Int64, - val229 Int64, - val230 Int64, - val231 Int64, - val232 Int64, - val233 Int64, - val234 Int64, - val235 Int64, - val236 Int64, - val237 Int64, - val238 Int64, - val239 Int64, - val240 Int64, - val241 Int64, - val242 Int64, - val243 Int64, - val244 Int64, - val245 Int64, - val246 Int64, - val247 Int64, - val248 Int64, - val249 Int64, - val250 Int64, - val251 Int64, - val252 Int64, - val253 Int64, - val254 Int64, - val255 Int64, - val256 Int64, - val257 Int64, - val258 Int64, - val259 Int64, - val260 Int64, - val261 Int64, - val262 Int64, - val263 Int64, - val264 Int64, - val265 Int64, - val266 Int64, - val267 Int64, - val268 Int64, - val269 Int64, - val270 Int64, - val271 Int64, - val272 Int64, - val273 Int64, - val274 Int64, - val275 Int64, - val276 Int64, - val277 Int64, - val278 Int64, - val279 Int64, - val280 Int64, - val281 Int64, - val282 Int64, - val283 Int64, - val284 Int64, - val285 Int64, - val286 Int64, - val287 Int64, - val288 Int64, - val289 Int64, - val290 Int64, - val291 Int64, - val292 Int64, - val293 Int64, - val294 Int64, - val295 Int64, - val296 Int64, - val297 Int64, - val298 Int64, - val299 Int64 - ) engine = MergeTree() order by tuple(); - - - - insert into test_sort select - number, number, number, number, number, number, number, number, number, number, - number, number, number, number, number, number, number, number, number, number, - number, number, number, number, number, number, number, number, number, number, - number, number, number, number, number, number, number, number, number, number, - number, number, number, number, number, number, number, number, number, number, - number, number, number, number, number, number, number, number, number, number, - number, number, number, number, number, number, number, number, number, number, - number, number, number, number, number, number, number, number, number, number, - number, number, number, number, number, number, number, number, number, number, - number, number, number, number, number, number, number, number, number, number, - number, number, number, number, number, number, number, number, number, number, - number, number, number, number, number, number, number, number, number, number, - number, number, number, number, number, number, number, number, number, number, - number, number, number, number, number, number, number, number, number, number, - number, number, number, number, number, number, number, number, number, number, - number, number, number, number, number, number, number, number, number, number, - number, number, number, number, number, number, number, number, number, number, - number, number, number, number, number, number, number, number, number, number, - number, number, number, number, number, number, number, number, number, number, - number, number, number, number, number, number, number, number, number, number, - number, number, number, number, number, number, number, number, number, number, - number, number, number, number, number, number, number, number, number, number, - number, number, number, number, number, number, number, number, number, number, - number, number, number, number, number, number, number, number, number, number, - number, number, number, number, number, number, number, number, number, number, - number, number, number, number, number, number, number, number, number, number, - number, number, number, number, number, number, number, number, number, number, - number, number, number, number, number, number, number, number, number, number, - number, number, number, number, number, number, number, number, number, number, - number, number, number, number, number, number, number, number, number, number - from numbers(1000000); - - - - 1 - - - - select * from test_sort order by val00 settings max_bytes_before_external_sort=50000000 format Null; - - - - drop table if exists test_sort - - From 6e61d671c803ef21314985e7101ea54ac3148db2 Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Mon, 6 May 2024 00:07:04 -0700 Subject: [PATCH 356/624] [Docs] Specify to use async_insert MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Raúl Marín --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 86dc3eb667a..474788c9812 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1925,7 +1925,7 @@ Default value: `16`. Enables or disables waiting for processing of asynchronous insertion. If enabled, server will return `OK` only after the data is inserted. Otherwise, it will return `OK` even if the data has not yet been inserted. -If you are using asynchronous inserts, we also recommend enabling [`async_insert`](#async-insert). +If you want to use asynchronous inserts, we need to also enable [`async_insert`](#async-insert). Possible values: From 6c415e9ec5fd984bea00b0b4c2a70ff1c59fe1a6 Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Mon, 6 May 2024 00:07:37 -0700 Subject: [PATCH 357/624] [Docs] wait_for_async_insert clarification MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Raúl Marín --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 474788c9812..8369c6b850f 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1923,7 +1923,7 @@ Default value: `16`. ### wait_for_async_insert {#wait-for-async-insert} -Enables or disables waiting for processing of asynchronous insertion. If enabled, server will return `OK` only after the data is inserted. Otherwise, it will return `OK` even if the data has not yet been inserted. +Enables or disables waiting for processing of asynchronous insertion. If enabled, server will return `OK` only after the data is inserted. Otherwise, it will return `OK` as soon it has received the data, but it might still fail to parse or insert it later (You can check in system.asynchronous_insert_log) If you want to use asynchronous inserts, we need to also enable [`async_insert`](#async-insert). From d6690f8384235531947a09cd65f00623095f3ed4 Mon Sep 17 00:00:00 2001 From: unashi Date: Mon, 6 May 2024 15:12:16 +0800 Subject: [PATCH 358/624] [feature] Raw as a synonym for TSVRaw --- docs/en/interfaces/formats.md | 6 +++--- .../Formats/Impl/TabSeparatedRowInputFormat.cpp | 8 ++++++++ .../Formats/Impl/TabSeparatedRowOutputFormat.cpp | 3 +++ .../0_stateless/00397_tsv_format_synonym.reference | 3 +++ tests/queries/0_stateless/00397_tsv_format_synonym.sql | 1 + 5 files changed, 18 insertions(+), 3 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 03cf345349e..937dfb52609 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`. +This format is also available under the name `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`. +This format is also available under the name `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`. +This format is also available under the name `TSVRawWithNamesAndNames`, `RawWithNamesAndNames`. ## Template {#format-template} diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index 85b1797dab8..09f8fa92e5f 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -402,6 +402,8 @@ void registerInputFormatTabSeparated(FormatFactory & factory) registerWithNamesAndTypes(is_raw ? "TabSeparatedRaw" : "TabSeparated", register_func); registerWithNamesAndTypes(is_raw ? "TSVRaw" : "TSV", register_func); + if (is_raw) + registerWithNamesAndTypes("Raw", register_func); } } @@ -433,6 +435,8 @@ void registerTSVSchemaReader(FormatFactory & factory) registerWithNamesAndTypes(is_raw ? "TabSeparatedRaw" : "TabSeparated", register_func); registerWithNamesAndTypes(is_raw ? "TSVRaw" : "TSV", register_func); + if (is_raw) + registerWithNamesAndTypes("Raw", register_func); } } @@ -506,8 +510,12 @@ void registerFileSegmentationEngineTabSeparated(FormatFactory & factory) registerWithNamesAndTypes(is_raw ? "TSVRaw" : "TSV", register_func); registerWithNamesAndTypes(is_raw ? "TabSeparatedRaw" : "TabSeparated", register_func); + if (is_raw) + registerWithNamesAndTypes("Raw", register_func); markFormatWithNamesAndTypesSupportsSamplingColumns(is_raw ? "TSVRaw" : "TSV", factory); markFormatWithNamesAndTypesSupportsSamplingColumns(is_raw ? "TabSeparatedRaw" : "TabSeparated", factory); + if (is_raw) + markFormatWithNamesAndTypesSupportsSamplingColumns("Raw", factory); } // We can use the same segmentation engine for TSKV. diff --git a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp index a4a5aea26cb..c8384c09be6 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp @@ -95,7 +95,10 @@ void registerOutputFormatTabSeparated(FormatFactory & factory) registerWithNamesAndTypes(is_raw ? "TSVRaw" : "TSV", register_func); registerWithNamesAndTypes(is_raw ? "TabSeparatedRaw" : "TabSeparated", register_func); if (is_raw) + { registerWithNamesAndTypes("LineAsString", register_func); + registerWithNamesAndTypes("Raw", register_func); + } } } diff --git a/tests/queries/0_stateless/00397_tsv_format_synonym.reference b/tests/queries/0_stateless/00397_tsv_format_synonym.reference index c4a86983be3..c91169a06fa 100644 --- a/tests/queries/0_stateless/00397_tsv_format_synonym.reference +++ b/tests/queries/0_stateless/00397_tsv_format_synonym.reference @@ -28,3 +28,6 @@ UInt8 String String 1 hello world 2 hello world 3 hello world +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 8c69a795857..51283c6ced9 100644 --- a/tests/queries/0_stateless/00397_tsv_format_synonym.sql +++ b/tests/queries/0_stateless/00397_tsv_format_synonym.sql @@ -9,3 +9,4 @@ 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; From 64b8b00f7bc16ec6476ed2c1679ada0f2f4b6ca4 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Mon, 6 May 2024 07:31:28 +0000 Subject: [PATCH 359/624] add test with session_timezone as http url param --- .../00265_http_content_type_format_timezone.reference | 2 ++ .../0_stateless/00265_http_content_type_format_timezone.sh | 6 ++++++ 2 files changed, 8 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 4b992161128..39bc7e8cab8 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 @@ -21,3 +21,5 @@ < X-ClickHouse-Timezone: CLICKHOUSE_TIMEZONE < X-ClickHouse-Timezone: Europe/Berlin < X-ClickHouse-Timezone: Africa/Cairo +< X-ClickHouse-Timezone: Europe/Berlin +< X-ClickHouse-Timezone: America/Denver 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 8b04bd74a96..e1a938c702c 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 @@ -16,3 +16,9 @@ ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}" --data-binary @- <<< "SELECT 1 FORMA ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}" --data-binary @- <<< "SELECT timezone() SETTINGS session_timezone='Europe/Berlin'" 2>&1 | grep '< X-ClickHouse-Timezone' | grep -v 'GET' | tr -d '\r'; ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}" --data-binary @- <<< "SELECT timezone() SETTINGS session_timezone='Africa/Cairo'" 2>&1 | grep '< X-ClickHouse-Timezone' | grep -v 'GET' | tr -d '\r'; + +# Not pretty but working way of removing randomized session_timezone for this part of test +CLICKHOUSE_URL_WO_SESSION_TZ=$(echo "${CLICKHOUSE_URL}" |sed 's/\&session_timezone\=\([A-Za-z]*\(\%2F\|\/\)[A-Za-z]*\|[A-Za-z0-9\+\-]*\)//g' | sed 's/\?session_timezone\=\([A-Za-z]*\(\%2F\|\/\)[A-Za-z]*\|[A-Za-z0-9\+\-]*\)\&/\?/g'); + +${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'; From 10907ec4ee18a1a7a6090a9e73cebfa0ce242e0f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 6 May 2024 11:05:16 +0200 Subject: [PATCH 360/624] Fix data race in reportBrokenPart --- src/Storages/MergeTree/MergeTreeData.cpp | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 9350b24c96a..99a5fc25f99 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7300,10 +7300,19 @@ void MergeTreeData::reportBrokenPart(MergeTreeData::DataPartPtr data_part) const broken_part_callback(part->name); } } - else if (data_part->getState() == MergeTreeDataPartState::Active) - broken_part_callback(data_part->name); else - LOG_DEBUG(log, "Will not check potentially broken part {} because it's not active", data_part->getNameWithState()); + { + MergeTreeDataPartState state = MergeTreeDataPartState::Temporary; + { + auto lock = lockParts(); + state = data_part->getState(); + } + + if (state == MergeTreeDataPartState::Active) + broken_part_callback(data_part->name); + else + LOG_DEBUG(log, "Will not check potentially broken part {} because it's not active", data_part->getNameWithState()); + } } MergeTreeData::MatcherFn MergeTreeData::getPartitionMatcher(const ASTPtr & partition_ast, ContextPtr local_context) const From 49bbee8d9c8938276f15b107cdf4e0bdc7b07be5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 6 May 2024 11:23:53 +0200 Subject: [PATCH 361/624] Review fixes --- src/Storages/RabbitMQ/RabbitMQConsumer.cpp | 4 ++-- src/Storages/RabbitMQ/RabbitMQSource.cpp | 11 +++++++---- src/Storages/RabbitMQ/RabbitMQSource.h | 6 ++++-- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 7 ++++--- src/Storages/RabbitMQ/StorageRabbitMQ.h | 4 ++-- 5 files changed, 19 insertions(+), 13 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQConsumer.cpp b/src/Storages/RabbitMQ/RabbitMQConsumer.cpp index 129a74f662a..d29194c888d 100644 --- a/src/Storages/RabbitMQ/RabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/RabbitMQConsumer.cpp @@ -149,14 +149,14 @@ bool RabbitMQConsumer::nackMessages(const CommitInfo & commit_info) { if (state != State::OK) { - LOG_TEST(log, "State is {}, will not ack messages", magic_enum::enum_name(state.load(std::memory_order_relaxed))); + LOG_TEST(log, "State is {}, will not nack messages", magic_enum::enum_name(state.load(std::memory_order_relaxed))); return false; } /// Nothing to nack. if (!commit_info.delivery_tag || commit_info.delivery_tag <= last_commited_delivery_tag) { - LOG_TEST(log, "Delivery tag is {}, last committed delivery tag: {}, Will not ack messages", + LOG_TEST(log, "Delivery tag is {}, last committed delivery tag: {}, Will not nack messages", commit_info.delivery_tag, last_commited_delivery_tag); return false; } diff --git a/src/Storages/RabbitMQ/RabbitMQSource.cpp b/src/Storages/RabbitMQ/RabbitMQSource.cpp index eb44e9e2ef3..ffe23dd8631 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSource.cpp @@ -47,7 +47,8 @@ RabbitMQSource::RabbitMQSource( UInt64 max_execution_time_, StreamingHandleErrorMode handle_error_mode_, bool nack_broken_messages_, - bool ack_in_suffix_) + bool ack_in_suffix_, + LoggerPtr log_) : RabbitMQSource( storage_, storage_snapshot_, @@ -58,7 +59,8 @@ RabbitMQSource::RabbitMQSource( max_execution_time_, handle_error_mode_, nack_broken_messages_, - ack_in_suffix_) + ack_in_suffix_, + log_) { } @@ -72,7 +74,8 @@ RabbitMQSource::RabbitMQSource( UInt64 max_execution_time_, StreamingHandleErrorMode handle_error_mode_, bool nack_broken_messages_, - bool ack_in_suffix_) + bool ack_in_suffix_, + LoggerPtr log_) : ISource(getSampleBlock(headers.first, headers.second)) , storage(storage_) , storage_snapshot(storage_snapshot_) @@ -84,7 +87,7 @@ RabbitMQSource::RabbitMQSource( , nack_broken_messages(nack_broken_messages_) , non_virtual_header(std::move(headers.first)) , virtual_header(std::move(headers.second)) - , log(storage.log) + , log(log_) , max_execution_time_ms(max_execution_time_) { storage.incrementReader(); diff --git a/src/Storages/RabbitMQ/RabbitMQSource.h b/src/Storages/RabbitMQ/RabbitMQSource.h index dc0af0043be..54a9f52de6d 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.h +++ b/src/Storages/RabbitMQ/RabbitMQSource.h @@ -21,7 +21,8 @@ public: UInt64 max_execution_time_, StreamingHandleErrorMode handle_error_mode_, bool nack_broken_messages_, - bool ack_in_suffix = false); + bool ack_in_suffix, + LoggerPtr log_); ~RabbitMQSource() override; @@ -69,7 +70,8 @@ private: UInt64 max_execution_time_, StreamingHandleErrorMode handle_error_mode_, bool nack_broken_messages_, - bool ack_in_suffix); + bool ack_in_suffix, + LoggerPtr log_); Chunk generateImpl(); }; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 5d8fd2c8502..0358b2bbc66 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -74,7 +74,6 @@ StorageRabbitMQ::StorageRabbitMQ( LoadingStrictnessLevel mode) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) - , log(getLogger("StorageRabbitMQ (" + table_id_.table_name + ")")) , rabbitmq_settings(std::move(rabbitmq_settings_)) , exchange_name(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_exchange_name)) , format_name(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_format)) @@ -86,6 +85,7 @@ StorageRabbitMQ::StorageRabbitMQ( , queue_base(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_queue_base)) , queue_settings_list(parseSettings(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_queue_settings_list))) , max_rows_per_message(rabbitmq_settings->rabbitmq_max_rows_per_message) + , log(getLogger("StorageRabbitMQ (" + table_id_.table_name + ")")) , persistent(rabbitmq_settings->rabbitmq_persistent.value) , use_user_setup(rabbitmq_settings->rabbitmq_queue_consume.value) , hash_exchange(num_consumers > 1 || num_queues > 1) @@ -747,7 +747,7 @@ void StorageRabbitMQ::read( auto rabbit_source = std::make_shared( *this, storage_snapshot, modified_context, column_names, /* max_block_size */1, max_execution_time_ms, rabbitmq_settings->rabbitmq_handle_error_mode, reject_unhandled_messages, - rabbitmq_settings->rabbitmq_commit_on_select); + /* ack_in_suffix */rabbitmq_settings->rabbitmq_commit_on_select, log); auto converting_dag = ActionsDAG::makeConvertingActions( rabbit_source->getPort().getHeader().getColumnsWithTypeAndName(), @@ -1083,7 +1083,8 @@ 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, reject_unhandled_messages); + max_execution_time_ms, rabbitmq_settings->rabbitmq_handle_error_mode, + reject_unhandled_messages, /* ack_in_suffix */false, log); sources.emplace_back(source); pipes.emplace_back(source); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index 0350cd05af7..59fab7cd1ea 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -77,8 +77,6 @@ public: void incrementReader(); void decrementReader(); - LoggerPtr log; - private: ContextMutablePtr rabbitmq_context; std::unique_ptr rabbitmq_settings; @@ -95,6 +93,8 @@ private: size_t max_rows_per_message; bool reject_unhandled_messages = false; + LoggerPtr log; + /// For insert query. Mark messages as durable. const bool persistent; From ab7d457c51d92123d1845486a2f3243324f06503 Mon Sep 17 00:00:00 2001 From: Yohann Jardin Date: Mon, 6 May 2024 11:35:48 +0200 Subject: [PATCH 362/624] Update ErrorCodes for functions using NUMBER_OF_ARGUMENTS_DOESNT_MATCH --- src/AggregateFunctions/AggregateFunctionAggThrow.cpp | 4 ++-- .../AggregateFunctionGroupArrayInsertAt.cpp | 3 ++- src/AggregateFunctions/AggregateFunctionMLMethod.cpp | 7 ++++--- src/AggregateFunctions/AggregateFunctionMannWhitney.cpp | 4 ++-- src/AggregateFunctions/AggregateFunctionQuantile.cpp | 4 ++-- .../AggregateFunctionQuantileBFloat16Weighted.cpp | 4 ++-- src/AggregateFunctions/AggregateFunctionQuantileDD.cpp | 4 ++-- .../AggregateFunctionQuantileExactHigh.cpp | 4 ++-- .../AggregateFunctionQuantileExactInclusive.cpp | 4 ++-- .../AggregateFunctionQuantileTiming.cpp | 4 ++-- src/AggregateFunctions/AggregateFunctionTopK.cpp | 4 ++-- src/AggregateFunctions/AggregateFunctionWelchTTest.cpp | 4 ++-- .../Combinators/AggregateFunctionArray.cpp | 4 ++-- src/AggregateFunctions/Combinators/AggregateFunctionIf.h | 4 ++-- src/Functions/Kusto/KqlArraySort.cpp | 4 ++-- src/Functions/MultiSearchFirstPositionImpl.h | 4 ++-- src/Functions/MultiSearchImpl.h | 4 ++-- src/Functions/array/arrayEnumerateExtended.h | 4 ++-- src/Functions/array/arrayEnumerateRanked.h | 7 ++++--- src/Functions/array/arrayFold.cpp | 6 +++--- src/Functions/array/arrayReduce.cpp | 4 ++-- src/Functions/array/arrayZip.cpp | 4 ++-- src/Functions/concat.cpp | 6 +++--- src/Functions/generateULID.cpp | 4 ++-- src/Functions/jsonMergePatch.cpp | 4 ++-- src/Functions/nested.cpp | 9 +++++---- src/Functions/now64.cpp | 4 ++-- src/Functions/nowInBlock.cpp | 4 ++-- src/Functions/parseTimeDelta.cpp | 7 ++++--- src/Functions/pointInPolygon.cpp | 4 ++-- src/TableFunctions/TableFunctionValues.cpp | 4 ++-- 31 files changed, 73 insertions(+), 68 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionAggThrow.cpp b/src/AggregateFunctions/AggregateFunctionAggThrow.cpp index 27432bc94ba..ebf6cabbcfd 100644 --- a/src/AggregateFunctions/AggregateFunctionAggThrow.cpp +++ b/src/AggregateFunctions/AggregateFunctionAggThrow.cpp @@ -16,7 +16,7 @@ struct Settings; namespace ErrorCodes { extern const int AGGREGATE_FUNCTION_THROW; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; } namespace @@ -116,7 +116,7 @@ void registerAggregateFunctionAggThrow(AggregateFunctionFactory & factory) if (parameters.size() == 1) throw_probability = parameters[0].safeGet(); else if (parameters.size() > 1) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} cannot have more than one parameter", name); + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} cannot have more than one parameter", name); return std::make_shared(argument_types, parameters, throw_probability); }); diff --git a/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.cpp b/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.cpp index 60e8df64283..8674aac2e90 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.cpp @@ -27,6 +27,7 @@ struct Settings; namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; extern const int TOO_LARGE_ARRAY_SIZE; extern const int CANNOT_CONVERT_TYPE; extern const int ILLEGAL_TYPE_OF_ARGUMENT; @@ -74,7 +75,7 @@ public: if (!params.empty()) { if (params.size() > 2) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at most two parameters.", getName()); + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} requires at most two parameters.", getName()); default_value = params[0]; diff --git a/src/AggregateFunctions/AggregateFunctionMLMethod.cpp b/src/AggregateFunctions/AggregateFunctionMLMethod.cpp index d2178437770..730b5cfa5cc 100644 --- a/src/AggregateFunctions/AggregateFunctionMLMethod.cpp +++ b/src/AggregateFunctions/AggregateFunctionMLMethod.cpp @@ -22,7 +22,8 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; } namespace @@ -34,12 +35,12 @@ namespace const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { if (parameters.size() > 4) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} requires at most four parameters: " "learning_rate, l2_regularization_coef, mini-batch size and weights_updater method", name); if (argument_types.size() < 2) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} requires at least two arguments: target and model's parameters", name); for (size_t i = 0; i < argument_types.size(); ++i) diff --git a/src/AggregateFunctions/AggregateFunctionMannWhitney.cpp b/src/AggregateFunctions/AggregateFunctionMannWhitney.cpp index e7bc5df335f..d185058dbd0 100644 --- a/src/AggregateFunctions/AggregateFunctionMannWhitney.cpp +++ b/src/AggregateFunctions/AggregateFunctionMannWhitney.cpp @@ -21,7 +21,7 @@ namespace ErrorCodes { extern const int NOT_IMPLEMENTED; extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; extern const int BAD_ARGUMENTS; } @@ -141,7 +141,7 @@ public: : IAggregateFunctionDataHelper ({arguments}, {}, createResultType()) { if (params.size() > 2) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} require two parameter or less", getName()); + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} require two parameter or less", getName()); if (params.empty()) { diff --git a/src/AggregateFunctions/AggregateFunctionQuantile.cpp b/src/AggregateFunctions/AggregateFunctionQuantile.cpp index 4d37ec69d26..f72b28030c3 100644 --- a/src/AggregateFunctions/AggregateFunctionQuantile.cpp +++ b/src/AggregateFunctions/AggregateFunctionQuantile.cpp @@ -14,7 +14,7 @@ struct Settings; namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NOT_IMPLEMENTED; } @@ -118,7 +118,7 @@ AggregateFunctionPtr createAggregateFunctionQuantile( const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *) { if (argument_types.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name); + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} requires at least one argument", name); const DataTypePtr & argument_type = argument_types[0]; WhichDataType which(argument_type); diff --git a/src/AggregateFunctions/AggregateFunctionQuantileBFloat16Weighted.cpp b/src/AggregateFunctions/AggregateFunctionQuantileBFloat16Weighted.cpp index 6d881b77c16..b0a39b1fdab 100644 --- a/src/AggregateFunctions/AggregateFunctionQuantileBFloat16Weighted.cpp +++ b/src/AggregateFunctions/AggregateFunctionQuantileBFloat16Weighted.cpp @@ -12,7 +12,7 @@ struct Settings; namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } @@ -27,7 +27,7 @@ AggregateFunctionPtr createAggregateFunctionQuantile( const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *) { if (argument_types.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name); + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} requires at least one argument", name); const DataTypePtr & argument_type = argument_types[0]; WhichDataType which(argument_type); diff --git a/src/AggregateFunctions/AggregateFunctionQuantileDD.cpp b/src/AggregateFunctions/AggregateFunctionQuantileDD.cpp index f3d6b26ee75..7b1ae43038a 100644 --- a/src/AggregateFunctions/AggregateFunctionQuantileDD.cpp +++ b/src/AggregateFunctions/AggregateFunctionQuantileDD.cpp @@ -13,7 +13,7 @@ struct Settings; namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } @@ -29,7 +29,7 @@ AggregateFunctionPtr createAggregateFunctionQuantile( const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *) { if (argument_types.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name); + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} requires at least one argument", name); const DataTypePtr & argument_type = argument_types[0]; WhichDataType which(argument_type); diff --git a/src/AggregateFunctions/AggregateFunctionQuantileExactHigh.cpp b/src/AggregateFunctions/AggregateFunctionQuantileExactHigh.cpp index b44df755158..6fa4158076c 100644 --- a/src/AggregateFunctions/AggregateFunctionQuantileExactHigh.cpp +++ b/src/AggregateFunctions/AggregateFunctionQuantileExactHigh.cpp @@ -13,7 +13,7 @@ struct Settings; namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } @@ -29,7 +29,7 @@ AggregateFunctionPtr createAggregateFunctionQuantile( const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *) { if (argument_types.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name); + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} requires at least one argument", name); const DataTypePtr & argument_type = argument_types[0]; WhichDataType which(argument_type); diff --git a/src/AggregateFunctions/AggregateFunctionQuantileExactInclusive.cpp b/src/AggregateFunctions/AggregateFunctionQuantileExactInclusive.cpp index d8287be86ca..bed5cf95bd8 100644 --- a/src/AggregateFunctions/AggregateFunctionQuantileExactInclusive.cpp +++ b/src/AggregateFunctions/AggregateFunctionQuantileExactInclusive.cpp @@ -13,7 +13,7 @@ struct Settings; namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } @@ -28,7 +28,7 @@ AggregateFunctionPtr createAggregateFunctionQuantile( const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *) { if (argument_types.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name); + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} requires at least one argument", name); const DataTypePtr & argument_type = argument_types[0]; WhichDataType which(argument_type); diff --git a/src/AggregateFunctions/AggregateFunctionQuantileTiming.cpp b/src/AggregateFunctions/AggregateFunctionQuantileTiming.cpp index 1bb77892c0d..e293fc6e59d 100644 --- a/src/AggregateFunctions/AggregateFunctionQuantileTiming.cpp +++ b/src/AggregateFunctions/AggregateFunctionQuantileTiming.cpp @@ -13,7 +13,7 @@ struct Settings; namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } @@ -28,7 +28,7 @@ AggregateFunctionPtr createAggregateFunctionQuantile( const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *) { if (argument_types.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name); + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} requires at least one argument", name); const DataTypePtr & argument_type = argument_types[0]; WhichDataType which(argument_type); diff --git a/src/AggregateFunctions/AggregateFunctionTopK.cpp b/src/AggregateFunctions/AggregateFunctionTopK.cpp index d0e4f507d46..26f756abe18 100644 --- a/src/AggregateFunctions/AggregateFunctionTopK.cpp +++ b/src/AggregateFunctions/AggregateFunctionTopK.cpp @@ -35,7 +35,7 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; } @@ -467,7 +467,7 @@ AggregateFunctionPtr createAggregateFunctionTopK(const std::string & name, const if (!params.empty()) { if (params.size() > 3) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Aggregate function '{}' requires three parameters or less", name); threshold = applyVisitor(FieldVisitorConvertToNumber(), params[0]); diff --git a/src/AggregateFunctions/AggregateFunctionWelchTTest.cpp b/src/AggregateFunctions/AggregateFunctionWelchTTest.cpp index 849f81279e7..7f1c4861fbc 100644 --- a/src/AggregateFunctions/AggregateFunctionWelchTTest.cpp +++ b/src/AggregateFunctions/AggregateFunctionWelchTTest.cpp @@ -7,7 +7,7 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; } namespace DB @@ -80,7 +80,7 @@ AggregateFunctionPtr createAggregateFunctionWelchTTest( assertBinary(name, argument_types); if (parameters.size() > 1) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires zero or one parameter.", name); + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} requires zero or one parameter.", name); if (!isNumber(argument_types[0]) || !isNumber(argument_types[1])) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Aggregate function {} only supports numerical types", name); diff --git a/src/AggregateFunctions/Combinators/AggregateFunctionArray.cpp b/src/AggregateFunctions/Combinators/AggregateFunctionArray.cpp index a54092f0bcb..6b60faadf07 100644 --- a/src/AggregateFunctions/Combinators/AggregateFunctionArray.cpp +++ b/src/AggregateFunctions/Combinators/AggregateFunctionArray.cpp @@ -9,7 +9,7 @@ struct Settings; namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } @@ -26,7 +26,7 @@ public: DataTypes transformArguments(const DataTypes & arguments) const override { if (arguments.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "-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) diff --git a/src/AggregateFunctions/Combinators/AggregateFunctionIf.h b/src/AggregateFunctions/Combinators/AggregateFunctionIf.h index a893fc91780..f57cfa41752 100644 --- a/src/AggregateFunctions/Combinators/AggregateFunctionIf.h +++ b/src/AggregateFunctions/Combinators/AggregateFunctionIf.h @@ -18,7 +18,7 @@ struct Settings; namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } @@ -42,7 +42,7 @@ public: , nested_func(nested), num_arguments(types.size()) { if (num_arguments == 0) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} require at least one argument", getName()); + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} require at least one argument", getName()); only_null_condition = types.back()->onlyNull(); diff --git a/src/Functions/Kusto/KqlArraySort.cpp b/src/Functions/Kusto/KqlArraySort.cpp index 5be36328cc3..ac301c474e8 100644 --- a/src/Functions/Kusto/KqlArraySort.cpp +++ b/src/Functions/Kusto/KqlArraySort.cpp @@ -11,7 +11,7 @@ namespace DB { namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_COLUMN; } @@ -35,7 +35,7 @@ public: { if (arguments.empty()) throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} needs at least one argument; passed {}.", getName(), arguments.size()); diff --git a/src/Functions/MultiSearchFirstPositionImpl.h b/src/Functions/MultiSearchFirstPositionImpl.h index 99dd3f9d394..aca96cabf04 100644 --- a/src/Functions/MultiSearchFirstPositionImpl.h +++ b/src/Functions/MultiSearchFirstPositionImpl.h @@ -10,7 +10,7 @@ namespace DB namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; } template @@ -37,7 +37,7 @@ struct MultiSearchFirstPositionImpl { // For performance of Volnitsky search, it is crucial to save only one byte for pattern number. if (needles_arr.size() > std::numeric_limits::max()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Number of arguments for function {} doesn't match: passed {}, should be at most {}", name, std::to_string(needles_arr.size()), std::to_string(std::numeric_limits::max())); diff --git a/src/Functions/MultiSearchImpl.h b/src/Functions/MultiSearchImpl.h index fb7d56f302a..d3d1680481f 100644 --- a/src/Functions/MultiSearchImpl.h +++ b/src/Functions/MultiSearchImpl.h @@ -10,7 +10,7 @@ namespace DB namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; } template @@ -37,7 +37,7 @@ struct MultiSearchImpl { // For performance of Volnitsky search, it is crucial to save only one byte for pattern number. if (needles_arr.size() > std::numeric_limits::max()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Number of arguments for function {} doesn't match: passed {}, should be at most {}", name, needles_arr.size(), std::to_string(std::numeric_limits::max())); diff --git a/src/Functions/array/arrayEnumerateExtended.h b/src/Functions/array/arrayEnumerateExtended.h index cf38afcfa5a..67c0981108f 100644 --- a/src/Functions/array/arrayEnumerateExtended.h +++ b/src/Functions/array/arrayEnumerateExtended.h @@ -18,7 +18,7 @@ namespace DB namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int SIZES_OF_ARRAYS_DONT_MATCH; @@ -43,7 +43,7 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (arguments.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Number of arguments for function {} doesn't match: passed {}, should be at least 1.", getName(), arguments.size()); diff --git a/src/Functions/array/arrayEnumerateRanked.h b/src/Functions/array/arrayEnumerateRanked.h index 04fa305368d..ad325fe542a 100644 --- a/src/Functions/array/arrayEnumerateRanked.h +++ b/src/Functions/array/arrayEnumerateRanked.h @@ -59,7 +59,8 @@ namespace DB { namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int SIZES_OF_ARRAYS_DONT_MATCH; } @@ -101,7 +102,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { if (arguments.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Number of arguments for function {} doesn't match: passed {}, should be at least 1.", getName(), arguments.size()); @@ -238,7 +239,7 @@ ColumnPtr FunctionArrayEnumerateRankedExtended::executeImpl( } if (offsets_by_depth.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "No arrays passed to function {}", getName()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "No arrays passed to function {}", getName()); auto res_nested = ColumnUInt32::create(); diff --git a/src/Functions/array/arrayFold.cpp b/src/Functions/array/arrayFold.cpp index 63c14f475fc..5170c9a5b5f 100644 --- a/src/Functions/array/arrayFold.cpp +++ b/src/Functions/array/arrayFold.cpp @@ -14,7 +14,7 @@ namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int SIZES_OF_ARRAYS_DONT_MATCH; extern const int TYPE_MISMATCH; } @@ -41,7 +41,7 @@ public: void getLambdaArgumentTypes(DataTypes & arguments) const override { if (arguments.size() < 3) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires as arguments a lambda function, at least one array and an accumulator", getName()); + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} requires as arguments a lambda function, at least one array and an accumulator", getName()); DataTypes accumulator_and_array_types(arguments.size() - 1); accumulator_and_array_types[0] = arguments.back(); @@ -64,7 +64,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { if (arguments.size() < 3) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires as arguments a lambda function, at least one array and an accumulator", getName()); + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} requires as arguments a lambda function, at least one array and an accumulator", getName()); const auto * lambda_function_type = checkAndGetDataType(arguments[0].type.get()); if (!lambda_function_type) diff --git a/src/Functions/array/arrayReduce.cpp b/src/Functions/array/arrayReduce.cpp index d47d1ae98cc..d70b8b40233 100644 --- a/src/Functions/array/arrayReduce.cpp +++ b/src/Functions/array/arrayReduce.cpp @@ -21,7 +21,7 @@ namespace DB namespace ErrorCodes { extern const int SIZES_OF_ARRAYS_DONT_MATCH; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int BAD_ARGUMENTS; @@ -73,7 +73,7 @@ DataTypePtr FunctionArrayReduce::getReturnTypeImpl(const ColumnsWithTypeAndName /// (possibly with parameters in parentheses, for example: "quantile(0.99)"). if (arguments.size() < 2) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Number of arguments for function {} doesn't match: passed {}, should be at least 2.", getName(), arguments.size()); diff --git a/src/Functions/array/arrayZip.cpp b/src/Functions/array/arrayZip.cpp index 44c323e3fe3..6c6fff5926b 100644 --- a/src/Functions/array/arrayZip.cpp +++ b/src/Functions/array/arrayZip.cpp @@ -14,7 +14,7 @@ namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int SIZES_OF_ARRAYS_DONT_MATCH; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_COLUMN; } @@ -39,7 +39,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { if (arguments.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} needs at least one argument; passed {}." , getName(), arguments.size()); DataTypes arguments_types; diff --git a/src/Functions/concat.cpp b/src/Functions/concat.cpp index c75a806559c..68cfcdb8d90 100644 --- a/src/Functions/concat.cpp +++ b/src/Functions/concat.cpp @@ -16,7 +16,7 @@ namespace DB { namespace ErrorCodes { -extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; } using namespace GatherUtils; @@ -48,7 +48,7 @@ public: { if (arguments.size() < 2) throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Number of arguments for function {} doesn't match: passed {}, should be at least 2", getName(), arguments.size()); @@ -225,7 +225,7 @@ public: { if (arguments.empty()) throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Number of arguments for function {} doesn't match: passed {}, should be at least 1.", getName(), arguments.size()); diff --git a/src/Functions/generateULID.cpp b/src/Functions/generateULID.cpp index 9c5c9403185..f2f2d8ae3b9 100644 --- a/src/Functions/generateULID.cpp +++ b/src/Functions/generateULID.cpp @@ -17,7 +17,7 @@ namespace DB namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; } class FunctionGenerateULID : public IFunction @@ -45,7 +45,7 @@ public: { if (arguments.size() > 1) throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Number of arguments for function {} doesn't match: passed {}, should be 0 or 1.", getName(), arguments.size()); diff --git a/src/Functions/jsonMergePatch.cpp b/src/Functions/jsonMergePatch.cpp index 65946721432..a83daacdbf6 100644 --- a/src/Functions/jsonMergePatch.cpp +++ b/src/Functions/jsonMergePatch.cpp @@ -25,7 +25,7 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int ILLEGAL_COLUMN; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } @@ -53,7 +53,7 @@ namespace DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { if (arguments.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires at least one argument.", getName()); + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} requires at least one argument.", getName()); for (const auto & arg : arguments) if (!isString(arg.type)) diff --git a/src/Functions/nested.cpp b/src/Functions/nested.cpp index 679bb4f73d8..bdaf57d65c9 100644 --- a/src/Functions/nested.cpp +++ b/src/Functions/nested.cpp @@ -18,9 +18,10 @@ namespace DB namespace ErrorCodes { + extern const int BAD_ARGUMENTS; extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int SIZES_OF_ARRAYS_DONT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; } namespace @@ -64,19 +65,19 @@ public: { size_t arguments_size = arguments.size(); if (arguments_size < 2) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Number of arguments for function {} doesn't match: passed {}, should be at least 2", getName(), arguments_size); Names nested_names = extractNestedNames(arguments[0].column); if (nested_names.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception(ErrorCodes::BAD_ARGUMENTS, "First argument for function {} must be constant column with array of strings", getName()); if (nested_names.size() != arguments_size - 1) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Size of nested names array for function {} does not match arrays arguments size. Actual {}. Expected {}", getName(), nested_names.size(), diff --git a/src/Functions/now64.cpp b/src/Functions/now64.cpp index 0f1e8a04236..d6f8474c984 100644 --- a/src/Functions/now64.cpp +++ b/src/Functions/now64.cpp @@ -18,7 +18,7 @@ namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int CANNOT_CLOCK_GETTIME; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; } namespace @@ -128,7 +128,7 @@ public: if (arguments.size() > 2) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Arguments size of function {} should be 0, or 1, or 2", getName()); + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Arguments size of function {} should be 0, or 1, or 2", getName()); } if (!arguments.empty()) { diff --git a/src/Functions/nowInBlock.cpp b/src/Functions/nowInBlock.cpp index 0d5f9c45780..74f420986c8 100644 --- a/src/Functions/nowInBlock.cpp +++ b/src/Functions/nowInBlock.cpp @@ -12,7 +12,7 @@ namespace DB namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } @@ -63,7 +63,7 @@ public: { if (arguments.size() > 1) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Arguments size of function {} should be 0 or 1", getName()); + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Arguments size of function {} should be 0 or 1", getName()); } if (arguments.size() == 1 && !isStringOrFixedString(arguments[0].type)) { diff --git a/src/Functions/parseTimeDelta.cpp b/src/Functions/parseTimeDelta.cpp index 7743a0cb664..44eeb1a289f 100644 --- a/src/Functions/parseTimeDelta.cpp +++ b/src/Functions/parseTimeDelta.cpp @@ -11,7 +11,8 @@ namespace DB namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int BAD_ARGUMENTS; } @@ -117,14 +118,14 @@ namespace { if (arguments.empty()) throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Number of arguments for function {} doesn't match: passed {}, should be 1.", getName(), arguments.size()); if (arguments.size() > 1) throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Number of arguments for function {} doesn't match: passed {}, should be 1.", getName(), arguments.size()); diff --git a/src/Functions/pointInPolygon.cpp b/src/Functions/pointInPolygon.cpp index 0e4467a8210..55f89b71d6f 100644 --- a/src/Functions/pointInPolygon.cpp +++ b/src/Functions/pointInPolygon.cpp @@ -37,7 +37,7 @@ namespace DB { namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int BAD_ARGUMENTS; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_COLUMN; @@ -87,7 +87,7 @@ public: { if (arguments.size() < 2) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires at least 2 arguments", getName()); + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} requires at least 2 arguments", getName()); } /** We allow function invocation in one of the following forms: diff --git a/src/TableFunctions/TableFunctionValues.cpp b/src/TableFunctions/TableFunctionValues.cpp index 7b2a61c25eb..4b56fa57091 100644 --- a/src/TableFunctions/TableFunctionValues.cpp +++ b/src/TableFunctions/TableFunctionValues.cpp @@ -25,7 +25,7 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; } @@ -112,7 +112,7 @@ void TableFunctionValues::parseArguments(const ASTPtr & ast_function, ContextPtr ASTs & args = args_func.at(0)->children; if (args.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' requires at least 1 argument", getName()); + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Table function '{}' requires at least 1 argument", getName()); const auto & literal = args[0]->as(); String value; From cb4f78d9d83de9ec29569652a0d4e6b826e9c0fb 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: Mon, 6 May 2024 13:04:28 +0300 Subject: [PATCH 363/624] Allow to create MaterializedMySQL database without connection --- .../MySQL/DatabaseMaterializedMySQL.cpp | 5 +--- .../materialized_with_ddl.py | 26 +++++++++++++++++++ .../test_materialized_mysql_database/test.py | 8 ++++++ 3 files changed, 35 insertions(+), 4 deletions(-) diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp index d8360a24bcb..6d89cc23590 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp @@ -81,12 +81,9 @@ LoadTaskPtr DatabaseMaterializedMySQL::startupDatabaseAsync(AsyncLoader & async_ base->goals(), TablesLoaderBackgroundStartupPoolId, fmt::format("startup MaterializedMySQL database {}", getDatabaseName()), - [this, mode] (AsyncLoader &, const LoadJobPtr &) + [this] (AsyncLoader &, const LoadJobPtr &) { LOG_TRACE(log, "Starting MaterializeMySQL database"); - if (mode < LoadingStrictnessLevel::FORCE_ATTACH) - materialize_thread.assertMySQLAvailable(); - materialize_thread.startSynchronization(); started_up = true; }); 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 57f2ccd720d..dcb2546bad3 100644 --- a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py @@ -3413,3 +3413,29 @@ def gtid_after_attach_test(clickhouse_node, mysql_node, replication): interval_seconds=1, retry_count=300, ) + + +def mysql_create_database_without_connection(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS create_without_connection") + clickhouse_node.query("DROP DATABASE IF EXISTS create_without_connection") + mysql_node.query("CREATE DATABASE create_without_connection") + mysql_node.query( + "CREATE TABLE create_without_connection.test ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;" + ) + + clickhouse_node.cluster.pause_container(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( + service_name + ) + ) + + clickhouse_node.cluster.unpause_container(service_name) + mysql_node.alloc_connection() + + check_query( + clickhouse_node, + "SHOW TABLES FROM create_without_connection FORMAT TSV", + "test\n", + ) diff --git a/tests/integration/test_materialized_mysql_database/test.py b/tests/integration/test_materialized_mysql_database/test.py index 57e496fe737..080a850a8c6 100644 --- a/tests/integration/test_materialized_mysql_database/test.py +++ b/tests/integration/test_materialized_mysql_database/test.py @@ -721,3 +721,11 @@ def test_binlog_client(started_cluster, started_mysql_8_0, replication): materialized_with_ddl.gtid_after_attach_test( node_db, started_mysql_8_0, replication ) + + +def test_create_database_without_mysql_connection( + started_cluster, started_mysql_8_0, clickhouse_node: ClickHouseInstance +): + materialized_with_ddl.mysql_create_database_without_connection( + clickhouse_node, started_mysql_8_0, "mysql80" + ) From b04cf7a698f92b9cdd5e045c336645f7067af43b Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 6 May 2024 12:25:28 +0200 Subject: [PATCH 364/624] fix 02815_join_algorithm_setting --- .../queries/0_stateless/02815_join_algorithm_setting.sql | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02815_join_algorithm_setting.sql b/tests/queries/0_stateless/02815_join_algorithm_setting.sql index c26f82d63bc..a8733eebc91 100644 --- a/tests/queries/0_stateless/02815_join_algorithm_setting.sql +++ b/tests/queries/0_stateless/02815_join_algorithm_setting.sql @@ -103,4 +103,10 @@ JOIN ( SELECT k AS key, k + 100 AS key2 FROM t2 ) AS t2 ON t1.key = t2.key OR t1 -- But for CROSS choose `hash` algorithm even though it's not enabled SELECT * FROM ( SELECT number AS key, number * 10 AS key2 FROM numbers_mt(10) ) AS t1 -CROSS JOIN ( SELECT k AS key, k + 100 AS key2 FROM t2 ) AS t2 FORMAT Null; +CROSS JOIN ( SELECT k AS key, k + 100 AS key2 FROM t2 ) AS t2 FORMAT Null +SETTINGS allow_experimental_analyzer = 1; + +-- ... (not for old analyzer) +SELECT * FROM ( SELECT number AS key, number * 10 AS key2 FROM numbers_mt(10) ) AS t1 +CROSS JOIN ( SELECT k AS key, k + 100 AS key2 FROM t2 ) AS t2 FORMAT Null +SETTINGS allow_experimental_analyzer = 0; -- { serverError NOT_IMPLEMENTED } From f15ee477ebb6fe030e0581b2f8b31b289f816f52 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 6 May 2024 12:39:52 +0200 Subject: [PATCH 365/624] Add test for #46335 --- .../03146_tcp_ds_grouping.reference | 0 .../0_stateless/03146_tcp_ds_grouping.sql | 56 +++++++++++++++++++ 2 files changed, 56 insertions(+) create mode 100644 tests/queries/0_stateless/03146_tcp_ds_grouping.reference create mode 100644 tests/queries/0_stateless/03146_tcp_ds_grouping.sql diff --git a/tests/queries/0_stateless/03146_tcp_ds_grouping.reference b/tests/queries/0_stateless/03146_tcp_ds_grouping.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03146_tcp_ds_grouping.sql b/tests/queries/0_stateless/03146_tcp_ds_grouping.sql new file mode 100644 index 00000000000..d37fb6f0425 --- /dev/null +++ b/tests/queries/0_stateless/03146_tcp_ds_grouping.sql @@ -0,0 +1,56 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/46335 +SET allow_experimental_analyzer = 1; +SELECT + key_a + key_b AS d, + rank() OVER () AS f +FROM + ( + SELECT + rand() % 10 AS key_a, + rand(1) % 5 AS key_b, + number + FROM numbers(100) + ) +GROUP BY + key_a, + key_b +WITH ROLLUP +ORDER BY multiIf(d = 0, key_a, NULL) ASC +FORMAT Null; + +SELECT + key_a + key_b AS d, + rank() OVER (PARTITION BY key_a + key_b) AS f +FROM + ( + SELECT + rand() % 10 AS key_a, + rand(1) % 5 AS key_b, + number + FROM numbers(100) + ) +GROUP BY + key_a, + key_b +WITH ROLLUP +ORDER BY multiIf(d = 0, key_a, NULL) ASC +FORMAT Null; + + +SELECT + grouping(key_a) + grouping(key_b) AS d, + rank() OVER (PARTITION BY grouping(key_a) + grouping(key_b), multiIf(grouping(key_b) = 0, key_a, NULL)) AS f +FROM + ( + SELECT + rand() % 10 AS key_a, + rand(1) % 5 AS key_b, + number + FROM numbers(100) + ) +GROUP BY + key_a, + key_b +WITH ROLLUP +ORDER BY multiIf(d = 0, key_a, NULL) ASC +FORMAT Null; From a51786532be43b451d779eb0f272f444f119e644 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 6 May 2024 13:19:34 +0200 Subject: [PATCH 366/624] Remove copy --- src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp | 2 +- src/Disks/IO/CachedOnDiskWriteBufferFromFile.h | 2 +- src/Interpreters/Cache/FileSegment.cpp | 14 +++++++++----- src/Interpreters/Cache/FileSegment.h | 2 +- 4 files changed, 12 insertions(+), 8 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp index f4e309f461e..382c4a80cc4 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp @@ -41,7 +41,7 @@ FileSegmentRangeWriter::FileSegmentRangeWriter( { } -bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind) +bool FileSegmentRangeWriter::write(char * data, size_t size, size_t offset, FileSegmentKind segment_kind) { if (finalized) return false; diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h index ad4f6b5916d..ba2952d9c56 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h @@ -39,7 +39,7 @@ public: * Write a range of file segments. Allocate file segment of `max_file_segment_size` and write to * it until it is full and then allocate next file segment. */ - bool write(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind); + bool write(char * data, size_t size, size_t offset, FileSegmentKind segment_kind); void finalize(); diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 20a3af60c8c..9459029dc4c 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -340,7 +340,7 @@ void FileSegment::setRemoteFileReader(RemoteFileReaderPtr remote_file_reader_) remote_file_reader = remote_file_reader_; } -void FileSegment::write(const char * from, size_t size, size_t offset) +void FileSegment::write(char * from, size_t size, size_t offset) { ProfileEventTimeIncrement watch(ProfileEvents::FileSegmentWriteMicroseconds); @@ -389,16 +389,20 @@ void FileSegment::write(const char * from, size_t size, size_t offset) try { - if (!cache_writer) - cache_writer = std::make_unique(file_segment_path); - #ifdef ABORT_ON_LOGICAL_ERROR /// This mutex is only needed to have a valid assertion in assertCacheCorrectness(), /// which is only executed in debug/sanitizer builds (under ABORT_ON_LOGICAL_ERROR). std::lock_guard lock(write_mutex); #endif - cache_writer->write(from, size); + if (!cache_writer) + cache_writer = std::make_unique(file_segment_path, /* buf_size */0); + + /// Size is equal to offset as offset for write buffer points to data end. + cache_writer->set(from, size, /* offset */size); + /// Reset the buffer when finished. + SCOPE_EXIT({ cache_writer->set(nullptr, 0); }); + /// Flush the buffer. cache_writer->next(); downloaded_size += size; diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index 7793c50d2d5..f28482a1ce4 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -204,7 +204,7 @@ public: bool reserve(size_t size_to_reserve, size_t lock_wait_timeout_milliseconds, FileCacheReserveStat * reserve_stat = nullptr); /// Write data into reserved space. - void write(const char * from, size_t size, size_t offset); + void write(char * from, size_t size, size_t offset); // Invariant: if state() != DOWNLOADING and remote file reader is present, the reader's // available() == 0, and getFileOffsetOfBufferEnd() == our getCurrentWriteOffset(). From c932d26799b90982f2ec2d221bf0ea8e7eb8f2ad Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 6 May 2024 13:27:19 +0200 Subject: [PATCH 367/624] Name --- ..._tcp_ds_grouping.reference => 03146_tpc_ds_grouping.reference} | 0 .../{03146_tcp_ds_grouping.sql => 03146_tpc_ds_grouping.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{03146_tcp_ds_grouping.reference => 03146_tpc_ds_grouping.reference} (100%) rename tests/queries/0_stateless/{03146_tcp_ds_grouping.sql => 03146_tpc_ds_grouping.sql} (100%) diff --git a/tests/queries/0_stateless/03146_tcp_ds_grouping.reference b/tests/queries/0_stateless/03146_tpc_ds_grouping.reference similarity index 100% rename from tests/queries/0_stateless/03146_tcp_ds_grouping.reference rename to tests/queries/0_stateless/03146_tpc_ds_grouping.reference diff --git a/tests/queries/0_stateless/03146_tcp_ds_grouping.sql b/tests/queries/0_stateless/03146_tpc_ds_grouping.sql similarity index 100% rename from tests/queries/0_stateless/03146_tcp_ds_grouping.sql rename to tests/queries/0_stateless/03146_tpc_ds_grouping.sql From 14c8af4d55080cb11720632f0a4c54267b0ab237 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Mon, 6 May 2024 11:27:32 +0000 Subject: [PATCH 368/624] make regexp shorter and better --- .../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 e1a938c702c..fb5ea456ed7 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 @@ -18,7 +18,7 @@ ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}" --data-binary @- <<< "SELECT timezon ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}" --data-binary @- <<< "SELECT timezone() SETTINGS session_timezone='Africa/Cairo'" 2>&1 | grep '< X-ClickHouse-Timezone' | grep -v 'GET' | tr -d '\r'; # Not pretty but working way of removing randomized session_timezone for this part of test -CLICKHOUSE_URL_WO_SESSION_TZ=$(echo "${CLICKHOUSE_URL}" |sed 's/\&session_timezone\=\([A-Za-z]*\(\%2F\|\/\)[A-Za-z]*\|[A-Za-z0-9\+\-]*\)//g' | sed 's/\?session_timezone\=\([A-Za-z]*\(\%2F\|\/\)[A-Za-z]*\|[A-Za-z0-9\+\-]*\)\&/\?/g'); +CLICKHOUSE_URL_WO_SESSION_TZ=$(echo "${CLICKHOUSE_URL}" |sed 's/\&session_timezone\=[A-Za-z0-9\/\%\_\-\+\-]*//g' | sed 's/\?session_timezone\=[A-Za-z0-9\/\%\_\-\+\-]*\&/\?/g'); ${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'; From d24d1e8979e36b07b8cb755652232567cf101cf3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 6 May 2024 13:52:59 +0200 Subject: [PATCH 369/624] Update log message --- 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 ffe23dd8631..15d013245d3 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSource.cpp @@ -208,9 +208,9 @@ Chunk RabbitMQSource::generateImpl() const auto exchange_name = storage.getExchange(); const auto & message = consumer->currentMessage(); - LOG_TEST(log, "Pulled {} rows, message delivery tag: {} on channel {}, " + 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, consumer->getChannelID(), commit_info.delivery_tag, message.redelivered, + 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 48d6c3760d3890280ce4308e9d51afe9b281232d Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 6 May 2024 12:05:37 +0000 Subject: [PATCH 370/624] workaround for `oklch()` inside canvas bug for firefox --- programs/server/dashboard.html | 50 +++++++++++++++++++++++++++++++++- 1 file changed, 49 insertions(+), 1 deletion(-) diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index 901211e8ad9..b21d4b86314 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -538,9 +538,57 @@ let params = default_params; /// Palette generation for charts function generatePalette(numColors) { + // oklch() does not work in firefox<=125 inside element so we convert it back to rgb for now. + // Based on https://github.com/color-js/color.js/blob/main/src/spaces/oklch.js + const multiplyMatrices = (A, B) => { + return [ + A[0]*B[0] + A[1]*B[1] + A[2]*B[2], + A[3]*B[0] + A[4]*B[1] + A[5]*B[2], + A[6]*B[0] + A[7]*B[1] + A[8]*B[2] + ]; + } + + const oklch2oklab = ([l, c, h]) => [ + l, + isNaN(h) ? 0 : c * Math.cos(h * Math.PI / 180), + isNaN(h) ? 0 : c * Math.sin(h * Math.PI / 180) + ] + + const srgbLinear2rgb = rgb => rgb.map(c => + Math.abs(c) > 0.0031308 ? + (c < 0 ? -1 : 1) * (1.055 * (Math.abs(c) ** (1 / 2.4)) - 0.055) : + 12.92 * c + ) + + const oklab2xyz = lab => { + const LMSg = multiplyMatrices([ + 1, 0.3963377773761749, 0.2158037573099136, + 1, -0.1055613458156586, -0.0638541728258133, + 1, -0.0894841775298119, -1.2914855480194092, + ], lab) + const LMS = LMSg.map(val => val ** 3) + return multiplyMatrices([ + 1.2268798758459243, -0.5578149944602171, 0.2813910456659647, + -0.0405757452148008, 1.1122868032803170, -0.0717110580655164, + -0.0763729366746601, -0.4214933324022432, 1.5869240198367816 + ], LMS) + } + + const xyz2rgbLinear = xyz => { + return multiplyMatrices([ + 3.2409699419045226, -1.537383177570094, -0.4986107602930034, + -0.9692436362808796, 1.8759675015077202, 0.04155505740717559, + 0.05563007969699366, -0.20397695888897652, 1.0569715142428786 + ], xyz) + } + + const oklch2rgb = lch => srgbLinear2rgb(xyz2rgbLinear(oklab2xyz(oklch2oklab(lch)))) + palette = []; for (let i = 0; i < numColors; i++) { - palette.push(`oklch(${theme != 'dark' ? 0.75 : 0.5}, 0.15, ${360 * i / numColors})`); + //palette.push(`oklch(${theme != 'dark' ? 0.75 : 0.5}, 0.15, ${360 * i / numColors})`); + let rgb = oklch2rgb([theme != 'dark' ? 0.75 : 0.5, 0.15, 360 * i / numColors]); + palette.push(`rgb(${rgb[0] * 255}, ${rgb[1] * 255}, ${rgb[2] * 255})`); } return palette; } From 731d05491cf44d8356f1d6971883004a862fcd0d Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Mon, 6 May 2024 12:08:02 +0000 Subject: [PATCH 371/624] simplify estimation of number of objects in bucket --- src/Storages/StorageS3.cpp | 33 ++++++++++++++------------------- src/Storages/StorageS3.h | 1 - 2 files changed, 14 insertions(+), 20 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index cb5734cfe0c..8a4e30fed1d 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -485,12 +485,15 @@ StorageS3Source::KeyWithInfoPtr StorageS3Source::DisclosedGlobIterator::next(siz size_t StorageS3Source::DisclosedGlobIterator::estimatedKeysCount() { - return pimpl->objectsCount(); -} - -bool StorageS3Source::DisclosedGlobIterator::hasMore() -{ - return pimpl->hasMore(); + if (pimpl->hasMore()) + { + /// 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. + return std::numeric_limits::max(); + } + else + return pimpl->objectsCount(); } class StorageS3Source::KeysIterator::Impl @@ -1285,21 +1288,13 @@ void ReadFromStorageS3Step::initializePipeline(QueryPipelineBuilder & pipeline, createIterator(nullptr); size_t estimated_keys_count = iterator_wrapper->estimatedKeysCount(); - const auto glob_iter = std::dynamic_pointer_cast(iterator_wrapper); - - if (!(glob_iter && glob_iter->hasMore())) + if (estimated_keys_count > 1) + num_streams = std::min(num_streams, estimated_keys_count); + else { - if (estimated_keys_count > 1) - num_streams = std::min(num_streams, estimated_keys_count); - else - { - /// The amount of keys (zero) was probably underestimated. We will keep one stream for this particular case. - num_streams = 1; - } + /// The amount of keys (zero) was probably underestimated. We will keep one stream for this particular case. + num_streams = 1; } - /// OTHERWISE, 1000 files were listed, but we cannot make any estimation of _how many_ 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. 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)); diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index b841e973a9b..c8ab28fb20e 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -83,7 +83,6 @@ public: KeyWithInfoPtr next(size_t idx = 0) override; /// NOLINT size_t estimatedKeysCount() override; - bool hasMore(); private: class Impl; From 8cdaa1a32a49c32b50e4006aa3a737c48e3e21cb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 6 May 2024 14:36:04 +0200 Subject: [PATCH 372/624] Make grouping never return low cardinality --- src/Functions/grouping.h | 4 ++++ .../queries/0_stateless/03146_tpc_ds_grouping.sql | 15 +++++++++++++++ 2 files changed, 19 insertions(+) diff --git a/src/Functions/grouping.h b/src/Functions/grouping.h index 830c509f1f5..22cbcf17d6c 100644 --- a/src/Functions/grouping.h +++ b/src/Functions/grouping.h @@ -47,6 +47,10 @@ public: bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + /// Change it to never return LowCardinality, making it consistent when using groupingForRollup / groupingForforCube + /// with __grouping_set + bool canBeExecutedOnLowCardinalityDictionary() const override { return false; } + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override { return std::make_shared(); diff --git a/tests/queries/0_stateless/03146_tpc_ds_grouping.sql b/tests/queries/0_stateless/03146_tpc_ds_grouping.sql index d37fb6f0425..f48c40e9bc4 100644 --- a/tests/queries/0_stateless/03146_tpc_ds_grouping.sql +++ b/tests/queries/0_stateless/03146_tpc_ds_grouping.sql @@ -54,3 +54,18 @@ GROUP BY WITH ROLLUP ORDER BY multiIf(d = 0, key_a, NULL) ASC FORMAT Null; + +SELECT grouping(key_a) + grouping(key_b) AS d +FROM + ( + SELECT + rand() % 10 AS key_a, + rand(toLowCardinality(1)) % 5 AS key_b, + number + FROM numbers(100) + ) +GROUP BY + key_a, + key_b +WITH ROLLUP +FORMAT Null; From 1dff80374c08cc657933e63a00f2b6adfdfeb474 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 6 May 2024 13:48:39 +0000 Subject: [PATCH 373/624] Docs: toStartOfInterval uses uppercase units For consistency with existing documentation https://clickhouse.com/docs/en/sql-reference/data-types/special-data-types/interval ... and with MySQL --- .../functions/date-time-functions.md | 32 +++++++++---------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index ed98f986139..e8ed1ed64a4 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1417,31 +1417,31 @@ toStartOfFifteenMinutes(toDateTime('2023-04-21 10:23:00')): 2023-04-21 10:15:00 This function generalizes other `toStartOf*()` functions with `toStartOfInterval(date_or_date_with_time, INTERVAL x unit [, time_zone])` syntax. For example, -- `toStartOfInterval(t, INTERVAL 1 year)` returns the same as `toStartOfYear(t)`, -- `toStartOfInterval(t, INTERVAL 1 month)` returns the same as `toStartOfMonth(t)`, -- `toStartOfInterval(t, INTERVAL 1 day)` returns the same as `toStartOfDay(t)`, -- `toStartOfInterval(t, INTERVAL 15 minute)` returns the same as `toStartOfFifteenMinutes(t)`. +- `toStartOfInterval(t, INTERVAL 1 YEAR)` returns the same as `toStartOfYear(t)`, +- `toStartOfInterval(t, INTERVAL 1 MONTH)` returns the same as `toStartOfMonth(t)`, +- `toStartOfInterval(t, INTERVAL 1 DAY)` returns the same as `toStartOfDay(t)`, +- `toStartOfInterval(t, INTERVAL 15 MINUTE)` returns the same as `toStartOfFifteenMinutes(t)`. The calculation is performed relative to specific points in time: | Interval | Start | |-------------|------------------------| -| year | year 0 | -| quarter | 1900 Q1 | -| month | 1900 January | -| week | 1970, 1st week (01-05) | -| day | 1970-01-01 | -| hour | (*) | -| minute | 1970-01-01 00:00:00 | -| second | 1970-01-01 00:00:00 | -| millisecond | 1970-01-01 00:00:00 | -| microsecond | 1970-01-01 00:00:00 | -| nanosecond | 1970-01-01 00:00:00 | +| YEAR | year 0 | +| QUARTER | 1900 Q1 | +| MONTH | 1900 January | +| WEEK | 1970, 1st week (01-05) | +| DAY | 1970-01-01 | +| HOUR | (*) | +| MINUTE | 1970-01-01 00:00:00 | +| SECOND | 1970-01-01 00:00:00 | +| MILLISECOND | 1970-01-01 00:00:00 | +| MICROSECOND | 1970-01-01 00:00:00 | +| NANOSECOND | 1970-01-01 00:00:00 | (*) hour intervals are special: the calculation is always performed relative to 00:00:00 (midnight) of the current day. As a result, only hour values between 1 and 23 are useful. -If unit `week` was specified, `toStartOfInterval` assumes that weeks start on Monday. Note that this behavior is different from that of function `toStartOfWeek` in which weeks start by default on Sunday. +If unit `WEEK` was specified, `toStartOfInterval` assumes that weeks start on Monday. Note that this behavior is different from that of function `toStartOfWeek` in which weeks start by default on Sunday. **See Also** From 53c089722cd3c2eb69a64d61e23f01e6fa933077 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 6 May 2024 14:04:34 +0000 Subject: [PATCH 374/624] Docs: Add a note about the naming of the inverted index in earlier versions --- .../engines/table-engines/mergetree-family/invertedindexes.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/engines/table-engines/mergetree-family/invertedindexes.md b/docs/en/engines/table-engines/mergetree-family/invertedindexes.md index 9374f6a3ac1..832ea1e3793 100644 --- a/docs/en/engines/table-engines/mergetree-family/invertedindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/invertedindexes.md @@ -53,6 +53,10 @@ ENGINE = MergeTree ORDER BY key ``` +:::note +In earlier versions of ClickHouse, the corresponding index type name was `inverted`. +::: + where `N` specifies the tokenizer: - `full_text(0)` (or shorter: `full_text()`) set the tokenizer to "tokens", i.e. split strings along spaces, From 942bd6af6107a9d1d66c2504f33afaa6f6a0f412 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Mon, 6 May 2024 16:15:32 +0200 Subject: [PATCH 375/624] Explain updating `result_details.timezone` --- src/Interpreters/executeQuery.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index cd1953f4b09..1923e8ae78a 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1525,6 +1525,8 @@ void executeQuery( throw; } + /// 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(); auto & pipeline = streams.pipeline; From b64ad9ac287fd17dd832f0ce5a75dad7e9bfe5ff Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 2 May 2024 07:20:38 +0000 Subject: [PATCH 376/624] Update s390x build docs --- docs/en/development/build-cross-s390x.md | 24 ++++++++++-------------- 1 file changed, 10 insertions(+), 14 deletions(-) diff --git a/docs/en/development/build-cross-s390x.md b/docs/en/development/build-cross-s390x.md index 3c1667350e1..4c111a76d82 100644 --- a/docs/en/development/build-cross-s390x.md +++ b/docs/en/development/build-cross-s390x.md @@ -5,22 +5,13 @@ title: How to Build, Run and Debug ClickHouse on Linux for s390x (zLinux) sidebar_label: Build on Linux for s390x (zLinux) --- -As of writing (2023/3/10) building for s390x considered to be experimental. Not all features can be enabled, has broken features and is currently under active development. +At the time of writing (2024 May), support for the s390x platform is considered experimental, i.e. some features are disabled or broken on s390x. +## Building ClickHouse for s390x -## Building - -s390x has two OpenSSL-related build options. -- By default, the s390x build will dynamically link to OpenSSL libraries. It will build OpenSSL shared objects, so it's not necessary to install OpenSSL beforehand. (This option is recommended in all cases.) -- Another option is to build OpenSSL in-tree. In this case two build flags need to be supplied to cmake -```bash --DENABLE_OPENSSL_DYNAMIC=0 -``` - -:::note -s390x builds are temporarily disabled in CI. -::: - +s390x has two OpenSSL-related build options: +- By default, OpenSSL is build on s390x as a shared library. This is different from all other platforms, where OpenSSL is build as static library. +- To build OpenSSL as a static library regardless, pass `-DENABLE_OPENSSL_DYNAMIC=0` to CMake. These instructions assume that the host machine is x86_64 and has all the tooling required to build natively based on the [build instructions](../development/build.md). It also assumes that the host is Ubuntu 22.04 but the following instructions should also work on Ubuntu 20.04. @@ -31,11 +22,16 @@ apt-get install binutils-s390x-linux-gnu libc6-dev-s390x-cross gcc-s390x-linux-g ``` If you wish to cross compile rust code install the rust cross compile target for s390x: + ```bash rustup target add s390x-unknown-linux-gnu ``` +The s390x build uses the mold linker, download it from https://github.com/rui314/mold/releases/download/v2.0.0/mold-2.0.0-x86_64-linux.tar.gz +and place it into your `$PATH`. + To build for s390x: + ```bash cmake -DCMAKE_TOOLCHAIN_FILE=cmake/linux/toolchain-s390x.cmake .. ninja From b00c64fe9db8fde5317ef2daf57920149765e83a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 6 May 2024 14:18:30 +0000 Subject: [PATCH 377/624] Docs: Remove tuple support from ANN indexes Indexes for approximate nearest neighbourhood (ANN) search (Annoy, USearch) can currently be build on columns of type Array(Float32) or Tuple(Float32[, Float32[, ...]]). In practice, only Arrays are relevant which makes sense as arrays store high-dimensional embeddings consecutively and the additional flexibility of different data types in a tuple is not needed. Therefore, removing support for ANN indexes over tuple columns to simplify the code, tests and docs. --- .../mergetree-family/annindexes.md | 55 +++---------------- 1 file changed, 7 insertions(+), 48 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 78a27d3ff86..5a81313f62e 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -22,9 +22,8 @@ ORDER BY Distance(vectors, Point) LIMIT N ``` -`vectors` contains N-dimensional values of type [Array](../../../sql-reference/data-types/array.md) or -[Tuple](../../../sql-reference/data-types/tuple.md), for example embeddings. Function `Distance` computes the distance between two vectors. -Often, the Euclidean (L2) distance is chosen as distance function but [other +`vectors` contains N-dimensional values of type [Array(Float32)](../../../sql-reference/data-types/array.md), for example embeddings. +Function `Distance` computes the distance between two vectors. Often, the Euclidean (L2) distance is chosen as distance function but [other distance functions](/docs/en/sql-reference/functions/distance-functions.md) are also possible. `Point` is the reference point, e.g. `(0.17, 0.33, ...)`, and `N` limits the number of search results. @@ -47,7 +46,7 @@ of the search space (using clustering, search trees, etc.) which allows to compu # Creating and Using ANN Indexes {#creating_using_ann_indexes} -Syntax to create an ANN index over an [Array](../../../sql-reference/data-types/array.md) column: +Syntax to create an ANN index over an [Array(Float32)](../../../sql-reference/data-types/array.md) column: ```sql CREATE TABLE table_with_ann_index @@ -60,19 +59,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_ann_index -( - `id` Int64, - `vectors` Tuple(Float32[, Float32[, ...]]), - INDEX [ann_index_name] vectors TYPE [ann_index_type]([ann_index_parameters]) [GRANULARITY [N]] -) -ENGINE = MergeTree -ORDER BY id; -``` - ANN indexes are built during column insertion and merge. As a result, `INSERT` and `OPTIMIZE` statements will be slower than for ordinary tables. ANNIndexes are ideally used only with immutable or rarely changed data, respectively when are far more read requests than write requests. @@ -164,7 +150,7 @@ linear surfaces (lines in 2D, planes in 3D etc.). -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 378/624] 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 379/624] 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 380/624] 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 381/624] 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 382/624] 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 383/624] 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 384/624] 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 385/624] 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 386/624] 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 387/624] 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 388/624] 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 389/624] 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 390/624] 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 391/624] 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 1b7b4fc858fe7918d274691d233b69df701654fa Mon Sep 17 00:00:00 2001 From: Maxim Alexeev Date: Mon, 6 May 2024 22:08:13 +0300 Subject: [PATCH 392/624] removed unnecessary setting --- src/Core/Settings.h | 1 - src/Interpreters/HashJoin.cpp | 12 ++++++------ src/Interpreters/TableJoin.cpp | 1 - src/Interpreters/TableJoin.h | 3 --- 4 files changed, 6 insertions(+), 11 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 5dc80912ebb..7ed175d38f4 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -463,7 +463,6 @@ class IColumn; 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) \ M(UInt64, join_on_disk_max_files_to_merge, 64, "For MergeJoin on disk set how much files it's allowed to sort simultaneously. Then this value bigger then more memory used and then less disk I/O needed. Minimum is 2.", 0) \ M(UInt64, max_rows_in_set_to_optimize_join, 0, "Maximal size of the set to filter joined tables by each other row sets before joining. 0 - disable.", 0) \ - M(UInt64, cross_join_max_bytes_inmemory, 1000000, "Maximal bytes to complete cross join inmemory.", 0) \ \ M(Bool, compatibility_ignore_collation_in_create_table, true, "Compatibility ignore collation in create table", 0) \ \ diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index e194df4e660..52c804c261a 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -2309,6 +2309,12 @@ void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) } }; + for (const Block & compressed_block_right : data->blocks) + { + auto block_right = compressed_block_right.decompress(); + process_right_block(block_right); + } + if (tmp_stream) { tmp_stream->finishWritingAsyncSafe(); @@ -2319,12 +2325,6 @@ void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) } } - for (const Block & compressed_block_right : data->blocks) - { - auto block_right = compressed_block_right.decompress(); - process_right_block(block_right); - } - start_right_block = 0; if (rows_added > max_joined_block_rows) diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 7ceb90704f3..1ee8ca14b2f 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -111,7 +111,6 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_, Temporary , 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) - , cross_join_max_bytes_inmemory(settings.cross_join_max_bytes_inmemory) , partial_merge_join_rows_in_right_blocks(settings.partial_merge_join_rows_in_right_blocks) , partial_merge_join_left_table_buffer_bytes(settings.partial_merge_join_left_table_buffer_bytes) , max_files_to_merge(settings.join_on_disk_max_files_to_merge) diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index d6920d6afbf..8e83233e54c 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -145,7 +145,6 @@ private: const UInt64 cross_join_min_bytes_to_compress = 10000; const size_t max_joined_block_rows = 0; std::vector join_algorithm; - const UInt64 cross_join_max_bytes_inmemory = 1000000; const size_t partial_merge_join_rows_in_right_blocks = 0; const size_t partial_merge_join_left_table_buffer_bytes = 0; const size_t max_files_to_merge = 0; @@ -281,8 +280,6 @@ public: bool allowParallelHashJoin() const; - UInt64 crossJoinMaxBytesInmemory() const { return cross_join_max_bytes_inmemory; } - bool joinUseNulls() const { return join_use_nulls; } UInt64 crossJoinMinRowsToCompress() const { return cross_join_min_rows_to_compress; } 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 393/624] 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 1660a4fe867d3d8d5e2fb0167c689f936226cfde Mon Sep 17 00:00:00 2001 From: Maxim Alexeev Date: Mon, 6 May 2024 23:22:41 +0300 Subject: [PATCH 394/624] first working version --- src/Interpreters/HashJoin.cpp | 24 +++++++++++++++++++----- 1 file changed, 19 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 52c804c261a..68186c00663 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -64,6 +64,7 @@ struct NotProcessedCrossJoin : public ExtraBlock { size_t left_position; size_t right_block; + std::unique_ptr reader; }; @@ -256,7 +257,6 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s , instance_log_id(!instance_id_.empty() ? "(" + instance_id_ + ") " : "") , log(getLogger("HashJoin")) { - LOG_INFO(log, "KEK CONSTRUCTOR {}\n", reserve_num); LOG_TRACE(log, "{}Keys: {}, datatype: {}, kind: {}, strictness: {}, right header: {}", instance_log_id, TableJoin::formatClauses(table_join->getClauses(), true), data->type, kind, strictness, right_sample_block.dumpStructure()); @@ -830,14 +830,23 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) if (shrink_blocks) block_to_save = block_to_save.shrinkToFit(); + size_t max_bytes_in_join = table_join->sizeLimits().max_bytes; + size_t max_rows_in_join = table_join->sizeLimits().max_rows; + if (!table_join->sizeLimits().hasLimits()) + { + max_bytes_in_join = table_join->defaultMaxBytes(); + } - if (kind == JoinKind::Cross) + if (kind == JoinKind::Cross + && (tmp_stream || (max_bytes_in_join && getTotalByteCount() + block_to_save.allocatedBytes() >= max_bytes_in_join) + || (max_rows_in_join && getTotalRowCount() + block_to_save.rows() >= max_rows_in_join))) { if (tmp_stream == nullptr) { tmp_stream = &tmp_data->createStream(right_sample_block); } tmp_stream->write(block_to_save); + return true; } size_t total_rows = 0; @@ -2251,11 +2260,13 @@ void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) { size_t start_left_row = 0; size_t start_right_block = 0; + std::unique_ptr reader = nullptr; if (not_processed) { auto & continuation = static_cast(*not_processed); start_left_row = continuation.left_position; start_right_block = continuation.right_block; + reader = std::move(continuation.reader); not_processed.reset(); } @@ -2317,8 +2328,11 @@ void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) if (tmp_stream) { - tmp_stream->finishWritingAsyncSafe(); - auto reader = tmp_stream->getReadStream(); + if (reader == nullptr) + { + tmp_stream->finishWritingAsyncSafe(); + reader = tmp_stream->getReadStream(); + } while (auto block_right = reader->read()) { process_right_block(block_right); @@ -2330,7 +2344,7 @@ void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) if (rows_added > max_joined_block_rows) { not_processed = std::make_shared( - NotProcessedCrossJoin{{block.cloneEmpty()}, left_row, block_number + 1}); + NotProcessedCrossJoin{{block.cloneEmpty()}, left_row, block_number + 1, std::move(reader)}); not_processed->block.swap(block); break; } From 3dd04e4e58585bd467fc3549200b9759886f8a88 Mon Sep 17 00:00:00 2001 From: Maxim Alexeev Date: Mon, 6 May 2024 23:27:28 +0300 Subject: [PATCH 395/624] removed garbage --- src/Interpreters/GraceHashJoin.cpp | 1 - src/Interpreters/HashJoin.h | 9 --------- 2 files changed, 10 deletions(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index ba51953e979..53d1f48c291 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -657,7 +657,6 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() GraceHashJoin::InMemoryJoinPtr GraceHashJoin::makeInMemoryJoin(const String & bucket_id, size_t reserve_num) { - LOG_INFO(log, "GreaceHashJoin\n"); return std::make_unique(table_join, right_sample_block, any_take_last_row, reserve_num, bucket_id); } diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 1eb4d0f8030..04a04b74dd0 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -415,15 +415,6 @@ public: void setMaxJoinedBlockRows(size_t value) { max_joined_block_rows = value; } - TemporaryFileStream* getStreamForCrossJoin() - { - auto streams = tmp_data->getStreams(); - assert(streams.size() <= 1); - if (streams.empty()) - return nullptr; - return streams[0]; - } - private: friend class NotJoinedHash; From d4a0325350673a89e874d4f3a5a1500044587ced Mon Sep 17 00:00:00 2001 From: Maxim Alexeev Date: Mon, 6 May 2024 23:30:27 +0300 Subject: [PATCH 396/624] added check that tmp_data is not nullptr --- src/Interpreters/HashJoin.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 68186c00663..f4408271fbf 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -837,7 +837,7 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) max_bytes_in_join = table_join->defaultMaxBytes(); } - if (kind == JoinKind::Cross + if (kind == JoinKind::Cross && tmp_data && (tmp_stream || (max_bytes_in_join && getTotalByteCount() + block_to_save.allocatedBytes() >= max_bytes_in_join) || (max_rows_in_join && getTotalRowCount() + block_to_save.rows() >= max_rows_in_join))) { From ac7e6fc69136153c8c99e1d7f76c884fb500815a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 6 May 2024 23:00:05 +0200 Subject: [PATCH 397/624] 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 d531e2006d9326bce2877efda9823aa880e4c8cb Mon Sep 17 00:00:00 2001 From: Maxim Alexeev Date: Tue, 7 May 2024 00:10:24 +0300 Subject: [PATCH 398/624] Deleted check of read type and added comment --- src/Interpreters/TemporaryDataOnDisk.cpp | 12 ------------ src/Interpreters/TemporaryDataOnDisk.h | 5 +---- 2 files changed, 1 insertion(+), 16 deletions(-) diff --git a/src/Interpreters/TemporaryDataOnDisk.cpp b/src/Interpreters/TemporaryDataOnDisk.cpp index e50d501f6d3..9bd0fa9e8ed 100644 --- a/src/Interpreters/TemporaryDataOnDisk.cpp +++ b/src/Interpreters/TemporaryDataOnDisk.cpp @@ -331,12 +331,6 @@ Block TemporaryFileStream::read() if (isEof()) return {}; - if (auto type = read_type.exchange(1); type == 2) - { - read_type.store(2); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Different type of reading was requested earlier"); - } - if (!in_reader) { in_reader = std::make_unique(getPath(), header, getSize()); @@ -360,12 +354,6 @@ std::unique_ptr TemporaryFileStream::getReadStream() if (isEof()) return nullptr; - if (auto type = read_type.exchange(2); type == 1) - { - read_type.store(1); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Different type of reading was requested earlier"); - } - return std::make_unique(getPath(), header, getSize()); } diff --git a/src/Interpreters/TemporaryDataOnDisk.h b/src/Interpreters/TemporaryDataOnDisk.h index 792988e94bd..a5141dd373e 100644 --- a/src/Interpreters/TemporaryDataOnDisk.h +++ b/src/Interpreters/TemporaryDataOnDisk.h @@ -149,7 +149,7 @@ struct InputReader /* * Data can be written into this stream and then read. - * After finish writing, call `finishWriting` and then `read` to read the data. + * After finish writing, call `finishWriting` and then either call `read` or 'getReadStream'(only one of the two) to read the data. * Account amount of data written to disk in parent scope. */ class TemporaryFileStream : boost::noncopyable @@ -204,9 +204,6 @@ private: Stat stat; - /// 0 - means that we haven't requested any read, 1 - read from function TemporaryFileStream::read, 2 - - std::atomic_char read_type{0}; - mutable std::mutex finish_writing; std::atomic_bool writing_finished{false}; From 9fcb5d2ebfb3e2105b5d6171a1a53ff75c0023f4 Mon Sep 17 00:00:00 2001 From: Yohann Jardin Date: Mon, 6 May 2024 23:44:27 +0200 Subject: [PATCH 399/624] 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 400/624] 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 401/624] 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 402/624] 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 403/624] 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 404/624] 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 405/624] 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 406/624] 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 407/624] 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 408/624] 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 409/624] 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 410/624] 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 411/624] [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 412/624] [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 413/624] 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 414/624] 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 415/624] 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 416/624] 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 417/624] 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 418/624] 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 419/624] 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 420/624] 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 421/624] 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 422/624] 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 423/624] 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 424/624] 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 425/624] 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 426/624] 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 427/624] 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 428/624] 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 429/624] 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 430/624] 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 fe18781bed4c26347cd7746dfc3be75745941eab Mon Sep 17 00:00:00 2001 From: skyoct Date: Tue, 7 May 2024 18:41:37 +0800 Subject: [PATCH 431/624] batter --- .../02415_all_new_functions_must_be_documented.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 3ddf165dec0..a152066a460 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -201,6 +201,7 @@ cbrt ceil char cityHash64 +clamp coalesce concat concatAssumeInjective From 1bae2d9d4ffa6b4757dc2aeccb9eccf89bebc072 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 7 May 2024 12:57:14 +0200 Subject: [PATCH 432/624] 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 433/624] 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 434/624] [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 435/624] [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 436/624] 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 437/624] 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 438/624] 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 d3155707e9fdf56671af19dd38388861468378a1 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 6 May 2024 12:09:12 +0000 Subject: [PATCH 439/624] no filter pushdow for group_by_use_nulls --- src/Processors/QueryPlan/AggregatingStep.h | 1 + src/Processors/QueryPlan/Optimizations/filterPushDown.cpp | 3 +++ 2 files changed, 4 insertions(+) diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index f446ecec846..ae43295024a 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -59,6 +59,7 @@ public: const Aggregator::Params & getParams() const { return params; } const auto & getGroupingSetsParamsList() const { return grouping_sets_params; } + bool isGroupByUseNulls() const { return group_by_use_nulls; } bool inOrder() const { return !sort_description_for_merging.empty(); } bool explicitSortingRequired() const { return explicit_sorting_required_for_aggregation_in_order; } diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 7dd526cbe95..5b3bcfc4468 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -428,6 +428,9 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes /// of the grouping sets, we could not push the filter down. if (aggregating->isGroupingSets()) { + /// Cannot push down filter if type has been changed. + if (aggregating->isGroupByUseNulls()) + return 0; const auto & actions = filter->getExpression(); const auto & filter_node = actions->findInOutputs(filter->getFilterColumnName()); From fd75522ea8c8f5ccd0dd6251ea5bb5480823f9c4 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 6 May 2024 12:09:47 +0000 Subject: [PATCH 440/624] analyzer: fix select * with grouping sets --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 52efee03ae4..5fd49aa4541 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -4815,6 +4815,19 @@ ProjectionNames QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, I } } + if (!scope.expressions_in_resolve_process_stack.hasAggregateFunction()) + { + for (auto & [node, _] : matched_expression_nodes_with_names) + { + auto it = scope.nullable_group_by_keys.find(node); + if (it != scope.nullable_group_by_keys.end()) + { + node = it->node->clone(); + node->convertToNullable(); + } + } + } + std::unordered_map> strict_transformer_to_used_column_names; for (const auto & transformer : matcher_node_typed.getColumnTransformers().getNodes()) { From 063db335480331bf0003f220ab74d93797a8ab17 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 6 May 2024 12:10:11 +0000 Subject: [PATCH 441/624] verbose error --- src/Functions/FunctionHelpers.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionHelpers.cpp b/src/Functions/FunctionHelpers.cpp index ce83a489a3d..b31127d3896 100644 --- a/src/Functions/FunctionHelpers.cpp +++ b/src/Functions/FunctionHelpers.cpp @@ -80,7 +80,7 @@ ColumnWithTypeAndName columnGetNested(const ColumnWithTypeAndName & col) return ColumnWithTypeAndName{ nullable_res, nested_type, col.name }; } else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column for DataTypeNullable"); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} for DataTypeNullable", col.dumpStructure()); } return col; } From cf9bc27c0b80c17b887170c4182439de1c6e8643 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 6 May 2024 12:12:11 +0000 Subject: [PATCH 442/624] add test --- ...grouping_sets_use_nulls_pushdown.reference | 8 +++++ ...03150_grouping_sets_use_nulls_pushdown.sql | 29 +++++++++++++++++++ 2 files changed, 37 insertions(+) create mode 100644 tests/queries/0_stateless/03150_grouping_sets_use_nulls_pushdown.reference create mode 100644 tests/queries/0_stateless/03150_grouping_sets_use_nulls_pushdown.sql diff --git a/tests/queries/0_stateless/03150_grouping_sets_use_nulls_pushdown.reference b/tests/queries/0_stateless/03150_grouping_sets_use_nulls_pushdown.reference new file mode 100644 index 00000000000..e9f85e9052f --- /dev/null +++ b/tests/queries/0_stateless/03150_grouping_sets_use_nulls_pushdown.reference @@ -0,0 +1,8 @@ +2023-01-05 hello, world +2023-01-05 \N +2023-01-05 hello, world +2023-01-05 \N +2023-01-05 10 +2023-01-05 10 +2023-01-05 hello, world +2023-01-05 \N diff --git a/tests/queries/0_stateless/03150_grouping_sets_use_nulls_pushdown.sql b/tests/queries/0_stateless/03150_grouping_sets_use_nulls_pushdown.sql new file mode 100644 index 00000000000..0f057a1c99d --- /dev/null +++ b/tests/queries/0_stateless/03150_grouping_sets_use_nulls_pushdown.sql @@ -0,0 +1,29 @@ +DROP TABLE IF EXISTS test_grouping_sets_predicate; + +CREATE TABLE test_grouping_sets_predicate ( day_ Date, type_1 String ) ENGINE=MergeTree ORDER BY day_; + +INSERT INTO test_grouping_sets_predicate SELECT toDate('2023-01-05') AS day_, 'hello, world' FROM numbers (10); + +SET group_by_use_nulls = true; + +SELECT * +FROM ( SELECT day_, type_1 FROM test_grouping_sets_predicate GROUP BY GROUPING SETS ( (day_, type_1), (day_) ) ) +WHERE day_ = '2023-01-05' +ORDER BY ALL; + + +SELECT * +FROM ( SELECT * FROM test_grouping_sets_predicate GROUP BY GROUPING SETS ( (day_, type_1), (day_) ) ) +WHERE day_ = '2023-01-05' +ORDER BY ALL; + +SELECT * +FROM ( SELECT day_, COUNT(*) FROM test_grouping_sets_predicate GROUP BY GROUPING SETS ( (day_, type_1), (day_) ) ) +WHERE day_ = '2023-01-05' +ORDER BY ALL; + + +SELECT t2.* +FROM ( SELECT t1.* FROM test_grouping_sets_predicate t1 GROUP BY GROUPING SETS ( (day_, type_1), (day_) ) ) t2 +WHERE day_ = '2023-01-05' +ORDER BY ALL; From cad9c97725e4943730d0dabaa3df2cdf008be948 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 7 May 2024 17:21:25 +0300 Subject: [PATCH 443/624] 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 c67a43e3a7354c3db488df44c05e5a0d56a454c5 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 7 May 2024 14:37:39 +0000 Subject: [PATCH 444/624] better --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 3 +++ src/Analyzer/ValidationUtils.cpp | 4 ++++ ..._grouping_sets_use_nulls_pushdown.reference | 6 ++++++ .../03150_grouping_sets_use_nulls_pushdown.sql | 18 ++++++++++++++++++ .../0_stateless/03151_where_asterisk.reference | 1 + .../0_stateless/03151_where_asterisk.sql | 3 +++ 6 files changed, 35 insertions(+) create mode 100644 tests/queries/0_stateless/03151_where_asterisk.reference create mode 100644 tests/queries/0_stateless/03151_where_asterisk.sql diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 5fd49aa4541..518c1272761 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -5021,7 +5021,10 @@ ProjectionNames QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, I scope.scope_node->formatASTForErrorMessage()); } + auto original_ast = matcher_node->getOriginalAST(); matcher_node = std::move(list); + if (original_ast) + matcher_node->setOriginalAST(original_ast); return result_projection_names; } diff --git a/src/Analyzer/ValidationUtils.cpp b/src/Analyzer/ValidationUtils.cpp index 60cc1dd521f..45a916016ce 100644 --- a/src/Analyzer/ValidationUtils.cpp +++ b/src/Analyzer/ValidationUtils.cpp @@ -26,6 +26,10 @@ namespace void validateFilter(const QueryTreeNodePtr & filter_node, std::string_view exception_place_message, const QueryTreeNodePtr & query_node) { + if (filter_node->getNodeType() == QueryTreeNodeType::LIST) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Unsupported expression '{}' in filter", filter_node->formatASTForErrorMessage()); + auto filter_node_result_type = filter_node->getResultType(); if (!filter_node_result_type->canBeUsedInBooleanContext()) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, diff --git a/tests/queries/0_stateless/03150_grouping_sets_use_nulls_pushdown.reference b/tests/queries/0_stateless/03150_grouping_sets_use_nulls_pushdown.reference index e9f85e9052f..209c455b6f5 100644 --- a/tests/queries/0_stateless/03150_grouping_sets_use_nulls_pushdown.reference +++ b/tests/queries/0_stateless/03150_grouping_sets_use_nulls_pushdown.reference @@ -2,6 +2,12 @@ 2023-01-05 \N 2023-01-05 hello, world 2023-01-05 \N +2023-01-05 +2023-01-05 +2023-01-05 hello, world +2023-01-05 \N +2023-01-05 hello, world +2023-01-05 \N 2023-01-05 10 2023-01-05 10 2023-01-05 hello, world diff --git a/tests/queries/0_stateless/03150_grouping_sets_use_nulls_pushdown.sql b/tests/queries/0_stateless/03150_grouping_sets_use_nulls_pushdown.sql index 0f057a1c99d..6162f55ac8f 100644 --- a/tests/queries/0_stateless/03150_grouping_sets_use_nulls_pushdown.sql +++ b/tests/queries/0_stateless/03150_grouping_sets_use_nulls_pushdown.sql @@ -17,6 +17,24 @@ FROM ( SELECT * FROM test_grouping_sets_predicate GROUP BY GROUPING SETS ( (day_ WHERE day_ = '2023-01-05' ORDER BY ALL; +SELECT * +FROM ( SELECT day_ FROM test_grouping_sets_predicate GROUP BY GROUPING SETS ( (day_, type_1), (day_) ) ) +WHERE day_ = '2023-01-05' +ORDER BY * +SETTINGS allow_experimental_analyzer=1; + +SELECT * +FROM ( SELECT * FROM test_grouping_sets_predicate GROUP BY GROUPING SETS ( (day_, type_1), (day_) ) ) +WHERE day_ = '2023-01-05' +GROUP BY * +SETTINGS allow_experimental_analyzer=1; + +SELECT * +FROM ( SELECT * FROM test_grouping_sets_predicate GROUP BY GROUPING SETS ( (*), (day_) ) ) +WHERE day_ = '2023-01-05' +GROUP BY GROUPING SETS (*) +SETTINGS allow_experimental_analyzer=1; + SELECT * FROM ( SELECT day_, COUNT(*) FROM test_grouping_sets_predicate GROUP BY GROUPING SETS ( (day_, type_1), (day_) ) ) WHERE day_ = '2023-01-05' diff --git a/tests/queries/0_stateless/03151_where_asterisk.reference b/tests/queries/0_stateless/03151_where_asterisk.reference new file mode 100644 index 00000000000..8b137891791 --- /dev/null +++ b/tests/queries/0_stateless/03151_where_asterisk.reference @@ -0,0 +1 @@ + diff --git a/tests/queries/0_stateless/03151_where_asterisk.sql b/tests/queries/0_stateless/03151_where_asterisk.sql new file mode 100644 index 00000000000..4b01d6e165c --- /dev/null +++ b/tests/queries/0_stateless/03151_where_asterisk.sql @@ -0,0 +1,3 @@ +SET allow_experimental_analyzer = 1; + +SELECT * FROM (SELECT 1) t1 WHERE *; -- { serverError BAD_ARGUMENTS } From c45c1444355b740ed01fe1849f18f17c689d826b Mon Sep 17 00:00:00 2001 From: Yohann Jardin Date: Tue, 7 May 2024 16:39:43 +0200 Subject: [PATCH 445/624] 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 446/624] 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 447/624] 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 448/624] 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 449/624] 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 450/624] 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 451/624] 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 452/624] 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 453/624] 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 454/624] 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 455/624] 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 456/624] 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 457/624] 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 134b468d917fd8b243b5b39e15872057acbc3015 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 7 May 2024 18:14:32 +0000 Subject: [PATCH 458/624] Another case. --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 14 +++++++++----- ...3_group_by_use_nulls_analyzer_crashes.reference | 10 ++++++++++ .../03023_group_by_use_nulls_analyzer_crashes.sql | 2 ++ 3 files changed, 21 insertions(+), 5 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 80294b7a428..426f3b6bde8 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -4120,8 +4120,8 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLook * SELECT id FROM ( SELECT ... ) AS subquery ARRAY JOIN [0] AS id INNER JOIN second_table USING (id) * In the example, identifier `id` should be resolved into one from USING (id) column. */ - auto alias_it = scope.alias_name_to_expression_node.find(identifier_lookup.identifier.getFullName()); - if (alias_it != scope.alias_name_to_expression_node.end() && alias_it->second->getNodeType() == QueryTreeNodeType::COLUMN) + auto alias_it = scope.alias_name_to_expression_node->find(identifier_lookup.identifier.getFullName()); + if (alias_it != scope.alias_name_to_expression_node->end() && alias_it->second->getNodeType() == QueryTreeNodeType::COLUMN) { const auto & column_node = alias_it->second->as(); if (column_node.getColumnSource()->getNodeType() == QueryTreeNodeType::ARRAY_JOIN) @@ -5225,8 +5225,12 @@ ProjectionNames QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_nod for (size_t i = 0; i < lambda_arguments_nodes_size; ++i) { auto & lambda_argument_node = lambda_arguments_nodes[i]; - auto & lambda_argument_node_typed = lambda_argument_node->as(); - const auto & lambda_argument_name = lambda_argument_node_typed.getIdentifier().getFullName(); + const auto * lambda_argument_identifier = lambda_argument_node->as(); + const auto * lambda_argument_column = lambda_argument_node->as(); + if (!lambda_argument_identifier && !lambda_argument_column) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected IDENTIFIER or COLUMN as lambda argument, got {}", lambda_node->dumpTree()); + const auto & lambda_argument_name = lambda_argument_identifier ? lambda_argument_identifier->getIdentifier().getFullName() + : lambda_argument_column->getColumnName(); bool has_expression_node = scope.alias_name_to_expression_node->contains(lambda_argument_name); bool has_alias_node = scope.alias_name_to_lambda_node.contains(lambda_argument_name); @@ -5236,7 +5240,7 @@ ProjectionNames QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_nod throw Exception(ErrorCodes::BAD_ARGUMENTS, "Alias name '{}' inside lambda {} cannot have same name as lambda argument. In scope {}", lambda_argument_name, - lambda_argument_node_typed.formatASTForErrorMessage(), + lambda_argument_node->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); } diff --git a/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.reference b/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.reference index 0eb9d94e85a..273e1567a9b 100644 --- a/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.reference +++ b/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.reference @@ -86,3 +86,13 @@ a a \N 0 nan \N 4 nan \N \N nan +[] +['.'] +['.','.'] +['.','.','.'] +['.','.','.','.'] +['.','.','.','.','.'] +['.','.','.','.','.','.'] +['.','.','.','.','.','.','.'] +['.','.','.','.','.','.','.','.'] +['.','.','.','.','.','.','.','.','.'] diff --git a/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.sql b/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.sql index 7311ce54e39..bba0a27560f 100644 --- a/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.sql +++ b/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.sql @@ -43,3 +43,5 @@ GROUP BY number + number AS b WITH CUBE SETTINGS group_by_use_nulls = 1; + +SELECT arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{2,3}', numbers(10)) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_use_nulls=1; From 95b76bf6a47f0e23d41ce33c2223cee93066ad3e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 7 May 2024 20:58:19 +0200 Subject: [PATCH 459/624] 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 460/624] 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 461/624] 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