From fb17d0d16939434be0990b18cfdd1fecc6227bb1 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 23 Jan 2024 11:31:12 +0000 Subject: [PATCH 001/192] 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 002/192] 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 003/192] 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 004/192] 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 005/192] 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 006/192] 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 2cd0d51b6947f0b7c4f62a8626ed164417cfe8a3 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 30 Jan 2024 02:22:07 +0000 Subject: [PATCH 007/192] 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 fce0cca924ded39196d338bd57416340ac75aa84 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 30 Jan 2024 08:37:41 +0000 Subject: [PATCH 008/192] 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 009/192] 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 010/192] 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 011/192] 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 bb55a0ca50fe2a93829e249df1f5f342ed11b4d2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 11 Mar 2024 19:18:15 +0100 Subject: [PATCH 012/192] 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 b0131a278bd13001f1d54413b4bc781eab8882a6 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 30 Jan 2024 21:38:07 +0100 Subject: [PATCH 013/192] impl --- .../MergeTree/MergeTreeDataWriter.cpp | 27 +++++++------------ src/Storages/MergeTree/MergeTreeDataWriter.h | 6 +++-- 2 files changed, 13 insertions(+), 20 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index cadd94867ec..ce11a535e1c 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -618,7 +618,8 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( if (projection_block.rows()) { - auto proj_temp_part = writeProjectionPart(data, log, projection_block, projection, new_data_part.get()); + auto proj_temp_part + = writeProjectionPart(data, log, projection_block, projection, new_data_part.get(), /*merge_is_needed=*/false); new_data_part->addProjectionPart(projection.name, std::move(proj_temp_part.part)); for (auto & stream : proj_temp_part.streams) temp_part.streams.emplace_back(std::move(stream)); @@ -647,7 +648,8 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( const MergeTreeData & data, LoggerPtr log, Block block, - const ProjectionDescription & projection) + const ProjectionDescription & projection, + bool merge_is_needed) { TemporaryPart temp_part; const auto & metadata_snapshot = projection.metadata; @@ -716,7 +718,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( ProfileEvents::increment(ProfileEvents::MergeTreeDataProjectionWriterBlocksAlreadySorted); } - if (projection.type == ProjectionDescription::Type::Aggregate) + if (projection.type == ProjectionDescription::Type::Aggregate && merge_is_needed) { ProfileEventTimeIncrement watch(ProfileEvents::MergeTreeDataProjectionWriterMergingBlocksMicroseconds); @@ -756,16 +758,11 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPart( LoggerPtr log, Block block, const ProjectionDescription & projection, - IMergeTreeDataPart * parent_part) + IMergeTreeDataPart * parent_part, + bool merge_is_needed) { return writeProjectionPartImpl( - projection.name, - false /* is_temp */, - parent_part, - data, - log, - std::move(block), - projection); + projection.name, false /* is_temp */, parent_part, data, log, std::move(block), projection, merge_is_needed); } /// This is used for projection materialization process which may contain multiple stages of @@ -780,13 +777,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempProjectionPart( { auto part_name = fmt::format("{}_{}", projection.name, block_num); return writeProjectionPartImpl( - part_name, - true /* is_temp */, - parent_part, - data, - log, - std::move(block), - projection); + part_name, true /* is_temp */, parent_part, data, log, std::move(block), projection, /*merge_is_needed=*/true); } } diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.h b/src/Storages/MergeTree/MergeTreeDataWriter.h index 9dffea0a471..863c951d957 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.h +++ b/src/Storages/MergeTree/MergeTreeDataWriter.h @@ -95,7 +95,8 @@ public: LoggerPtr log, Block block, const ProjectionDescription & projection, - IMergeTreeDataPart * parent_part); + IMergeTreeDataPart * parent_part, + bool merge_is_needed); /// For mutation: MATERIALIZE PROJECTION. static TemporaryPart writeTempProjectionPart( @@ -129,7 +130,8 @@ private: const MergeTreeData & data, LoggerPtr log, Block block, - const ProjectionDescription & projection); + const ProjectionDescription & projection, + bool merge_is_needed); MergeTreeData & data; LoggerPtr log; From 02ff01f2468b36479bd40abe23138bb28a4071b6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 19 Mar 2024 16:48:42 +0100 Subject: [PATCH 014/192] 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 255e4b4bda442968197148051ac507266f1708ac Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 25 Mar 2024 14:22:39 +0000 Subject: [PATCH 015/192] fix test --- .../queries/0_stateless/02982_perf_introspection_for_inserts.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02982_perf_introspection_for_inserts.sh b/tests/queries/0_stateless/02982_perf_introspection_for_inserts.sh index f5fb54b54d3..409bd996cbd 100755 --- a/tests/queries/0_stateless/02982_perf_introspection_for_inserts.sh +++ b/tests/queries/0_stateless/02982_perf_introspection_for_inserts.sh @@ -36,7 +36,7 @@ FROM numbers_mt(1000000); $CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS" $CLICKHOUSE_CLIENT -q """ SELECT - ProfileEvents['MergeTreeDataProjectionWriterMergingBlocksMicroseconds'] > 0, + ProfileEvents['MergeTreeDataProjectionWriterMergingBlocksMicroseconds'] = 0, ProfileEvents['MergeTreeDataProjectionWriterSortingBlocksMicroseconds'] > 0, ProfileEvents['MergeTreeDataWriterSortingBlocksMicroseconds'] > 0, ProfileEvents['MergeTreeDataWriterProjectionsCalculationMicroseconds'] > 0, From 4bb827e987781c9a754b1c7c5eb1f46acd28a90a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=B1=AA=E8=82=A5=E8=82=A5?= Date: Thu, 28 Mar 2024 14:28:42 +0800 Subject: [PATCH 016/192] fix npy big endianness --- src/Processors/Formats/Impl/NpyRowInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/NpyRowInputFormat.cpp b/src/Processors/Formats/Impl/NpyRowInputFormat.cpp index 795ad12ac98..65e0f9dd192 100644 --- a/src/Processors/Formats/Impl/NpyRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/NpyRowInputFormat.cpp @@ -131,7 +131,7 @@ std::shared_ptr parseType(String type) NumpyDataType::Endianness endianness; if (type[0] == '<') endianness = NumpyDataType::Endianness::LITTLE; - else if (type[1] == '>') + else if (type[0] == '>') endianness = NumpyDataType::Endianness::BIG; else if (type[0] == '|') endianness = NumpyDataType::Endianness::NONE; From fd925770b2de0ea067541a156c796552a94f3d11 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 28 Mar 2024 17:00:54 +0100 Subject: [PATCH 017/192] 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 acd8a1f32e6640db767fb44e675c24cce4ea564c Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 29 Mar 2024 09:15:45 +0000 Subject: [PATCH 018/192] Enable custom parquet encoder by default --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 84e709294aa..f4e34aeea3c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1111,7 +1111,7 @@ class IColumn; M(ParquetVersion, output_format_parquet_version, "2.latest", "Parquet format version for output format. Supported versions: 1.0, 2.4, 2.6 and 2.latest (default)", 0) \ M(ParquetCompression, output_format_parquet_compression_method, "zstd", "Compression method for Parquet output format. Supported codecs: snappy, lz4, brotli, zstd, gzip, none (uncompressed)", 0) \ M(Bool, output_format_parquet_compliant_nested_types, true, "In parquet file schema, use name 'element' instead of 'item' for list elements. This is a historical artifact of Arrow library implementation. Generally increases compatibility, except perhaps with some old versions of Arrow.", 0) \ - M(Bool, output_format_parquet_use_custom_encoder, false, "Use a faster Parquet encoder implementation.", 0) \ + M(Bool, output_format_parquet_use_custom_encoder, true, "Use a faster Parquet encoder implementation.", 0) \ M(Bool, output_format_parquet_parallel_encoding, true, "Do Parquet encoding in multiple threads. Requires output_format_parquet_use_custom_encoder.", 0) \ M(UInt64, output_format_parquet_data_page_size, 1024 * 1024, "Target page size in bytes, before compression.", 0) \ M(UInt64, output_format_parquet_batch_size, 1024, "Check page size every this many rows. Consider decreasing if you have columns with average values size above a few KBs.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 170836cb980..7ea721ec955 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -125,6 +125,7 @@ static std::map sett {"azure_max_upload_part_size", 5ull*1024*1024*1024, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to Azure blob storage."}, {"azure_upload_part_size_multiply_factor", 2, 2, "Multiply azure_min_upload_part_size by this factor each time azure_multiply_parts_count_threshold parts were uploaded from a single write to Azure blob storage."}, {"azure_upload_part_size_multiply_parts_count_threshold", 500, 500, "Each time this number of parts was uploaded to Azure blob storage, azure_min_upload_part_size is multiplied by azure_upload_part_size_multiply_factor."}, + {"output_format_parquet_use_custom_encoder", false, true, "Enable custom Parquet encoder."}, }}, {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, From 1cd73b907c1493729699578231fa223f57de2f35 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 5 Apr 2024 12:59:57 +0200 Subject: [PATCH 019/192] 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 020/192] optimize drop tables --- src/Interpreters/DDLTask.cpp | 9 --------- src/Interpreters/InterpreterDropQuery.cpp | 2 +- src/Parsers/ASTDropQuery.h | 3 ++- src/Parsers/ParserDropQuery.cpp | 11 +++++++++++ src/Parsers/tests/gtest_dictionary_parser.cpp | 12 ++++-------- .../0_stateless/02961_drop_tables.reference | 3 +++ tests/queries/0_stateless/02961_drop_tables.sql | 14 ++++++++++++++ 7 files changed, 35 insertions(+), 19 deletions(-) diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index 37954850851..a37b4db029a 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -16,7 +16,6 @@ #include #include #include -#include namespace DB @@ -201,14 +200,6 @@ void DDLTaskBase::parseQueryFromEntry(ContextPtr context) ParserQuery parser_query(end, settings.allow_settings_after_format_in_insert); String description; query = parseQuery(parser_query, begin, end, description, 0, settings.max_parser_depth, settings.max_parser_backtracks); - if (auto * query_drop = query->as()) - { - ASTs drops = query_drop->getRewrittenASTsOfSingleTable(); - if (drops.size() > 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Not supports drop multiple tables for ddl task."); - - query = drops[0]; - } } void DDLTaskBase::formatRewrittenQuery(ContextPtr context) diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index e29e59ee4c3..0e591a7782a 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -557,7 +557,7 @@ bool InterpreterDropQuery::supportsTransactions() const return drop.cluster.empty() && !drop.temporary && drop.kind == ASTDropQuery::Kind::Truncate - && drop.database_and_tables; + && drop.table; } void registerInterpreterDropQuery(InterpreterFactory & factory) diff --git a/src/Parsers/ASTDropQuery.h b/src/Parsers/ASTDropQuery.h index e0e908733e5..7e25e990bc8 100644 --- a/src/Parsers/ASTDropQuery.h +++ b/src/Parsers/ASTDropQuery.h @@ -40,7 +40,7 @@ public: // We detach the object permanently, so it will not be reattached back during server restart. bool permanently{false}; - /// Example: Drop TABLE t1, t2, t3... + /// Used to drop multiple tables only, example: Drop TABLE t1, t2, t3... ASTPtr database_and_tables; /** Get the text that identifies this element. */ @@ -52,6 +52,7 @@ public: return removeOnCluster(clone(), params.default_database); } + /** Convert an AST that deletes multiple tables into multiple ASTs that delete a single table. */ ASTs getRewrittenASTsOfSingleTable(); QueryKind getQueryKind() const override { return QueryKind::Drop; } diff --git a/src/Parsers/ParserDropQuery.cpp b/src/Parsers/ParserDropQuery.cpp index 09f15e9649f..9fe8306c0c2 100644 --- a/src/Parsers/ParserDropQuery.cpp +++ b/src/Parsers/ParserDropQuery.cpp @@ -7,6 +7,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int SYNTAX_ERROR; +} + namespace { @@ -84,6 +89,9 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, cons if (!tables_p.parse(pos, database_and_tables, expected)) return false; + + if (database_and_tables->size() > 1 && kind != ASTDropQuery::Kind::Drop) + throw Exception(ErrorCodes::SYNTAX_ERROR, "Only Support DROP multiple tables currently"); } /// common for tables / dictionaries / databases @@ -123,6 +131,9 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, cons query->cluster = cluster_str; + if (database_and_tables && database_and_tables->size() == 1) + node = query->getRewrittenASTsOfSingleTable()[0]; + return true; } diff --git a/src/Parsers/tests/gtest_dictionary_parser.cpp b/src/Parsers/tests/gtest_dictionary_parser.cpp index 44205975cdc..a1ba46125a7 100644 --- a/src/Parsers/tests/gtest_dictionary_parser.cpp +++ b/src/Parsers/tests/gtest_dictionary_parser.cpp @@ -301,10 +301,8 @@ TEST(ParserDictionaryDDL, ParseDropQuery) ASTDropQuery * drop1 = ast1->as(); EXPECT_TRUE(drop1->is_dictionary); - auto & database_and_tables1 = drop1->database_and_tables->as(); - auto identifier1 = dynamic_pointer_cast(database_and_tables1.children[0]); - EXPECT_EQ(identifier1->getDatabaseName(), "test"); - EXPECT_EQ(identifier1->shortName(), "dict1"); + EXPECT_EQ(drop1->getDatabase(), "test"); + EXPECT_EQ(drop1->getTable(), "dict1"); auto str1 = serializeAST(*drop1); EXPECT_EQ(input1, str1); @@ -314,10 +312,8 @@ TEST(ParserDictionaryDDL, ParseDropQuery) ASTDropQuery * drop2 = ast2->as(); EXPECT_TRUE(drop2->is_dictionary); - auto & database_and_tables2 = drop2->database_and_tables->as(); - auto identifier2 = dynamic_pointer_cast(database_and_tables2.children[0]); - EXPECT_EQ(identifier2->getDatabaseName(), ""); - EXPECT_EQ(identifier2->shortName(), "dict2"); + EXPECT_EQ(drop2->getDatabase(), ""); + EXPECT_EQ(drop2->getTable(), "dict2"); auto str2 = serializeAST(*drop2); EXPECT_EQ(input2, str2); } diff --git a/tests/queries/0_stateless/02961_drop_tables.reference b/tests/queries/0_stateless/02961_drop_tables.reference index c0465dc592a..8ccdec0a1b0 100644 --- a/tests/queries/0_stateless/02961_drop_tables.reference +++ b/tests/queries/0_stateless/02961_drop_tables.reference @@ -6,3 +6,6 @@ Test when deletion of existing table fails -- check which tables exist in 02961_db1 -- check which tables exist in 02961_db2 02961_tb5 +Test when deletion of not empty table fails +tab2 +tab3 diff --git a/tests/queries/0_stateless/02961_drop_tables.sql b/tests/queries/0_stateless/02961_drop_tables.sql index e91ac4bfe19..f84fffbef75 100644 --- a/tests/queries/0_stateless/02961_drop_tables.sql +++ b/tests/queries/0_stateless/02961_drop_tables.sql @@ -27,6 +27,20 @@ SHOW TABLES FROM 02961_db1; SELECT '-- check which tables exist in 02961_db2'; SHOW TABLES FROM 02961_db2; +DROP TABLE IF EXISTS tab1, tab2, tab3; +CREATE TABLE IF NOT EXISTS tab1 (id UInt32) Engine=Memory(); +CREATE TABLE IF NOT EXISTS tab2 (id UInt32) Engine=Memory(); +CREATE TABLE IF NOT EXISTS tab3 (id UInt32) Engine=Memory(); + +INSERT INTO tab2 SELECT number FROM system.numbers limit 10; + +DROP TABLE IF EMPTY tab1, tab2, tab3; -- { serverError TABLE_NOT_EMPTY } +SELECT 'Test when deletion of not empty table fails'; +SHOW TABLES; + +TRUNCATE TABLE tab2, tab3; -- { clientError SYNTAX_ERROR } + +DROP TABLE IF EXISTS tab1, tab2, tab3; DROP DATABASE IF EXISTS 02961_db1; DROP DATABASE IF EXISTS 02961_db2; From d5aa23659043ae17a267bef3353a383e5383a24e Mon Sep 17 00:00:00 2001 From: zhongyuankai <54787696+zhongyuankai@users.noreply.github.com> Date: Mon, 8 Apr 2024 19:40:27 +0800 Subject: [PATCH 021/192] 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 022/192] 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 023/192] Add a test --- .../integration/test_filesystem_cache/test.py | 57 +++++++++++++++++++ 1 file changed, 57 insertions(+) diff --git a/tests/integration/test_filesystem_cache/test.py b/tests/integration/test_filesystem_cache/test.py index dfab462732a..17a8dd8b6e1 100644 --- a/tests/integration/test_filesystem_cache/test.py +++ b/tests/integration/test_filesystem_cache/test.py @@ -501,3 +501,60 @@ INSERT INTO test SELECT 1, 'test'; node.query("SELECT * FROM test FORMAT Null") assert key not in node.query("SYSTEM SYNC FILESYSTEM CACHE") + + +def test_keep_up_size_ratio(cluster): + node = cluster.instances["node"] + max_elements = 20 + elements_ratio = 0.5 + cache_name = "keep_up_size_ratio" + node.query( + f""" +DROP TABLE IF EXISTS test; + +CREATE TABLE test (a String) +ENGINE = MergeTree() ORDER BY tuple() +SETTINGS disk = disk(type = cache, + name = {cache_name}, + max_size = '100Ki', + max_elements = {max_elements}, + max_file_segment_size = 10, + boundary_alignment = 10, + path = "test_keep_up_size_ratio", + keep_free_space_size_ratio = 0.5, + keep_free_space_elements_ratio = {elements_ratio}, + disk = hdd_blob), + min_bytes_for_wide_part = 10485760; + +INSERT INTO test SELECT randomString(200); + """ + ) + + query_id = "test_keep_up_size_ratio_1" + node.query( + "SELECT * FROM test FORMAT Null SETTINGS enable_filesystem_cache_log = 1", + query_id=query_id, + ) + count = int( + node.query( + f""" + SYSTEM FLUSH LOGS; + SELECT uniqExact(concat(key, toString(offset))) + FROM system.filesystem_cache_log + WHERE read_type = 'READ_FROM_FS_AND_DOWNLOADED_TO_CACHE'; + """ + ) + ) + assert count > max_elements + + expected = 10 + for _ in range(100): + elements = int( + node.query( + f"SELECT count() FROM system.filesystem_cache WHERE cache_name = '{cache_name}'" + ) + ) + if elements <= expected: + break + time.sleep(1) + assert elements <= expected From ed6e4fbe162e9398a5f3b2ce24ed4afb31141f97 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 8 Apr 2024 13:56:15 +0000 Subject: [PATCH 024/192] Improve trivial insert select from files, add separate max_parsing_threads setting --- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.h | 1 + src/Formats/FormatFactory.cpp | 2 +- src/Storages/StorageFile.cpp | 2 +- src/Storages/StorageS3.cpp | 4 +-- src/Storages/StorageURL.cpp | 6 ++-- .../trivial_insert_select_from_files.xml | 30 +++++++++++++++++++ 7 files changed, 39 insertions(+), 7 deletions(-) create mode 100644 tests/performance/trivial_insert_select_from_files.xml diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 84e709294aa..67a12d1e6d5 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -50,6 +50,7 @@ class IColumn; M(MaxThreads, max_threads, 0, "The maximum number of threads to execute the request. By default, it is determined automatically.", 0) \ M(Bool, use_concurrency_control, true, "Respect the server's concurrency control (see the `concurrent_threads_soft_limit_num` and `concurrent_threads_soft_limit_ratio_to_cores` global server settings). If disabled, it allows using a larger number of threads even if the server is overloaded (not recommended for normal usage, and needed mostly for tests).", 0) \ M(MaxThreads, max_download_threads, 4, "The maximum number of threads to download data (e.g. for URL engine).", 0) \ + M(MaxThreads, max_parsing_threads, 0, "The maximum number of threads to parse data in input formats that support parallel parsing. By default, it is determined automatically", 0) \ M(UInt64, max_download_buffer_size, 10*1024*1024, "The maximal size of buffer for parallel downloading (e.g. for URL engine) per each thread.", 0) \ M(UInt64, max_read_buffer_size, DBMS_DEFAULT_BUFFER_SIZE, "The maximum size of the buffer to read from the filesystem.", 0) \ M(UInt64, max_read_buffer_size_local_fs, 128*1024, "The maximum size of the buffer to read from local filesystem. If set to 0 then max_read_buffer_size will be used.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 768b6aa6cbd..a26e215ce35 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -86,6 +86,7 @@ namespace SettingsChangesHistory static std::map settings_changes_history = { {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, + {"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"}, }}, {"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"}, {"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"}, diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 8cbb1b9e563..1cc3dabd954 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -303,7 +303,7 @@ InputFormatPtr FormatFactory::getInput( auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context); const Settings & settings = context->getSettingsRef(); - size_t max_parsing_threads = _max_parsing_threads.value_or(settings.max_threads); + size_t max_parsing_threads = _max_parsing_threads.value_or(settings.max_parsing_threads); size_t max_download_threads = _max_download_threads.value_or(settings.max_download_threads); RowInputFormatParams row_input_format_params; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 0d220f2fd5d..54ba567b838 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1392,7 +1392,7 @@ Chunk StorageFileSource::generate() chassert(file_num > 0); - const auto max_parsing_threads = std::max(settings.max_threads / file_num, 1UL); + const auto max_parsing_threads = std::max(settings.max_parsing_threads / file_num, 1UL); input_format = FormatFactory::instance().getInput( storage->format_name, *read_buf, block_for_format, getContext(), max_block_size, storage->format_settings, max_parsing_threads, std::nullopt, /*is_remote_fs*/ false, CompressionMethod::None, need_only_count); diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 2d3aef312bf..c7b80f4912a 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1206,8 +1206,8 @@ void ReadFromStorageS3Step::initializePipeline(QueryPipelineBuilder & pipeline, /// Disclosed glob iterator can underestimate the amount of keys in some cases. We will keep one stream for this particular case. num_streams = 1; - const size_t max_threads = context->getSettingsRef().max_threads; - const size_t max_parsing_threads = num_streams >= max_threads ? 1 : (max_threads / std::max(num_streams, 1ul)); + const auto & settings = context->getSettingsRef(); + const size_t max_parsing_threads = num_streams >= settings.max_parsing_threads ? 1 : (settings.max_parsing_threads / std::max(num_streams, 1ul)); LOG_DEBUG(getLogger("StorageS3"), "Reading in {} streams, {} threads per stream", num_streams, max_parsing_threads); Pipes pipes; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index cc46cc8f8dc..be23ff50647 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -1172,8 +1172,8 @@ void ReadFromURL::initializePipeline(QueryPipelineBuilder & pipeline, const Buil Pipes pipes; pipes.reserve(num_streams); - const size_t max_threads = context->getSettingsRef().max_threads; - const size_t max_parsing_threads = num_streams >= max_threads ? 1 : (max_threads / num_streams); + const auto & settings = context->getSettingsRef(); + const size_t max_parsing_threads = num_streams >= settings.max_parsing_threads ? 1 : (settings.max_parsing_threads / num_streams); for (size_t i = 0; i < num_streams; ++i) { @@ -1204,7 +1204,7 @@ void ReadFromURL::initializePipeline(QueryPipelineBuilder & pipeline, const Buil auto pipe = Pipe::unitePipes(std::move(pipes)); size_t output_ports = pipe.numOutputPorts(); - const bool parallelize_output = context->getSettingsRef().parallelize_output_from_storages; + const bool parallelize_output = settings.parallelize_output_from_storages; if (parallelize_output && storage->parallelizeOutputAfterReading(context) && output_ports > 0 && output_ports < max_num_streams) pipe.resize(max_num_streams); diff --git a/tests/performance/trivial_insert_select_from_files.xml b/tests/performance/trivial_insert_select_from_files.xml new file mode 100644 index 00000000000..f6ec6500c10 --- /dev/null +++ b/tests/performance/trivial_insert_select_from_files.xml @@ -0,0 +1,30 @@ + + + + + format + + TabSeparated + TabSeparatedWithNames + TabSeparatedWithNamesAndTypes + CSV + CSVWithNames + JSONEachRow + JSONCompactEachRow + JSONCompactEachRowWithNamesAndTypes + TSKV + + + + +CREATE TABLE IF NOT EXISTS table_src_{format} ENGINE = File({format}) AS test.hits +CREATE TABLE IF NOT EXISTS table_dst_{format} AS test.hits + +INSERT INTO table_src_{format} SELECT * FROM test.hits LIMIT 100000 + +INSERT INTO table_dst_{format} SELECT * FROM table_src_{format} + +DROP TABLE IF EXISTS table_src_{format} +DROP TABLE IF EXISTS table_dst_{format} + + From 19f00e7c42bd9ba0af63be4987a89aad3457b3c8 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 9 Apr 2024 12:33:18 +0200 Subject: [PATCH 025/192] Restart CI --- tests/performance/trivial_insert_select_from_files.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/performance/trivial_insert_select_from_files.xml b/tests/performance/trivial_insert_select_from_files.xml index f6ec6500c10..64bd3e54b95 100644 --- a/tests/performance/trivial_insert_select_from_files.xml +++ b/tests/performance/trivial_insert_select_from_files.xml @@ -19,7 +19,6 @@ CREATE TABLE IF NOT EXISTS table_src_{format} ENGINE = File({format}) AS test.hits CREATE TABLE IF NOT EXISTS table_dst_{format} AS test.hits - INSERT INTO table_src_{format} SELECT * FROM test.hits LIMIT 100000 INSERT INTO table_dst_{format} SELECT * FROM table_src_{format} From 41e0720c0e4e1f3dd1262ffc5932285e700c8dd4 Mon Sep 17 00:00:00 2001 From: zhongyuankai Date: Wed, 10 Apr 2024 09:50:47 +0800 Subject: [PATCH 026/192] cmment --- src/Parsers/ASTDropQuery.h | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Parsers/ASTDropQuery.h b/src/Parsers/ASTDropQuery.h index 7e25e990bc8..e18043b771b 100644 --- a/src/Parsers/ASTDropQuery.h +++ b/src/Parsers/ASTDropQuery.h @@ -37,13 +37,13 @@ public: bool sync{false}; - // We detach the object permanently, so it will not be reattached back during server restart. + /// We detach the object permanently, so it will not be reattached back during server restart. bool permanently{false}; - /// Used to drop multiple tables only, example: Drop TABLE t1, t2, t3... + /// Used to drop multiple tables only, example: DROP TABLE t1, t2, t3... ASTPtr database_and_tables; - /** Get the text that identifies this element. */ + /// Get the text that identifies this element. String getID(char) const override; ASTPtr clone() const override; @@ -52,7 +52,7 @@ public: return removeOnCluster(clone(), params.default_database); } - /** Convert an AST that deletes multiple tables into multiple ASTs that delete a single table. */ + /// Convert an AST that deletes multiple tables into multiple ASTs that delete a single table. ASTs getRewrittenASTsOfSingleTable(); QueryKind getQueryKind() const override { return QueryKind::Drop; } From b6b7c3f80f6461e8198e9d70f6b4742f62d6435d Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 10 Apr 2024 13:41:21 +0200 Subject: [PATCH 027/192] 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 dc07496fd668f5db3e28a123fbfb9347b3ce7c63 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 10 Apr 2024 15:45:03 +0000 Subject: [PATCH 028/192] Fix validation of special MergeTree columns --- src/Storages/MergeTree/MergeTreeData.cpp | 90 ++++++++++++++++++- .../03093_special_column_errors.reference | 0 .../03093_special_column_errors.sql | 36 ++++++++ 3 files changed, 122 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/03093_special_column_errors.reference create mode 100644 tests/queries/0_stateless/03093_special_column_errors.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 3e4350c2fbb..7eb862c8c72 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -861,6 +861,42 @@ void MergeTreeData::checkTTLExpressions(const StorageInMemoryMetadata & new_meta } } +namespace +{ +template +void checkSpecialColumn(const std::string_view column_meta_name, const AlterCommand & command) +{ + if (command.type == AlterCommand::MODIFY_COLUMN) + { + if (!typeid_cast(command.data_type.get())) + { + throw Exception( + ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, + "Cannot alter {} column ({}) to type {}, because it must have type {}", + column_meta_name, + command.column_name, + command.data_type->getName(), + TypeName); + } + } + else if (command.type == AlterCommand::DROP_COLUMN) + { + throw Exception( + ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, + "Trying to ALTER DROP {} ({}) column", + column_meta_name, + backQuoteIfNeed(command.column_name)); + } + else if (command.type == AlterCommand::RENAME_COLUMN) + { + throw Exception( + ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, + "Trying to ALTER RENAME {} ({}) column", + column_meta_name, + backQuoteIfNeed(command.column_name)); + } +}; +} void MergeTreeData::checkStoragePolicy(const StoragePolicyPtr & new_storage_policy) const { @@ -993,10 +1029,20 @@ void MergeTreeData::MergingParams::check(const StorageInMemoryMetadata & metadat { return column_to_sum == Nested::extractTableName(name_and_type.name); }; - if (columns.end() == std::find_if(columns.begin(), columns.end(), check_column_to_sum_exists)) - throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, - "Column {} listed in columns to sum does not exist in table declaration.", - column_to_sum); + const auto column_it = std::find_if(columns.begin(), columns.end(), check_column_to_sum_exists); + + if (columns.end() == column_it) + { + throw Exception( + ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, + "Column {} listed in columns to sum does not exist in table declaration", + column_to_sum); + } + else if (!isNumber(column_it->type)) + { + throw Exception( + ErrorCodes::BAD_TYPE_OF_FIELD, "Column {} listed in columns to sum does not have number type", column_to_sum); + } } /// Check that summing columns are not in partition key. @@ -3114,6 +3160,42 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context } } + // Technically it is possible to specify the same column for `version` and `is_deleted`, thus let's be sure and don't use -if-else here + if (command.column_name == merging_params.is_deleted_column) + { + checkSpecialColumn("is_deleted", command); + } + else if (command.column_name == merging_params.sign_column) + { + checkSpecialColumn("sign", command); + } + else if (std::ranges::any_of( + merging_params.columns_to_sum, [&](const String & column_to_sum) { return command.column_name == column_to_sum; })) + { + if (command.type == AlterCommand::MODIFY_COLUMN && !isNumber(command.data_type)) + { + throw Exception( + ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, + "Cannot alter summarizing column ({}) to type {}, because it must have numeric type", + command.column_name, + command.data_type->getName()); + } + else if (command.type == AlterCommand::DROP_COLUMN) + { + throw Exception( + ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, + "Trying to ALTER DROP summarizing column ({})", + backQuoteIfNeed(command.column_name)); + } + else if (command.type == AlterCommand::RENAME_COLUMN) + { + throw Exception( + ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, + "Trying to ALTER RENAME summarizing column ({})", + backQuoteIfNeed(command.column_name)); + } + } + if (command.type == AlterCommand::MODIFY_QUERY) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "ALTER MODIFY QUERY is not supported by MergeTree engines family"); diff --git a/tests/queries/0_stateless/03093_special_column_errors.reference b/tests/queries/0_stateless/03093_special_column_errors.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03093_special_column_errors.sql b/tests/queries/0_stateless/03093_special_column_errors.sql new file mode 100644 index 00000000000..2bff551738e --- /dev/null +++ b/tests/queries/0_stateless/03093_special_column_errors.sql @@ -0,0 +1,36 @@ +CREATE TABLE replacing_wrong (key Int64, ver Int64, is_deleted UInt16) ENGINE = ReplacingMergeTree(ver, is_deleted) ORDER BY key; -- { serverError BAD_TYPE_OF_FIELD } +CREATE TABLE replacing_wrong (key Int64, ver String, is_deleted UInt8) ENGINE = ReplacingMergeTree(ver, is_deleted) ORDER BY key; -- { serverError BAD_TYPE_OF_FIELD } + +CREATE TABLE replacing (key Int64, ver Int64, is_deleted UInt8) ENGINE = ReplacingMergeTree(ver, is_deleted) ORDER BY key; +ALTER TABLE replacing MODIFY COLUMN ver String; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE replacing MODIFY COLUMN ver Int128; +ALTER TABLE replacing MODIFY COLUMN is_deleted String; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE replacing MODIFY COLUMN is_deleted UInt16; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE replacing MODIFY COLUMN is_deleted Int8; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE replacing DROP COLUMN ver; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE replacing DROP COLUMN is_deleted; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE replacing RENAME COLUMN ver TO ver2; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE replacing RENAME COLUMN is_deleted TO is_deleted2; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } + +CREATE TABLE summing_wrong (key Int64, sum1 Int64, sum2 String) ENGINE = SummingMergeTree((sum1, sum2)) ORDER BY key; -- { serverError BAD_TYPE_OF_FIELD } +CREATE TABLE summing_wrong (key Int64, sum1 String, sum2 Int64) ENGINE = SummingMergeTree((sum1, sum2)) ORDER BY key; -- { serverError BAD_TYPE_OF_FIELD } +CREATE TABLE summing_wrong (key Int64, sum1 String, sum2 Int64) ENGINE = SummingMergeTree(sum_doesnt_exists) ORDER BY key; -- { serverError NO_SUCH_COLUMN_IN_TABLE } + +CREATE TABLE summing (key Int64, sum1 Int64, sum2 UInt64, not_sum String) ENGINE = SummingMergeTree((sum1, sum2)) ORDER BY key; +ALTER TABLE summing MODIFY COLUMN sum1 Int32, MODIFY COLUMN sum2 IPv4; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE summing MODIFY COLUMN sum2 String; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE summing MODIFY COLUMN sum1 Int32, MODIFY COLUMN sum2 UInt256; +ALTER TABLE summing DROP COLUMN sum1; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE summing DROP COLUMN sum2; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE summing RENAME COLUMN sum1 TO sum3; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE summing RENAME COLUMN sum2 TO sum3; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE summing RENAME COLUMN not_sum TO still_not_sum; + +CREATE TABLE collapsing_wrong (key Int64, sign Int16) ENGINE = CollapsingMergeTree(sign) ORDER BY key; -- { serverError BAD_TYPE_OF_FIELD } +CREATE TABLE collapsing_wrong (key Int64, sign UInt8) ENGINE = CollapsingMergeTree(sign) ORDER BY key; -- { serverError BAD_TYPE_OF_FIELD } +CREATE TABLE collapsing_wrong (key Int64, sign UInt8) ENGINE = CollapsingMergeTree(not_existing) ORDER BY key; -- { serverError NO_SUCH_COLUMN_IN_TABLE } + +CREATE TABLE collapsing (key Int64, sign Int8) ENGINE = CollapsingMergeTree(sign) ORDER BY key; +ALTER TABLE collapsing MODIFY COLUMN sign String; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE collapsing DROP COLUMN sign; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } +ALTER TABLE collapsing RENAME COLUMN sign TO sign2; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } From 05f55504bb487148d5ced68feb2f9e1ad9ca2597 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 10 Apr 2024 19:32:11 +0200 Subject: [PATCH 029/192] 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 ce457e76256e397bea451db5c1012b8c49e8d70f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Thu, 11 Apr 2024 09:01:14 +0000 Subject: [PATCH 030/192] Add checks to prevent using the same column for multiple special columns --- src/Storages/MergeTree/MergeTreeData.cpp | 15 ++++++++++++--- .../0_stateless/03093_special_column_errors.sql | 6 ++++++ 2 files changed, 18 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 7eb862c8c72..8f97ec9cbc2 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1022,6 +1022,11 @@ void MergeTreeData::MergingParams::check(const StorageInMemoryMetadata & metadat if (mode == MergingParams::Summing) { + auto columns_to_sum_copy = columns_to_sum; + std::sort(columns_to_sum_copy.begin(), columns_to_sum_copy.end()); + if (const auto it = std::adjacent_find(columns_to_sum_copy.begin(), columns_to_sum_copy.end()); it != columns_to_sum_copy.end()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Column {} is listed multiple times in the list of columns to sum", *it); + /// If columns_to_sum are set, then check that such columns exist. for (const auto & column_to_sum : columns_to_sum) { @@ -1063,12 +1068,18 @@ void MergeTreeData::MergingParams::check(const StorageInMemoryMetadata & metadat if (mode == MergingParams::Replacing) { + if (version_column == is_deleted_column) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The version and is_deleted column cannot be the same column ({})", version_column); + check_is_deleted_column(true, "ReplacingMergeTree"); check_version_column(true, "ReplacingMergeTree"); } if (mode == MergingParams::VersionedCollapsing) { + if (version_column == sign_column) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The version and sign column cannot be the same column ({})", version_column); + check_sign_column(false, "VersionedCollapsingMergeTree"); check_version_column(false, "VersionedCollapsingMergeTree"); } @@ -3159,9 +3170,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context "Trying to ALTER RENAME version {} column", backQuoteIfNeed(command.column_name)); } } - - // Technically it is possible to specify the same column for `version` and `is_deleted`, thus let's be sure and don't use -if-else here - if (command.column_name == merging_params.is_deleted_column) + else if (command.column_name == merging_params.is_deleted_column) { checkSpecialColumn("is_deleted", command); } diff --git a/tests/queries/0_stateless/03093_special_column_errors.sql b/tests/queries/0_stateless/03093_special_column_errors.sql index 2bff551738e..bbdf1bb7183 100644 --- a/tests/queries/0_stateless/03093_special_column_errors.sql +++ b/tests/queries/0_stateless/03093_special_column_errors.sql @@ -1,5 +1,6 @@ CREATE TABLE replacing_wrong (key Int64, ver Int64, is_deleted UInt16) ENGINE = ReplacingMergeTree(ver, is_deleted) ORDER BY key; -- { serverError BAD_TYPE_OF_FIELD } CREATE TABLE replacing_wrong (key Int64, ver String, is_deleted UInt8) ENGINE = ReplacingMergeTree(ver, is_deleted) ORDER BY key; -- { serverError BAD_TYPE_OF_FIELD } +CREATE TABLE replacing_wrong (key Int64, ver Int64, is_deleted UInt8) ENGINE = ReplacingMergeTree(is_deleted, is_deleted) ORDER BY key; -- { serverError BAD_ARGUMENTS } CREATE TABLE replacing (key Int64, ver Int64, is_deleted UInt8) ENGINE = ReplacingMergeTree(ver, is_deleted) ORDER BY key; ALTER TABLE replacing MODIFY COLUMN ver String; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } @@ -15,6 +16,7 @@ ALTER TABLE replacing RENAME COLUMN is_deleted TO is_deleted2; -- { serverError CREATE TABLE summing_wrong (key Int64, sum1 Int64, sum2 String) ENGINE = SummingMergeTree((sum1, sum2)) ORDER BY key; -- { serverError BAD_TYPE_OF_FIELD } CREATE TABLE summing_wrong (key Int64, sum1 String, sum2 Int64) ENGINE = SummingMergeTree((sum1, sum2)) ORDER BY key; -- { serverError BAD_TYPE_OF_FIELD } CREATE TABLE summing_wrong (key Int64, sum1 String, sum2 Int64) ENGINE = SummingMergeTree(sum_doesnt_exists) ORDER BY key; -- { serverError NO_SUCH_COLUMN_IN_TABLE } +CREATE TABLE summing_wrong (key Int64, sum1 String, sum2 Int64) ENGINE = SummingMergeTree((sum1, sum2, sum1)) ORDER BY key; -- { serverError BAD_ARGUMENTS } CREATE TABLE summing (key Int64, sum1 Int64, sum2 UInt64, not_sum String) ENGINE = SummingMergeTree((sum1, sum2)) ORDER BY key; ALTER TABLE summing MODIFY COLUMN sum1 Int32, MODIFY COLUMN sum2 IPv4; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } @@ -34,3 +36,7 @@ CREATE TABLE collapsing (key Int64, sign Int8) ENGINE = CollapsingMergeTree(sign ALTER TABLE collapsing MODIFY COLUMN sign String; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } ALTER TABLE collapsing DROP COLUMN sign; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } ALTER TABLE collapsing RENAME COLUMN sign TO sign2; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } + +CREATE TABLE versioned_collapsing_wrong (key Int64, version UInt8, sign Int8) ENGINE = VersionedCollapsingMergeTree(sign, sign) ORDER BY key; -- { serverError BAD_ARGUMENTS } + +CREATE TABLE versioned_collapsing (key Int64, version UInt8, sign Int8) ENGINE = VersionedCollapsingMergeTree(sign, version) ORDER BY key; From 583bdff2bc51c3dbff275d76fbdbd7e9e209915c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Thu, 11 Apr 2024 10:57:33 +0000 Subject: [PATCH 031/192] Abandon validating the special columns of SummingMergeTree, as they are way more complex --- src/Storages/MergeTree/MergeTreeData.cpp | 39 +------------------ .../03093_special_column_errors.sql | 15 ------- 2 files changed, 2 insertions(+), 52 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 8f97ec9cbc2..d09bb23de90 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1034,20 +1034,11 @@ void MergeTreeData::MergingParams::check(const StorageInMemoryMetadata & metadat { return column_to_sum == Nested::extractTableName(name_and_type.name); }; - const auto column_it = std::find_if(columns.begin(), columns.end(), check_column_to_sum_exists); - - if (columns.end() == column_it) - { + if (columns.end() == std::find_if(columns.begin(), columns.end(), check_column_to_sum_exists)) throw Exception( ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, - "Column {} listed in columns to sum does not exist in table declaration", + "Column {} listed in columns to sum does not exist in table declaration.", column_to_sum); - } - else if (!isNumber(column_it->type)) - { - throw Exception( - ErrorCodes::BAD_TYPE_OF_FIELD, "Column {} listed in columns to sum does not have number type", column_to_sum); - } } /// Check that summing columns are not in partition key. @@ -3178,32 +3169,6 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context { checkSpecialColumn("sign", command); } - else if (std::ranges::any_of( - merging_params.columns_to_sum, [&](const String & column_to_sum) { return command.column_name == column_to_sum; })) - { - if (command.type == AlterCommand::MODIFY_COLUMN && !isNumber(command.data_type)) - { - throw Exception( - ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, - "Cannot alter summarizing column ({}) to type {}, because it must have numeric type", - command.column_name, - command.data_type->getName()); - } - else if (command.type == AlterCommand::DROP_COLUMN) - { - throw Exception( - ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, - "Trying to ALTER DROP summarizing column ({})", - backQuoteIfNeed(command.column_name)); - } - else if (command.type == AlterCommand::RENAME_COLUMN) - { - throw Exception( - ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, - "Trying to ALTER RENAME summarizing column ({})", - backQuoteIfNeed(command.column_name)); - } - } if (command.type == AlterCommand::MODIFY_QUERY) throw Exception(ErrorCodes::NOT_IMPLEMENTED, diff --git a/tests/queries/0_stateless/03093_special_column_errors.sql b/tests/queries/0_stateless/03093_special_column_errors.sql index bbdf1bb7183..1464927db7e 100644 --- a/tests/queries/0_stateless/03093_special_column_errors.sql +++ b/tests/queries/0_stateless/03093_special_column_errors.sql @@ -13,21 +13,6 @@ ALTER TABLE replacing DROP COLUMN is_deleted; -- { serverError ALTER_OF_COLUMN_I ALTER TABLE replacing RENAME COLUMN ver TO ver2; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } ALTER TABLE replacing RENAME COLUMN is_deleted TO is_deleted2; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } -CREATE TABLE summing_wrong (key Int64, sum1 Int64, sum2 String) ENGINE = SummingMergeTree((sum1, sum2)) ORDER BY key; -- { serverError BAD_TYPE_OF_FIELD } -CREATE TABLE summing_wrong (key Int64, sum1 String, sum2 Int64) ENGINE = SummingMergeTree((sum1, sum2)) ORDER BY key; -- { serverError BAD_TYPE_OF_FIELD } -CREATE TABLE summing_wrong (key Int64, sum1 String, sum2 Int64) ENGINE = SummingMergeTree(sum_doesnt_exists) ORDER BY key; -- { serverError NO_SUCH_COLUMN_IN_TABLE } -CREATE TABLE summing_wrong (key Int64, sum1 String, sum2 Int64) ENGINE = SummingMergeTree((sum1, sum2, sum1)) ORDER BY key; -- { serverError BAD_ARGUMENTS } - -CREATE TABLE summing (key Int64, sum1 Int64, sum2 UInt64, not_sum String) ENGINE = SummingMergeTree((sum1, sum2)) ORDER BY key; -ALTER TABLE summing MODIFY COLUMN sum1 Int32, MODIFY COLUMN sum2 IPv4; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } -ALTER TABLE summing MODIFY COLUMN sum2 String; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } -ALTER TABLE summing MODIFY COLUMN sum1 Int32, MODIFY COLUMN sum2 UInt256; -ALTER TABLE summing DROP COLUMN sum1; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } -ALTER TABLE summing DROP COLUMN sum2; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } -ALTER TABLE summing RENAME COLUMN sum1 TO sum3; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } -ALTER TABLE summing RENAME COLUMN sum2 TO sum3; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } -ALTER TABLE summing RENAME COLUMN not_sum TO still_not_sum; - CREATE TABLE collapsing_wrong (key Int64, sign Int16) ENGINE = CollapsingMergeTree(sign) ORDER BY key; -- { serverError BAD_TYPE_OF_FIELD } CREATE TABLE collapsing_wrong (key Int64, sign UInt8) ENGINE = CollapsingMergeTree(sign) ORDER BY key; -- { serverError BAD_TYPE_OF_FIELD } CREATE TABLE collapsing_wrong (key Int64, sign UInt8) ENGINE = CollapsingMergeTree(not_existing) ORDER BY key; -- { serverError NO_SUCH_COLUMN_IN_TABLE } From 78c34916f93c11a913bc16149e028c9b688c902d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Thu, 11 Apr 2024 13:44:23 +0000 Subject: [PATCH 032/192] Fix incorrect error when no special columns are specified --- src/Storages/MergeTree/MergeTreeData.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d09bb23de90..7a138331207 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1059,7 +1059,7 @@ void MergeTreeData::MergingParams::check(const StorageInMemoryMetadata & metadat if (mode == MergingParams::Replacing) { - if (version_column == is_deleted_column) + if (!version_column.empty() && version_column == is_deleted_column) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The version and is_deleted column cannot be the same column ({})", version_column); check_is_deleted_column(true, "ReplacingMergeTree"); @@ -1068,7 +1068,7 @@ void MergeTreeData::MergingParams::check(const StorageInMemoryMetadata & metadat if (mode == MergingParams::VersionedCollapsing) { - if (version_column == sign_column) + if (!version_column.empty() && version_column == sign_column) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The version and sign column cannot be the same column ({})", version_column); check_sign_column(false, "VersionedCollapsingMergeTree"); From d3a58ffbbd5e5efc5c0c9679d4b56737fe6de40f Mon Sep 17 00:00:00 2001 From: pet74alex <167422282+pet74alex@users.noreply.github.com> Date: Mon, 22 Apr 2024 19:22:40 +0300 Subject: [PATCH 033/192] Added generateUUIDv7* functions --- src/Functions/generateUUIDv7.cpp | 373 +++++++++++++++++++++++++++++++ 1 file changed, 373 insertions(+) create mode 100644 src/Functions/generateUUIDv7.cpp diff --git a/src/Functions/generateUUIDv7.cpp b/src/Functions/generateUUIDv7.cpp new file mode 100644 index 00000000000..4bd540d28db --- /dev/null +++ b/src/Functions/generateUUIDv7.cpp @@ -0,0 +1,373 @@ +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +namespace +{ +constexpr auto bits_in_counter = 42; +constexpr uint64_t counter_limit = (uint64_t{1} << bits_in_counter); +constexpr uint8_t random_data_offset = 6; +constexpr uint8_t random_data_count = 10; +constexpr uint8_t next_count_random_data_offset = 12; +constexpr uint8_t next_count_random_data_count = 4; + +using UUIDAsArray = std::array; + +inline uint64_t getTimestampMs() +{ + timespec tp; + clock_gettime(CLOCK_REALTIME, &tp); + const uint64_t sec = tp.tv_sec; + return sec * 1000 + tp.tv_nsec / 1000000; +} + +inline void fillTimestamp(UUIDAsArray & uuid, uint64_t timestamp) +{ + uuid[0] = (timestamp >> 40) & 0xFF; + uuid[1] = (timestamp >> 32) & 0xFF; + uuid[2] = (timestamp >> 24) & 0xFF; + uuid[3] = (timestamp >> 16) & 0xFF; + uuid[4] = (timestamp >> 8) & 0xFF; + uuid[5] = (timestamp)&0xFF; +} +} + +#define DECLARE_SEVERAL_IMPLEMENTATIONS(...) \ + DECLARE_DEFAULT_CODE(__VA_ARGS__) \ + DECLARE_AVX2_SPECIFIC_CODE(__VA_ARGS__) + +DECLARE_SEVERAL_IMPLEMENTATIONS( + + namespace UUIDv7Impl { + inline void store(UUID & new_uuid, UUIDAsArray & uuid) + { + uuid[6] = (uuid[6] & 0x0f) | 0x70; // version 7 + uuid[8] = (uuid[8] & 0x3f) | 0x80; // variant 2 + + DB::UUIDHelpers::getHighBytes(new_uuid) = unalignedLoadBigEndian(uuid.data()); + DB::UUIDHelpers::getLowBytes(new_uuid) = unalignedLoadBigEndian(uuid.data() + 8); + } + + struct UUIDv7Base + { + UUIDAsArray & uuid; + UUIDv7Base(UUIDAsArray & u) : uuid(u) { } + }; + + struct RandomData + { + static constexpr auto name = "generateUUIDv7"; + struct Data : UUIDv7Base + { + UUIDAsArray uuid_data; + + Data() : UUIDv7Base(uuid_data) { } + + void generate(UUID & new_uuid) + { + fillTimestamp(uuid, getTimestampMs()); + memcpy(uuid.data() + random_data_offset, &new_uuid, random_data_count); + store(new_uuid, uuid); + } + }; + }; + + struct CounterDataCommon : UUIDv7Base + { + CounterDataCommon(UUIDAsArray & u) : UUIDv7Base(u) { } + + uint64_t getCounter() + { + uint64_t counter = uuid[6] & 0x0f; + counter = (counter << 8) | uuid[7]; + counter = (counter << 6) | (uuid[8] & 0x3f); + counter = (counter << 8) | uuid[9]; + counter = (counter << 8) | uuid[10]; + counter = (counter << 8) | uuid[11]; + return counter; + } + + void generate(UUID & newUUID) + { + uint64_t timestamp = 0; + /// Get timestamp of the previous uuid + for (int i = 0; i != 6; ++i) + { + timestamp = (timestamp << 8) | uuid[i]; + } + + const uint64_t unix_time_ms = getTimestampMs(); + // continue incrementing counter when clock slightly goes back or when counter overflow happened during the previous UUID generation + bool need_to_increment_counter = (timestamp == unix_time_ms || timestamp < unix_time_ms + 10000); + uint64_t counter = 0; + if (need_to_increment_counter) + { + counter = getCounter(); + } + else + { + timestamp = unix_time_ms; + } + + bool counter_incremented = false; + if (need_to_increment_counter) + { + if (++counter == counter_limit) + { + ++timestamp; + // counter bytes will be filled by the random data + } + else + { + uuid[6] = counter >> 38; + uuid[7] = counter >> 30; + uuid[8] = counter >> 24; + uuid[9] = counter >> 16; + uuid[10] = counter >> 8; + uuid[11] = counter; + counter_incremented = true; + } + } + + fillTimestamp(uuid, timestamp); + + // Get the required number of random bytes: 4 in the case of incrementing existing counter, 10 in the case of renewing counter + memcpy( + uuid.data() + (counter_incremented ? next_count_random_data_offset : random_data_offset), + &newUUID, + counter_incremented ? next_count_random_data_count : random_data_count); + + store(newUUID, uuid); + } + }; + + struct ThreadLocalCounter + { + static constexpr auto name = "generateUUIDv7WithFastCounter"; + struct Data : CounterDataCommon + { + // Implement counter monotony only within one thread so function doesn't require mutexes and doesn't affect performance of the same function running simultenaously on other threads + static inline thread_local UUIDAsArray uuid_data; + Data() : CounterDataCommon(uuid_data) { } + }; + }; + + struct GlobalCounter + { + static constexpr auto name = "generateUUIDv7WithCounter"; + struct Data : std::lock_guard, CounterDataCommon + { + // Implement counter monotony whithin one timestamp accross all threads generating UUIDv7 with counter simultaneously + static inline UUIDAsArray uuid_data; + static inline std::mutex mtx; + Data() : std::lock_guard(mtx), CounterDataCommon(uuid_data) { } + }; + }; + } // namespace UUIDv7Impl + + + template + class FunctionGenerateUUIDv7Base + : public IFunction, + public FillPolicy { + public: + using FillPolicy::name; + using FillPolicyData = typename FillPolicy::Data; + + FunctionGenerateUUIDv7Base() = default; + + String getName() const final + { + return name; + } + + size_t getNumberOfArguments() const final + { + return 0; + } + + bool isDeterministicInScopeOfQuery() const final + { + return false; + } + bool useDefaultImplementationForNulls() const final + { + return false; + } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const final + { + return false; + } + bool isVariadic() const final + { + return true; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.size() > 1) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be 0 or 1.", + getName(), + arguments.size()); + + return std::make_shared(); + } + + bool isDeterministic() const override + { + return false; + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override + { + auto col_res = ColumnVector::create(); + typename ColumnVector::Container & vec_to = col_res->getData(); + + size_t size = input_rows_count; + vec_to.resize(size); + + /// RandImpl is target-dependent and is not the same in different TargetSpecific namespaces. + /// Not all random bytes produced here are required for the UUIDv7 but it's the simplest way to get the required number of them by using RandImpl + RandImpl::execute(reinterpret_cast(vec_to.data()), vec_to.size() * sizeof(UUID)); + + for (UUID & new_uuid : vec_to) + { + FillPolicyData data; + data.generate(new_uuid); + } + + return col_res; + } + }; + + using FunctionGenerateUUIDv7 = FunctionGenerateUUIDv7Base; + using FunctionGenerateUUIDv7WithCounter = FunctionGenerateUUIDv7Base; + using FunctionGenerateUUIDv7WithFastCounter = FunctionGenerateUUIDv7Base; + + ) // DECLARE_SEVERAL_IMPLEMENTATIONS +#undef DECLARE_SEVERAL_IMPLEMENTATIONS + + +class FunctionGenerateUUIDv7 : public TargetSpecific::Default::FunctionGenerateUUIDv7 +{ +public: + explicit FunctionGenerateUUIDv7(ContextPtr context) : selector(context) + { + selector.registerImplementation(); + +#if USE_MULTITARGET_CODE + selector.registerImplementation(); +#endif + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + return selector.selectAndExecute(arguments, result_type, input_rows_count); + } + + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } + +private: + ImplementationSelector selector; +}; + +class FunctionGenerateUUIDv7WithCounter : public TargetSpecific::Default::FunctionGenerateUUIDv7WithCounter +{ +public: + explicit FunctionGenerateUUIDv7WithCounter(ContextPtr context) : selector(context) + { + selector.registerImplementation(); + +#if USE_MULTITARGET_CODE + selector.registerImplementation(); +#endif + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + return selector.selectAndExecute(arguments, result_type, input_rows_count); + } + + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } + +private: + ImplementationSelector selector; +}; + + +class FunctionGenerateUUIDv7WithFastCounter : public TargetSpecific::Default::FunctionGenerateUUIDv7WithFastCounter +{ +public: + explicit FunctionGenerateUUIDv7WithFastCounter(ContextPtr context) : selector(context) + { + selector.registerImplementation(); + +#if USE_MULTITARGET_CODE + selector.registerImplementation(); +#endif + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + return selector.selectAndExecute(arguments, result_type, input_rows_count); + } + + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } + +private: + ImplementationSelector selector; +}; + + +REGISTER_FUNCTION(GenerateUUIDv7) +{ + factory.registerFunction( + FunctionDocumentation{ + .description = R"( +Generates a UUID of version 7 with current Unix time having milliseconds precision followed by random data. +This function takes an optional argument, the value of which is discarded to generate different values in case the function is called multiple times. +The function returns a value of type UUID. +)", + .examples{{"uuid", "SELECT generateUUIDv7()", ""}, {"multiple", "SELECT generateUUIDv7(1), generateUUIDv7(2)", ""}}, + .categories{"UUID"}}, + FunctionFactory::CaseSensitive); + + factory.registerFunction( + FunctionDocumentation{ + .description = R"( +Generates a UUID of version 7 with current Unix time having milliseconds precision, a monotonic counter within the same timestamp starting from the random value, and followed by 4 random bytes. +This function takes an optional argument, the value of which is discarded to generate different values in case the function is called multiple times. +The function returns a value of type UUID. +)", + .examples{ + {"uuid", "SELECT generateUUIDv7WithCounter()", ""}, + {"multiple", "SELECT generateUUIDv7WithCounter(1), generateUUIDv7WithCounter(2)", ""}}, + .categories{"UUID"}}, + FunctionFactory::CaseSensitive); + + factory.registerFunction( + FunctionDocumentation{ + .description = R"( +Generates a UUID of version 7 with current Unix time having milliseconds precision, a monotonic counter within the same timestamp and the same request starting from the random value, and followed by 4 random bytes. +This function takes an optional argument, the value of which is discarded to generate different values in case the function is called multiple times. +This function is a little bit faster version of the function GenerateUUIDv7WithCounter. It doesn't guarantee the counter monotony withing the same timestamp accross different requests. It means that two UUIDs having +The function returns a value of type UUID. +)", + .examples{ + {"uuid", "SELECT generateUUIDv7WithFastCounter()", ""}, + {"multiple", "SELECT generateUUIDv7WithFastCounter(1), generateUUIDv7WithFastCounter(2)", ""}}, + .categories{"UUID"}}, + FunctionFactory::CaseSensitive); +} + +} From 142ce60b4a9c01ca865cc2cf531238d162d0b994 Mon Sep 17 00:00:00 2001 From: pet74alex <167422282+pet74alex@users.noreply.github.com> Date: Mon, 22 Apr 2024 19:26:15 +0300 Subject: [PATCH 034/192] Added UUIDToNum and UUDv7ToDateTime functions --- src/Functions/FunctionsCodingUUID.cpp | 185 ++++++++++++++++++++++++++ 1 file changed, 185 insertions(+) diff --git a/src/Functions/FunctionsCodingUUID.cpp b/src/Functions/FunctionsCodingUUID.cpp index dd9170e44ad..dceff894c34 100644 --- a/src/Functions/FunctionsCodingUUID.cpp +++ b/src/Functions/FunctionsCodingUUID.cpp @@ -1,14 +1,18 @@ #include +#include #include #include +#include #include #include #include #include #include +#include #include #include #include +#include #include #include #include @@ -319,10 +323,191 @@ public: } }; + +class FunctionUUIDToNum : public IFunction +{ +public: + static constexpr auto name = "UUIDToNum"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 0; } + bool isInjective(const ColumnsWithTypeAndName &) const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + bool isVariadic() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + checkArgumentCount(arguments, name); + + if (!isUUID(arguments[0])) + { + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of first argument of function {}, expected UUID", + arguments[0]->getName(), + getName()); + } + + checkFormatArgument(arguments, name); + + return std::make_shared(uuid_bytes_length); + } + + bool useDefaultImplementationForConstants() const override { return true; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + { + const ColumnWithTypeAndName & col_type_name = arguments[0]; + const ColumnPtr & column = col_type_name.column; + + const bool defaultFormat = (parseVariant(arguments) == UUIDSerializer::Variant::Default); + if (const auto * col_in = checkAndGetColumn(column.get())) + { + const auto & vec_in = col_in->getData(); + const UUID * uuids = vec_in.data(); + const size_t size = vec_in.size(); + + auto col_res = ColumnFixedString::create(uuid_bytes_length); + + ColumnString::Chars & vec_res = col_res->getChars(); + vec_res.resize(size * uuid_bytes_length); + + size_t dst_offset = 0; + + for (size_t i = 0; i < size; ++i) + { + uint64_t hiBytes = DB::UUIDHelpers::getHighBytes(uuids[i]); + uint64_t loBytes = DB::UUIDHelpers::getLowBytes(uuids[i]); + unalignedStoreBigEndian(&vec_res[dst_offset], hiBytes); + unalignedStoreBigEndian(&vec_res[dst_offset + sizeof(hiBytes)], loBytes); + if (!defaultFormat) + { + std::swap(vec_res[dst_offset], vec_res[dst_offset + 3]); + std::swap(vec_res[dst_offset + 1], vec_res[dst_offset + 2]); + } + dst_offset += uuid_bytes_length; + } + + return col_res; + } + else + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", arguments[0].column->getName(), getName()); + } +}; + +class FunctionUUIDv7ToDateTime : public IFunction +{ +public: + static constexpr auto name = "UUIDv7ToDateTime"; + static constexpr UInt32 DATETIME_SCALE = 3; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 0; } + bool isInjective(const ColumnsWithTypeAndName &) const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + bool isVariadic() const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (arguments.empty() || arguments.size() > 2) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Wrong number of arguments for function {}: should be 1 or 2", getName()); + + if (!checkAndGetDataType(arguments[0].type.get())) + { + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of first argument of function {}, expected UUID", + arguments[0].type->getName(), + getName()); + } + + String timezone; + if (arguments.size() == 2) + { + timezone = extractTimeZoneNameFromColumn(arguments[1].column.get(), arguments[1].name); + + if (timezone.empty()) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Function {} supports a 2nd argument (optional) that must be a valid time zone", + getName()); + } + + return std::make_shared(DATETIME_SCALE, timezone); + } + + bool useDefaultImplementationForConstants() const override { return true; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + { + const ColumnWithTypeAndName & col_type_name = arguments[0]; + const ColumnPtr & column = col_type_name.column; + + if (const auto * col_in = checkAndGetColumn(column.get())) + { + const auto & vec_in = col_in->getData(); + const UUID * uuids = vec_in.data(); + const size_t size = vec_in.size(); + + auto col_res = ColumnDateTime64::create(size, DATETIME_SCALE); + auto & vec_res = col_res->getData(); + + for (size_t i = 0; i < size; ++i) + { + uint64_t hiBytes = DB::UUIDHelpers::getHighBytes(uuids[i]); + if ((hiBytes & 0xf000) == 0x7000) + { + uint64_t ms = hiBytes >> 16; + vec_res[i] = DecimalUtils::decimalFromComponents( + ms / intExp10(DATETIME_SCALE), ms % intExp10(DATETIME_SCALE), DATETIME_SCALE); + } + } + + return col_res; + } + else + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", arguments[0].column->getName(), getName()); + } +}; + REGISTER_FUNCTION(CodingUUID) { factory.registerFunction(); factory.registerFunction(); + factory.registerFunction( + FunctionDocumentation{ + .description = R"( +This function accepts a UUID and returns a FixedString(16) as its binary representation, with its format optionally specified by variant (Big-endian by default). +)", + .examples{ + {"uuid", + "select toUUID(UUIDNumToString(toFixedString('a/<@];!~p{jTj={)', 16))) as uuid, UUIDToNum(uuid) as uuidNum, " + "UUIDToNum(uuid, 2) as uuidMsNum", + R"( +┌─uuid─────────────────────────────────┬─uuidNum──────────┬─uuidMsNum────────┐ +│ 612f3c40-5d3b-217e-707b-6a546a3d7b29 │ a/<@];!~p{jTj={) │ @( + FunctionDocumentation{ + .description = R"( +This function extracts the timestamp from a UUID and returns it as a DateTime64(3) typed value. +The function expects the UUID having version 7 to be provided as the first argument. +An optional second argument can be passed to specify a timezone for the timestamp. +)", + .examples{ + {"uuid","select UUIDv7ToDateTime(generateUUIDv7())", ""}, + {"uuid","select generateUUIDv7() as uuid, UUIDv7ToDateTime(uuid), UUIDv7ToDateTime(uuid, 'America/New_York')", ""}}, + .categories{"UUID"}}, + FunctionFactory::CaseSensitive); } } From 4e4e72ead8436ec7e352be043f185412730dfdc8 Mon Sep 17 00:00:00 2001 From: pet74alex <167422282+pet74alex@users.noreply.github.com> Date: Mon, 22 Apr 2024 19:30:25 +0300 Subject: [PATCH 035/192] Update English version of uuid-functions.md --- .../sql-reference/functions/uuid-functions.md | 251 ++++++++++++++++++ 1 file changed, 251 insertions(+) diff --git a/docs/en/sql-reference/functions/uuid-functions.md b/docs/en/sql-reference/functions/uuid-functions.md index c338add3a57..5f1246fc3ad 100644 --- a/docs/en/sql-reference/functions/uuid-functions.md +++ b/docs/en/sql-reference/functions/uuid-functions.md @@ -51,6 +51,169 @@ SELECT generateUUIDv4(1), generateUUIDv4(2) └──────────────────────────────────────┴──────────────────────────────────────┘ ``` +## generateUUIDv7 + +Generates the [UUID](../data-types/uuid.md) of [version 7](https://datatracker.ietf.org/doc/html/draft-peabody-dispatch-new-uuid-format-04). The generated UUID contains current timestamp in milliseconds followed by version 7 and variant 2 markers and random data in the following bit layout. +``` + 0 1 2 3 + 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| unix_ts_ms | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| unix_ts_ms | ver | rand_a | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +|var| rand_b | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| rand_b | +└─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┘ +``` + +**Syntax** + +``` sql +generateUUIDv7([x]) +``` + +**Arguments** + +- `x` — [Expression](../../sql-reference/syntax.md#syntax-expressions) resulting in any of the [supported data types](../../sql-reference/data-types/index.md#data_types). The resulting value is discarded, but the expression itself if used for bypassing [common subexpression elimination](../../sql-reference/functions/index.md#common-subexpression-elimination) if the function is called multiple times in one query. Optional parameter. + +**Returned value** + +The UUID type value. + +**Usage example** + +This example demonstrates creating a table with the UUID type column and inserting a UUIDv7 value into the table. + +``` sql +CREATE TABLE t_uuid (x UUID) ENGINE=TinyLog + +INSERT INTO t_uuid SELECT generateUUIDv7() + +SELECT * FROM t_uuid +``` + +```response +┌────────────────────────────────────x─┐ +│ 018f05af-f4a8-778f-beee-1bedbc95c93b │ +└──────────────────────────────────────┘ +``` + +**Usage example if it is needed to generate multiple values in one row** + +```sql +SELECT generateUUIDv7(1), generateUUIDv7(2) +┌─generateUUIDv7(1)────────────────────┬─generateUUIDv7(2)────────────────────┐ +│ 018f05b1-8c2e-7567-a988-48d09606ae8c │ 018f05b1-8c2e-7946-895b-fcd7635da9a0 │ +└──────────────────────────────────────┴──────────────────────────────────────┘ +``` + +## generateUUIDv7WithCounter + +Generates the [UUID](../data-types/uuid.md) of [version 7](https://datatracker.ietf.org/doc/html/draft-peabody-dispatch-new-uuid-format-04). +The generated UUID contains current timestamp in milliseconds followed by version 7 and variant 2 markers, counter and random data in the following bit layout. At any given new timestamp in unix_ts_ms the counter starts from some random value and then it's being increased by 1 on each new UUID v7 with counter generation until current timestamp changes. The counter overflow causes unix_ts_ms field increment by 1 and the counter restart from a random value. Counter increment monotony at one timestamp is guaranteed across all `generateUUIDv7WithCounter` functions running simultaneously. +``` + 0 1 2 3 + 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| unix_ts_ms | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| unix_ts_ms | ver | counter_high_bits | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +|var| counter_low_bits | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| rand_b | +└─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┘ +``` + +**Syntax** + +``` sql +generateUUIDv7WithCounter([x]) +``` + +**Arguments** + +- `x` — [Expression](../../sql-reference/syntax.md#syntax-expressions) resulting in any of the [supported data types](../../sql-reference/data-types/index.md#data_types). The resulting value is discarded, but the expression itself if used for bypassing [common subexpression elimination](../../sql-reference/functions/index.md#common-subexpression-elimination) if the function is called multiple times in one query. Optional parameter. + +**Returned value** + +The UUID type value. + +**Usage example** + +This example demonstrates creating a table with the UUID type column and inserting a UUIDv7 value into the table. + +``` sql +CREATE TABLE t_uuid (x UUID) ENGINE=TinyLog + +INSERT INTO t_uuid SELECT generateUUIDv7WithCounter() + +SELECT * FROM t_uuid +``` + +```response +┌────────────────────────────────────x─┐ +│ 018f05c7-56e3-7ac3-93e9-1d93c4218e0e │ +└──────────────────────────────────────┘ +``` + +**Usage example if it is needed to generate multiple values in one row** + +```sql +SELECT generateUUIDv7WithCounter(1), generateUUIDv7WithCounter(2) +┌─generateUUIDv7WithCounter(1)─────────┬─generateUUIDv7WithCounter(2)─────────┐ +│ 018f05c9-4ab8-7b86-b64e-c9f03fbd45d1 │ 018f05c9-4ab8-7b86-b64e-c9f12efb7e16 │ +└──────────────────────────────────────┴──────────────────────────────────────┘ +``` + +## generateUUIDv7WithFastCounter + +Generates the [UUID](../data-types/uuid.md) of [version 7](https://datatracker.ietf.org/doc/html/draft-peabody-dispatch-new-uuid-format-04). +This function is a faster version of `generateUUIDv7WithCounter` function giving no guarantee on counter monotony across different requests running simultaneously. Counter increment monotony at one timestamp is guaranteed only within one thread calling this function to generate many UUIDs. + +**Syntax** + +``` sql +generateUUIDv7WithFastCounter([x]) +``` + +**Arguments** + +- `x` — [Expression](../../sql-reference/syntax.md#syntax-expressions) resulting in any of the [supported data types](../../sql-reference/data-types/index.md#data_types). The resulting value is discarded, but the expression itself if used for bypassing [common subexpression elimination](../../sql-reference/functions/index.md#common-subexpression-elimination) if the function is called multiple times in one query. Optional parameter. + +**Returned value** + +The UUID type value. + +**Usage example** + +This example demonstrates creating a table with the UUID type column and inserting a UUIDv7 value into the table. + +``` sql +CREATE TABLE t_uuid (x UUID) ENGINE=TinyLog + +INSERT INTO t_uuid SELECT generateUUIDv7WithFastCounter() + +SELECT * FROM t_uuid +``` + +```response +┌────────────────────────────────────x─┐ +│ 018f05e2-e3b2-70cb-b8be-64b09b626d32 │ +└──────────────────────────────────────┘ +``` + +**Usage example if it is needed to generate multiple values in one row** + +```sql +SELECT generateUUIDv7WithFastCounter(1), generateUUIDv7WithFastCounter(2) +┌─generateUUIDv7WithFastCounter(1)─────┬─generateUUIDv7WithFastCounter(2)─────┐ +│ 018f05e1-14ee-7bc5-9906-207153b400b1 │ 018f05e1-14ee-7bc5-9906-2072b8e96758 │ +└──────────────────────────────────────┴──────────────────────────────────────┘ +``` + ## empty Checks whether the input UUID is empty. @@ -341,6 +504,94 @@ SELECT └──────────────────┴──────────────────────────────────────┘ ``` +## UUIDToNum + +Accepts `UUID` and returns a [FixedString(16)](../../sql-reference/data-types/fixedstring.md) as its binary representation, with its format optionally specified by `variant` (`Big-endian` by default). This function replaces calls to two separate functions `UUIDStringToNum(toString(uuid))` so intermediate conversion from UUID to string is not required to extract bytes from a UUID. + +**Syntax** + +``` sql +UUIDToNum(uuid[, variant = 1]) +``` + +**Arguments** + +- `uuid` — [UUID](../data-types/uuid.md). +- `variant` — Integer, representing a variant as specified by [RFC4122](https://datatracker.ietf.org/doc/html/rfc4122#section-4.1.1). 1 = `Big-endian` (default), 2 = `Microsoft`. + +**Returned value** + +FixedString(16) + +**Usage examples** + +``` sql +SELECT + toUUID('612f3c40-5d3b-217e-707b-6a546a3d7b29') AS uuid, + UUIDToNum(uuid) AS bytes +``` + +```response +┌─uuid─────────────────────────────────┬─bytes────────────┐ +│ 612f3c40-5d3b-217e-707b-6a546a3d7b29 │ a/<@];!~p{jTj={) │ +└──────────────────────────────────────┴──────────────────┘ +``` + +``` sql +SELECT + toUUID('612f3c40-5d3b-217e-707b-6a546a3d7b29') AS uuid, + UUIDToNum(uuid, 2) AS bytes +``` + +```response +┌─uuid─────────────────────────────────┬─bytes────────────┐ +│ 612f3c40-5d3b-217e-707b-6a546a3d7b29 │ @ Date: Mon, 22 Apr 2024 20:01:27 +0300 Subject: [PATCH 036/192] Small fix in generateUUIDv7WithFastCounter documentation --- src/Functions/generateUUIDv7.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/generateUUIDv7.cpp b/src/Functions/generateUUIDv7.cpp index 4bd540d28db..1a7f358f263 100644 --- a/src/Functions/generateUUIDv7.cpp +++ b/src/Functions/generateUUIDv7.cpp @@ -360,7 +360,7 @@ The function returns a value of type UUID. .description = R"( Generates a UUID of version 7 with current Unix time having milliseconds precision, a monotonic counter within the same timestamp and the same request starting from the random value, and followed by 4 random bytes. This function takes an optional argument, the value of which is discarded to generate different values in case the function is called multiple times. -This function is a little bit faster version of the function GenerateUUIDv7WithCounter. It doesn't guarantee the counter monotony withing the same timestamp accross different requests. It means that two UUIDs having +This function is a little bit faster version of the function GenerateUUIDv7WithCounter. It doesn't guarantee the counter monotony withing the same timestamp accross different requests. The function returns a value of type UUID. )", .examples{ From 447aa5bf6926a8bf6442727d9bb2989340cda342 Mon Sep 17 00:00:00 2001 From: Alexey Petrunyaka <167422282+pet74alex@users.noreply.github.com> Date: Tue, 23 Apr 2024 12:32:08 +0300 Subject: [PATCH 037/192] Mistypes fixes in generateUUIDv7.cpp --- src/Functions/generateUUIDv7.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/generateUUIDv7.cpp b/src/Functions/generateUUIDv7.cpp index 1a7f358f263..9354e75d4f3 100644 --- a/src/Functions/generateUUIDv7.cpp +++ b/src/Functions/generateUUIDv7.cpp @@ -165,7 +165,7 @@ DECLARE_SEVERAL_IMPLEMENTATIONS( static constexpr auto name = "generateUUIDv7WithCounter"; struct Data : std::lock_guard, CounterDataCommon { - // Implement counter monotony whithin one timestamp accross all threads generating UUIDv7 with counter simultaneously + // Implement counter monotony within one timestamp across all threads generating UUIDv7 with counter simultaneously static inline UUIDAsArray uuid_data; static inline std::mutex mtx; Data() : std::lock_guard(mtx), CounterDataCommon(uuid_data) { } @@ -360,7 +360,7 @@ The function returns a value of type UUID. .description = R"( Generates a UUID of version 7 with current Unix time having milliseconds precision, a monotonic counter within the same timestamp and the same request starting from the random value, and followed by 4 random bytes. This function takes an optional argument, the value of which is discarded to generate different values in case the function is called multiple times. -This function is a little bit faster version of the function GenerateUUIDv7WithCounter. It doesn't guarantee the counter monotony withing the same timestamp accross different requests. +This function is a little bit faster version of the function GenerateUUIDv7WithCounter. It doesn't guarantee the counter monotony within the same timestamp across different requests. The function returns a value of type UUID. )", .examples{ From e9f80b8631341b57a8f3f72e7b2ff5de2fa810b7 Mon Sep 17 00:00:00 2001 From: Alexey Petrunyaka <167422282+pet74alex@users.noreply.github.com> Date: Tue, 23 Apr 2024 12:36:07 +0300 Subject: [PATCH 038/192] Update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index e63a7608210..811bf3f8e9c 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1008,7 +1008,9 @@ URLPathHierarchy USearch UUIDNumToString UUIDStringToNum +UUIDToNum UUIDs +UUIDv UUid Uber Uint @@ -1043,6 +1045,8 @@ Wether WikiStat WindowView Winkler +WithCounter +WithFastCounter WithNames WithNamesAndTypes WordNet From 2ba6be6d8b88a1f6d876e9ea86a88a49fa5f9a3d Mon Sep 17 00:00:00 2001 From: Alexey Petrunyaka <167422282+pet74alex@users.noreply.github.com> Date: Tue, 23 Apr 2024 12:39:29 +0300 Subject: [PATCH 039/192] Small style fix in generateUUIDv7.cpp --- src/Functions/generateUUIDv7.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Functions/generateUUIDv7.cpp b/src/Functions/generateUUIDv7.cpp index 9354e75d4f3..e6f78543c0d 100644 --- a/src/Functions/generateUUIDv7.cpp +++ b/src/Functions/generateUUIDv7.cpp @@ -46,7 +46,8 @@ inline void fillTimestamp(UUIDAsArray & uuid, uint64_t timestamp) DECLARE_SEVERAL_IMPLEMENTATIONS( - namespace UUIDv7Impl { + namespace UUIDv7Impl + { inline void store(UUID & new_uuid, UUIDAsArray & uuid) { uuid[6] = (uuid[6] & 0x0f) | 0x70; // version 7 From 9c744e50ec6494a14f24004a58ba0d67686388e4 Mon Sep 17 00:00:00 2001 From: Alexey Petrunyaka <167422282+pet74alex@users.noreply.github.com> Date: Tue, 23 Apr 2024 12:55:17 +0300 Subject: [PATCH 040/192] Update generateUUIDv7.cpp for style check test --- src/Functions/generateUUIDv7.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Functions/generateUUIDv7.cpp b/src/Functions/generateUUIDv7.cpp index e6f78543c0d..17619caffec 100644 --- a/src/Functions/generateUUIDv7.cpp +++ b/src/Functions/generateUUIDv7.cpp @@ -172,8 +172,7 @@ DECLARE_SEVERAL_IMPLEMENTATIONS( Data() : std::lock_guard(mtx), CounterDataCommon(uuid_data) { } }; }; - } // namespace UUIDv7Impl - + } template class FunctionGenerateUUIDv7Base From 35d700a5af99a8046faa4f75993973291d92d0c9 Mon Sep 17 00:00:00 2001 From: Alexey Petrunyaka <167422282+pet74alex@users.noreply.github.com> Date: Tue, 23 Apr 2024 14:22:58 +0300 Subject: [PATCH 041/192] Update generateUUIDv7.cpp small fixes for clang-tidy checks --- src/Functions/generateUUIDv7.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/generateUUIDv7.cpp b/src/Functions/generateUUIDv7.cpp index 17619caffec..101fe4a279d 100644 --- a/src/Functions/generateUUIDv7.cpp +++ b/src/Functions/generateUUIDv7.cpp @@ -60,7 +60,7 @@ DECLARE_SEVERAL_IMPLEMENTATIONS( struct UUIDv7Base { UUIDAsArray & uuid; - UUIDv7Base(UUIDAsArray & u) : uuid(u) { } + explicit UUIDv7Base(UUIDAsArray & u) : uuid(u) { } }; struct RandomData @@ -83,7 +83,7 @@ DECLARE_SEVERAL_IMPLEMENTATIONS( struct CounterDataCommon : UUIDv7Base { - CounterDataCommon(UUIDAsArray & u) : UUIDv7Base(u) { } + explicit CounterDataCommon(UUIDAsArray & u) : UUIDv7Base(u) { } uint64_t getCounter() { From f0faac2e8bcef83289e858e4cbfe374f68cf2ce4 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 23 Apr 2024 14:02:26 +0000 Subject: [PATCH 042/192] Move MergeTree*BloomFilter into the same header/source file --- .../MergeTreeIndexAggregatorBloomFilter.cpp | 65 -- .../MergeTreeIndexAggregatorBloomFilter.h | 30 - .../MergeTree/MergeTreeIndexBloomFilter.cpp | 866 +++++++++++++++++- .../MergeTree/MergeTreeIndexBloomFilter.h | 121 ++- .../MergeTreeIndexConditionBloomFilter.cpp | 729 --------------- .../MergeTreeIndexConditionBloomFilter.h | 87 -- .../MergeTreeIndexGranuleBloomFilter.cpp | 102 --- .../MergeTreeIndexGranuleBloomFilter.h | 35 - 8 files changed, 979 insertions(+), 1056 deletions(-) delete mode 100644 src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp delete mode 100644 src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.h delete mode 100644 src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp delete mode 100644 src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h delete mode 100644 src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp delete mode 100644 src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.h diff --git a/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp deleted file mode 100644 index c69c54f1c0d..00000000000 --- a/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp +++ /dev/null @@ -1,65 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -MergeTreeIndexAggregatorBloomFilter::MergeTreeIndexAggregatorBloomFilter( - size_t bits_per_row_, size_t hash_functions_, const Names & columns_name_) - : bits_per_row(bits_per_row_), hash_functions(hash_functions_), index_columns_name(columns_name_), column_hashes(columns_name_.size()) -{ - assert(bits_per_row != 0); - assert(hash_functions != 0); -} - -bool MergeTreeIndexAggregatorBloomFilter::empty() const -{ - return !total_rows; -} - -MergeTreeIndexGranulePtr MergeTreeIndexAggregatorBloomFilter::getGranuleAndReset() -{ - const auto granule = std::make_shared(bits_per_row, hash_functions, column_hashes); - total_rows = 0; - column_hashes.clear(); - return granule; -} - -void MergeTreeIndexAggregatorBloomFilter::update(const Block & block, size_t * pos, size_t limit) -{ - if (*pos >= block.rows()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "The provided position is not less than the number of block rows. " - "Position: {}, Block rows: {}.", *pos, block.rows()); - - Block granule_index_block; - size_t max_read_rows = std::min(block.rows() - *pos, limit); - - for (size_t column = 0; column < index_columns_name.size(); ++column) - { - const auto & column_and_type = block.getByName(index_columns_name[column]); - auto index_column = BloomFilterHash::hashWithColumn(column_and_type.type, column_and_type.column, *pos, max_read_rows); - - const auto & index_col = checkAndGetColumn(index_column.get()); - const auto & index_data = index_col->getData(); - for (const auto & hash: index_data) - column_hashes[column].insert(hash); - } - - *pos += max_read_rows; - total_rows += max_read_rows; -} - -} diff --git a/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.h b/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.h deleted file mode 100644 index d20653b7689..00000000000 --- a/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.h +++ /dev/null @@ -1,30 +0,0 @@ -#pragma once - -#include -#include -#include - -namespace DB -{ - -class MergeTreeIndexAggregatorBloomFilter final : public IMergeTreeIndexAggregator -{ -public: - MergeTreeIndexAggregatorBloomFilter(size_t bits_per_row_, size_t hash_functions_, const Names & columns_name_); - - bool empty() const override; - - MergeTreeIndexGranulePtr getGranuleAndReset() override; - - void update(const Block & block, size_t * pos, size_t limit) override; - -private: - size_t bits_per_row; - size_t hash_functions; - const Names index_columns_name; - - std::vector> column_hashes; - size_t total_rows = 0; -}; - -} diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp index dbd33609a00..ed091022a91 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp @@ -1,13 +1,36 @@ #include -#include -#include -#include -#include -#include -#include + +#include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include +#include +#include +#include +#include +#include #include +#include +#include +#include +#include +#include +#include +#include +#include namespace DB @@ -17,8 +40,839 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int INCORRECT_QUERY; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int LOGICAL_ERROR; +} + +MergeTreeIndexGranuleBloomFilter::MergeTreeIndexGranuleBloomFilter(size_t bits_per_row_, size_t hash_functions_, size_t index_columns_) + : bits_per_row(bits_per_row_), hash_functions(hash_functions_), bloom_filters(index_columns_) +{ + total_rows = 0; + for (size_t column = 0; column < index_columns_; ++column) + bloom_filters[column] = std::make_shared(bits_per_row, hash_functions, 0); +} + +MergeTreeIndexGranuleBloomFilter::MergeTreeIndexGranuleBloomFilter( + size_t bits_per_row_, size_t hash_functions_, const std::vector>& column_hashes_) + : bits_per_row(bits_per_row_), hash_functions(hash_functions_), bloom_filters(column_hashes_.size()) +{ + if (column_hashes_.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Granule_index_blocks empty or total_rows is zero."); + + size_t bloom_filter_max_size = 0; + for (const auto & column_hash : column_hashes_) + bloom_filter_max_size = std::max(bloom_filter_max_size, column_hash.size()); + + static size_t atom_size = 8; + + // If multiple columns are given, we will initialize all the bloom filters + // with the size of the highest-cardinality one. This is done for compatibility with + // existing binary serialization format + total_rows = bloom_filter_max_size; + size_t bytes_size = (bits_per_row * total_rows + atom_size - 1) / atom_size; + + for (size_t column = 0, columns = column_hashes_.size(); column < columns; ++column) + { + bloom_filters[column] = std::make_shared(bytes_size, hash_functions, 0); + fillingBloomFilter(bloom_filters[column], column_hashes_[column]); + } +} + +bool MergeTreeIndexGranuleBloomFilter::empty() const +{ + return !total_rows; +} + +void MergeTreeIndexGranuleBloomFilter::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) +{ + if (version != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown index version {}.", version); + + readVarUInt(total_rows, istr); + + static size_t atom_size = 8; + size_t bytes_size = (bits_per_row * total_rows + atom_size - 1) / atom_size; + size_t read_size = bytes_size; + for (auto & filter : bloom_filters) + { + filter->resize(bytes_size); +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + read_size = filter->getFilter().size() * sizeof(BloomFilter::UnderType); +#endif + istr.readStrict(reinterpret_cast(filter->getFilter().data()), read_size); + } +} + +void MergeTreeIndexGranuleBloomFilter::serializeBinary(WriteBuffer & ostr) const +{ + if (empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to write empty bloom filter index."); + + writeVarUInt(total_rows, ostr); + + static size_t atom_size = 8; + size_t write_size = (bits_per_row * total_rows + atom_size - 1) / atom_size; + for (const auto & bloom_filter : bloom_filters) + { +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + write_size = bloom_filter->getFilter().size() * sizeof(BloomFilter::UnderType); +#endif + ostr.write(reinterpret_cast(bloom_filter->getFilter().data()), write_size); + } +} + +void MergeTreeIndexGranuleBloomFilter::fillingBloomFilter(BloomFilterPtr & bf, const HashSet &hashes) const +{ + for (const auto & bf_base_hash : hashes) + for (size_t i = 0; i < hash_functions; ++i) + bf->addHashWithSeed(bf_base_hash.getKey(), BloomFilterHash::bf_hash_seed[i]); +} + +namespace +{ + +ColumnWithTypeAndName getPreparedSetInfo(const ConstSetPtr & prepared_set) +{ + if (prepared_set->getDataTypes().size() == 1) + return {prepared_set->getSetElements()[0], prepared_set->getElementsTypes()[0], "dummy"}; + + Columns set_elements; + for (auto & set_element : prepared_set->getSetElements()) + + set_elements.emplace_back(set_element->convertToFullColumnIfConst()); + + return {ColumnTuple::create(set_elements), std::make_shared(prepared_set->getElementsTypes()), "dummy"}; +} + +bool hashMatchesFilter(const BloomFilterPtr& bloom_filter, UInt64 hash, size_t hash_functions) +{ + return std::all_of(BloomFilterHash::bf_hash_seed, + BloomFilterHash::bf_hash_seed + hash_functions, + [&](const auto &hash_seed) + { + return bloom_filter->findHashWithSeed(hash, + hash_seed); + }); +} + +bool maybeTrueOnBloomFilter(const IColumn * hash_column, const BloomFilterPtr & bloom_filter, size_t hash_functions, bool match_all) +{ + const auto * const_column = typeid_cast(hash_column); + const auto * non_const_column = typeid_cast(hash_column); + + if (!const_column && !non_const_column) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Hash column must be Const or UInt64."); + + if (const_column) + { + return hashMatchesFilter(bloom_filter, + const_column->getValue(), + hash_functions); + } + + const ColumnUInt64::Container & hashes = non_const_column->getData(); + + if (match_all) + { + return std::all_of(hashes.begin(), + hashes.end(), + [&](const auto& hash_row) + { + return hashMatchesFilter(bloom_filter, + hash_row, + hash_functions); + }); + } + else + { + return std::any_of(hashes.begin(), + hashes.end(), + [&](const auto& hash_row) + { + return hashMatchesFilter(bloom_filter, + hash_row, + hash_functions); + }); + } +} + +} + +MergeTreeIndexConditionBloomFilter::MergeTreeIndexConditionBloomFilter( + const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, const Block & header_, size_t hash_functions_) + : WithContext(context_), header(header_), hash_functions(hash_functions_) +{ + if (!filter_actions_dag) + { + rpn.push_back(RPNElement::FUNCTION_UNKNOWN); + return; + } + + RPNBuilder builder( + filter_actions_dag->getOutputs().at(0), + context_, + [&](const RPNBuilderTreeNode & node, RPNElement & out) { return extractAtomFromTree(node, out); }); + rpn = std::move(builder).extractRPN(); +} + +bool MergeTreeIndexConditionBloomFilter::alwaysUnknownOrTrue() const +{ + std::vector rpn_stack; + + for (const auto & element : rpn) + { + if (element.function == RPNElement::FUNCTION_UNKNOWN + || element.function == RPNElement::ALWAYS_TRUE) + { + rpn_stack.push_back(true); + } + else if (element.function == RPNElement::FUNCTION_EQUALS + || element.function == RPNElement::FUNCTION_NOT_EQUALS + || element.function == RPNElement::FUNCTION_HAS + || element.function == RPNElement::FUNCTION_HAS_ANY + || element.function == RPNElement::FUNCTION_HAS_ALL + || element.function == RPNElement::FUNCTION_IN + || element.function == RPNElement::FUNCTION_NOT_IN + || element.function == RPNElement::ALWAYS_FALSE) + { + rpn_stack.push_back(false); + } + else if (element.function == RPNElement::FUNCTION_NOT) + { + // do nothing + } + else if (element.function == RPNElement::FUNCTION_AND) + { + auto arg1 = rpn_stack.back(); + rpn_stack.pop_back(); + auto arg2 = rpn_stack.back(); + rpn_stack.back() = arg1 && arg2; + } + else if (element.function == RPNElement::FUNCTION_OR) + { + auto arg1 = rpn_stack.back(); + rpn_stack.pop_back(); + auto arg2 = rpn_stack.back(); + rpn_stack.back() = arg1 || arg2; + } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected function type in KeyCondition::RPNElement"); + } + + return rpn_stack[0]; +} + +bool MergeTreeIndexConditionBloomFilter::mayBeTrueOnGranule(const MergeTreeIndexGranuleBloomFilter * granule) const +{ + std::vector rpn_stack; + const auto & filters = granule->getFilters(); + + for (const auto & element : rpn) + { + if (element.function == RPNElement::FUNCTION_UNKNOWN) + { + rpn_stack.emplace_back(true, true); + } + else if (element.function == RPNElement::FUNCTION_IN + || element.function == RPNElement::FUNCTION_NOT_IN + || element.function == RPNElement::FUNCTION_EQUALS + || element.function == RPNElement::FUNCTION_NOT_EQUALS + || element.function == RPNElement::FUNCTION_HAS + || element.function == RPNElement::FUNCTION_HAS_ANY + || element.function == RPNElement::FUNCTION_HAS_ALL) + { + bool match_rows = true; + bool match_all = element.function == RPNElement::FUNCTION_HAS_ALL; + const auto & predicate = element.predicate; + for (size_t index = 0; match_rows && index < predicate.size(); ++index) + { + const auto & query_index_hash = predicate[index]; + const auto & filter = filters[query_index_hash.first]; + const ColumnPtr & hash_column = query_index_hash.second; + + match_rows = maybeTrueOnBloomFilter(&*hash_column, + filter, + hash_functions, + match_all); + } + + rpn_stack.emplace_back(match_rows, true); + if (element.function == RPNElement::FUNCTION_NOT_EQUALS || element.function == RPNElement::FUNCTION_NOT_IN) + rpn_stack.back() = !rpn_stack.back(); + } + else if (element.function == RPNElement::FUNCTION_NOT) + { + rpn_stack.back() = !rpn_stack.back(); + } + else if (element.function == RPNElement::FUNCTION_OR) + { + auto arg1 = rpn_stack.back(); + rpn_stack.pop_back(); + auto arg2 = rpn_stack.back(); + rpn_stack.back() = arg1 | arg2; + } + else if (element.function == RPNElement::FUNCTION_AND) + { + auto arg1 = rpn_stack.back(); + rpn_stack.pop_back(); + auto arg2 = rpn_stack.back(); + rpn_stack.back() = arg1 & arg2; + } + else if (element.function == RPNElement::ALWAYS_TRUE) + { + rpn_stack.emplace_back(true, false); + } + else if (element.function == RPNElement::ALWAYS_FALSE) + { + rpn_stack.emplace_back(false, true); + } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected function type in KeyCondition::RPNElement"); + } + + if (rpn_stack.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected stack size in KeyCondition::mayBeTrueInRange"); + + return rpn_stack[0].can_be_true; +} + +bool MergeTreeIndexConditionBloomFilter::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNElement & out) +{ + { + Field const_value; + DataTypePtr const_type; + + if (node.tryGetConstant(const_value, const_type)) + { + if (const_value.getType() == Field::Types::UInt64) + { + out.function = const_value.get() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + return true; + } + + if (const_value.getType() == Field::Types::Int64) + { + out.function = const_value.get() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + return true; + } + + if (const_value.getType() == Field::Types::Float64) + { + out.function = const_value.get() != 0.0 ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + return true; + } + } + } + + return traverseFunction(node, out, nullptr /*parent*/); +} + +bool MergeTreeIndexConditionBloomFilter::traverseFunction(const RPNBuilderTreeNode & node, RPNElement & out, const RPNBuilderTreeNode * parent) +{ + bool maybe_useful = false; + + if (node.isFunction()) + { + const auto function = node.toFunctionNode(); + auto arguments_size = function.getArgumentsSize(); + auto function_name = function.getFunctionName(); + + for (size_t i = 0; i < arguments_size; ++i) + { + auto argument = function.getArgumentAt(i); + if (traverseFunction(argument, out, &node)) + maybe_useful = true; + } + + if (arguments_size != 2) + return false; + + auto lhs_argument = function.getArgumentAt(0); + auto rhs_argument = function.getArgumentAt(1); + + if (functionIsInOrGlobalInOperator(function_name)) + { + if (auto future_set = rhs_argument.tryGetPreparedSet(); future_set) + { + if (auto prepared_set = future_set->buildOrderedSetInplace(rhs_argument.getTreeContext().getQueryContext()); prepared_set) + { + if (prepared_set->hasExplicitSetElements()) + { + const auto prepared_info = getPreparedSetInfo(prepared_set); + if (traverseTreeIn(function_name, lhs_argument, prepared_set, prepared_info.type, prepared_info.column, out)) + maybe_useful = true; + } + } + } + } + else if (function_name == "equals" || + function_name == "notEquals" || + function_name == "has" || + function_name == "mapContains" || + function_name == "indexOf" || + function_name == "hasAny" || + function_name == "hasAll") + { + Field const_value; + DataTypePtr const_type; + + if (rhs_argument.tryGetConstant(const_value, const_type)) + { + if (traverseTreeEquals(function_name, lhs_argument, const_type, const_value, out, parent)) + maybe_useful = true; + } + else if (lhs_argument.tryGetConstant(const_value, const_type)) + { + if (traverseTreeEquals(function_name, rhs_argument, const_type, const_value, out, parent)) + maybe_useful = true; + } + } + } + + return maybe_useful; +} + +bool MergeTreeIndexConditionBloomFilter::traverseTreeIn( + const String & function_name, + const RPNBuilderTreeNode & key_node, + const ConstSetPtr & prepared_set, + const DataTypePtr & type, + const ColumnPtr & column, + RPNElement & out) +{ + auto key_node_column_name = key_node.getColumnName(); + + if (header.has(key_node_column_name)) + { + size_t row_size = column->size(); + size_t position = header.getPositionByName(key_node_column_name); + const DataTypePtr & index_type = header.getByPosition(position).type; + const auto & converted_column = castColumn(ColumnWithTypeAndName{column, type, ""}, index_type); + out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithColumn(index_type, converted_column, 0, row_size))); + + if (function_name == "in" || function_name == "globalIn") + out.function = RPNElement::FUNCTION_IN; + + if (function_name == "notIn" || function_name == "globalNotIn") + out.function = RPNElement::FUNCTION_NOT_IN; + + return true; + } + + if (key_node.isFunction()) + { + auto key_node_function = key_node.toFunctionNode(); + auto key_node_function_name = key_node_function.getFunctionName(); + size_t key_node_function_arguments_size = key_node_function.getArgumentsSize(); + + WhichDataType which(type); + + if (which.isTuple() && key_node_function_name == "tuple") + { + const auto & tuple_column = typeid_cast(column.get()); + const auto & tuple_data_type = typeid_cast(type.get()); + + if (tuple_data_type->getElements().size() != key_node_function_arguments_size || tuple_column->getColumns().size() != key_node_function_arguments_size) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal types of arguments of function {}", function_name); + + bool match_with_subtype = false; + const auto & sub_columns = tuple_column->getColumns(); + const auto & sub_data_types = tuple_data_type->getElements(); + + for (size_t index = 0; index < key_node_function_arguments_size; ++index) + match_with_subtype |= traverseTreeIn(function_name, key_node_function.getArgumentAt(index), nullptr, sub_data_types[index], sub_columns[index], out); + + return match_with_subtype; + } + + if (key_node_function_name == "arrayElement") + { + /** Try to parse arrayElement for mapKeys index. + * It is important to ignore keys like column_map['Key'] IN ('') because if key does not exists in map + * we return default value for arrayElement. + * + * We cannot skip keys that does not exist in map if comparison is with default type value because + * that way we skip necessary granules where map key does not exists. + */ + if (!prepared_set) + return false; + + auto default_column_to_check = type->createColumnConstWithDefaultValue(1)->convertToFullColumnIfConst(); + ColumnWithTypeAndName default_column_with_type_to_check { default_column_to_check, type, "" }; + ColumnsWithTypeAndName default_columns_with_type_to_check = {default_column_with_type_to_check}; + auto set_contains_default_value_predicate_column = prepared_set->execute(default_columns_with_type_to_check, false /*negative*/); + const auto & set_contains_default_value_predicate_column_typed = assert_cast(*set_contains_default_value_predicate_column); + bool set_contain_default_value = set_contains_default_value_predicate_column_typed.getData()[0]; + if (set_contain_default_value) + return false; + + auto first_argument = key_node_function.getArgumentAt(0); + const auto column_name = first_argument.getColumnName(); + auto map_keys_index_column_name = fmt::format("mapKeys({})", column_name); + auto map_values_index_column_name = fmt::format("mapValues({})", column_name); + + if (header.has(map_keys_index_column_name)) + { + /// For mapKeys we serialize key argument with bloom filter + + auto second_argument = key_node_function.getArgumentAt(1); + + Field constant_value; + DataTypePtr constant_type; + + if (second_argument.tryGetConstant(constant_value, constant_type)) + { + size_t position = header.getPositionByName(map_keys_index_column_name); + const DataTypePtr & index_type = header.getByPosition(position).type; + const DataTypePtr actual_type = BloomFilter::getPrimitiveType(index_type); + out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), constant_value))); + } + else + { + return false; + } + } + else if (header.has(map_values_index_column_name)) + { + /// For mapValues we serialize set with bloom filter + + size_t row_size = column->size(); + size_t position = header.getPositionByName(map_values_index_column_name); + const DataTypePtr & index_type = header.getByPosition(position).type; + const auto & array_type = assert_cast(*index_type); + const auto & array_nested_type = array_type.getNestedType(); + const auto & converted_column = castColumn(ColumnWithTypeAndName{column, type, ""}, array_nested_type); + out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithColumn(array_nested_type, converted_column, 0, row_size))); + } + else + { + return false; + } + + if (function_name == "in" || function_name == "globalIn") + out.function = RPNElement::FUNCTION_IN; + + if (function_name == "notIn" || function_name == "globalNotIn") + out.function = RPNElement::FUNCTION_NOT_IN; + + return true; + } + } + + return false; +} + + +static bool indexOfCanUseBloomFilter(const RPNBuilderTreeNode * parent) +{ + if (!parent) + return true; + + if (!parent->isFunction()) + return false; + + auto function = parent->toFunctionNode(); + auto function_name = function.getFunctionName(); + + /// `parent` is a function where `indexOf` is located. + /// Example: `indexOf(arr, x) = 1`, parent is a function named `equals`. + if (function_name == "and") + { + return true; + } + else if (function_name == "equals" /// notEquals is not applicable + || function_name == "greater" || function_name == "greaterOrEquals" + || function_name == "less" || function_name == "lessOrEquals") + { + size_t function_arguments_size = function.getArgumentsSize(); + if (function_arguments_size != 2) + return false; + + /// We don't allow constant expressions like `indexOf(arr, x) = 1 + 0` but it's negligible. + + /// We should return true when the corresponding expression implies that the array contains the element. + /// Example: when `indexOf(arr, x)` > 10 is written, it means that arr definitely should contain the element + /// (at least at 11th position but it does not matter). + + bool reversed = false; + Field constant_value; + DataTypePtr constant_type; + + if (function.getArgumentAt(0).tryGetConstant(constant_value, constant_type)) + { + reversed = true; + } + else if (function.getArgumentAt(1).tryGetConstant(constant_value, constant_type)) + { + } + else + { + return false; + } + + Field zero(0); + bool constant_equal_zero = applyVisitor(FieldVisitorAccurateEquals(), constant_value, zero); + + if (function_name == "equals" && !constant_equal_zero) + { + /// indexOf(...) = c, c != 0 + return true; + } + else if (function_name == "notEquals" && constant_equal_zero) + { + /// indexOf(...) != c, c = 0 + return true; + } + else if (function_name == (reversed ? "less" : "greater") && !applyVisitor(FieldVisitorAccurateLess(), constant_value, zero)) + { + /// indexOf(...) > c, c >= 0 + return true; + } + else if (function_name == (reversed ? "lessOrEquals" : "greaterOrEquals") && applyVisitor(FieldVisitorAccurateLess(), zero, constant_value)) + { + /// indexOf(...) >= c, c > 0 + return true; + } + + return false; + } + + return false; +} + + +bool MergeTreeIndexConditionBloomFilter::traverseTreeEquals( + const String & function_name, + const RPNBuilderTreeNode & key_node, + const DataTypePtr & value_type, + const Field & value_field, + RPNElement & out, + const RPNBuilderTreeNode * parent) +{ + auto key_column_name = key_node.getColumnName(); + + if (header.has(key_column_name)) + { + size_t position = header.getPositionByName(key_column_name); + const DataTypePtr & index_type = header.getByPosition(position).type; + const auto * array_type = typeid_cast(index_type.get()); + + if (function_name == "has" || function_name == "indexOf") + { + if (!array_type) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be an array.", function_name); + + /// We can treat `indexOf` function similar to `has`. + /// But it is little more cumbersome, compare: `has(arr, elem)` and `indexOf(arr, elem) != 0`. + /// The `parent` in this context is expected to be function `!=` (`notEquals`). + if (function_name == "has" || indexOfCanUseBloomFilter(parent)) + { + out.function = RPNElement::FUNCTION_HAS; + const DataTypePtr actual_type = BloomFilter::getPrimitiveType(array_type->getNestedType()); + auto converted_field = convertFieldToType(value_field, *actual_type, value_type.get()); + if (converted_field.isNull()) + return false; + + out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), converted_field))); + } + } + else if (function_name == "hasAny" || function_name == "hasAll") + { + if (!array_type) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be an array.", function_name); + + if (value_field.getType() != Field::Types::Array) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument for function {} must be an array.", function_name); + + const DataTypePtr actual_type = BloomFilter::getPrimitiveType(array_type->getNestedType()); + ColumnPtr column; + { + const bool is_nullable = actual_type->isNullable(); + auto mutable_column = actual_type->createColumn(); + + for (const auto & f : value_field.get()) + { + if ((f.isNull() && !is_nullable) || f.isDecimal(f.getType())) /// NOLINT(readability-static-accessed-through-instance) + return false; + + auto converted = convertFieldToType(f, *actual_type); + if (converted.isNull()) + return false; + + mutable_column->insert(converted); + } + + column = std::move(mutable_column); + } + + out.function = function_name == "hasAny" ? + RPNElement::FUNCTION_HAS_ANY : + RPNElement::FUNCTION_HAS_ALL; + out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithColumn(actual_type, column, 0, column->size()))); + } + else + { + if (array_type) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "An array type of bloom_filter supports only has(), indexOf(), and hasAny() functions."); + + out.function = function_name == "equals" ? RPNElement::FUNCTION_EQUALS : RPNElement::FUNCTION_NOT_EQUALS; + const DataTypePtr actual_type = BloomFilter::getPrimitiveType(index_type); + auto converted_field = convertFieldToType(value_field, *actual_type, value_type.get()); + if (converted_field.isNull()) + return false; + + out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), converted_field))); + } + + return true; + } + + if (function_name == "mapContains" || function_name == "has") + { + auto map_keys_index_column_name = fmt::format("mapKeys({})", key_column_name); + if (!header.has(map_keys_index_column_name)) + return false; + + size_t position = header.getPositionByName(map_keys_index_column_name); + const DataTypePtr & index_type = header.getByPosition(position).type; + const auto * array_type = typeid_cast(index_type.get()); + + if (!array_type) + return false; + + out.function = RPNElement::FUNCTION_HAS; + const DataTypePtr actual_type = BloomFilter::getPrimitiveType(array_type->getNestedType()); + auto converted_field = convertFieldToType(value_field, *actual_type, value_type.get()); + if (converted_field.isNull()) + return false; + + out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), converted_field))); + return true; + } + + if (key_node.isFunction()) + { + WhichDataType which(value_type); + + auto key_node_function = key_node.toFunctionNode(); + auto key_node_function_name = key_node_function.getFunctionName(); + size_t key_node_function_arguments_size = key_node_function.getArgumentsSize(); + + if (which.isTuple() && key_node_function_name == "tuple") + { + const Tuple & tuple = value_field.get(); + const auto * value_tuple_data_type = typeid_cast(value_type.get()); + + if (tuple.size() != key_node_function_arguments_size) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal types of arguments of function {}", function_name); + + bool match_with_subtype = false; + const DataTypes & subtypes = value_tuple_data_type->getElements(); + + for (size_t index = 0; index < tuple.size(); ++index) + match_with_subtype |= traverseTreeEquals(function_name, key_node_function.getArgumentAt(index), subtypes[index], tuple[index], out, &key_node); + + return match_with_subtype; + } + + if (key_node_function_name == "arrayElement" && (function_name == "equals" || function_name == "notEquals")) + { + /** Try to parse arrayElement for mapKeys index. + * It is important to ignore keys like column_map['Key'] = '' because if key does not exists in map + * we return default value for arrayElement. + * + * We cannot skip keys that does not exist in map if comparison is with default type value because + * that way we skip necessary granules where map key does not exists. + */ + if (value_field == value_type->getDefault()) + return false; + + auto first_argument = key_node_function.getArgumentAt(0); + const auto column_name = first_argument.getColumnName(); + + auto map_keys_index_column_name = fmt::format("mapKeys({})", column_name); + auto map_values_index_column_name = fmt::format("mapValues({})", column_name); + + size_t position = 0; + Field const_value = value_field; + DataTypePtr const_type; + + if (header.has(map_keys_index_column_name)) + { + position = header.getPositionByName(map_keys_index_column_name); + auto second_argument = key_node_function.getArgumentAt(1); + + if (!second_argument.tryGetConstant(const_value, const_type)) + return false; + } + else if (header.has(map_values_index_column_name)) + { + position = header.getPositionByName(map_values_index_column_name); + } + else + { + return false; + } + + out.function = function_name == "equals" ? RPNElement::FUNCTION_EQUALS : RPNElement::FUNCTION_NOT_EQUALS; + + const auto & index_type = header.getByPosition(position).type; + const auto actual_type = BloomFilter::getPrimitiveType(index_type); + out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), const_value))); + + return true; + } + } + + return false; +} + +MergeTreeIndexAggregatorBloomFilter::MergeTreeIndexAggregatorBloomFilter( + size_t bits_per_row_, size_t hash_functions_, const Names & columns_name_) + : bits_per_row(bits_per_row_), hash_functions(hash_functions_), index_columns_name(columns_name_), column_hashes(columns_name_.size()) +{ + assert(bits_per_row != 0); + assert(hash_functions != 0); +} + +bool MergeTreeIndexAggregatorBloomFilter::empty() const +{ + return !total_rows; +} + +MergeTreeIndexGranulePtr MergeTreeIndexAggregatorBloomFilter::getGranuleAndReset() +{ + const auto granule = std::make_shared(bits_per_row, hash_functions, column_hashes); + total_rows = 0; + column_hashes.clear(); + return granule; +} + +void MergeTreeIndexAggregatorBloomFilter::update(const Block & block, size_t * pos, size_t limit) +{ + if (*pos >= block.rows()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "The provided position is not less than the number of block rows. " + "Position: {}, Block rows: {}.", *pos, block.rows()); + + Block granule_index_block; + size_t max_read_rows = std::min(block.rows() - *pos, limit); + + for (size_t column = 0; column < index_columns_name.size(); ++column) + { + const auto & column_and_type = block.getByName(index_columns_name[column]); + auto index_column = BloomFilterHash::hashWithColumn(column_and_type.type, column_and_type.column, *pos, max_read_rows); + + const auto & index_col = checkAndGetColumn(index_column.get()); + const auto & index_data = index_col->getData(); + for (const auto & hash: index_data) + column_hashes[column].insert(hash); + } + + *pos += max_read_rows; + total_rows += max_read_rows; } MergeTreeIndexBloomFilter::MergeTreeIndexBloomFilter( diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.h b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.h index d6f4d6f2cf5..eeaa938551c 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.h +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.h @@ -1,13 +1,130 @@ #pragma once +#include +#include #include +#include #include -#include -#include namespace DB { +class MergeTreeIndexGranuleBloomFilter final : public IMergeTreeIndexGranule +{ +public: + MergeTreeIndexGranuleBloomFilter(size_t bits_per_row_, size_t hash_functions_, size_t index_columns_); + + MergeTreeIndexGranuleBloomFilter(size_t bits_per_row_, size_t hash_functions_, const std::vector> & column_hashes); + + bool empty() const override; + + void serializeBinary(WriteBuffer & ostr) const override; + void deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) override; + + const std::vector & getFilters() const { return bloom_filters; } + +private: + const size_t bits_per_row; + const size_t hash_functions; + + size_t total_rows = 0; + std::vector bloom_filters; + + void fillingBloomFilter(BloomFilterPtr & bf, const HashSet & hashes) const; +}; + +class MergeTreeIndexConditionBloomFilter final : public IMergeTreeIndexCondition, WithContext +{ +public: + struct RPNElement + { + enum Function + { + /// Atoms of a Boolean expression. + FUNCTION_EQUALS, + FUNCTION_NOT_EQUALS, + FUNCTION_HAS, + FUNCTION_HAS_ANY, + FUNCTION_HAS_ALL, + FUNCTION_IN, + FUNCTION_NOT_IN, + FUNCTION_UNKNOWN, /// Can take any value. + /// Operators of the logical expression. + FUNCTION_NOT, + FUNCTION_AND, + FUNCTION_OR, + /// Constants + ALWAYS_FALSE, + ALWAYS_TRUE, + }; + + RPNElement(Function function_ = FUNCTION_UNKNOWN) : function(function_) {} /// NOLINT + + Function function = FUNCTION_UNKNOWN; + std::vector> predicate; + }; + + MergeTreeIndexConditionBloomFilter(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, const Block & header_, size_t hash_functions_); + + bool alwaysUnknownOrTrue() const override; + + bool mayBeTrueOnGranule(MergeTreeIndexGranulePtr granule) const override + { + if (const auto & bf_granule = typeid_cast(granule.get())) + return mayBeTrueOnGranule(bf_granule); + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Requires bloom filter index granule."); + } + +private: + const Block & header; + const size_t hash_functions; + std::vector rpn; + + bool mayBeTrueOnGranule(const MergeTreeIndexGranuleBloomFilter * granule) const; + + bool extractAtomFromTree(const RPNBuilderTreeNode & node, RPNElement & out); + + bool traverseFunction(const RPNBuilderTreeNode & node, RPNElement & out, const RPNBuilderTreeNode * parent); + + bool traverseTreeIn( + const String & function_name, + const RPNBuilderTreeNode & key_node, + const ConstSetPtr & prepared_set, + const DataTypePtr & type, + const ColumnPtr & column, + RPNElement & out); + + bool traverseTreeEquals( + const String & function_name, + const RPNBuilderTreeNode & key_node, + const DataTypePtr & value_type, + const Field & value_field, + RPNElement & out, + const RPNBuilderTreeNode * parent); +}; + +class MergeTreeIndexAggregatorBloomFilter final : public IMergeTreeIndexAggregator +{ +public: + MergeTreeIndexAggregatorBloomFilter(size_t bits_per_row_, size_t hash_functions_, const Names & columns_name_); + + bool empty() const override; + + MergeTreeIndexGranulePtr getGranuleAndReset() override; + + void update(const Block & block, size_t * pos, size_t limit) override; + +private: + size_t bits_per_row; + size_t hash_functions; + const Names index_columns_name; + + std::vector> column_hashes; + size_t total_rows = 0; +}; + + class MergeTreeIndexBloomFilter final : public IMergeTreeIndex { public: diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp deleted file mode 100644 index 7ab90dac5b0..00000000000 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ /dev/null @@ -1,729 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int LOGICAL_ERROR; -} - -namespace -{ - -ColumnWithTypeAndName getPreparedSetInfo(const ConstSetPtr & prepared_set) -{ - if (prepared_set->getDataTypes().size() == 1) - return {prepared_set->getSetElements()[0], prepared_set->getElementsTypes()[0], "dummy"}; - - Columns set_elements; - for (auto & set_element : prepared_set->getSetElements()) - - set_elements.emplace_back(set_element->convertToFullColumnIfConst()); - - return {ColumnTuple::create(set_elements), std::make_shared(prepared_set->getElementsTypes()), "dummy"}; -} - -bool hashMatchesFilter(const BloomFilterPtr& bloom_filter, UInt64 hash, size_t hash_functions) -{ - return std::all_of(BloomFilterHash::bf_hash_seed, - BloomFilterHash::bf_hash_seed + hash_functions, - [&](const auto &hash_seed) - { - return bloom_filter->findHashWithSeed(hash, - hash_seed); - }); -} - -bool maybeTrueOnBloomFilter(const IColumn * hash_column, const BloomFilterPtr & bloom_filter, size_t hash_functions, bool match_all) -{ - const auto * const_column = typeid_cast(hash_column); - const auto * non_const_column = typeid_cast(hash_column); - - if (!const_column && !non_const_column) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Hash column must be Const or UInt64."); - - if (const_column) - { - return hashMatchesFilter(bloom_filter, - const_column->getValue(), - hash_functions); - } - - const ColumnUInt64::Container & hashes = non_const_column->getData(); - - if (match_all) - { - return std::all_of(hashes.begin(), - hashes.end(), - [&](const auto& hash_row) - { - return hashMatchesFilter(bloom_filter, - hash_row, - hash_functions); - }); - } - else - { - return std::any_of(hashes.begin(), - hashes.end(), - [&](const auto& hash_row) - { - return hashMatchesFilter(bloom_filter, - hash_row, - hash_functions); - }); - } -} - -} - -MergeTreeIndexConditionBloomFilter::MergeTreeIndexConditionBloomFilter( - const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, const Block & header_, size_t hash_functions_) - : WithContext(context_), header(header_), hash_functions(hash_functions_) -{ - if (!filter_actions_dag) - { - rpn.push_back(RPNElement::FUNCTION_UNKNOWN); - return; - } - - RPNBuilder builder( - filter_actions_dag->getOutputs().at(0), - context_, - [&](const RPNBuilderTreeNode & node, RPNElement & out) { return extractAtomFromTree(node, out); }); - rpn = std::move(builder).extractRPN(); -} - -bool MergeTreeIndexConditionBloomFilter::alwaysUnknownOrTrue() const -{ - std::vector rpn_stack; - - for (const auto & element : rpn) - { - if (element.function == RPNElement::FUNCTION_UNKNOWN - || element.function == RPNElement::ALWAYS_TRUE) - { - rpn_stack.push_back(true); - } - else if (element.function == RPNElement::FUNCTION_EQUALS - || element.function == RPNElement::FUNCTION_NOT_EQUALS - || element.function == RPNElement::FUNCTION_HAS - || element.function == RPNElement::FUNCTION_HAS_ANY - || element.function == RPNElement::FUNCTION_HAS_ALL - || element.function == RPNElement::FUNCTION_IN - || element.function == RPNElement::FUNCTION_NOT_IN - || element.function == RPNElement::ALWAYS_FALSE) - { - rpn_stack.push_back(false); - } - else if (element.function == RPNElement::FUNCTION_NOT) - { - // do nothing - } - else if (element.function == RPNElement::FUNCTION_AND) - { - auto arg1 = rpn_stack.back(); - rpn_stack.pop_back(); - auto arg2 = rpn_stack.back(); - rpn_stack.back() = arg1 && arg2; - } - else if (element.function == RPNElement::FUNCTION_OR) - { - auto arg1 = rpn_stack.back(); - rpn_stack.pop_back(); - auto arg2 = rpn_stack.back(); - rpn_stack.back() = arg1 || arg2; - } - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected function type in KeyCondition::RPNElement"); - } - - return rpn_stack[0]; -} - -bool MergeTreeIndexConditionBloomFilter::mayBeTrueOnGranule(const MergeTreeIndexGranuleBloomFilter * granule) const -{ - std::vector rpn_stack; - const auto & filters = granule->getFilters(); - - for (const auto & element : rpn) - { - if (element.function == RPNElement::FUNCTION_UNKNOWN) - { - rpn_stack.emplace_back(true, true); - } - else if (element.function == RPNElement::FUNCTION_IN - || element.function == RPNElement::FUNCTION_NOT_IN - || element.function == RPNElement::FUNCTION_EQUALS - || element.function == RPNElement::FUNCTION_NOT_EQUALS - || element.function == RPNElement::FUNCTION_HAS - || element.function == RPNElement::FUNCTION_HAS_ANY - || element.function == RPNElement::FUNCTION_HAS_ALL) - { - bool match_rows = true; - bool match_all = element.function == RPNElement::FUNCTION_HAS_ALL; - const auto & predicate = element.predicate; - for (size_t index = 0; match_rows && index < predicate.size(); ++index) - { - const auto & query_index_hash = predicate[index]; - const auto & filter = filters[query_index_hash.first]; - const ColumnPtr & hash_column = query_index_hash.second; - - match_rows = maybeTrueOnBloomFilter(&*hash_column, - filter, - hash_functions, - match_all); - } - - rpn_stack.emplace_back(match_rows, true); - if (element.function == RPNElement::FUNCTION_NOT_EQUALS || element.function == RPNElement::FUNCTION_NOT_IN) - rpn_stack.back() = !rpn_stack.back(); - } - else if (element.function == RPNElement::FUNCTION_NOT) - { - rpn_stack.back() = !rpn_stack.back(); - } - else if (element.function == RPNElement::FUNCTION_OR) - { - auto arg1 = rpn_stack.back(); - rpn_stack.pop_back(); - auto arg2 = rpn_stack.back(); - rpn_stack.back() = arg1 | arg2; - } - else if (element.function == RPNElement::FUNCTION_AND) - { - auto arg1 = rpn_stack.back(); - rpn_stack.pop_back(); - auto arg2 = rpn_stack.back(); - rpn_stack.back() = arg1 & arg2; - } - else if (element.function == RPNElement::ALWAYS_TRUE) - { - rpn_stack.emplace_back(true, false); - } - else if (element.function == RPNElement::ALWAYS_FALSE) - { - rpn_stack.emplace_back(false, true); - } - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected function type in KeyCondition::RPNElement"); - } - - if (rpn_stack.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected stack size in KeyCondition::mayBeTrueInRange"); - - return rpn_stack[0].can_be_true; -} - -bool MergeTreeIndexConditionBloomFilter::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNElement & out) -{ - { - Field const_value; - DataTypePtr const_type; - - if (node.tryGetConstant(const_value, const_type)) - { - if (const_value.getType() == Field::Types::UInt64) - { - out.function = const_value.get() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; - return true; - } - - if (const_value.getType() == Field::Types::Int64) - { - out.function = const_value.get() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; - return true; - } - - if (const_value.getType() == Field::Types::Float64) - { - out.function = const_value.get() != 0.0 ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; - return true; - } - } - } - - return traverseFunction(node, out, nullptr /*parent*/); -} - -bool MergeTreeIndexConditionBloomFilter::traverseFunction(const RPNBuilderTreeNode & node, RPNElement & out, const RPNBuilderTreeNode * parent) -{ - bool maybe_useful = false; - - if (node.isFunction()) - { - const auto function = node.toFunctionNode(); - auto arguments_size = function.getArgumentsSize(); - auto function_name = function.getFunctionName(); - - for (size_t i = 0; i < arguments_size; ++i) - { - auto argument = function.getArgumentAt(i); - if (traverseFunction(argument, out, &node)) - maybe_useful = true; - } - - if (arguments_size != 2) - return false; - - auto lhs_argument = function.getArgumentAt(0); - auto rhs_argument = function.getArgumentAt(1); - - if (functionIsInOrGlobalInOperator(function_name)) - { - if (auto future_set = rhs_argument.tryGetPreparedSet(); future_set) - { - if (auto prepared_set = future_set->buildOrderedSetInplace(rhs_argument.getTreeContext().getQueryContext()); prepared_set) - { - if (prepared_set->hasExplicitSetElements()) - { - const auto prepared_info = getPreparedSetInfo(prepared_set); - if (traverseTreeIn(function_name, lhs_argument, prepared_set, prepared_info.type, prepared_info.column, out)) - maybe_useful = true; - } - } - } - } - else if (function_name == "equals" || - function_name == "notEquals" || - function_name == "has" || - function_name == "mapContains" || - function_name == "indexOf" || - function_name == "hasAny" || - function_name == "hasAll") - { - Field const_value; - DataTypePtr const_type; - - if (rhs_argument.tryGetConstant(const_value, const_type)) - { - if (traverseTreeEquals(function_name, lhs_argument, const_type, const_value, out, parent)) - maybe_useful = true; - } - else if (lhs_argument.tryGetConstant(const_value, const_type)) - { - if (traverseTreeEquals(function_name, rhs_argument, const_type, const_value, out, parent)) - maybe_useful = true; - } - } - } - - return maybe_useful; -} - -bool MergeTreeIndexConditionBloomFilter::traverseTreeIn( - const String & function_name, - const RPNBuilderTreeNode & key_node, - const ConstSetPtr & prepared_set, - const DataTypePtr & type, - const ColumnPtr & column, - RPNElement & out) -{ - auto key_node_column_name = key_node.getColumnName(); - - if (header.has(key_node_column_name)) - { - size_t row_size = column->size(); - size_t position = header.getPositionByName(key_node_column_name); - const DataTypePtr & index_type = header.getByPosition(position).type; - const auto & converted_column = castColumn(ColumnWithTypeAndName{column, type, ""}, index_type); - out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithColumn(index_type, converted_column, 0, row_size))); - - if (function_name == "in" || function_name == "globalIn") - out.function = RPNElement::FUNCTION_IN; - - if (function_name == "notIn" || function_name == "globalNotIn") - out.function = RPNElement::FUNCTION_NOT_IN; - - return true; - } - - if (key_node.isFunction()) - { - auto key_node_function = key_node.toFunctionNode(); - auto key_node_function_name = key_node_function.getFunctionName(); - size_t key_node_function_arguments_size = key_node_function.getArgumentsSize(); - - WhichDataType which(type); - - if (which.isTuple() && key_node_function_name == "tuple") - { - const auto & tuple_column = typeid_cast(column.get()); - const auto & tuple_data_type = typeid_cast(type.get()); - - if (tuple_data_type->getElements().size() != key_node_function_arguments_size || tuple_column->getColumns().size() != key_node_function_arguments_size) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal types of arguments of function {}", function_name); - - bool match_with_subtype = false; - const auto & sub_columns = tuple_column->getColumns(); - const auto & sub_data_types = tuple_data_type->getElements(); - - for (size_t index = 0; index < key_node_function_arguments_size; ++index) - match_with_subtype |= traverseTreeIn(function_name, key_node_function.getArgumentAt(index), nullptr, sub_data_types[index], sub_columns[index], out); - - return match_with_subtype; - } - - if (key_node_function_name == "arrayElement") - { - /** Try to parse arrayElement for mapKeys index. - * It is important to ignore keys like column_map['Key'] IN ('') because if key does not exists in map - * we return default value for arrayElement. - * - * We cannot skip keys that does not exist in map if comparison is with default type value because - * that way we skip necessary granules where map key does not exists. - */ - if (!prepared_set) - return false; - - auto default_column_to_check = type->createColumnConstWithDefaultValue(1)->convertToFullColumnIfConst(); - ColumnWithTypeAndName default_column_with_type_to_check { default_column_to_check, type, "" }; - ColumnsWithTypeAndName default_columns_with_type_to_check = {default_column_with_type_to_check}; - auto set_contains_default_value_predicate_column = prepared_set->execute(default_columns_with_type_to_check, false /*negative*/); - const auto & set_contains_default_value_predicate_column_typed = assert_cast(*set_contains_default_value_predicate_column); - bool set_contain_default_value = set_contains_default_value_predicate_column_typed.getData()[0]; - if (set_contain_default_value) - return false; - - auto first_argument = key_node_function.getArgumentAt(0); - const auto column_name = first_argument.getColumnName(); - auto map_keys_index_column_name = fmt::format("mapKeys({})", column_name); - auto map_values_index_column_name = fmt::format("mapValues({})", column_name); - - if (header.has(map_keys_index_column_name)) - { - /// For mapKeys we serialize key argument with bloom filter - - auto second_argument = key_node_function.getArgumentAt(1); - - Field constant_value; - DataTypePtr constant_type; - - if (second_argument.tryGetConstant(constant_value, constant_type)) - { - size_t position = header.getPositionByName(map_keys_index_column_name); - const DataTypePtr & index_type = header.getByPosition(position).type; - const DataTypePtr actual_type = BloomFilter::getPrimitiveType(index_type); - out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), constant_value))); - } - else - { - return false; - } - } - else if (header.has(map_values_index_column_name)) - { - /// For mapValues we serialize set with bloom filter - - size_t row_size = column->size(); - size_t position = header.getPositionByName(map_values_index_column_name); - const DataTypePtr & index_type = header.getByPosition(position).type; - const auto & array_type = assert_cast(*index_type); - const auto & array_nested_type = array_type.getNestedType(); - const auto & converted_column = castColumn(ColumnWithTypeAndName{column, type, ""}, array_nested_type); - out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithColumn(array_nested_type, converted_column, 0, row_size))); - } - else - { - return false; - } - - if (function_name == "in" || function_name == "globalIn") - out.function = RPNElement::FUNCTION_IN; - - if (function_name == "notIn" || function_name == "globalNotIn") - out.function = RPNElement::FUNCTION_NOT_IN; - - return true; - } - } - - return false; -} - - -static bool indexOfCanUseBloomFilter(const RPNBuilderTreeNode * parent) -{ - if (!parent) - return true; - - if (!parent->isFunction()) - return false; - - auto function = parent->toFunctionNode(); - auto function_name = function.getFunctionName(); - - /// `parent` is a function where `indexOf` is located. - /// Example: `indexOf(arr, x) = 1`, parent is a function named `equals`. - if (function_name == "and") - { - return true; - } - else if (function_name == "equals" /// notEquals is not applicable - || function_name == "greater" || function_name == "greaterOrEquals" - || function_name == "less" || function_name == "lessOrEquals") - { - size_t function_arguments_size = function.getArgumentsSize(); - if (function_arguments_size != 2) - return false; - - /// We don't allow constant expressions like `indexOf(arr, x) = 1 + 0` but it's negligible. - - /// We should return true when the corresponding expression implies that the array contains the element. - /// Example: when `indexOf(arr, x)` > 10 is written, it means that arr definitely should contain the element - /// (at least at 11th position but it does not matter). - - bool reversed = false; - Field constant_value; - DataTypePtr constant_type; - - if (function.getArgumentAt(0).tryGetConstant(constant_value, constant_type)) - { - reversed = true; - } - else if (function.getArgumentAt(1).tryGetConstant(constant_value, constant_type)) - { - } - else - { - return false; - } - - Field zero(0); - bool constant_equal_zero = applyVisitor(FieldVisitorAccurateEquals(), constant_value, zero); - - if (function_name == "equals" && !constant_equal_zero) - { - /// indexOf(...) = c, c != 0 - return true; - } - else if (function_name == "notEquals" && constant_equal_zero) - { - /// indexOf(...) != c, c = 0 - return true; - } - else if (function_name == (reversed ? "less" : "greater") && !applyVisitor(FieldVisitorAccurateLess(), constant_value, zero)) - { - /// indexOf(...) > c, c >= 0 - return true; - } - else if (function_name == (reversed ? "lessOrEquals" : "greaterOrEquals") && applyVisitor(FieldVisitorAccurateLess(), zero, constant_value)) - { - /// indexOf(...) >= c, c > 0 - return true; - } - - return false; - } - - return false; -} - - -bool MergeTreeIndexConditionBloomFilter::traverseTreeEquals( - const String & function_name, - const RPNBuilderTreeNode & key_node, - const DataTypePtr & value_type, - const Field & value_field, - RPNElement & out, - const RPNBuilderTreeNode * parent) -{ - auto key_column_name = key_node.getColumnName(); - - if (header.has(key_column_name)) - { - size_t position = header.getPositionByName(key_column_name); - const DataTypePtr & index_type = header.getByPosition(position).type; - const auto * array_type = typeid_cast(index_type.get()); - - if (function_name == "has" || function_name == "indexOf") - { - if (!array_type) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be an array.", function_name); - - /// We can treat `indexOf` function similar to `has`. - /// But it is little more cumbersome, compare: `has(arr, elem)` and `indexOf(arr, elem) != 0`. - /// The `parent` in this context is expected to be function `!=` (`notEquals`). - if (function_name == "has" || indexOfCanUseBloomFilter(parent)) - { - out.function = RPNElement::FUNCTION_HAS; - const DataTypePtr actual_type = BloomFilter::getPrimitiveType(array_type->getNestedType()); - auto converted_field = convertFieldToType(value_field, *actual_type, value_type.get()); - if (converted_field.isNull()) - return false; - - out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), converted_field))); - } - } - else if (function_name == "hasAny" || function_name == "hasAll") - { - if (!array_type) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be an array.", function_name); - - if (value_field.getType() != Field::Types::Array) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument for function {} must be an array.", function_name); - - const DataTypePtr actual_type = BloomFilter::getPrimitiveType(array_type->getNestedType()); - ColumnPtr column; - { - const bool is_nullable = actual_type->isNullable(); - auto mutable_column = actual_type->createColumn(); - - for (const auto & f : value_field.get()) - { - if ((f.isNull() && !is_nullable) || f.isDecimal(f.getType())) /// NOLINT(readability-static-accessed-through-instance) - return false; - - auto converted = convertFieldToType(f, *actual_type); - if (converted.isNull()) - return false; - - mutable_column->insert(converted); - } - - column = std::move(mutable_column); - } - - out.function = function_name == "hasAny" ? - RPNElement::FUNCTION_HAS_ANY : - RPNElement::FUNCTION_HAS_ALL; - out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithColumn(actual_type, column, 0, column->size()))); - } - else - { - if (array_type) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "An array type of bloom_filter supports only has(), indexOf(), and hasAny() functions."); - - out.function = function_name == "equals" ? RPNElement::FUNCTION_EQUALS : RPNElement::FUNCTION_NOT_EQUALS; - const DataTypePtr actual_type = BloomFilter::getPrimitiveType(index_type); - auto converted_field = convertFieldToType(value_field, *actual_type, value_type.get()); - if (converted_field.isNull()) - return false; - - out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), converted_field))); - } - - return true; - } - - if (function_name == "mapContains" || function_name == "has") - { - auto map_keys_index_column_name = fmt::format("mapKeys({})", key_column_name); - if (!header.has(map_keys_index_column_name)) - return false; - - size_t position = header.getPositionByName(map_keys_index_column_name); - const DataTypePtr & index_type = header.getByPosition(position).type; - const auto * array_type = typeid_cast(index_type.get()); - - if (!array_type) - return false; - - out.function = RPNElement::FUNCTION_HAS; - const DataTypePtr actual_type = BloomFilter::getPrimitiveType(array_type->getNestedType()); - auto converted_field = convertFieldToType(value_field, *actual_type, value_type.get()); - if (converted_field.isNull()) - return false; - - out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), converted_field))); - return true; - } - - if (key_node.isFunction()) - { - WhichDataType which(value_type); - - auto key_node_function = key_node.toFunctionNode(); - auto key_node_function_name = key_node_function.getFunctionName(); - size_t key_node_function_arguments_size = key_node_function.getArgumentsSize(); - - if (which.isTuple() && key_node_function_name == "tuple") - { - const Tuple & tuple = value_field.get(); - const auto * value_tuple_data_type = typeid_cast(value_type.get()); - - if (tuple.size() != key_node_function_arguments_size) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal types of arguments of function {}", function_name); - - bool match_with_subtype = false; - const DataTypes & subtypes = value_tuple_data_type->getElements(); - - for (size_t index = 0; index < tuple.size(); ++index) - match_with_subtype |= traverseTreeEquals(function_name, key_node_function.getArgumentAt(index), subtypes[index], tuple[index], out, &key_node); - - return match_with_subtype; - } - - if (key_node_function_name == "arrayElement" && (function_name == "equals" || function_name == "notEquals")) - { - /** Try to parse arrayElement for mapKeys index. - * It is important to ignore keys like column_map['Key'] = '' because if key does not exists in map - * we return default value for arrayElement. - * - * We cannot skip keys that does not exist in map if comparison is with default type value because - * that way we skip necessary granules where map key does not exists. - */ - if (value_field == value_type->getDefault()) - return false; - - auto first_argument = key_node_function.getArgumentAt(0); - const auto column_name = first_argument.getColumnName(); - - auto map_keys_index_column_name = fmt::format("mapKeys({})", column_name); - auto map_values_index_column_name = fmt::format("mapValues({})", column_name); - - size_t position = 0; - Field const_value = value_field; - DataTypePtr const_type; - - if (header.has(map_keys_index_column_name)) - { - position = header.getPositionByName(map_keys_index_column_name); - auto second_argument = key_node_function.getArgumentAt(1); - - if (!second_argument.tryGetConstant(const_value, const_type)) - return false; - } - else if (header.has(map_values_index_column_name)) - { - position = header.getPositionByName(map_values_index_column_name); - } - else - { - return false; - } - - out.function = function_name == "equals" ? RPNElement::FUNCTION_EQUALS : RPNElement::FUNCTION_NOT_EQUALS; - - const auto & index_type = header.getByPosition(position).type; - const auto actual_type = BloomFilter::getPrimitiveType(index_type); - out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), const_value))); - - return true; - } - } - - return false; -} - -} diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h deleted file mode 100644 index 8029d6d405b..00000000000 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h +++ /dev/null @@ -1,87 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -class MergeTreeIndexConditionBloomFilter final : public IMergeTreeIndexCondition, WithContext -{ -public: - struct RPNElement - { - enum Function - { - /// Atoms of a Boolean expression. - FUNCTION_EQUALS, - FUNCTION_NOT_EQUALS, - FUNCTION_HAS, - FUNCTION_HAS_ANY, - FUNCTION_HAS_ALL, - FUNCTION_IN, - FUNCTION_NOT_IN, - FUNCTION_UNKNOWN, /// Can take any value. - /// Operators of the logical expression. - FUNCTION_NOT, - FUNCTION_AND, - FUNCTION_OR, - /// Constants - ALWAYS_FALSE, - ALWAYS_TRUE, - }; - - RPNElement(Function function_ = FUNCTION_UNKNOWN) : function(function_) {} /// NOLINT - - Function function = FUNCTION_UNKNOWN; - std::vector> predicate; - }; - - MergeTreeIndexConditionBloomFilter(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, const Block & header_, size_t hash_functions_); - - bool alwaysUnknownOrTrue() const override; - - bool mayBeTrueOnGranule(MergeTreeIndexGranulePtr granule) const override - { - if (const auto & bf_granule = typeid_cast(granule.get())) - return mayBeTrueOnGranule(bf_granule); - - throw Exception(ErrorCodes::LOGICAL_ERROR, "Requires bloom filter index granule."); - } - -private: - const Block & header; - const size_t hash_functions; - std::vector rpn; - - bool mayBeTrueOnGranule(const MergeTreeIndexGranuleBloomFilter * granule) const; - - bool extractAtomFromTree(const RPNBuilderTreeNode & node, RPNElement & out); - - bool traverseFunction(const RPNBuilderTreeNode & node, RPNElement & out, const RPNBuilderTreeNode * parent); - - bool traverseTreeIn( - const String & function_name, - const RPNBuilderTreeNode & key_node, - const ConstSetPtr & prepared_set, - const DataTypePtr & type, - const ColumnPtr & column, - RPNElement & out); - - bool traverseTreeEquals( - const String & function_name, - const RPNBuilderTreeNode & key_node, - const DataTypePtr & value_type, - const Field & value_field, - RPNElement & out, - const RPNBuilderTreeNode * parent); -}; - -} diff --git a/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp deleted file mode 100644 index 8355cac8033..00000000000 --- a/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp +++ /dev/null @@ -1,102 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -MergeTreeIndexGranuleBloomFilter::MergeTreeIndexGranuleBloomFilter(size_t bits_per_row_, size_t hash_functions_, size_t index_columns_) - : bits_per_row(bits_per_row_), hash_functions(hash_functions_), bloom_filters(index_columns_) -{ - total_rows = 0; - for (size_t column = 0; column < index_columns_; ++column) - bloom_filters[column] = std::make_shared(bits_per_row, hash_functions, 0); -} - -MergeTreeIndexGranuleBloomFilter::MergeTreeIndexGranuleBloomFilter( - size_t bits_per_row_, size_t hash_functions_, const std::vector>& column_hashes_) - : bits_per_row(bits_per_row_), hash_functions(hash_functions_), bloom_filters(column_hashes_.size()) -{ - if (column_hashes_.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Granule_index_blocks empty or total_rows is zero."); - - size_t bloom_filter_max_size = 0; - for (const auto & column_hash : column_hashes_) - bloom_filter_max_size = std::max(bloom_filter_max_size, column_hash.size()); - - static size_t atom_size = 8; - - // If multiple columns are given, we will initialize all the bloom filters - // with the size of the highest-cardinality one. This is done for compatibility with - // existing binary serialization format - total_rows = bloom_filter_max_size; - size_t bytes_size = (bits_per_row * total_rows + atom_size - 1) / atom_size; - - for (size_t column = 0, columns = column_hashes_.size(); column < columns; ++column) - { - bloom_filters[column] = std::make_shared(bytes_size, hash_functions, 0); - fillingBloomFilter(bloom_filters[column], column_hashes_[column]); - } -} - -bool MergeTreeIndexGranuleBloomFilter::empty() const -{ - return !total_rows; -} - -void MergeTreeIndexGranuleBloomFilter::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) -{ - if (version != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown index version {}.", version); - - readVarUInt(total_rows, istr); - - static size_t atom_size = 8; - size_t bytes_size = (bits_per_row * total_rows + atom_size - 1) / atom_size; - size_t read_size = bytes_size; - for (auto & filter : bloom_filters) - { - filter->resize(bytes_size); -#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - read_size = filter->getFilter().size() * sizeof(BloomFilter::UnderType); -#endif - istr.readStrict(reinterpret_cast(filter->getFilter().data()), read_size); - } -} - -void MergeTreeIndexGranuleBloomFilter::serializeBinary(WriteBuffer & ostr) const -{ - if (empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to write empty bloom filter index."); - - writeVarUInt(total_rows, ostr); - - static size_t atom_size = 8; - size_t write_size = (bits_per_row * total_rows + atom_size - 1) / atom_size; - for (const auto & bloom_filter : bloom_filters) - { -#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - write_size = bloom_filter->getFilter().size() * sizeof(BloomFilter::UnderType); -#endif - ostr.write(reinterpret_cast(bloom_filter->getFilter().data()), write_size); - } -} - -void MergeTreeIndexGranuleBloomFilter::fillingBloomFilter(BloomFilterPtr & bf, const HashSet &hashes) const -{ - for (const auto & bf_base_hash : hashes) - for (size_t i = 0; i < hash_functions; ++i) - bf->addHashWithSeed(bf_base_hash.getKey(), BloomFilterHash::bf_hash_seed[i]); -} - -} diff --git a/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.h b/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.h deleted file mode 100644 index a3434daa5a4..00000000000 --- a/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.h +++ /dev/null @@ -1,35 +0,0 @@ -#pragma once - -#include -#include -#include - -namespace DB -{ - -class MergeTreeIndexGranuleBloomFilter final : public IMergeTreeIndexGranule -{ -public: - MergeTreeIndexGranuleBloomFilter(size_t bits_per_row_, size_t hash_functions_, size_t index_columns_); - - MergeTreeIndexGranuleBloomFilter(size_t bits_per_row_, size_t hash_functions_, const std::vector> & column_hashes); - - bool empty() const override; - - void serializeBinary(WriteBuffer & ostr) const override; - void deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) override; - - const std::vector & getFilters() const { return bloom_filters; } - -private: - const size_t bits_per_row; - const size_t hash_functions; - - size_t total_rows = 0; - std::vector bloom_filters; - - void fillingBloomFilter(BloomFilterPtr & bf, const HashSet & hashes) const; -}; - - -} From 0ae422d33c0538f759c64444cc8b7a6f8f2a5b63 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 23 Apr 2024 14:11:12 +0000 Subject: [PATCH 043/192] Rename MergeTreeIndexFullText --> MergeTreeIndexBloomFilterText --- src/Interpreters/GinFilter.cpp | 2 +- .../MergeTree/MergeTreeIndexBloomFilter.cpp | 4 +- ....cpp => MergeTreeIndexBloomFilterText.cpp} | 56 +++++++++---------- ...Text.h => MergeTreeIndexBloomFilterText.h} | 28 +++++----- .../MergeTree/MergeTreeIndexInverted.h | 1 - src/Storages/MergeTree/MergeTreeIndices.cpp | 12 ++-- src/Storages/MergeTree/MergeTreeIndices.h | 6 +- .../tests/gtest_SplitTokenExtractor.cpp | 2 +- 8 files changed, 55 insertions(+), 56 deletions(-) rename src/Storages/MergeTree/{MergeTreeIndexFullText.cpp => MergeTreeIndexBloomFilterText.cpp} (92%) rename src/Storages/MergeTree/{MergeTreeIndexFullText.h => MergeTreeIndexBloomFilterText.h} (84%) diff --git a/src/Interpreters/GinFilter.cpp b/src/Interpreters/GinFilter.cpp index 5d823318313..1ce26ed1fd1 100644 --- a/src/Interpreters/GinFilter.cpp +++ b/src/Interpreters/GinFilter.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp index ed091022a91..4f25a014382 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp @@ -921,7 +921,7 @@ static void assertIndexColumnsType(const Block & header) } } -MergeTreeIndexPtr bloomFilterIndexCreatorNew( +MergeTreeIndexPtr bloomFilterIndexCreator( const IndexDescription & index) { double max_conflict_probability = 0.025; @@ -938,7 +938,7 @@ MergeTreeIndexPtr bloomFilterIndexCreatorNew( index, bits_per_row_and_size_of_hash_functions.first, bits_per_row_and_size_of_hash_functions.second); } -void bloomFilterIndexValidatorNew(const IndexDescription & index, bool attach) +void bloomFilterIndexValidator(const IndexDescription & index, bool attach) { assertIndexColumnsType(index.sample_block); diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.cpp similarity index 92% rename from src/Storages/MergeTree/MergeTreeIndexFullText.cpp rename to src/Storages/MergeTree/MergeTreeIndexBloomFilterText.cpp index 4cd616513ac..826b149cf01 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include @@ -32,7 +32,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -MergeTreeIndexGranuleFullText::MergeTreeIndexGranuleFullText( +MergeTreeIndexGranuleBloomFilterText::MergeTreeIndexGranuleBloomFilterText( const String & index_name_, size_t columns_number, const BloomFilterParameters & params_) @@ -44,7 +44,7 @@ MergeTreeIndexGranuleFullText::MergeTreeIndexGranuleFullText( { } -void MergeTreeIndexGranuleFullText::serializeBinary(WriteBuffer & ostr) const +void MergeTreeIndexGranuleBloomFilterText::serializeBinary(WriteBuffer & ostr) const { if (empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to write empty fulltext index {}.", backQuote(index_name)); @@ -53,7 +53,7 @@ void MergeTreeIndexGranuleFullText::serializeBinary(WriteBuffer & ostr) const ostr.write(reinterpret_cast(bloom_filter.getFilter().data()), params.filter_size); } -void MergeTreeIndexGranuleFullText::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) +void MergeTreeIndexGranuleBloomFilterText::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) { if (version != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown index version {}.", version); @@ -66,7 +66,7 @@ void MergeTreeIndexGranuleFullText::deserializeBinary(ReadBuffer & istr, MergeTr } -MergeTreeIndexAggregatorFullText::MergeTreeIndexAggregatorFullText( +MergeTreeIndexAggregatorBloomFilterText::MergeTreeIndexAggregatorBloomFilterText( const Names & index_columns_, const String & index_name_, const BloomFilterParameters & params_, @@ -76,20 +76,20 @@ MergeTreeIndexAggregatorFullText::MergeTreeIndexAggregatorFullText( , params(params_) , token_extractor(token_extractor_) , granule( - std::make_shared( + std::make_shared( index_name, index_columns.size(), params)) { } -MergeTreeIndexGranulePtr MergeTreeIndexAggregatorFullText::getGranuleAndReset() +MergeTreeIndexGranulePtr MergeTreeIndexAggregatorBloomFilterText::getGranuleAndReset() { - auto new_granule = std::make_shared( + auto new_granule = std::make_shared( index_name, index_columns.size(), params); new_granule.swap(granule); return new_granule; } -void MergeTreeIndexAggregatorFullText::update(const Block & block, size_t * pos, size_t limit) +void MergeTreeIndexAggregatorBloomFilterText::update(const Block & block, size_t * pos, size_t limit) { if (*pos >= block.rows()) throw Exception(ErrorCodes::LOGICAL_ERROR, "The provided position is not less than the number of block rows. " @@ -137,7 +137,7 @@ void MergeTreeIndexAggregatorFullText::update(const Block & block, size_t * pos, *pos += rows_read; } -MergeTreeConditionFullText::MergeTreeConditionFullText( +MergeTreeConditionBloomFilterText::MergeTreeConditionBloomFilterText( const ActionsDAGPtr & filter_actions_dag, ContextPtr context, const Block & index_sample_block, @@ -162,7 +162,7 @@ MergeTreeConditionFullText::MergeTreeConditionFullText( } /// Keep in-sync with MergeTreeConditionGinFilter::alwaysUnknownOrTrue -bool MergeTreeConditionFullText::alwaysUnknownOrTrue() const +bool MergeTreeConditionBloomFilterText::alwaysUnknownOrTrue() const { /// Check like in KeyCondition. std::vector rpn_stack; @@ -212,10 +212,10 @@ bool MergeTreeConditionFullText::alwaysUnknownOrTrue() const } /// Keep in-sync with MergeTreeIndexConditionGin::mayBeTrueOnTranuleInPart -bool MergeTreeConditionFullText::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const +bool MergeTreeConditionBloomFilterText::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const { - std::shared_ptr granule - = std::dynamic_pointer_cast(idx_granule); + std::shared_ptr granule + = std::dynamic_pointer_cast(idx_granule); if (!granule) throw Exception(ErrorCodes::LOGICAL_ERROR, "BloomFilter index condition got a granule with the wrong type."); @@ -323,13 +323,13 @@ bool MergeTreeConditionFullText::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx return rpn_stack[0].can_be_true; } -std::optional MergeTreeConditionFullText::getKeyIndex(const std::string & key_column_name) +std::optional MergeTreeConditionBloomFilterText::getKeyIndex(const std::string & key_column_name) { const auto it = std::ranges::find(index_columns, key_column_name); return it == index_columns.end() ? std::nullopt : std::make_optional(std::ranges::distance(index_columns.cbegin(), it)); } -bool MergeTreeConditionFullText::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNElement & out) +bool MergeTreeConditionBloomFilterText::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNElement & out) { { Field const_value; @@ -419,7 +419,7 @@ bool MergeTreeConditionFullText::extractAtomFromTree(const RPNBuilderTreeNode & return false; } -bool MergeTreeConditionFullText::traverseTreeEquals( +bool MergeTreeConditionBloomFilterText::traverseTreeEquals( const String & function_name, const RPNBuilderTreeNode & key_node, const DataTypePtr & value_type, @@ -638,7 +638,7 @@ bool MergeTreeConditionFullText::traverseTreeEquals( return false; } -bool MergeTreeConditionFullText::tryPrepareSetBloomFilter( +bool MergeTreeConditionBloomFilterText::tryPrepareSetBloomFilter( const RPNBuilderTreeNode & left_argument, const RPNBuilderTreeNode & right_argument, RPNElement & out) @@ -714,23 +714,23 @@ bool MergeTreeConditionFullText::tryPrepareSetBloomFilter( return true; } -MergeTreeIndexGranulePtr MergeTreeIndexFullText::createIndexGranule() const +MergeTreeIndexGranulePtr MergeTreeIndexBloomFilterText::createIndexGranule() const { - return std::make_shared(index.name, index.column_names.size(), params); + return std::make_shared(index.name, index.column_names.size(), params); } -MergeTreeIndexAggregatorPtr MergeTreeIndexFullText::createIndexAggregator(const MergeTreeWriterSettings & /*settings*/) const +MergeTreeIndexAggregatorPtr MergeTreeIndexBloomFilterText::createIndexAggregator(const MergeTreeWriterSettings & /*settings*/) const { - return std::make_shared(index.column_names, index.name, params, token_extractor.get()); + return std::make_shared(index.column_names, index.name, params, token_extractor.get()); } -MergeTreeIndexConditionPtr MergeTreeIndexFullText::createIndexCondition( +MergeTreeIndexConditionPtr MergeTreeIndexBloomFilterText::createIndexCondition( const ActionsDAGPtr & filter_dag, ContextPtr context) const { - return std::make_shared(filter_dag, context, index.sample_block, params, token_extractor.get()); + return std::make_shared(filter_dag, context, index.sample_block, params, token_extractor.get()); } -MergeTreeIndexPtr bloomFilterIndexCreator( +MergeTreeIndexPtr bloomFilterIndexTextCreator( const IndexDescription & index) { if (index.type == NgramTokenExtractor::getName()) @@ -743,7 +743,7 @@ MergeTreeIndexPtr bloomFilterIndexCreator( auto tokenizer = std::make_unique(n); - return std::make_shared(index, params, std::move(tokenizer)); + return std::make_shared(index, params, std::move(tokenizer)); } else if (index.type == SplitTokenExtractor::getName()) { @@ -754,7 +754,7 @@ MergeTreeIndexPtr bloomFilterIndexCreator( auto tokenizer = std::make_unique(); - return std::make_shared(index, params, std::move(tokenizer)); + return std::make_shared(index, params, std::move(tokenizer)); } else { @@ -762,7 +762,7 @@ MergeTreeIndexPtr bloomFilterIndexCreator( } } -void bloomFilterIndexValidator(const IndexDescription & index, bool /*attach*/) +void bloomFilterIndexTextValidator(const IndexDescription & index, bool /*attach*/) { for (const auto & index_data_type : index.data_types) { diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.h b/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.h similarity index 84% rename from src/Storages/MergeTree/MergeTreeIndexFullText.h rename to src/Storages/MergeTree/MergeTreeIndexBloomFilterText.h index e66f498ce1d..6fd969030df 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.h +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.h @@ -11,14 +11,14 @@ namespace DB { -struct MergeTreeIndexGranuleFullText final : public IMergeTreeIndexGranule +struct MergeTreeIndexGranuleBloomFilterText final : public IMergeTreeIndexGranule { - explicit MergeTreeIndexGranuleFullText( + explicit MergeTreeIndexGranuleBloomFilterText( const String & index_name_, size_t columns_number, const BloomFilterParameters & params_); - ~MergeTreeIndexGranuleFullText() override = default; + ~MergeTreeIndexGranuleBloomFilterText() override = default; void serializeBinary(WriteBuffer & ostr) const override; void deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) override; @@ -32,17 +32,17 @@ struct MergeTreeIndexGranuleFullText final : public IMergeTreeIndexGranule bool has_elems; }; -using MergeTreeIndexGranuleFullTextPtr = std::shared_ptr; +using MergeTreeIndexGranuleBloomFilterTextPtr = std::shared_ptr; -struct MergeTreeIndexAggregatorFullText final : IMergeTreeIndexAggregator +struct MergeTreeIndexAggregatorBloomFilterText final : IMergeTreeIndexAggregator { - explicit MergeTreeIndexAggregatorFullText( + explicit MergeTreeIndexAggregatorBloomFilterText( const Names & index_columns_, const String & index_name_, const BloomFilterParameters & params_, TokenExtractorPtr token_extractor_); - ~MergeTreeIndexAggregatorFullText() override = default; + ~MergeTreeIndexAggregatorBloomFilterText() override = default; bool empty() const override { return !granule || granule->empty(); } MergeTreeIndexGranulePtr getGranuleAndReset() override; @@ -54,21 +54,21 @@ struct MergeTreeIndexAggregatorFullText final : IMergeTreeIndexAggregator BloomFilterParameters params; TokenExtractorPtr token_extractor; - MergeTreeIndexGranuleFullTextPtr granule; + MergeTreeIndexGranuleBloomFilterTextPtr granule; }; -class MergeTreeConditionFullText final : public IMergeTreeIndexCondition +class MergeTreeConditionBloomFilterText final : public IMergeTreeIndexCondition { public: - MergeTreeConditionFullText( + MergeTreeConditionBloomFilterText( const ActionsDAGPtr & filter_actions_dag, ContextPtr context, const Block & index_sample_block, const BloomFilterParameters & params_, TokenExtractorPtr token_extactor_); - ~MergeTreeConditionFullText() override = default; + ~MergeTreeConditionBloomFilterText() override = default; bool alwaysUnknownOrTrue() const override; bool mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const override; @@ -146,10 +146,10 @@ private: RPN rpn; }; -class MergeTreeIndexFullText final : public IMergeTreeIndex +class MergeTreeIndexBloomFilterText final : public IMergeTreeIndex { public: - MergeTreeIndexFullText( + MergeTreeIndexBloomFilterText( const IndexDescription & index_, const BloomFilterParameters & params_, std::unique_ptr && token_extractor_) @@ -157,7 +157,7 @@ public: , params(params_) , token_extractor(std::move(token_extractor_)) {} - ~MergeTreeIndexFullText() override = default; + ~MergeTreeIndexBloomFilterText() override = default; MergeTreeIndexGranulePtr createIndexGranule() const override; MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override; diff --git a/src/Storages/MergeTree/MergeTreeIndexInverted.h b/src/Storages/MergeTree/MergeTreeIndexInverted.h index f3c1f37e364..bab4e122aa6 100644 --- a/src/Storages/MergeTree/MergeTreeIndexInverted.h +++ b/src/Storages/MergeTree/MergeTreeIndexInverted.h @@ -5,7 +5,6 @@ #include #include #include -#include #include namespace DB diff --git a/src/Storages/MergeTree/MergeTreeIndices.cpp b/src/Storages/MergeTree/MergeTreeIndices.cpp index 322cdd35afe..be8b4c795f0 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.cpp +++ b/src/Storages/MergeTree/MergeTreeIndices.cpp @@ -115,14 +115,14 @@ MergeTreeIndexFactory::MergeTreeIndexFactory() registerCreator("set", setIndexCreator); registerValidator("set", setIndexValidator); - registerCreator("ngrambf_v1", bloomFilterIndexCreator); - registerValidator("ngrambf_v1", bloomFilterIndexValidator); + registerCreator("ngrambf_v1", bloomFilterIndexTextCreator); + registerValidator("ngrambf_v1", bloomFilterIndexTextValidator); - registerCreator("tokenbf_v1", bloomFilterIndexCreator); - registerValidator("tokenbf_v1", bloomFilterIndexValidator); + registerCreator("tokenbf_v1", bloomFilterIndexTextCreator); + registerValidator("tokenbf_v1", bloomFilterIndexTextValidator); - registerCreator("bloom_filter", bloomFilterIndexCreatorNew); - registerValidator("bloom_filter", bloomFilterIndexValidatorNew); + registerCreator("bloom_filter", bloomFilterIndexCreator); + registerValidator("bloom_filter", bloomFilterIndexValidator); registerCreator("hypothesis", hypothesisIndexCreator); registerValidator("hypothesis", hypothesisIndexValidator); diff --git a/src/Storages/MergeTree/MergeTreeIndices.h b/src/Storages/MergeTree/MergeTreeIndices.h index 8fdadb4e5eb..900e6b6658c 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.h +++ b/src/Storages/MergeTree/MergeTreeIndices.h @@ -221,12 +221,12 @@ void minmaxIndexValidator(const IndexDescription & index, bool attach); MergeTreeIndexPtr setIndexCreator(const IndexDescription & index); void setIndexValidator(const IndexDescription & index, bool attach); +MergeTreeIndexPtr bloomFilterIndexTextCreator(const IndexDescription & index); +void bloomFilterIndexTextValidator(const IndexDescription & index, bool attach); + MergeTreeIndexPtr bloomFilterIndexCreator(const IndexDescription & index); void bloomFilterIndexValidator(const IndexDescription & index, bool attach); -MergeTreeIndexPtr bloomFilterIndexCreatorNew(const IndexDescription & index); -void bloomFilterIndexValidatorNew(const IndexDescription & index, bool attach); - MergeTreeIndexPtr hypothesisIndexCreator(const IndexDescription & index); void hypothesisIndexValidator(const IndexDescription & index, bool attach); diff --git a/src/Storages/tests/gtest_SplitTokenExtractor.cpp b/src/Storages/tests/gtest_SplitTokenExtractor.cpp index 62389639c11..e01673359bd 100644 --- a/src/Storages/tests/gtest_SplitTokenExtractor.cpp +++ b/src/Storages/tests/gtest_SplitTokenExtractor.cpp @@ -1,4 +1,4 @@ -#include +#include #include From 7c24d4f48de6b62faf76445e729eea536dbec010 Mon Sep 17 00:00:00 2001 From: Alexey Petrunyaka <167422282+pet74alex@users.noreply.github.com> Date: Wed, 24 Apr 2024 03:33:53 +0300 Subject: [PATCH 044/192] Update Russian version of uuid-functions.md --- .../sql-reference/functions/uuid-functions.md | 238 ++++++++++++++++++ 1 file changed, 238 insertions(+) diff --git a/docs/ru/sql-reference/functions/uuid-functions.md b/docs/ru/sql-reference/functions/uuid-functions.md index 65d13079ee8..8f41d2ab6f4 100644 --- a/docs/ru/sql-reference/functions/uuid-functions.md +++ b/docs/ru/sql-reference/functions/uuid-functions.md @@ -51,6 +51,166 @@ SELECT generateUUIDv4(1), generateUUIDv4(2) └──────────────────────────────────────┴──────────────────────────────────────┘ ``` +## generateUUIDv7 {#uuidv7-function-generate} + +Генерирует идентификатор [UUID версии 7](https://datatracker.ietf.org/doc/html/draft-peabody-dispatch-new-uuid-format-04). Генерируемый UUID состоит из 48-битной временной метки (Unix time в миллисекундах), маркеров версии 7 и варианта 2, и случайных данных в следующей последовательности: +``` + 0 1 2 3 + 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| unix_ts_ms | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| unix_ts_ms | ver | rand_a | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +|var| rand_b | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| rand_b | +└─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┘ +``` + +**Синтаксис** + +``` sql +generateUUIDv7([x]) +``` + +**Аргументы** + +- `x` — [выражение](../syntax.md#syntax-expressions), возвращающее значение одного из [поддерживаемых типов данных](../data-types/index.md#data_types). Значение используется, чтобы избежать [склейки одинаковых выражений](index.md#common-subexpression-elimination), если функция вызывается несколько раз в одном запросе. Необязательный параметр. + +**Возвращаемое значение** + +Значение типа [UUID](../../sql-reference/functions/uuid-functions.md). + +**Пример использования** + +Этот пример демонстрирует, как создать таблицу с UUID-колонкой и добавить в нее сгенерированный UUIDv7. + +``` sql +CREATE TABLE t_uuid (x UUID) ENGINE=TinyLog + +INSERT INTO t_uuid SELECT generateUUIDv7() + +SELECT * FROM t_uuid +``` + +``` text +┌────────────────────────────────────x─┐ +│ 018f05af-f4a8-778f-beee-1bedbc95c93b │ +└──────────────────────────────────────┘ +``` + +**Пример использования, для генерации нескольких значений в одной строке** + +```sql +SELECT generateUUIDv7(1), generateUUIDv7(7) +┌─generateUUIDv7(1)────────────────────┬─generateUUIDv7(2)────────────────────┐ +│ 018f05b1-8c2e-7567-a988-48d09606ae8c │ 018f05b1-8c2e-7946-895b-fcd7635da9a0 │ +└──────────────────────────────────────┴──────────────────────────────────────┘ +``` + +## generateUUIDv7WithCounter {#uuidv7withcounter-function-generate} + +Генерирует идентификатор [UUID версии 7](https://datatracker.ietf.org/doc/html/draft-peabody-dispatch-new-uuid-format-04). Генерируемый UUID состоит из 48-битной временной метки (Unix time в миллисекундах), маркеров версии 7 и варианта 2, монотонно возрастающего счётчика для данной временной метки и случайных данных в указанной ниже последовательности. Для каждой новой временной метки счётчик стартует с нового случайного значения, а для следующих UUIDv7 он увеличивается на единицу. В случае переполнения счётчика временная метка принудительно увеличивается на 1, и счётчик снова стартует со случайного значения. Монотонность возрастания счётчика для каждой временной метки гарантируется между всеми одновременно работающими функциями `generateUUIDv7WithCounter`. +``` + 0 1 2 3 + 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| unix_ts_ms | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| unix_ts_ms | ver | counter_high_bits | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +|var| counter_low_bits | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| rand_b | +└─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┘ +``` +**Синтаксис** + +``` sql +generateUUIDv7WithCounter([x]) +``` + +**Аргументы** + +- `x` — [выражение](../syntax.md#syntax-expressions), возвращающее значение одного из [поддерживаемых типов данных](../data-types/index.md#data_types). Значение используется, чтобы избежать [склейки одинаковых выражений](index.md#common-subexpression-elimination), если функция вызывается несколько раз в одном запросе. Необязательный параметр. + +**Возвращаемое значение** + +Значение типа [UUID](../../sql-reference/functions/uuid-functions.md). + +**Пример использования** + +Этот пример демонстрирует, как создать таблицу с UUID-колонкой и добавить в нее сгенерированный UUIDv7. + +``` sql +CREATE TABLE t_uuid (x UUID) ENGINE=TinyLog + +INSERT INTO t_uuid SELECT generateUUIDv7WithCounter() + +SELECT * FROM t_uuid +``` + +``` text +┌────────────────────────────────────x─┐ +│ 018f05c7-56e3-7ac3-93e9-1d93c4218e0e │ +└──────────────────────────────────────┘ +``` + +**Пример использования, для генерации нескольких значений в одной строке** + +```sql +SELECT generateUUIDv7WithCounter(1), generateUUIDv7WithCounter(7) +┌─generateUUIDv7WithCounter(1)─────────┬─generateUUIDv7WithCounter(2)─────────┐ +│ 018f05c9-4ab8-7b86-b64e-c9f03fbd45d1 │ 018f05c9-4ab8-7b86-b64e-c9f12efb7e16 │ +└──────────────────────────────────────┴──────────────────────────────────────┘ +``` + +## generateUUIDv7WithFastCounter {#uuidv7withfastcounter-function-generate} + +Генерирует идентификатор [UUID версии 7](https://datatracker.ietf.org/doc/html/draft-peabody-dispatch-new-uuid-format-04). Данная функция является ускоренным аналогом функции `generateUUIDv7WithCounter` за счёт потери гарантии монотонности счётчика при одной и той же метке времени между одновременно исполняемыми разными запросами. Монотонность счётчика гарантируется только в пределах одного треда, исполняющего данную функцию для генерации нескольких UUID. + +**Синтаксис** + +``` sql +generateUUIDv7WithFastCounter([x]) +``` + +**Аргументы** + +- `x` — [выражение](../syntax.md#syntax-expressions), возвращающее значение одного из [поддерживаемых типов данных](../data-types/index.md#data_types). Значение используется, чтобы избежать [склейки одинаковых выражений](index.md#common-subexpression-elimination), если функция вызывается несколько раз в одном запросе. Необязательный параметр. + +**Возвращаемое значение** + +Значение типа [UUID](../../sql-reference/functions/uuid-functions.md). + +**Пример использования** + +Этот пример демонстрирует, как создать таблицу с UUID-колонкой и добавить в нее сгенерированный UUIDv7. + +``` sql +CREATE TABLE t_uuid (x UUID) ENGINE=TinyLog + +INSERT INTO t_uuid SELECT generateUUIDv7WithFastCounter() + +SELECT * FROM t_uuid +``` + +``` text +┌────────────────────────────────────x─┐ +│ 018f05e2-e3b2-70cb-b8be-64b09b626d32 │ +└──────────────────────────────────────┘ +``` + +**Пример использования, для генерации нескольких значений в одной строке** + +```sql +SELECT generateUUIDv7WithFastCounter(1), generateUUIDv7WithFastCounter(7) +┌─generateUUIDv7WithFastCounter(1)─────┬─generateUUIDv7WithFastCounter(2)─────┐ +│ 018f05e1-14ee-7bc5-9906-207153b400b1 │ 018f05e1-14ee-7bc5-9906-2072b8e96758 │ +└──────────────────────────────────────┴──────────────────────────────────────┘ +``` + ## empty {#empty} Проверяет, является ли входной UUID пустым. @@ -259,6 +419,84 @@ SELECT └──────────────────┴──────────────────────────────────────┘ ``` +## UUIDToNum {#uuidtonum} + +Принимает UUID и возвращает в виде набора байт в [FixedString(16)](../../sql-reference/functions/uuid-functions.md). Также принимает необязательный второй параметр - вариант представления UUID, по умолчанию 1 - `Big-endian` (2 означает представление в формате `Microsoft`). Данная функция заменяет последовательность из двух отдельных функций `UUIDStringToNum(toString(uuid))`, так что промежуточная конвертация из UUID в String для извлечения набора байт из UUID не требуется. + +``` sql +UUIDToNum(UUID[, variant = 1]) +``` + +**Возвращаемое значение** + +FixedString(16) + +**Примеры использования** + +``` sql +SELECT + toUUID('612f3c40-5d3b-217e-707b-6a546a3d7b29') AS uuid, + UUIDToNum(uuid) AS bytes +``` + +``` text +┌─uuid─────────────────────────────────┬─bytes────────────┐ +│ 612f3c40-5d3b-217e-707b-6a546a3d7b29 │ a/<@];!~p{jTj={) │ +└──────────────────────────────────────┴──────────────────┘ +``` +``` sql +SELECT + toUUID('612f3c40-5d3b-217e-707b-6a546a3d7b29') AS uuid, + UUIDToNum(uuid, 2) AS bytes +``` + +```text +┌─uuid─────────────────────────────────┬─bytes────────────┐ +│ 612f3c40-5d3b-217e-707b-6a546a3d7b29 │ @ Date: Tue, 23 Apr 2024 14:19:56 +0000 Subject: [PATCH 045/192] Rename MergeTreeIndexInverted* to MergeTreeIndexFullText* --- .../mergetree-family/invertedindexes.md | 46 ++++++------- src/Core/Settings.h | 2 +- src/Interpreters/GinFilter.cpp | 2 +- .../MergeTreeDataPartWriterOnDisk.cpp | 8 +-- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 6 +- .../MergeTree/MergeTreeIndexBloomFilter.h | 5 ++ ...nverted.cpp => MergeTreeIndexFullText.cpp} | 66 +++++++++---------- ...dexInverted.h => MergeTreeIndexFullText.h} | 28 ++++---- src/Storages/MergeTree/MergeTreeIndices.cpp | 6 +- src/Storages/MergeTree/MergeTreeIndices.h | 4 +- src/Storages/MergeTree/MutateTask.cpp | 4 +- ...> 02346_fulltext_index_bug47393.reference} | 0 ....sql => 02346_fulltext_index_bug47393.sql} | 2 +- ...> 02346_fulltext_index_bug52019.reference} | 0 ....sql => 02346_fulltext_index_bug52019.sql} | 2 +- ...> 02346_fulltext_index_bug59039.reference} | 0 ....sql => 02346_fulltext_index_bug59039.sql} | 2 +- ...46_fulltext_index_detach_attach.reference} | 0 ...=> 02346_fulltext_index_detach_attach.sql} | 2 +- ..._fulltext_index_match_predicate.reference} | 0 ... 02346_fulltext_index_match_predicate.sql} | 2 +- ... => 02346_fulltext_index_search.reference} | 20 +++--- ...ch.sql => 02346_fulltext_index_search.sql} | 64 +++++++++--------- 23 files changed, 138 insertions(+), 133 deletions(-) rename src/Storages/MergeTree/{MergeTreeIndexInverted.cpp => MergeTreeIndexFullText.cpp} (93%) rename src/Storages/MergeTree/{MergeTreeIndexInverted.h => MergeTreeIndexFullText.h} (87%) rename tests/queries/0_stateless/{02346_inverted_index_bug47393.reference => 02346_fulltext_index_bug47393.reference} (100%) rename tests/queries/0_stateless/{02346_inverted_index_bug47393.sql => 02346_fulltext_index_bug47393.sql} (93%) rename tests/queries/0_stateless/{02346_inverted_index_bug52019.reference => 02346_fulltext_index_bug52019.reference} (100%) rename tests/queries/0_stateless/{02346_inverted_index_bug52019.sql => 02346_fulltext_index_bug52019.sql} (91%) rename tests/queries/0_stateless/{02346_inverted_index_bug59039.reference => 02346_fulltext_index_bug59039.reference} (100%) rename tests/queries/0_stateless/{02346_inverted_index_bug59039.sql => 02346_fulltext_index_bug59039.sql} (93%) rename tests/queries/0_stateless/{02346_inverted_index_detach_attach.reference => 02346_fulltext_index_detach_attach.reference} (100%) rename tests/queries/0_stateless/{02346_inverted_index_detach_attach.sql => 02346_fulltext_index_detach_attach.sql} (82%) rename tests/queries/0_stateless/{02346_inverted_index_match_predicate.reference => 02346_fulltext_index_match_predicate.reference} (100%) rename tests/queries/0_stateless/{02346_inverted_index_match_predicate.sql => 02346_fulltext_index_match_predicate.sql} (97%) rename tests/queries/0_stateless/{02346_inverted_index_search.reference => 02346_fulltext_index_search.reference} (74%) rename tests/queries/0_stateless/{02346_inverted_index_search.sql => 02346_fulltext_index_search.sql} (87%) diff --git a/docs/en/engines/table-engines/mergetree-family/invertedindexes.md b/docs/en/engines/table-engines/mergetree-family/invertedindexes.md index 7e5140b4c4f..9374f6a3ac1 100644 --- a/docs/en/engines/table-engines/mergetree-family/invertedindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/invertedindexes.md @@ -1,19 +1,19 @@ --- slug: /en/engines/table-engines/mergetree-family/invertedindexes -sidebar_label: Inverted Indexes +sidebar_label: Full-text Indexes description: Quickly find search terms in text. keywords: [full-text search, text search, inverted, index, indices] --- -# Full-text Search using Inverted Indexes [experimental] +# Full-text Search using Full-text Indexes [experimental] -Inverted indexes are an experimental type of [secondary indexes](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#available-types-of-indices) which provide fast text search +Full-text indexes are an experimental type of [secondary indexes](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#available-types-of-indices) which provide fast text search capabilities for [String](/docs/en/sql-reference/data-types/string.md) or [FixedString](/docs/en/sql-reference/data-types/fixedstring.md) -columns. The main idea of an inverted index is to store a mapping from "terms" to the rows which contain these terms. "Terms" are +columns. The main idea of a full-text index is to store a mapping from "terms" to the rows which contain these terms. "Terms" are tokenized cells of the string column. For example, the string cell "I will be a little late" is by default tokenized into six terms "I", "will", "be", "a", "little" and "late". Another kind of tokenizer is n-grams. For example, the result of 3-gram tokenization will be 21 terms "I w", " wi", "wil", "ill", "ll ", "l b", " be" etc. The more fine-granular the input strings are tokenized, the bigger but also the more -useful the resulting inverted index will be. +useful the resulting full-text index will be.