From 53d57ffb52542583aac6c772797aa611d83890c8 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Tue, 24 Nov 2020 14:24:48 +0000 Subject: [PATCH 01/46] Part movement between shards Integrate query deduplication from #17348 --- src/Core/Settings.h | 6 +- src/Parsers/ParserAlterQuery.cpp | 5 + src/Storages/DataDestinationType.h | 1 + src/Storages/MergeTree/MergeTreeData.cpp | 18 + src/Storages/MergeTree/MergeTreeData.h | 10 + .../MergeTree/MergeTreeDataSelectExecutor.cpp | 24 +- .../MergeTree/MergeTreeDataSelectExecutor.h | 1 + src/Storages/MergeTree/MergeTreeSettings.h | 1 + .../PartMovesBetweenShardsOrchestrator.cpp | 411 ++++++++++++++++++ .../PartMovesBetweenShardsOrchestrator.h | 158 +++++++ src/Storages/MergeTree/PinnedPartUUIDs.cpp | 36 ++ src/Storages/MergeTree/PinnedPartUUIDs.h | 27 ++ .../ReplicatedMergeTreeBlockOutputStream.cpp | 4 +- .../ReplicatedMergeTreeBlockOutputStream.h | 2 +- .../MergeTree/ReplicatedMergeTreeLogEntry.cpp | 20 + .../MergeTree/ReplicatedMergeTreeLogEntry.h | 13 + .../MergeTree/ReplicatedMergeTreeQueue.cpp | 25 ++ .../MergeTree/ReplicatedMergeTreeQueue.h | 4 + src/Storages/PartitionCommands.cpp | 10 +- src/Storages/PartitionCommands.h | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 178 ++++++++ src/Storages/StorageReplicatedMergeTree.h | 13 +- .../StorageSystemPartMovesBetweenShards.cpp | 135 ++++++ .../StorageSystemPartMovesBetweenShards.h | 27 ++ src/Storages/System/attachSystemTables.cpp | 2 + .../test_move_part_to_shard/__init__.py | 0 .../configs/merge_tree.xml | 6 + .../configs/remote_servers.xml | 26 ++ .../test_move_part_to_shard/test.py | 162 +++++++ .../configs/profiles.xml | 7 + .../test_query_deduplication/test.py | 9 +- 31 files changed, 1328 insertions(+), 14 deletions(-) create mode 100644 src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp create mode 100644 src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h create mode 100644 src/Storages/MergeTree/PinnedPartUUIDs.cpp create mode 100644 src/Storages/MergeTree/PinnedPartUUIDs.h create mode 100644 src/Storages/System/StorageSystemPartMovesBetweenShards.cpp create mode 100644 src/Storages/System/StorageSystemPartMovesBetweenShards.h create mode 100644 tests/integration/test_move_part_to_shard/__init__.py create mode 100644 tests/integration/test_move_part_to_shard/configs/merge_tree.xml create mode 100644 tests/integration/test_move_part_to_shard/configs/remote_servers.xml create mode 100644 tests/integration/test_move_part_to_shard/test.py create mode 100644 tests/integration/test_query_deduplication/configs/profiles.xml diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 51ea501b949..3489331acd4 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -440,7 +440,11 @@ class IColumn; \ M(Bool, optimize_rewrite_sum_if_to_count_if, true, "Rewrite sumIf() and sum(if()) function countIf() function when logically equivalent", 0) \ M(UInt64, insert_shard_id, 0, "If non zero, when insert into a distributed table, the data will be inserted into the shard `insert_shard_id` synchronously. Possible values range from 1 to `shards_number` of corresponding distributed table", 0) \ - M(Bool, allow_experimental_query_deduplication, false, "Allow sending parts' UUIDs for a query in order to deduplicate data parts if any", 0) \ + \ + /** Experimental feature for moving data between shards. */ \ + \ + M(Bool, allow_experimental_query_deduplication, false, "Experimental data deduplication for SELECT queries based on part UUIDs", 0) \ + M(Bool, experimental_query_deduplication_send_all_part_uuids, false, "If false only part UUIDs for currently moving parts are sent. If true all read part UUIDs are sent (useful only for testing).", 0) \ M(Bool, engine_file_empty_if_not_exists, false, "Allows to select data from a file engine table without file", 0) \ M(Bool, engine_file_truncate_on_insert, false, "Enables or disables truncate before insert in file engine tables", 0) \ M(Bool, allow_experimental_database_replicated, false, "Allow to create databases with Replicated engine", 0) \ diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index de524342fb4..36dd81e129c 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -79,6 +79,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_to_disk("TO DISK"); ParserKeyword s_to_volume("TO VOLUME"); ParserKeyword s_to_table("TO TABLE"); + ParserKeyword s_to_shard("TO SHARD"); ParserKeyword s_delete("DELETE"); ParserKeyword s_update("UPDATE"); @@ -295,6 +296,10 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected return false; command->move_destination_type = DataDestinationType::TABLE; } + else if (s_to_shard.ignore(pos)) + { + command->move_destination_type = DataDestinationType::SHARD; + } else return false; diff --git a/src/Storages/DataDestinationType.h b/src/Storages/DataDestinationType.h index 05d1d89c2b5..4729019b5cb 100644 --- a/src/Storages/DataDestinationType.h +++ b/src/Storages/DataDestinationType.h @@ -10,6 +10,7 @@ enum class DataDestinationType VOLUME, TABLE, DELETE, + SHARD, }; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 5d1400138b4..572df6486be 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -151,6 +151,7 @@ MergeTreeData::MergeTreeData( , log_name(table_id_.getNameForLogs()) , log(&Poco::Logger::get(log_name)) , storage_settings(std::move(storage_settings_)) + , pinned_part_uuids(std::make_shared()) , data_parts_by_info(data_parts_indexes.get()) , data_parts_by_state_and_info(data_parts_indexes.get()) , parts_mover(this) @@ -2920,6 +2921,11 @@ void MergeTreeData::movePartitionToVolume(const ASTPtr & partition, const String throw Exception("Cannot move parts because moves are manually disabled", ErrorCodes::ABORTED); } +void MergeTreeData::movePartitionToShard(const ASTPtr & /*partition*/, bool /*move_part*/, const String & /*to*/, const Context & /*query_context*/) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MOVE PARTITION TO SHARD is not supported by storage {}", getName()); +} + void MergeTreeData::fetchPartition( const ASTPtr & /*partition*/, const StorageMetadataPtr & /*metadata_snapshot*/, @@ -2969,10 +2975,16 @@ Pipe MergeTreeData::alterPartition( break; case PartitionCommand::MoveDestinationType::TABLE: + { checkPartitionCanBeDropped(command.partition); String dest_database = query_context->resolveDatabase(command.to_database); auto dest_storage = DatabaseCatalog::instance().getTable({dest_database, command.to_table}, query_context); movePartitionToTable(dest_storage, command.partition, query_context); + } + break; + + case PartitionCommand::MoveDestinationType::SHARD: + movePartitionToShard(command.partition, command.part, command.move_destination_name, query_context); break; } } @@ -4054,6 +4066,12 @@ catch (...) tryLogCurrentException(log, __PRETTY_FUNCTION__); } +StorageMergeTree::PinnedPartUUIDsPtr MergeTreeData::getPinnedPartUUIDs() const +{ + std::lock_guard lock(pinned_part_uuids_mutex); + return pinned_part_uuids; +} + MergeTreeData::CurrentlyMovingPartsTagger::CurrentlyMovingPartsTagger(MergeTreeMovingParts && moving_parts_, MergeTreeData & data_) : parts_to_move(std::move(moving_parts_)), data(data_) { diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 46c0014d9f7..eb7f479bf9b 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -20,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -127,6 +128,8 @@ public: using DataPartStates = std::initializer_list; using DataPartStateVector = std::vector; + using PinnedPartUUIDsPtr = std::shared_ptr; + constexpr static auto FORMAT_VERSION_FILE_NAME = "format_version.txt"; constexpr static auto DETACHED_DIR_NAME = "detached"; @@ -781,6 +784,8 @@ public: /// Mutex for currently_moving_parts mutable std::mutex moving_parts_mutex; + PinnedPartUUIDsPtr getPinnedPartUUIDs() const; + /// Return main processing background job, like merge/mutate/fetch and so on virtual std::optional getDataProcessingJob() = 0; /// Return job to move parts between disks/volumes and so on. @@ -835,6 +840,10 @@ protected: /// Use get and set to receive readonly versions. MultiVersion storage_settings; + /// Used to determine which UUIDs to send to root query executor for deduplication. + mutable std::shared_mutex pinned_part_uuids_mutex; + PinnedPartUUIDsPtr pinned_part_uuids; + /// Work with data parts struct TagByInfo{}; @@ -970,6 +979,7 @@ protected: virtual void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, ContextPtr context) = 0; /// Makes sense only for replicated tables + virtual void movePartitionToShard(const ASTPtr & partition, bool move_part, const String & to, const Context & query_context); virtual void fetchPartition( const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 8245364d87a..d94460f6d1c 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -291,7 +291,14 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( auto index_stats = std::make_unique(); if (query_context->getSettingsRef().allow_experimental_query_deduplication) - selectPartsToReadWithUUIDFilter(parts, part_values, minmax_idx_condition, minmax_columns_types, partition_pruner, max_block_numbers_to_read, query_context, part_filter_counters); + selectPartsToReadWithUUIDFilter( + parts, + part_values, + data.getPinnedPartUUIDs(), + minmax_idx_condition, + minmax_columns_types, partition_pruner, + max_block_numbers_to_read, + query_context, part_filter_counters); else selectPartsToRead(parts, part_values, minmax_idx_condition, minmax_columns_types, partition_pruner, max_block_numbers_to_read, part_filter_counters); @@ -1922,6 +1929,7 @@ void MergeTreeDataSelectExecutor::selectPartsToRead( void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( MergeTreeData::DataPartsVector & parts, const std::unordered_set & part_values, + MergeTreeData::PinnedPartUUIDsPtr pinned_part_uuids, const std::optional & minmax_idx_condition, const DataTypes & minmax_columns_types, std::optional & partition_pruner, @@ -1929,6 +1937,8 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( ContextPtr query_context, PartFilterCounters & counters) const { + const Settings & settings = query_context.getSettings(); + /// process_parts prepare parts that have to be read for the query, /// returns false if duplicated parts' UUID have been met auto select_parts = [&] (MergeTreeData::DataPartsVector & selected_parts) -> bool @@ -1985,9 +1995,12 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( /// populate UUIDs and exclude ignored parts if enabled if (part->uuid != UUIDHelpers::Nil) { - auto result = temp_part_uuids.insert(part->uuid); - if (!result.second) - throw Exception("Found a part with the same UUID on the same replica.", ErrorCodes::LOGICAL_ERROR); + if (settings.experimental_query_deduplication_send_all_part_uuids || pinned_part_uuids->contains(part->uuid)) + { + auto result = temp_part_uuids.insert(part->uuid); + if (!result.second) + throw Exception("Found a part with the same UUID on the same replica.", ErrorCodes::LOGICAL_ERROR); + } } selected_parts.push_back(part); @@ -2011,7 +2024,8 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( /// Process parts that have to be read for a query. auto needs_retry = !select_parts(parts); - /// If any duplicated part UUIDs met during the first step, try to ignore them in second pass + /// If any duplicated part UUIDs met during the first step, try to ignore them in second pass. + /// This may happen when `prefer_localhost_replica` is set and "distributed" stage runs in the same process with "remote" stage. if (needs_retry) { LOG_DEBUG(log, "Found duplicate uuids locally, will retry part selection without them"); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index d7193fbfbfa..6cb1295271b 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -150,6 +150,7 @@ private: void selectPartsToReadWithUUIDFilter( MergeTreeData::DataPartsVector & parts, const std::unordered_set & part_values, + MergeTreeData::PinnedPartUUIDsPtr pinned_part_uuids, const std::optional & minmax_idx_condition, const DataTypes & minmax_columns_types, std::optional & partition_pruner, diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index f422f00f4dc..ced28974849 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -127,6 +127,7 @@ struct Settings; M(UInt64, max_concurrent_queries, 0, "Max number of concurrently executed queries related to the MergeTree table (0 - disabled). Queries will still be limited by other max_concurrent_queries settings.", 0) \ M(UInt64, min_marks_to_honor_max_concurrent_queries, 0, "Minimal number of marks to honor the MergeTree-level's max_concurrent_queries (0 - disabled). Queries will still be limited by other max_concurrent_queries settings.", 0) \ M(UInt64, min_bytes_to_rebalance_partition_over_jbod, 0, "Minimal amount of bytes to enable part rebalance over JBOD array (0 - disabled).", 0) \ + M(UInt64, part_moves_between_shards_delay_seconds, 30, "Time to wait before/after moving parts between shards.", 0) \ \ /** Obsolete settings. Kept for backward compatibility only. */ \ M(UInt64, min_relative_delay_to_yield_leadership, 120, "Obsolete setting, does nothing.", 0) \ diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp new file mode 100644 index 00000000000..9247f2e2fde --- /dev/null +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp @@ -0,0 +1,411 @@ +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +PartMovesBetweenShardsOrchestrator::PartMovesBetweenShardsOrchestrator(StorageReplicatedMergeTree & storage_) + : storage(storage_) + , zookeeper_path(storage.zookeeper_path) + , logger_name(storage.getStorageID().getFullTableName() + " (PartMovesBetweenShardsOrchestrator)") + , log(&Poco::Logger::get(logger_name)) + , entries_znode_path(zookeeper_path + "/part_moves_shard") +{ + task = storage.global_context.getSchedulePool().createTask(logger_name, [this]{ run(); }); +} + +void PartMovesBetweenShardsOrchestrator::run() +{ + if (need_stop) + return; + + auto sleep_ms = 10; + + try + { + sync(); + + if (step()) + sync(); + else + sleep_ms = 3 * 1000; + } + catch (...) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + } + + + task->scheduleAfter(sleep_ms); +} + +void PartMovesBetweenShardsOrchestrator::shutdown() +{ + need_stop = true; + task->deactivate(); + LOG_TRACE(log, "PartMovesBetweenShardsOrchestrator thread finished"); +} + +void PartMovesBetweenShardsOrchestrator::sync() +{ + std::lock_guard lock(state_mutex); + + entries.clear(); + + auto zk = storage.getZooKeeper(); + + Strings task_names = zk->getChildren(entries_znode_path); + for (auto const & task_name : task_names) + { + PartMovesBetweenShardsOrchestrator::Entry e; + Coordination::Stat stat; + + e.znode_path = entries_znode_path + "/" + task_name; + + auto entry_str = zk->get(e.znode_path, &stat); + e.fromString(entry_str); + + e.version = stat.version; + e.znode_name = task_name; + + entries[task_name] = std::move(e); + } +} + +bool PartMovesBetweenShardsOrchestrator::step() +{ + if (!storage.is_leader) + return false; + + auto zk = storage.getZooKeeper(); + + std::optional entry_to_process; + + /// Try find an entry to process and copy it. + { + std::lock_guard lock(state_mutex); + + for (auto const & entry : entries | boost::adaptors::map_values) + { + if (entry.state.value == EntryState::DONE || entry.state.value == EntryState::CANCELLED) + continue; + + entry_to_process.emplace(entry); + break; + } + } + + if (!entry_to_process.has_value()) + return false; + + try + { + + /// Since some state transitions are long running (waiting on replicas acknowledgement we create this lock to avoid + /// other replicas trying to do the same work. All state transitions should be idempotent so is is safe to lose the + /// lock and have another replica retry. + /// + /// Note: This blocks all other entries from being executed. Technical debt. + auto entry_node_holder = zkutil::EphemeralNodeHolder::create(entry_to_process->znode_path + "/lock_holder", *zk, storage.replica_name); + } + catch (const Coordination::Exception & e) + { + if (e.code == Coordination::Error::ZNODEEXISTS) + { + LOG_DEBUG(log, "Task {} is being processed by another replica", entry_to_process->znode_name); + return false; + } + + throw; + } + + try + { + stepEntry(entry_to_process.value()); + } + catch (...) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + + Entry entry_copy = entry_to_process.value(); + entry_copy.last_exception_msg = getCurrentExceptionMessage(false); + entry_copy.update_time = std::time(nullptr); + zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); + + return false; + } + + return true; +} + +void PartMovesBetweenShardsOrchestrator::stepEntry(const Entry & entry) +{ + auto zk = storage.getZooKeeper(); + + switch (entry.state.value) + { + case EntryState::DONE: + break; + + case EntryState::CANCELLED: + break; + + case EntryState::TODO: + { + /// State transition. + Entry entry_copy = entry; + entry_copy.state = EntryState::SYNC_SOURCE; + entry_copy.update_time = std::time(nullptr); + zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); + } + break; + + case EntryState::SYNC_SOURCE: + { + { + /// Log entry. + Coordination::Requests ops; + ReplicatedMergeTreeLogEntryData log_entry; + log_entry.type = ReplicatedMergeTreeLogEntryData::SYNC_PINNED_PART_UUIDS; + log_entry.create_time = std::time(nullptr); + log_entry.source_replica = storage.replica_name; + ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/log", "", -1)); + ops.emplace_back(zkutil::makeCreateRequest( + zookeeper_path + "/log/log-", log_entry.toString(), zkutil::CreateMode::PersistentSequential)); + + Coordination::Responses responses; + Coordination::Error rc = zk->tryMulti(ops, responses); + zkutil::KeeperMultiException::check(rc, ops, responses); + + String log_znode_path = dynamic_cast(*responses.back()).path_created; + log_entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); + + storage.waitForAllReplicasToProcessLogEntry(log_entry, true); + } + + { + /// State transition. + Entry entry_copy = entry; + entry_copy.state = EntryState::SYNC_DESTINATION; + entry_copy.update_time = std::time(nullptr); + zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); + } + } + break; + + case EntryState::SYNC_DESTINATION: + { + { + /// Log entry. + Coordination::Requests ops; + ReplicatedMergeTreeLogEntryData log_entry; + log_entry.type = ReplicatedMergeTreeLogEntryData::SYNC_PINNED_PART_UUIDS; + log_entry.create_time = std::time(nullptr); + log_entry.source_replica = storage.replica_name; + log_entry.source_shard = zookeeper_path; + + ops.emplace_back(zkutil::makeSetRequest(entry.to_shard + "/log", "", -1)); + ops.emplace_back(zkutil::makeCreateRequest( + entry.to_shard + "/log/log-", log_entry.toString(), zkutil::CreateMode::PersistentSequential)); + + Coordination::Responses responses; + Coordination::Error rc = zk->tryMulti(ops, responses); + zkutil::KeeperMultiException::check(rc, ops, responses); + + String log_znode_path = dynamic_cast(*responses.back()).path_created; + log_entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); + + storage.waitForAllTableReplicasToProcessLogEntry(entry.to_shard, log_entry, true); + } + + { + /// State transition. + Entry entry_copy = entry; + entry_copy.state = EntryState::DESTINATION_FETCH; + entry_copy.update_time = std::time(nullptr); + zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); + } + } + break; + + case EntryState::DESTINATION_FETCH: + { + /// There is a chance that attach on destination will fail and this task will be left in the queue forever. + /// Make sure table structure doesn't change when there are part movements in progress. + /// + /// DESTINATION_FETCH is tricky from idempotency standpoint. We must ensure that no orchestrator + /// issues a CLONE_PART_FROM_SHARD entry after the source part is dropped. + /// + /// `makeSetRequest` on the entry is a sloppy mechanism for ensuring that the thread that makes the state + /// transition is the last one to issue a CLONE_PART_FROM_SHARD event. We assume here that if last event + /// was processed then all the ones prior to that succeeded as well. + /// + /// If we could somehow create just one entry in the log and record log entry name in the same + /// transaction so we could wait for it later the problem would be solved. + { + Coordination::Requests ops; + ops.emplace_back(zkutil::makeSetRequest(entry.znode_path, entry.toString(), entry.version)); + + /// Log entry. + ReplicatedMergeTreeLogEntryData log_entry; + log_entry.type = ReplicatedMergeTreeLogEntryData::CLONE_PART_FROM_SHARD; + log_entry.create_time = std::time(nullptr); + log_entry.new_part_name = entry.part_name; + log_entry.source_replica = storage.replica_name; + log_entry.source_shard = zookeeper_path; + log_entry.block_id = toString(entry.task_uuid); + ops.emplace_back(zkutil::makeSetRequest(entry.to_shard + "/log", "", -1)); + ops.emplace_back(zkutil::makeCreateRequest( + entry.to_shard + "/log/log-", log_entry.toString(), zkutil::CreateMode::PersistentSequential)); + + /// Submit. + Coordination::Responses responses; + Coordination::Error rc = zk->tryMulti(ops, responses); + zkutil::KeeperMultiException::check(rc, ops, responses); + + String log_znode_path = dynamic_cast(*responses.back()).path_created; + log_entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); + + storage.waitForAllTableReplicasToProcessLogEntry(entry.to_shard, log_entry, true); + } + + { + /// State transition. + Entry entry_copy = entry; + entry_copy.state = EntryState::SOURCE_DROP_PRE_DELAY; + entry_copy.update_time = std::time(nullptr); + zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version + 1); + } + } + break; + + case EntryState::SOURCE_DROP_PRE_DELAY: + { + std::this_thread::sleep_for(std::chrono::seconds(storage.getSettings()->part_moves_between_shards_delay_seconds)); + + /// State transition. + Entry entry_copy = entry; + entry_copy.state = EntryState::SOURCE_DROP; + entry_copy.update_time = std::time(nullptr); + zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); + } + break; + + case EntryState::SOURCE_DROP: + { + { + ReplicatedMergeTreeLogEntry log_entry; + if (storage.dropPart(zk, entry.part_name, log_entry,false, false)) + storage.waitForAllReplicasToProcessLogEntry(log_entry, true); + } + + { + /// State transition. + Entry entry_copy = entry; + entry_copy.state = EntryState::SOURCE_DROP_POST_DELAY; + entry_copy.update_time = std::time(nullptr); + zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); + } + } + break; + + case EntryState::SOURCE_DROP_POST_DELAY: + { + std::this_thread::sleep_for(std::chrono::seconds(storage.getSettings()->part_moves_between_shards_delay_seconds)); + + /// State transition. + Entry entry_copy = entry; + entry_copy.state = EntryState::REMOVE_UUID_PIN; + entry_copy.update_time = std::time(nullptr); + zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); + } + break; + + case EntryState::REMOVE_UUID_PIN: + { + { + PinnedPartUUIDs src_pins; + PinnedPartUUIDs dst_pins; + + { + String s = zk->get(zookeeper_path + "/pinned_part_uuids", &src_pins.stat); + src_pins.fromString(s); + } + + { + String s = zk->get(entry.to_shard + "/pinned_part_uuids", &dst_pins.stat); + dst_pins.fromString(s); + } + + src_pins.part_uuids.erase(entry.part_uuid); + dst_pins.part_uuids.erase(entry.part_uuid); + + Coordination::Requests ops; + ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/pinned_part_uuids", src_pins.toString(), src_pins.stat.version)); + ops.emplace_back(zkutil::makeSetRequest(entry.to_shard + "/pinned_part_uuids", dst_pins.toString(), dst_pins.stat.version)); + + zk->multi(ops); + } + + /// State transition. + Entry entry_copy = entry; + entry_copy.state = EntryState::DONE; + entry_copy.update_time = std::time(nullptr); + zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); + } + break; + } +} + +std::vector PartMovesBetweenShardsOrchestrator::getEntries() const +{ + std::lock_guard lock(state_mutex); + + auto res = std::vector(); + + for (const auto& e : entries) + res.push_back(e.second); + + return res; +} + +String PartMovesBetweenShardsOrchestrator::Entry::toString() const +{ + Poco::JSON::Object json; + + json.set(JSON_KEY_CREATE_TIME, DB::toString(create_time)); + json.set(JSON_KEY_UPDATE_TIME, DB::toString(update_time)); + json.set(JSON_KEY_TASK_UUID, DB::toString(task_uuid)); + json.set(JSON_KEY_PART_NAME, part_name); + json.set(JSON_KEY_PART_UUID, DB::toString(part_uuid)); + json.set(JSON_KEY_TO_SHARD, to_shard); + json.set(JSON_KEY_STATE, state.toString()); + json.set(JSON_KEY_LAST_EX_MSG, last_exception_msg); + + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + oss.exceptions(std::ios::failbit); + json.stringify(oss); + + return oss.str(); +} + +void PartMovesBetweenShardsOrchestrator::Entry::fromString(const String & buf) +{ + Poco::JSON::Parser parser; + auto json = parser.parse(buf).extract(); + + create_time = parseFromString(json->getValue(JSON_KEY_CREATE_TIME)); + update_time = parseFromString(json->getValue(JSON_KEY_UPDATE_TIME)); + task_uuid = parseFromString(json->getValue(JSON_KEY_TASK_UUID)); + part_name = json->getValue(JSON_KEY_PART_NAME); + part_uuid = parseFromString(json->getValue(JSON_KEY_PART_UUID)); + to_shard = json->getValue(JSON_KEY_TO_SHARD); + state.value = EntryState::fromString(json->getValue(JSON_KEY_STATE)); + last_exception_msg = json->getValue(JSON_KEY_LAST_EX_MSG); +} + +} diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h new file mode 100644 index 00000000000..efbb140c56a --- /dev/null +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h @@ -0,0 +1,158 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +class StorageReplicatedMergeTree; + +/// Cross shard part movement workflow orchestration. +class PartMovesBetweenShardsOrchestrator +{ +public: + struct EntryState + { + enum Value + { + TODO, + SYNC_SOURCE, + SYNC_DESTINATION, + DESTINATION_FETCH, + SOURCE_DROP_PRE_DELAY, + SOURCE_DROP, + SOURCE_DROP_POST_DELAY, + REMOVE_UUID_PIN, + DONE, + CANCELLED, + }; + + EntryState(): value(TODO) {} + EntryState(Value value_): value(value_) {} + + Value value; + + String toString() const + { + switch (value) + { + case TODO: return "TODO"; + case SYNC_SOURCE: return "SYNC_SOURCE"; + case SYNC_DESTINATION: return "SYNC_DESTINATION"; + case DESTINATION_FETCH: return "DESTINATION_FETCH"; + case SOURCE_DROP_PRE_DELAY: return "SOURCE_DROP_PRE_DELAY"; + case SOURCE_DROP: return "SOURCE_DROP"; + case SOURCE_DROP_POST_DELAY: return "SOURCE_DROP_POST_DELAY"; + case REMOVE_UUID_PIN: return "REMOVE_UUID_PIN"; + case DONE: return "DONE"; + case CANCELLED: return "CANCELLED"; + } + + throw Exception("Unknown EntryState: " + DB::toString(value), ErrorCodes::LOGICAL_ERROR); + } + + static EntryState::Value fromString(String in) + { + if (in == "TODO") return TODO; + else if (in == "SYNC_SOURCE") return SYNC_SOURCE; + else if (in == "SYNC_DESTINATION") return SYNC_DESTINATION; + else if (in == "DESTINATION_FETCH") return DESTINATION_FETCH; + else if (in == "SOURCE_DROP_PRE_DELAY") return SOURCE_DROP_PRE_DELAY; + else if (in == "SOURCE_DROP") return SOURCE_DROP; + else if (in == "SOURCE_DROP_POST_DELAY") return SOURCE_DROP_POST_DELAY; + else if (in == "REMOVE_UUID_PIN") return REMOVE_UUID_PIN; + else if (in == "DONE") return DONE; + else if (in == "CANCELLED") return CANCELLED; + else throw Exception("Unknown state: " + in, ErrorCodes::LOGICAL_ERROR); + } + }; + + struct Entry + { + friend class PartMovesBetweenShardsOrchestrator; + + time_t create_time = 0; + time_t update_time = 0; + + /// Globally unique identifier used for attaching parts on destination. + /// Using `part_uuid` results in part names being reused when moving parts back and forth. + UUID task_uuid; + + String part_name; + UUID part_uuid; + String to_shard; + + EntryState state; + + String last_exception_msg; + + String znode_name; + + private: + /// Transient value for CAS. + uint32_t version; + + String znode_path; + + public: + String toString() const; + void fromString(const String & buf); + }; + +private: + static constexpr auto JSON_KEY_CREATE_TIME = "create_time"; + static constexpr auto JSON_KEY_UPDATE_TIME = "update_time"; + static constexpr auto JSON_KEY_TASK_UUID = "task_uuid"; + static constexpr auto JSON_KEY_PART_NAME = "part_name"; + static constexpr auto JSON_KEY_PART_UUID = "part_uuid"; + static constexpr auto JSON_KEY_TO_SHARD = "to_shard"; + static constexpr auto JSON_KEY_STATE = "state"; + static constexpr auto JSON_KEY_LAST_EX_MSG = "last_exception"; + +public: + PartMovesBetweenShardsOrchestrator(StorageReplicatedMergeTree & storage_); + + void start() { task->activateAndSchedule(); } + void wakeup() { task->schedule(); } + void shutdown(); + + void sync(); + + /// We could have one thread per Entry and worry about concurrency issues. + /// Or we could have a single thread trying to run one step at a time. + bool step(); + + std::vector getEntries() const; + +private: + void run(); + void stepEntry(const Entry & entry); + +private: + StorageReplicatedMergeTree & storage; + + String zookeeper_path; + String logger_name; + Poco::Logger * log = nullptr; + std::atomic need_stop{false}; + + BackgroundSchedulePool::TaskHolder task; + + mutable std::mutex state_mutex; + std::map entries; + +public: + String entries_znode_path; +}; + +} diff --git a/src/Storages/MergeTree/PinnedPartUUIDs.cpp b/src/Storages/MergeTree/PinnedPartUUIDs.cpp new file mode 100644 index 00000000000..1302d492c8c --- /dev/null +++ b/src/Storages/MergeTree/PinnedPartUUIDs.cpp @@ -0,0 +1,36 @@ +#include "PinnedPartUUIDs.h" +#include +#include +#include +#include +#include + +namespace DB +{ + +String PinnedPartUUIDs::toString() const +{ + std::vector vec(part_uuids.begin(), part_uuids.end()); + + Poco::JSON::Object json; + json.set(JSON_KEY_UUIDS, DB::toString(vec)); + + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + oss.exceptions(std::ios::failbit); + json.stringify(oss); + + return oss.str(); +} + +void PinnedPartUUIDs::fromString(const String & buf) +{ + Poco::JSON::Parser parser; + auto json = parser.parse(buf).extract(); + + std::vector vec = parseFromString>(json->getValue(PinnedPartUUIDs::JSON_KEY_UUIDS)); + + part_uuids.clear(); + std::copy(vec.begin(), vec.end(), std::inserter(part_uuids, part_uuids.begin())); +} + +} diff --git a/src/Storages/MergeTree/PinnedPartUUIDs.h b/src/Storages/MergeTree/PinnedPartUUIDs.h new file mode 100644 index 00000000000..a8f6c1fceda --- /dev/null +++ b/src/Storages/MergeTree/PinnedPartUUIDs.h @@ -0,0 +1,27 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +struct PinnedPartUUIDs +{ + std::set part_uuids; + Coordination::Stat stat{}; + + bool contains(const UUID & part_uuid) const + { + return part_uuids.contains(part_uuid); + } + + String toString() const; + void fromString(const String & buf); + +private: + static constexpr auto JSON_KEY_UUIDS = "part_uuids"; +}; + +} diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index df4f9124980..29342cf4d9b 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -183,7 +183,7 @@ void ReplicatedMergeTreeBlockOutputStream::write(const Block & block) } -void ReplicatedMergeTreeBlockOutputStream::writeExistingPart(MergeTreeData::MutableDataPartPtr & part) +void ReplicatedMergeTreeBlockOutputStream::writeExistingPart(MergeTreeData::MutableDataPartPtr & part, String block_id) { last_block_is_duplicate = false; @@ -199,7 +199,7 @@ void ReplicatedMergeTreeBlockOutputStream::writeExistingPart(MergeTreeData::Muta try { - commitPart(zookeeper, part, ""); + commitPart(zookeeper, part, block_id); PartLog::addNewPart(storage.getContext(), part, watch.elapsed()); } catch (...) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h index 6ea16491d64..314e6b92d87 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h @@ -40,7 +40,7 @@ public: void write(const Block & block) override; /// For ATTACHing existing data on filesystem. - void writeExistingPart(MergeTreeData::MutableDataPartPtr & part); + void writeExistingPart(MergeTreeData::MutableDataPartPtr & part, String block_id = ""); /// For proper deduplication in MaterializedViews bool lastBlockIsDuplicate() const diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp index 7d8ba0e4a30..88570ac33ef 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp @@ -53,6 +53,12 @@ void ReplicatedMergeTreeLogEntryData::writeText(WriteBuffer & out) const out << "get\n" << new_part_name; break; + case CLONE_PART_FROM_SHARD: + out << "clone_part_from_shard\n" + << new_part_name << "\n" + << "source_shard: " << source_shard; + break; + case ATTACH_PART: out << "attach\n" << new_part_name << "\n" << "part_checksum: " << part_checksum; @@ -141,6 +147,10 @@ void ReplicatedMergeTreeLogEntryData::writeText(WriteBuffer & out) const out << metadata_str; break; + case SYNC_PINNED_PART_UUIDS: + out << "sync_pinned_part_uuids\n"; + break; + default: throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown log entry type: {}", static_cast(type)); } @@ -305,6 +315,16 @@ void ReplicatedMergeTreeLogEntryData::readText(ReadBuffer & in) metadata_str.resize(metadata_size); in.readStrict(&metadata_str[0], metadata_size); } + else if (type_str == "sync_pinned_part_uuids") + { + type = SYNC_PINNED_PART_UUIDS; + } + else if (type_str == "clone_part_from_shard") + { + type = CLONE_PART_FROM_SHARD; + in >> new_part_name; + in >> "\nsource_shard: " >> source_shard; + } if (!trailing_newline_found) in >> "\n"; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h index 309120560e7..82748dcd4b0 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h @@ -41,6 +41,8 @@ struct ReplicatedMergeTreeLogEntryData REPLACE_RANGE, /// Drop certain range of partitions and replace them by new ones MUTATE_PART, /// Apply one or several mutations to the part. ALTER_METADATA, /// Apply alter modification according to global /metadata and /columns paths + SYNC_PINNED_PART_UUIDS, /// Synchronization point for ensuring that all replicas have up to date in-memory state. + CLONE_PART_FROM_SHARD, /// Clone part from another shard. }; static String typeToString(Type type) @@ -56,6 +58,8 @@ struct ReplicatedMergeTreeLogEntryData case ReplicatedMergeTreeLogEntryData::REPLACE_RANGE: return "REPLACE_RANGE"; case ReplicatedMergeTreeLogEntryData::MUTATE_PART: return "MUTATE_PART"; case ReplicatedMergeTreeLogEntryData::ALTER_METADATA: return "ALTER_METADATA"; + case ReplicatedMergeTreeLogEntryData::SYNC_PINNED_PART_UUIDS: return "SYNC_PINNED_PART_UUIDS"; + case ReplicatedMergeTreeLogEntryData::CLONE_PART_FROM_SHARD: return "CLONE_PART_FROM_SHARD"; default: throw Exception("Unknown log entry type: " + DB::toString(type), ErrorCodes::LOGICAL_ERROR); } @@ -74,6 +78,7 @@ struct ReplicatedMergeTreeLogEntryData Type type = EMPTY; String source_replica; /// Empty string means that this entry was added to the queue immediately, and not copied from the log. + String source_shard; String part_checksum; /// Part checksum for ATTACH_PART, empty otherwise. @@ -150,6 +155,14 @@ struct ReplicatedMergeTreeLogEntryData return res; } + /// Doesn't produce any part. + if (type == SYNC_PINNED_PART_UUIDS) + return {}; + + /// Doesn't produce any part by itself. + if (type == CLONE_PART_FROM_SHARD) + return {}; + return {new_part_name}; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index ad41bbe1a08..fc848eb6be2 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1849,6 +1849,17 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( merges_version = queue_.pullLogsToQueue(zookeeper); + { + /// We avoid returning here a version to be used in a lightweight transaction. + /// + /// When pinned parts set is changed a log entry is added to the queue in the same transaction. + /// The log entry serves as a synchronization point, and it also increments `merges_version`. + /// + /// If pinned parts are fetched after logs are pulled then we can safely say that it contains all locks up to `merges_version`. + String s = zookeeper->get(queue.zookeeper_path + "/pinned_part_uuids"); + pinned_part_uuids.fromString(s); + } + Coordination::GetResponse quorum_status_response = quorum_status_future.get(); if (quorum_status_response.error == Coordination::Error::ZOK) { @@ -1919,6 +1930,13 @@ bool ReplicatedMergeTreeMergePredicate::canMergeTwoParts( for (const MergeTreeData::DataPartPtr & part : {left, right}) { + if (pinned_part_uuids.part_uuids.contains(part->uuid)) + { + if (out_reason) + *out_reason = "Part " + part->name + " has uuid " + toString(part->uuid) + " which is currently pinned"; + return false; + } + if (part->name == inprogress_quorum_part) { if (out_reason) @@ -2014,6 +2032,13 @@ bool ReplicatedMergeTreeMergePredicate::canMergeSinglePart( const MergeTreeData::DataPartPtr & part, String * out_reason) const { + if (pinned_part_uuids.part_uuids.contains(part->uuid)) + { + if (out_reason) + *out_reason = "Part " + part->name + " has uuid " + toString(part->uuid) + " which is currently pinned"; + return false; + } + if (part->name == inprogress_quorum_part) { if (out_reason) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 00ef3ee7292..551ed9cb55c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -501,6 +502,9 @@ private: /// (loaded at some later time than prev_virtual_parts). std::unordered_map> committing_blocks; + /// List of UUIDs for parts that have their identity "pinned". + PinnedPartUUIDs pinned_part_uuids; + /// Quorum state taken at some later time than prev_virtual_parts. String inprogress_quorum_part; diff --git a/src/Storages/PartitionCommands.cpp b/src/Storages/PartitionCommands.cpp index f09f60887e8..45eeecd2e0c 100644 --- a/src/Storages/PartitionCommands.cpp +++ b/src/Storages/PartitionCommands.cpp @@ -13,6 +13,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + std::optional PartitionCommand::parse(const ASTAlterCommand * command_ast) { if (command_ast->type == ASTAlterCommand::DROP_PARTITION) @@ -59,8 +64,11 @@ std::optional PartitionCommand::parse(const ASTAlterCommand * res.to_database = command_ast->to_database; res.to_table = command_ast->to_table; break; - default: + case DataDestinationType::SHARD: + res.move_destination_type = PartitionCommand::MoveDestinationType::SHARD; break; + case DataDestinationType::DELETE: + throw Exception("ALTER with this destination type is not handled. This is a bug.", ErrorCodes::LOGICAL_ERROR); } if (res.move_destination_type != PartitionCommand::MoveDestinationType::TABLE) res.move_destination_name = command_ast->move_destination_name; diff --git a/src/Storages/PartitionCommands.h b/src/Storages/PartitionCommands.h index 9f89d44bd4e..1875f0f98ef 100644 --- a/src/Storages/PartitionCommands.h +++ b/src/Storages/PartitionCommands.h @@ -62,6 +62,7 @@ struct PartitionCommand DISK, VOLUME, TABLE, + SHARD, }; std::optional move_destination_type; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 3b4a1ec4e16..31dee005e42 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -131,6 +132,7 @@ namespace ErrorCodes extern const int NO_SUCH_DATA_PART; extern const int INTERSERVER_SCHEME_DOESNT_MATCH; extern const int DUPLICATE_DATA_PART; + extern const int BAD_ARGUMENTS; } namespace ActionLocks @@ -263,6 +265,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( , cleanup_thread(*this) , part_check_thread(*this) , restarting_thread(*this) + , part_moves_between_shards_orchestrator(*this) , allow_renaming(allow_renaming_) , replicated_fetches_pool_size(getContext()->getSettingsRef().background_fetches_pool_size) { @@ -440,6 +443,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( } createNewZooKeeperNodes(); + syncPinnedPartUUIDs(); } @@ -560,6 +564,10 @@ void StorageReplicatedMergeTree::createNewZooKeeperNodes() zookeeper->createIfNotExists(zookeeper_path + "/zero_copy_s3", String()); zookeeper->createIfNotExists(zookeeper_path + "/zero_copy_s3/shared", String()); } + + /// Part movement. + zookeeper->createIfNotExists(zookeeper_path + "/part_moves_shard", String()); + zookeeper->createIfNotExists(zookeeper_path + "/pinned_part_uuids", getPinnedPartUUIDs()->toString()); } @@ -1199,6 +1207,27 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) } +void StorageReplicatedMergeTree::syncPinnedPartUUIDs() +{ + auto zookeeper = getZooKeeper(); + + Coordination::Stat stat; + String s = zookeeper->get(zookeeper_path + "/pinned_part_uuids", &stat); + + std::lock_guard lock(pinned_part_uuids_mutex); + + /// Unsure whether or not this can be called concurrently. + if (pinned_part_uuids->stat.version < stat.version) + { + auto new_pinned_part_uuids = std::make_shared(); + new_pinned_part_uuids->fromString(s); + new_pinned_part_uuids->stat = stat; + + pinned_part_uuids = new_pinned_part_uuids; + } +} + + void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil::ZooKeeperPtr & zookeeper, const DataPartPtr & part, Coordination::Requests & ops, String part_name, NameSet * absent_replicas_paths) { @@ -1487,6 +1516,12 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) break; case LogEntry::ALTER_METADATA: return executeMetadataAlter(entry); + case LogEntry::SYNC_PINNED_PART_UUIDS: + syncPinnedPartUUIDs(); + return true; + case LogEntry::CLONE_PART_FROM_SHARD: + executeClonePartFromShard(entry); + return true; default: throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected log entry type: {}", static_cast(entry.type)); } @@ -2479,6 +2514,47 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) } +void StorageReplicatedMergeTree::executeClonePartFromShard(const LogEntry & entry) +{ + auto zookeeper = getZooKeeper(); + + Strings replicas = zookeeper->getChildren(entry.source_shard + "/replicas"); + std::shuffle(replicas.begin(), replicas.end(), thread_local_rng); + String replica = replicas.front(); + + LOG_INFO(log, "Will clone part from shard " + entry.source_shard + " and replica " + replica); + + MutableDataPartPtr part; + + { + auto metadata_snapshot = getInMemoryMetadataPtr(); + String source_replica_path = entry.source_shard + "/replicas/" + replica; + ReplicatedMergeTreeAddress address(getZooKeeper()->get(source_replica_path + "/host")); + auto timeouts = ConnectionTimeouts::getHTTPTimeouts(global_context); + auto user_password = global_context.getInterserverCredentials(); + String interserver_scheme = global_context.getInterserverScheme(); + + auto get_part = [&, address, timeouts, user_password, interserver_scheme]() + { + if (interserver_scheme != address.scheme) + throw Exception("Interserver schemes are different: '" + interserver_scheme + + "' != '" + address.scheme + "', can't fetch part from " + address.host, + ErrorCodes::LOGICAL_ERROR); + + return fetcher.fetchPart( + metadata_snapshot, entry.new_part_name, source_replica_path, + address.host, address.replication_port, + timeouts, user_password.first, user_password.second, interserver_scheme, true); + }; + + part = get_part(); + + ReplicatedMergeTreeBlockOutputStream output(*this, metadata_snapshot, 0, 0, 0, false, false, false); + output.writeExistingPart(part, "clone_part_from_shard_" + entry.block_id); + } +} + + void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coordination::Stat source_is_lost_stat, zkutil::ZooKeeperPtr & zookeeper) { String source_path = zookeeper_path + "/replicas/" + source_replica; @@ -4005,6 +4081,8 @@ void StorageReplicatedMergeTree::startup() /// between the assignment of queue_task_handle and queueTask that use the queue_task_handle. background_executor.start(); startBackgroundMovesIfNeeded(); + + part_moves_between_shards_orchestrator.start(); } catch (...) { @@ -4035,6 +4113,7 @@ void StorageReplicatedMergeTree::shutdown() restarting_thread.shutdown(); background_executor.finish(); + part_moves_between_shards_orchestrator.shutdown(); { auto lock = queue.lockQueue(); @@ -5245,6 +5324,11 @@ void StorageReplicatedMergeTree::getQueue(LogEntriesData & res, String & replica queue.getEntries(res); } +std::vector StorageReplicatedMergeTree::getPartMovesBetweenShardsEntries() +{ + return part_moves_between_shards_orchestrator.getEntries(); +} + time_t StorageReplicatedMergeTree::getAbsoluteDelay() const { time_t min_unprocessed_insert_time = 0; @@ -6432,6 +6516,100 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta cleanLastPartNode(partition_id); } +void StorageReplicatedMergeTree::movePartitionToShard( + const ASTPtr & partition, bool move_part, const String & to, const Context & /*query_context*/) +{ + /// This is a lightweight operation that only optimistically checks if it could succeed and queues tasks. + + if (!move_part) + throw Exception("MOVE PARTITION TO SHARD is not supported, use MOVE PART instead", ErrorCodes::NOT_IMPLEMENTED); + + if (normalizeZooKeeperPath(zookeeper_path) == normalizeZooKeeperPath(to)) + throw Exception("Source and destination are the same", ErrorCodes::BAD_ARGUMENTS); + + auto zookeeper = getZooKeeper(); + + String part_name = partition->as().value.safeGet(); + auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version); + + auto part = getPartIfExists(part_info, {MergeTreeDataPartState::Committed}); + if (!part) + throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "Part {} not found locally", part_name); + + if (part->uuid == UUIDHelpers::Nil) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Part {} does not have an uuid assigned and it can't be moved between shards", part_name); + + + ReplicatedMergeTreeMergePredicate merge_pred = queue.getMergePredicate(zookeeper); + + /// The following block is pretty much copy & paste from StorageReplicatedMergeTree::dropPart to avoid conflicts while this is WIP. + /// Extract it to a common method and re-use it before merging. + { + if (partIsLastQuorumPart(part->info)) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Part {} is last inserted part with quorum in partition. Would not be able to drop", part_name); + } + + /// canMergeSinglePart is overlapping with dropPart, let's try to use the same code. + String out_reason; + if (!merge_pred.canMergeSinglePart(part, &out_reason)) + throw Exception(ErrorCodes::PART_IS_TEMPORARILY_LOCKED, "Part is busy, reason: " + out_reason); + } + + { + /// Optimistic check that for compatible destination table structure. + checkTableStructure(to, getInMemoryMetadataPtr()); + } + + PinnedPartUUIDs src_pins; + PinnedPartUUIDs dst_pins; + + { + String s = zookeeper->get(zookeeper_path + "/pinned_part_uuids", &src_pins.stat); + src_pins.fromString(s); + } + + { + String s = zookeeper->get(to + "/pinned_part_uuids", &dst_pins.stat); + dst_pins.fromString(s); + } + + if (src_pins.part_uuids.contains(part->uuid) || dst_pins.part_uuids.contains(part->uuid)) + throw Exception(ErrorCodes::PART_IS_TEMPORARILY_LOCKED, "Part {} has it's uuid ({}) already pinned.", part_name, toString(part->uuid)); + + src_pins.part_uuids.insert(part->uuid); + dst_pins.part_uuids.insert(part->uuid); + + PartMovesBetweenShardsOrchestrator::Entry part_move_entry; + part_move_entry.create_time = std::time(nullptr); + part_move_entry.update_time = part_move_entry.create_time; + part_move_entry.task_uuid = UUIDHelpers::generateV4(); + part_move_entry.part_name = part->name; + part_move_entry.part_uuid = part->uuid; + part_move_entry.to_shard = to; + + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCheckRequest(zookeeper_path + "/log", merge_pred.getVersion())); /// Make sure no new events were added to the log. + ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/pinned_part_uuids", src_pins.toString(), src_pins.stat.version)); + ops.emplace_back(zkutil::makeSetRequest(to + "/pinned_part_uuids", dst_pins.toString(), dst_pins.stat.version)); + ops.emplace_back(zkutil::makeCreateRequest( + part_moves_between_shards_orchestrator.entries_znode_path + "/task-", + part_move_entry.toString(), + zkutil::CreateMode::PersistentSequential)); + + Coordination::Responses responses; + Coordination::Error rc = zookeeper->tryMulti(ops, responses); + zkutil::KeeperMultiException::check(rc, ops, responses); + + String task_znode_path = dynamic_cast(*responses.back()).path_created; + LOG_DEBUG(log, "Created task for part movement between shards at " + task_znode_path); + + /// Force refresh local state for making system table up to date after this operation succeeds. + part_moves_between_shards_orchestrator.sync(); + + // TODO: Add support for `replication_alter_partitions_sync`. +} + void StorageReplicatedMergeTree::getCommitPartOps( Coordination::Requests & ops, MutableDataPartPtr & part, diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index c70556f40df..fa5b725588c 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -20,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -183,6 +184,8 @@ public: using LogEntriesData = std::vector; void getQueue(LogEntriesData & res, String & replica_name); + std::vector getPartMovesBetweenShardsEntries(); + /// Get replica delay relative to current time. time_t getAbsoluteDelay() const; @@ -252,6 +255,7 @@ private: friend struct ReplicatedMergeTreeLogEntry; friend class ScopedPartitionMergeLock; friend class ReplicatedMergeTreeQueue; + friend class PartMovesBetweenShardsOrchestrator; friend class MergeTreeData; using MergeStrategyPicker = ReplicatedMergeTreeMergeStrategyPicker; @@ -305,7 +309,6 @@ private: DataPartsExchange::Fetcher fetcher; - /// When activated, replica is initialized and startup() method could exit Poco::Event startup_event; @@ -350,6 +353,8 @@ private: /// A thread that processes reconnection to ZooKeeper when the session expires. ReplicatedMergeTreeRestartingThread restarting_thread; + PartMovesBetweenShardsOrchestrator part_moves_between_shards_orchestrator; + /// True if replica was created for existing table with fixed granularity bool other_replicas_fixed_granularity = false; @@ -387,6 +392,10 @@ private: */ void checkParts(bool skip_sanity_checks); + /// Synchronize the list of part uuids which are currently pinned. These should be sent to root query executor + /// to be used for deduplication. + void syncPinnedPartUUIDs(); + /** Check that the part's checksum is the same as the checksum of the same part on some other replica. * If no one has such a part, nothing checks. * Not very reliable: if two replicas add a part almost at the same time, no checks will occur. @@ -457,6 +466,7 @@ private: bool executeFetch(LogEntry & entry); bool executeReplaceRange(const LogEntry & entry); + void executeClonePartFromShard(const LogEntry & entry); /** Updates the queue. */ @@ -633,6 +643,7 @@ private: PartitionCommandsResultInfo attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool part, ContextPtr query_context) override; void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, ContextPtr query_context) override; void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, ContextPtr query_context) override; + void movePartitionToShard(const ASTPtr & partition, bool move_part, const String & to, const Context & query_context) override; void fetchPartition( const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, diff --git a/src/Storages/System/StorageSystemPartMovesBetweenShards.cpp b/src/Storages/System/StorageSystemPartMovesBetweenShards.cpp new file mode 100644 index 00000000000..57946468bf3 --- /dev/null +++ b/src/Storages/System/StorageSystemPartMovesBetweenShards.cpp @@ -0,0 +1,135 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + + +NamesAndTypesList StorageSystemPartMovesBetweenShards::getNamesAndTypes() +{ + return { + /// Table properties. + { "database", std::make_shared() }, + { "table", std::make_shared() }, + + /// Constant element properties. + { "task_name", std::make_shared() }, + { "task_uuid", std::make_shared() }, + { "create_time", std::make_shared() }, + { "part_name", std::make_shared() }, + { "part_uuid", std::make_shared() }, + { "to_shard", std::make_shared() }, + + /// Processing status of item. + { "update_time", std::make_shared() }, + { "state", std::make_shared() }, + { "num_tries", std::make_shared() }, + { "last_exception", std::make_shared() }, + }; +} + + +void StorageSystemPartMovesBetweenShards::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const +{ + const auto access = context.getAccess(); + const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); + + std::map> replicated_tables; + for (const auto & db : DatabaseCatalog::instance().getDatabases()) + { + /// Check if database can contain replicated tables + if (!db.second->canContainMergeTreeTables()) + continue; + + const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, db.first); + + for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) + { + const auto & table = iterator->table(); + if (!table) + continue; + if (!dynamic_cast(table.get())) + continue; + if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, db.first, iterator->name())) + continue; + replicated_tables[db.first][iterator->name()] = table; + } + } + + + MutableColumnPtr col_database_mut = ColumnString::create(); + MutableColumnPtr col_table_mut = ColumnString::create(); + + for (auto & db : replicated_tables) + { + for (auto & table : db.second) + { + col_database_mut->insert(db.first); + col_table_mut->insert(table.first); + } + } + + ColumnPtr col_database_to_filter = std::move(col_database_mut); + ColumnPtr col_table_to_filter = std::move(col_table_mut); + + /// Determine what tables are needed by the conditions in the query. + { + Block filtered_block + { + { col_database_to_filter, std::make_shared(), "database" }, + { col_table_to_filter, std::make_shared(), "table" }, + }; + + VirtualColumnUtils::filterBlockWithQuery(query_info.query, filtered_block, context); + + if (!filtered_block.rows()) + return; + + col_database_to_filter = filtered_block.getByName("database").column; + col_table_to_filter = filtered_block.getByName("table").column; + } + + for (size_t i = 0, tables_size = col_database_to_filter->size(); i < tables_size; ++i) + { + String database = (*col_database_to_filter)[i].safeGet(); + String table = (*col_table_to_filter)[i].safeGet(); + + auto moves = dynamic_cast(*replicated_tables[database][table]).getPartMovesBetweenShardsEntries(); + + for (auto & entry : moves) + { + size_t col_num = 0; + + /// Table properties. + res_columns[col_num++]->insert(database); + res_columns[col_num++]->insert(table); + + /// Constant element properties. + res_columns[col_num++]->insert(entry.znode_name); + res_columns[col_num++]->insert(entry.task_uuid); + res_columns[col_num++]->insert(entry.create_time); + res_columns[col_num++]->insert(entry.part_name); + res_columns[col_num++]->insert(entry.part_uuid); + res_columns[col_num++]->insert(entry.to_shard); + + /// Processing status of item. + res_columns[col_num++]->insert(entry.update_time); + res_columns[col_num++]->insert(entry.state.toString()); + res_columns[col_num++]->insert(0); + res_columns[col_num++]->insert(entry.last_exception_msg); + } + } +} + +} diff --git a/src/Storages/System/StorageSystemPartMovesBetweenShards.h b/src/Storages/System/StorageSystemPartMovesBetweenShards.h new file mode 100644 index 00000000000..1d0e7e40037 --- /dev/null +++ b/src/Storages/System/StorageSystemPartMovesBetweenShards.h @@ -0,0 +1,27 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class Context; + + +class StorageSystemPartMovesBetweenShards final : public ext::shared_ptr_helper, public IStorageSystemOneBlock +{ + friend struct ext::shared_ptr_helper; +public: + std::string getName() const override { return "SystemShardMoves"; } + + static NamesAndTypesList getNamesAndTypes(); + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 673cf671548..3f3c19bb663 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -25,6 +25,7 @@ #include #include #include +#include #include #include #include @@ -144,6 +145,7 @@ void attachSystemTablesServer(IDatabase & system_database, bool has_zookeeper) attach(system_database, "graphite_retentions"); attach(system_database, "macros"); attach(system_database, "replicated_fetches"); + attach(system_database, "part_moves_between_shards"); if (has_zookeeper) attach(system_database, "zookeeper"); diff --git a/tests/integration/test_move_part_to_shard/__init__.py b/tests/integration/test_move_part_to_shard/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_move_part_to_shard/configs/merge_tree.xml b/tests/integration/test_move_part_to_shard/configs/merge_tree.xml new file mode 100644 index 00000000000..d4b61fa08a7 --- /dev/null +++ b/tests/integration/test_move_part_to_shard/configs/merge_tree.xml @@ -0,0 +1,6 @@ + + + 1 + 3 + + diff --git a/tests/integration/test_move_part_to_shard/configs/remote_servers.xml b/tests/integration/test_move_part_to_shard/configs/remote_servers.xml new file mode 100644 index 00000000000..4812d84a918 --- /dev/null +++ b/tests/integration/test_move_part_to_shard/configs/remote_servers.xml @@ -0,0 +1,26 @@ + + + + + + s0r0 + 9000 + + + s0r1 + 9000 + + + + + s1r0 + 9000 + + + s1r1 + 9000 + + + + + diff --git a/tests/integration/test_move_part_to_shard/test.py b/tests/integration/test_move_part_to_shard/test.py new file mode 100644 index 00000000000..eda0cba95e2 --- /dev/null +++ b/tests/integration/test_move_part_to_shard/test.py @@ -0,0 +1,162 @@ +import random +import time + +import pytest + +from helpers.client import QueryRuntimeException +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + +cluster = ClickHouseCluster(__file__) + +s0r0 = cluster.add_instance( + 's0r0', + main_configs=['configs/remote_servers.xml', 'configs/merge_tree.xml'], + with_zookeeper=True) + +s0r1 = cluster.add_instance( + 's0r1', + main_configs=['configs/remote_servers.xml', 'configs/merge_tree.xml'], + with_zookeeper=True) + +s1r0 = cluster.add_instance( + 's1r0', + main_configs=['configs/remote_servers.xml', 'configs/merge_tree.xml'], + with_zookeeper=True) + +s1r1 = cluster.add_instance( + 's1r1', + main_configs=['configs/remote_servers.xml', 'configs/merge_tree.xml'], + with_zookeeper=True) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_move(started_cluster): + for shard_ix, rs in enumerate([[s0r0, s0r1], [s1r0, s1r1]]): + for replica_ix, r in enumerate(rs): + r.query(""" + CREATE TABLE t(v UInt64) + ENGINE ReplicatedMergeTree('/clickhouse/shard_{}/tables/t', '{}') + ORDER BY tuple() + """.format(shard_ix, replica_ix)) + + s0r0.query("SYSTEM STOP MERGES t") + + s0r0.query("INSERT INTO t VALUES (1)") + s0r0.query("INSERT INTO t VALUES (2)") + + assert "2" == s0r0.query("SELECT count() FROM t").strip() + assert "0" == s1r0.query("SELECT count() FROM t").strip() + + s0r0.query("ALTER TABLE t MOVE PART 'all_0_0_0' TO SHARD '/clickhouse/shard_1/tables/t'") + + print(s0r0.query("SELECT * FROM system.part_moves_between_shards")) + + s0r0.query("SYSTEM START MERGES t") + s0r0.query("OPTIMIZE TABLE t FINAL") + + while True: + time.sleep(3) + + print(s0r0.query("SELECT * FROM system.part_moves_between_shards")) + + # Eventually. + if "DONE" == s0r0.query("SELECT state FROM system.part_moves_between_shards").strip(): + break + + for n in [s0r0, s0r1]: + assert "1" == n.query("SELECT count() FROM t").strip() + + for n in [s1r0, s1r1]: + assert "1" == n.query("SELECT count() FROM t").strip() + + # Move part back + s1r0.query("ALTER TABLE t MOVE PART 'all_0_0_0' TO SHARD '/clickhouse/shard_0/tables/t'") + + while True: + time.sleep(3) + + print(s1r0.query("SELECT * FROM system.part_moves_between_shards")) + + # Eventually. + if "DONE" == s1r0.query("SELECT state FROM system.part_moves_between_shards").strip(): + break + + for n in [s0r0, s0r1]: + assert "2" == n.query("SELECT count() FROM t").strip() + + for n in [s1r0, s1r1]: + assert "0" == n.query("SELECT count() FROM t").strip() + + # Cleanup. + for n in started_cluster.instances.values(): + n.query("DROP TABLE t SYNC") + + +def test_deduplication_while_move(started_cluster): + for shard_ix, rs in enumerate([[s0r0, s0r1], [s1r0, s1r1]]): + for replica_ix, r in enumerate(rs): + r.query(""" + CREATE TABLE t(v UInt64) + ENGINE ReplicatedMergeTree('/clickhouse/shard_{}/tables/t', '{}') + ORDER BY tuple() + """.format(shard_ix, replica_ix)) + + r.query(""" + CREATE TABLE t_d AS t + ENGINE Distributed('test_cluster', '', t) + """) + + s0r0.query("SYSTEM STOP MERGES t") + + s0r0.query("INSERT INTO t VALUES (1)") + s0r0.query("INSERT INTO t VALUES (2)") + s0r1.query("SYSTEM SYNC REPLICA t") + + assert "2" == s0r0.query("SELECT count() FROM t").strip() + assert "0" == s1r0.query("SELECT count() FROM t").strip() + + s0r0.query("ALTER TABLE t MOVE PART 'all_0_0_0' TO SHARD '/clickhouse/shard_1/tables/t'") + s0r0.query("SYSTEM START MERGES t") + + expected = """ +1 +2 +""" + + # Verify that we get consisntent result at all times while the part is moving from one shard to another. + while "DONE" != s0r0.query("SELECT state FROM system.part_moves_between_shards ORDER BY create_time DESC LIMIT 1").strip(): + n = random.choice(list(started_cluster.instances.values())) + + assert TSV(n.query("SELECT * FROM t_d ORDER BY v", settings={ + "allow_experimental_query_deduplication": 1 + })) == TSV(expected) + + +def test_move_not_permitted(started_cluster): + for ix, n in enumerate([s0r0, s1r0]): + n.query(""" + CREATE TABLE not_permitted(v_{} UInt64) + ENGINE ReplicatedMergeTree('/clickhouse/shard_{}/tables/not_permitted', 'r') + ORDER BY tuple() + """.format(ix, ix)) + + s0r0.query("INSERT INTO not_permitted VALUES (1)") + + with pytest.raises(QueryRuntimeException) as exc: + s0r0.query("ALTER TABLE not_permitted MOVE PART 'all_0_0_0' TO SHARD '/clickhouse/shard_1/tables/not_permitted'") + + assert "DB::Exception: Table columns structure in ZooKeeper is different from local table structure." in str(exc.value) + + with pytest.raises(QueryRuntimeException) as exc: + s0r0.query("ALTER TABLE not_permitted MOVE PART 'all_0_0_0' TO SHARD '/clickhouse/shard_0/tables/not_permitted'") + + assert "DB::Exception: Source and destination are the same" in str(exc.value) diff --git a/tests/integration/test_query_deduplication/configs/profiles.xml b/tests/integration/test_query_deduplication/configs/profiles.xml new file mode 100644 index 00000000000..4c15556ab99 --- /dev/null +++ b/tests/integration/test_query_deduplication/configs/profiles.xml @@ -0,0 +1,7 @@ + + + + 1 + + + \ No newline at end of file diff --git a/tests/integration/test_query_deduplication/test.py b/tests/integration/test_query_deduplication/test.py index 8d935b98579..1088b539414 100644 --- a/tests/integration/test_query_deduplication/test.py +++ b/tests/integration/test_query_deduplication/test.py @@ -11,15 +11,18 @@ cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( 'node1', - main_configs=['configs/remote_servers.xml', 'configs/deduplication_settings.xml']) + main_configs=['configs/remote_servers.xml', 'configs/deduplication_settings.xml'], + user_configs=['configs/profiles.xml']) node2 = cluster.add_instance( 'node2', - main_configs=['configs/remote_servers.xml', 'configs/deduplication_settings.xml']) + main_configs=['configs/remote_servers.xml', 'configs/deduplication_settings.xml'], + user_configs=['configs/profiles.xml']) node3 = cluster.add_instance( 'node3', - main_configs=['configs/remote_servers.xml', 'configs/deduplication_settings.xml']) + main_configs=['configs/remote_servers.xml', 'configs/deduplication_settings.xml'], + user_configs=['configs/profiles.xml']) @pytest.fixture(scope="module") From b930ca5d59eba9ef4d885981eba4813717ef3a7d Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 5 Apr 2021 16:39:40 +0300 Subject: [PATCH 02/46] Followup fix --- src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp index 88570ac33ef..39a51fe2646 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp @@ -64,6 +64,12 @@ void ReplicatedMergeTreeLogEntryData::writeText(WriteBuffer & out) const << "part_checksum: " << part_checksum; break; + case CLONE_PART_FROM_SHARD: + out << "clone_part_from_shard\n" + << new_part_name << "\n" + << "source_shard: " << source_shard; + break; + case MERGE_PARTS: out << "merge\n"; for (const String & s : source_parts) From d12deed53dc57f4a4eb6056a78975f71657ce791 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 6 Apr 2021 13:30:40 +0300 Subject: [PATCH 03/46] bump ci --- src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp index 9247f2e2fde..b2894bbb895 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp @@ -262,7 +262,6 @@ void PartMovesBetweenShardsOrchestrator::stepEntry(const Entry & entry) ops.emplace_back(zkutil::makeCreateRequest( entry.to_shard + "/log/log-", log_entry.toString(), zkutil::CreateMode::PersistentSequential)); - /// Submit. Coordination::Responses responses; Coordination::Error rc = zk->tryMulti(ops, responses); zkutil::KeeperMultiException::check(rc, ops, responses); From 94f3db3c78a5c511896f6acefe3f0fc27107757a Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Wed, 7 Apr 2021 10:38:57 +0100 Subject: [PATCH 04/46] Use ZK check requests as barriers to prevent issuing tasks based on stale state information --- .../PartMovesBetweenShardsOrchestrator.cpp | 39 ++++++++----------- .../PartMovesBetweenShardsOrchestrator.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 4 +- 3 files changed, 20 insertions(+), 25 deletions(-) diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp index b2894bbb895..26c0a992688 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp @@ -27,10 +27,10 @@ void PartMovesBetweenShardsOrchestrator::run() try { - sync(); + fetchStateFromZK(); if (step()) - sync(); + fetchStateFromZK(); else sleep_ms = 3 * 1000; } @@ -50,7 +50,7 @@ void PartMovesBetweenShardsOrchestrator::shutdown() LOG_TRACE(log, "PartMovesBetweenShardsOrchestrator thread finished"); } -void PartMovesBetweenShardsOrchestrator::sync() +void PartMovesBetweenShardsOrchestrator::fetchStateFromZK() { std::lock_guard lock(state_mutex); @@ -102,15 +102,16 @@ bool PartMovesBetweenShardsOrchestrator::step() if (!entry_to_process.has_value()) return false; + /// Since some state transitions are long running (waiting on replicas acknowledgement we create this lock to avoid + /// other replicas trying to do the same work. All state transitions should be idempotent so is is safe to lose the + /// lock and have another replica retry. + /// + /// Note: This blocks all other entries from being executed. Technical debt. + zkutil::EphemeralNodeHolder::Ptr entry_node_holder; + try { - - /// Since some state transitions are long running (waiting on replicas acknowledgement we create this lock to avoid - /// other replicas trying to do the same work. All state transitions should be idempotent so is is safe to lose the - /// lock and have another replica retry. - /// - /// Note: This blocks all other entries from being executed. Technical debt. - auto entry_node_holder = zkutil::EphemeralNodeHolder::create(entry_to_process->znode_path + "/lock_holder", *zk, storage.replica_name); + entry_node_holder = zkutil::EphemeralNodeHolder::create(entry_to_process->znode_path + "/lock_holder", *zk, storage.replica_name); } catch (const Coordination::Exception & e) { @@ -169,6 +170,8 @@ void PartMovesBetweenShardsOrchestrator::stepEntry(const Entry & entry) { /// Log entry. Coordination::Requests ops; + ops.emplace_back(zkutil::makeCheckRequest(entry.znode_path, entry.version)); + ReplicatedMergeTreeLogEntryData log_entry; log_entry.type = ReplicatedMergeTreeLogEntryData::SYNC_PINNED_PART_UUIDS; log_entry.create_time = std::time(nullptr); @@ -202,6 +205,8 @@ void PartMovesBetweenShardsOrchestrator::stepEntry(const Entry & entry) { /// Log entry. Coordination::Requests ops; + ops.emplace_back(zkutil::makeCheckRequest(entry.znode_path, entry.version)); + ReplicatedMergeTreeLogEntryData log_entry; log_entry.type = ReplicatedMergeTreeLogEntryData::SYNC_PINNED_PART_UUIDS; log_entry.create_time = std::time(nullptr); @@ -236,19 +241,9 @@ void PartMovesBetweenShardsOrchestrator::stepEntry(const Entry & entry) { /// There is a chance that attach on destination will fail and this task will be left in the queue forever. /// Make sure table structure doesn't change when there are part movements in progress. - /// - /// DESTINATION_FETCH is tricky from idempotency standpoint. We must ensure that no orchestrator - /// issues a CLONE_PART_FROM_SHARD entry after the source part is dropped. - /// - /// `makeSetRequest` on the entry is a sloppy mechanism for ensuring that the thread that makes the state - /// transition is the last one to issue a CLONE_PART_FROM_SHARD event. We assume here that if last event - /// was processed then all the ones prior to that succeeded as well. - /// - /// If we could somehow create just one entry in the log and record log entry name in the same - /// transaction so we could wait for it later the problem would be solved. { Coordination::Requests ops; - ops.emplace_back(zkutil::makeSetRequest(entry.znode_path, entry.toString(), entry.version)); + ops.emplace_back(zkutil::makeCheckRequest(entry.znode_path, entry.version)); /// Log entry. ReplicatedMergeTreeLogEntryData log_entry; @@ -277,7 +272,7 @@ void PartMovesBetweenShardsOrchestrator::stepEntry(const Entry & entry) Entry entry_copy = entry; entry_copy.state = EntryState::SOURCE_DROP_PRE_DELAY; entry_copy.update_time = std::time(nullptr); - zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version + 1); + zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); } } break; diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h index efbb140c56a..196629620d4 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h @@ -126,7 +126,7 @@ public: void wakeup() { task->schedule(); } void shutdown(); - void sync(); + void fetchStateFromZK(); /// We could have one thread per Entry and worry about concurrency issues. /// Or we could have a single thread trying to run one step at a time. diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 31dee005e42..f664f67ea1f 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -6604,8 +6604,8 @@ void StorageReplicatedMergeTree::movePartitionToShard( String task_znode_path = dynamic_cast(*responses.back()).path_created; LOG_DEBUG(log, "Created task for part movement between shards at " + task_znode_path); - /// Force refresh local state for making system table up to date after this operation succeeds. - part_moves_between_shards_orchestrator.sync(); + /// Force refresh local state. This will make the task immediately visible in `system.part_moves_between_shards` table. + part_moves_between_shards_orchestrator.fetchStateFromZK(); // TODO: Add support for `replication_alter_partitions_sync`. } From 1fa5871ff7e24e6b3df88e6416e4f1458929b4a7 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Tue, 20 Apr 2021 13:26:05 +0100 Subject: [PATCH 05/46] Fix bad rebase and introduce part_moves_between_shards_enable setting --- src/Core/Settings.h | 1 + src/Storages/MergeTree/MergeTreeData.cpp | 10 ++++++++-- src/Storages/MergeTree/MergeTreeData.h | 4 ++-- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- src/Storages/MergeTree/MergeTreeSettings.h | 3 +++ .../MergeTree/PartMovesBetweenShardsOrchestrator.cpp | 5 ++++- .../MergeTree/ReplicatedMergeTreeLogEntry.cpp | 6 ------ src/Storages/StorageReplicatedMergeTree.cpp | 12 ++++++------ src/Storages/StorageReplicatedMergeTree.h | 2 +- .../System/StorageSystemPartMovesBetweenShards.cpp | 4 ++-- .../System/StorageSystemPartMovesBetweenShards.h | 2 +- .../__init__.py | 0 .../configs/merge_tree.xml | 1 + .../configs/remote_servers.xml | 0 .../test.py | 0 15 files changed, 30 insertions(+), 22 deletions(-) rename tests/integration/{test_move_part_to_shard => test_part_moves_between_shards}/__init__.py (100%) rename tests/integration/{test_move_part_to_shard => test_part_moves_between_shards}/configs/merge_tree.xml (71%) rename tests/integration/{test_move_part_to_shard => test_part_moves_between_shards}/configs/remote_servers.xml (100%) rename tests/integration/{test_move_part_to_shard => test_part_moves_between_shards}/test.py (100%) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3489331acd4..ac5bda5fda8 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -445,6 +445,7 @@ class IColumn; \ M(Bool, allow_experimental_query_deduplication, false, "Experimental data deduplication for SELECT queries based on part UUIDs", 0) \ M(Bool, experimental_query_deduplication_send_all_part_uuids, false, "If false only part UUIDs for currently moving parts are sent. If true all read part UUIDs are sent (useful only for testing).", 0) \ + \ M(Bool, engine_file_empty_if_not_exists, false, "Allows to select data from a file engine table without file", 0) \ M(Bool, engine_file_truncate_on_insert, false, "Enables or disables truncate before insert in file engine tables", 0) \ M(Bool, allow_experimental_database_replicated, false, "Allow to create databases with Replicated engine", 0) \ diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 572df6486be..a1924009ffa 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2921,7 +2921,7 @@ void MergeTreeData::movePartitionToVolume(const ASTPtr & partition, const String throw Exception("Cannot move parts because moves are manually disabled", ErrorCodes::ABORTED); } -void MergeTreeData::movePartitionToShard(const ASTPtr & /*partition*/, bool /*move_part*/, const String & /*to*/, const Context & /*query_context*/) +void MergeTreeData::movePartitionToShard(const ASTPtr & /*partition*/, bool /*move_part*/, const String & /*to*/, ContextPtr /*query_context*/) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MOVE PARTITION TO SHARD is not supported by storage {}", getName()); } @@ -2984,8 +2984,14 @@ Pipe MergeTreeData::alterPartition( break; case PartitionCommand::MoveDestinationType::SHARD: + { + if (!getSettings()->part_moves_between_shards_enable) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Moving parts between shards is experimental and work in progress" + ", see part_moves_between_shards_enable setting"); movePartitionToShard(command.partition, command.part, command.move_destination_name, query_context); - break; + } + break; } } break; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index eb7f479bf9b..def64d536de 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -978,8 +978,6 @@ protected: virtual void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, ContextPtr context) = 0; virtual void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, ContextPtr context) = 0; - /// Makes sense only for replicated tables - virtual void movePartitionToShard(const ASTPtr & partition, bool move_part, const String & to, const Context & query_context); virtual void fetchPartition( const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, @@ -987,6 +985,8 @@ protected: bool fetch_part, ContextPtr query_context); + virtual void movePartitionToShard(const ASTPtr & partition, bool move_part, const String & to, ContextPtr query_context); + void writePartLog( PartLogElement::Type type, const ExecutionStatus & execution_status, diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index d94460f6d1c..31ee85091e4 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1937,7 +1937,7 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( ContextPtr query_context, PartFilterCounters & counters) const { - const Settings & settings = query_context.getSettings(); + const Settings & settings = query_context->getSettings(); /// process_parts prepare parts that have to be read for the query, /// returns false if duplicated parts' UUID have been met diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index ced28974849..6e43d0fad77 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -127,6 +127,9 @@ struct Settings; M(UInt64, max_concurrent_queries, 0, "Max number of concurrently executed queries related to the MergeTree table (0 - disabled). Queries will still be limited by other max_concurrent_queries settings.", 0) \ M(UInt64, min_marks_to_honor_max_concurrent_queries, 0, "Minimal number of marks to honor the MergeTree-level's max_concurrent_queries (0 - disabled). Queries will still be limited by other max_concurrent_queries settings.", 0) \ M(UInt64, min_bytes_to_rebalance_partition_over_jbod, 0, "Minimal amount of bytes to enable part rebalance over JBOD array (0 - disabled).", 0) \ + \ + /** Experimental/work in progress feature. Unsafe for production. */ \ + M(UInt64, part_moves_between_shards_enable, 0, "Experimental/Incomplete feature to move parts between shards. Does not take into account sharding expressions.", 0) \ M(UInt64, part_moves_between_shards_delay_seconds, 30, "Time to wait before/after moving parts between shards.", 0) \ \ /** Obsolete settings. Kept for backward compatibility only. */ \ diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp index 26c0a992688..c90c083888e 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp @@ -15,11 +15,14 @@ PartMovesBetweenShardsOrchestrator::PartMovesBetweenShardsOrchestrator(StorageRe , log(&Poco::Logger::get(logger_name)) , entries_znode_path(zookeeper_path + "/part_moves_shard") { - task = storage.global_context.getSchedulePool().createTask(logger_name, [this]{ run(); }); + task = storage.getContext()->getSchedulePool().createTask(logger_name, [this]{ run(); }); } void PartMovesBetweenShardsOrchestrator::run() { + if (!storage.getSettings()->part_moves_between_shards_enable) + return; + if (need_stop) return; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp index 39a51fe2646..88570ac33ef 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp @@ -64,12 +64,6 @@ void ReplicatedMergeTreeLogEntryData::writeText(WriteBuffer & out) const << "part_checksum: " << part_checksum; break; - case CLONE_PART_FROM_SHARD: - out << "clone_part_from_shard\n" - << new_part_name << "\n" - << "source_shard: " << source_shard; - break; - case MERGE_PARTS: out << "merge\n"; for (const String & s : source_parts) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index f664f67ea1f..9ec0c626c46 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2530,11 +2530,11 @@ void StorageReplicatedMergeTree::executeClonePartFromShard(const LogEntry & entr auto metadata_snapshot = getInMemoryMetadataPtr(); String source_replica_path = entry.source_shard + "/replicas/" + replica; ReplicatedMergeTreeAddress address(getZooKeeper()->get(source_replica_path + "/host")); - auto timeouts = ConnectionTimeouts::getHTTPTimeouts(global_context); - auto user_password = global_context.getInterserverCredentials(); - String interserver_scheme = global_context.getInterserverScheme(); + auto timeouts = ConnectionTimeouts::getHTTPTimeouts(getContext()); + auto credentials = getContext()->getInterserverCredentials(); + String interserver_scheme = getContext()->getInterserverScheme(); - auto get_part = [&, address, timeouts, user_password, interserver_scheme]() + auto get_part = [&, address, timeouts, credentials, interserver_scheme]() { if (interserver_scheme != address.scheme) throw Exception("Interserver schemes are different: '" + interserver_scheme @@ -2544,7 +2544,7 @@ void StorageReplicatedMergeTree::executeClonePartFromShard(const LogEntry & entr return fetcher.fetchPart( metadata_snapshot, entry.new_part_name, source_replica_path, address.host, address.replication_port, - timeouts, user_password.first, user_password.second, interserver_scheme, true); + timeouts, credentials->getUser(), credentials->getPassword(), interserver_scheme, true); }; part = get_part(); @@ -6517,7 +6517,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta } void StorageReplicatedMergeTree::movePartitionToShard( - const ASTPtr & partition, bool move_part, const String & to, const Context & /*query_context*/) + const ASTPtr & partition, bool move_part, const String & to, ContextPtr /*query_context*/) { /// This is a lightweight operation that only optimistically checks if it could succeed and queues tasks. diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index fa5b725588c..5ddfe5e7585 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -643,7 +643,7 @@ private: PartitionCommandsResultInfo attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool part, ContextPtr query_context) override; void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, ContextPtr query_context) override; void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, ContextPtr query_context) override; - void movePartitionToShard(const ASTPtr & partition, bool move_part, const String & to, const Context & query_context) override; + void movePartitionToShard(const ASTPtr & partition, bool move_part, const String & to, ContextPtr query_context) override; void fetchPartition( const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, diff --git a/src/Storages/System/StorageSystemPartMovesBetweenShards.cpp b/src/Storages/System/StorageSystemPartMovesBetweenShards.cpp index 57946468bf3..70bc473e241 100644 --- a/src/Storages/System/StorageSystemPartMovesBetweenShards.cpp +++ b/src/Storages/System/StorageSystemPartMovesBetweenShards.cpp @@ -40,9 +40,9 @@ NamesAndTypesList StorageSystemPartMovesBetweenShards::getNamesAndTypes() } -void StorageSystemPartMovesBetweenShards::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const +void StorageSystemPartMovesBetweenShards::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const { - const auto access = context.getAccess(); + const auto access = context->getAccess(); const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); std::map> replicated_tables; diff --git a/src/Storages/System/StorageSystemPartMovesBetweenShards.h b/src/Storages/System/StorageSystemPartMovesBetweenShards.h index 1d0e7e40037..892f816d48e 100644 --- a/src/Storages/System/StorageSystemPartMovesBetweenShards.h +++ b/src/Storages/System/StorageSystemPartMovesBetweenShards.h @@ -21,7 +21,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; + void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; }; } diff --git a/tests/integration/test_move_part_to_shard/__init__.py b/tests/integration/test_part_moves_between_shards/__init__.py similarity index 100% rename from tests/integration/test_move_part_to_shard/__init__.py rename to tests/integration/test_part_moves_between_shards/__init__.py diff --git a/tests/integration/test_move_part_to_shard/configs/merge_tree.xml b/tests/integration/test_part_moves_between_shards/configs/merge_tree.xml similarity index 71% rename from tests/integration/test_move_part_to_shard/configs/merge_tree.xml rename to tests/integration/test_part_moves_between_shards/configs/merge_tree.xml index d4b61fa08a7..7bf77e26249 100644 --- a/tests/integration/test_move_part_to_shard/configs/merge_tree.xml +++ b/tests/integration/test_part_moves_between_shards/configs/merge_tree.xml @@ -1,6 +1,7 @@ 1 + 1 3 diff --git a/tests/integration/test_move_part_to_shard/configs/remote_servers.xml b/tests/integration/test_part_moves_between_shards/configs/remote_servers.xml similarity index 100% rename from tests/integration/test_move_part_to_shard/configs/remote_servers.xml rename to tests/integration/test_part_moves_between_shards/configs/remote_servers.xml diff --git a/tests/integration/test_move_part_to_shard/test.py b/tests/integration/test_part_moves_between_shards/test.py similarity index 100% rename from tests/integration/test_move_part_to_shard/test.py rename to tests/integration/test_part_moves_between_shards/test.py From 1d47dcbe7f071faf7d3fb059195868c89f42de85 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Tue, 27 Apr 2021 14:22:26 +0100 Subject: [PATCH 06/46] Cleanup env before running tests --- contrib/grpc | 2 +- contrib/zlib-ng | 2 +- tests/integration/test_part_moves_between_shards/test.py | 5 +++++ 3 files changed, 7 insertions(+), 2 deletions(-) diff --git a/contrib/grpc b/contrib/grpc index 1085a941238..8d558f03fe3 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit 1085a941238e66b13e3fb89c310533745380acbc +Subproject commit 8d558f03fe370240081424fafa76cdc9301ea14b diff --git a/contrib/zlib-ng b/contrib/zlib-ng index 5cc4d232020..7f254522fd6 160000 --- a/contrib/zlib-ng +++ b/contrib/zlib-ng @@ -1 +1 @@ -Subproject commit 5cc4d232020dc66d1d6c5438834457e2a2f6127b +Subproject commit 7f254522fd676ff4e906c6d4e9b30d4df4214c2d diff --git a/tests/integration/test_part_moves_between_shards/test.py b/tests/integration/test_part_moves_between_shards/test.py index eda0cba95e2..2af2a12f8cb 100644 --- a/tests/integration/test_part_moves_between_shards/test.py +++ b/tests/integration/test_part_moves_between_shards/test.py @@ -42,6 +42,7 @@ def started_cluster(): def test_move(started_cluster): for shard_ix, rs in enumerate([[s0r0, s0r1], [s1r0, s1r1]]): for replica_ix, r in enumerate(rs): + r.query("DROP TABLE IF EXISTS t") r.query(""" CREATE TABLE t(v UInt64) ENGINE ReplicatedMergeTree('/clickhouse/shard_{}/tables/t', '{}') @@ -104,6 +105,9 @@ def test_move(started_cluster): def test_deduplication_while_move(started_cluster): for shard_ix, rs in enumerate([[s0r0, s0r1], [s1r0, s1r1]]): for replica_ix, r in enumerate(rs): + r.query("DROP TABLE IF EXISTS t") + r.query("DROP TABLE IF EXISTS t_d") + r.query(""" CREATE TABLE t(v UInt64) ENGINE ReplicatedMergeTree('/clickhouse/shard_{}/tables/t', '{}') @@ -143,6 +147,7 @@ def test_deduplication_while_move(started_cluster): def test_move_not_permitted(started_cluster): for ix, n in enumerate([s0r0, s1r0]): + n.query("DROP TABLE IF EXISTS not_permitted") n.query(""" CREATE TABLE not_permitted(v_{} UInt64) ENGINE ReplicatedMergeTree('/clickhouse/shard_{}/tables/not_permitted', 'r') From 67d34c0136426f79d33d9bd7a4e5bee6cc1915d0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 17 May 2021 14:13:18 +0300 Subject: [PATCH 07/46] merge with master --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 1 + src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp | 8 ++++++++ src/Storages/StorageReplicatedMergeTree.cpp | 2 +- tests/integration/test_part_moves_between_shards/test.py | 2 +- 4 files changed, 11 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 33ce87f1604..20cadb53098 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -528,6 +528,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( selectPartsToReadWithUUIDFilter( parts, part_values, + data.getPinnedPartUUIDs(), minmax_idx_condition, minmax_columns_types, partition_pruner, diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp index 9c4f39b95f3..dbd55bc4ff3 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp @@ -439,6 +439,14 @@ Strings ReplicatedMergeTreeLogEntryData::getVirtualPartNames(MergeTreeDataFormat return res; } + /// Doesn't produce any part. + if (type == SYNC_PINNED_PART_UUIDS) + return {}; + + /// Doesn't produce any part by itself. + if (type == CLONE_PART_FROM_SHARD) + return {}; + return {new_part_name}; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 9d0e623fcea..a4a959c9fba 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2610,7 +2610,7 @@ void StorageReplicatedMergeTree::executeClonePartFromShard(const LogEntry & entr part = get_part(); - ReplicatedMergeTreeBlockOutputStream output(*this, metadata_snapshot, 0, 0, 0, false, false, false); + ReplicatedMergeTreeBlockOutputStream output(*this, metadata_snapshot, 0, 0, 0, false, false, getContext(), false); output.writeExistingPart(part, "clone_part_from_shard_" + entry.block_id); } } diff --git a/tests/integration/test_part_moves_between_shards/test.py b/tests/integration/test_part_moves_between_shards/test.py index 2af2a12f8cb..f6d3942a4c4 100644 --- a/tests/integration/test_part_moves_between_shards/test.py +++ b/tests/integration/test_part_moves_between_shards/test.py @@ -42,7 +42,7 @@ def started_cluster(): def test_move(started_cluster): for shard_ix, rs in enumerate([[s0r0, s0r1], [s1r0, s1r1]]): for replica_ix, r in enumerate(rs): - r.query("DROP TABLE IF EXISTS t") + r.query("DROP TABLE IF EXISTS t SYNC") r.query(""" CREATE TABLE t(v UInt64) ENGINE ReplicatedMergeTree('/clickhouse/shard_{}/tables/t', '{}') From 77b1aeaac25aa52d1eba4ba35eacd52f201ab52a Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 17 May 2021 14:30:23 +0300 Subject: [PATCH 08/46] Use same zookeeper object to have an exception if something went wrong --- .../MergeTree/PartMovesBetweenShardsOrchestrator.cpp | 8 ++++---- .../MergeTree/PartMovesBetweenShardsOrchestrator.h | 3 ++- src/Storages/ya.make | 3 +++ 3 files changed, 9 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp index c90c083888e..dd06a6b12b9 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp @@ -129,7 +129,9 @@ bool PartMovesBetweenShardsOrchestrator::step() try { - stepEntry(entry_to_process.value()); + /// Use the same ZooKeeper connection. If we'd lost the lock then connection + /// will become expired and all consequent operations will fail. + stepEntry(entry_to_process.value(), zk); } catch (...) { @@ -146,10 +148,8 @@ bool PartMovesBetweenShardsOrchestrator::step() return true; } -void PartMovesBetweenShardsOrchestrator::stepEntry(const Entry & entry) +void PartMovesBetweenShardsOrchestrator::stepEntry(const Entry & entry, zkutil::ZooKeeperPtr zk) { - auto zk = storage.getZooKeeper(); - switch (entry.state.value) { case EntryState::DONE: diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h index 196629620d4..b7bb0ad1b55 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -136,7 +137,7 @@ public: private: void run(); - void stepEntry(const Entry & entry); + void stepEntry(const Entry & entry, zkutil::ZooKeeperPtr zk); private: StorageReplicatedMergeTree & storage; diff --git a/src/Storages/ya.make b/src/Storages/ya.make index d83ba7f6490..f907d2a7080 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -92,7 +92,9 @@ SRCS( MergeTree/MergeType.cpp MergeTree/MergedBlockOutputStream.cpp MergeTree/MergedColumnOnlyOutputStream.cpp + MergeTree/PartMovesBetweenShardsOrchestrator.cpp MergeTree/PartitionPruner.cpp + MergeTree/PinnedPartUUIDs.cpp MergeTree/ReplicatedFetchList.cpp MergeTree/ReplicatedMergeTreeAddress.cpp MergeTree/ReplicatedMergeTreeAltersSequence.cpp @@ -176,6 +178,7 @@ SRCS( System/StorageSystemMutations.cpp System/StorageSystemNumbers.cpp System/StorageSystemOne.cpp + System/StorageSystemPartMovesBetweenShards.cpp System/StorageSystemParts.cpp System/StorageSystemPartsBase.cpp System/StorageSystemPartsColumns.cpp From d8994350c13af1e35d663ed13979715a58530b78 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 17 May 2021 14:39:37 +0300 Subject: [PATCH 09/46] Useful comments --- src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp index dd06a6b12b9..6322446f248 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp @@ -15,6 +15,7 @@ PartMovesBetweenShardsOrchestrator::PartMovesBetweenShardsOrchestrator(StorageRe , log(&Poco::Logger::get(logger_name)) , entries_znode_path(zookeeper_path + "/part_moves_shard") { + /// Schedule pool is not designed for long-running tasks. TODO replace with a separate thread? task = storage.getContext()->getSchedulePool().createTask(logger_name, [this]{ run(); }); } @@ -190,6 +191,9 @@ void PartMovesBetweenShardsOrchestrator::stepEntry(const Entry & entry, zkutil:: String log_znode_path = dynamic_cast(*responses.back()).path_created; log_entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); + /// This wait in background schedule pool is useless. It'd be + /// better to have some notification which will call `step` + /// function when all replicated will finish. TODO. storage.waitForAllReplicasToProcessLogEntry(log_entry, true); } From 023b592f9789cfd900bfdfd5a6963d70eeb0783e Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 17 May 2021 14:42:14 +0300 Subject: [PATCH 10/46] Fix style --- src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp index 6322446f248..12160b0e578 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp @@ -366,9 +366,9 @@ std::vector PartMovesBetweenShardsOrc { std::lock_guard lock(state_mutex); - auto res = std::vector(); + std::vector res; - for (const auto& e : entries) + for (const auto & e : entries) res.push_back(e.second); return res; From 5563a0d7f35b05fd150835cfa4b740e972f79594 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 17 May 2021 14:44:47 +0300 Subject: [PATCH 11/46] Fix style --- src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp | 2 +- src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index 952053443ac..c4f281c2e78 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -183,7 +183,7 @@ void ReplicatedMergeTreeBlockOutputStream::write(const Block & block) } -void ReplicatedMergeTreeBlockOutputStream::writeExistingPart(MergeTreeData::MutableDataPartPtr & part, String block_id) +void ReplicatedMergeTreeBlockOutputStream::writeExistingPart(MergeTreeData::MutableDataPartPtr & part, const String & block_id) { last_block_is_duplicate = false; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h index 02d1843abf0..4b88a082cf1 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h @@ -40,7 +40,7 @@ public: void write(const Block & block) override; /// For ATTACHing existing data on filesystem. - void writeExistingPart(MergeTreeData::MutableDataPartPtr & part, String block_id = ""); + void writeExistingPart(MergeTreeData::MutableDataPartPtr & part, const String & block_id = ""); /// For proper deduplication in MaterializedViews bool lastBlockIsDuplicate() const From 87c6e39a2951f6ade2690b817d24a3d8a14be56a Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 17 May 2021 23:23:32 +0300 Subject: [PATCH 12/46] PVS Studio fixes --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 12 ++++-------- .../MergeTree/PartMovesBetweenShardsOrchestrator.h | 2 +- 2 files changed, 5 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 20cadb53098..889818b7ad4 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -466,6 +466,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( real_column_names.push_back(ExpressionActions::getSmallestColumn(available_real_columns)); metadata_snapshot->check(real_column_names, data.getVirtuals(), data.getStorageID()); + auto index_stats = use_cache ? std::move(cache->index_stats) : std::make_unique(); // Build and check if primary key is used when necessary std::optional key_condition; @@ -483,15 +484,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( fmt::join(primary_key_columns, ", ")); } LOG_DEBUG(log, "Key condition: {}", key_condition->toString()); - } - const auto & select = query_info.query->as(); - auto query_context = context->hasQueryContext() ? context->getQueryContext() : context; - auto index_stats = use_cache ? std::move(cache->index_stats) : std::make_unique(); - - // Select parts to read and do partition pruning via partition value and minmax indices - if (!use_cache) - { + /// Select parts to read and do partition pruning via partition value and minmax indices std::optional partition_pruner; std::optional minmax_idx_condition; DataTypes minmax_columns_types; @@ -523,6 +517,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( } } + auto query_context = context->hasQueryContext() ? context->getQueryContext() : context; PartFilterCounters part_filter_counters; if (query_context->getSettingsRef().allow_experimental_query_deduplication) selectPartsToReadWithUUIDFilter( @@ -574,6 +569,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( } } + const auto & select = query_info.query->as(); /// Sampling. MergeTreeDataSelectSamplingData sampling = use_cache ? std::move(cache->sampling) : MergeTreeDataSelectSamplingData{}; if (!use_cache) diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h index b7bb0ad1b55..a5ea09050a6 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h @@ -101,7 +101,7 @@ public: private: /// Transient value for CAS. - uint32_t version; + uint32_t version = 0; String znode_path; From 22cc1c953f2eb5ebe8c4006527328f8127df5358 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 18 May 2021 10:34:16 +0300 Subject: [PATCH 13/46] Revert "PVS Studio fixes" This reverts commit 87c6e39a2951f6ade2690b817d24a3d8a14be56a. --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 12 ++++++++---- .../MergeTree/PartMovesBetweenShardsOrchestrator.h | 2 +- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 889818b7ad4..20cadb53098 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -466,7 +466,6 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( real_column_names.push_back(ExpressionActions::getSmallestColumn(available_real_columns)); metadata_snapshot->check(real_column_names, data.getVirtuals(), data.getStorageID()); - auto index_stats = use_cache ? std::move(cache->index_stats) : std::make_unique(); // Build and check if primary key is used when necessary std::optional key_condition; @@ -484,8 +483,15 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( fmt::join(primary_key_columns, ", ")); } LOG_DEBUG(log, "Key condition: {}", key_condition->toString()); + } - /// Select parts to read and do partition pruning via partition value and minmax indices + const auto & select = query_info.query->as(); + auto query_context = context->hasQueryContext() ? context->getQueryContext() : context; + auto index_stats = use_cache ? std::move(cache->index_stats) : std::make_unique(); + + // Select parts to read and do partition pruning via partition value and minmax indices + if (!use_cache) + { std::optional partition_pruner; std::optional minmax_idx_condition; DataTypes minmax_columns_types; @@ -517,7 +523,6 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( } } - auto query_context = context->hasQueryContext() ? context->getQueryContext() : context; PartFilterCounters part_filter_counters; if (query_context->getSettingsRef().allow_experimental_query_deduplication) selectPartsToReadWithUUIDFilter( @@ -569,7 +574,6 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( } } - const auto & select = query_info.query->as(); /// Sampling. MergeTreeDataSelectSamplingData sampling = use_cache ? std::move(cache->sampling) : MergeTreeDataSelectSamplingData{}; if (!use_cache) diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h index a5ea09050a6..b7bb0ad1b55 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h @@ -101,7 +101,7 @@ public: private: /// Transient value for CAS. - uint32_t version = 0; + uint32_t version; String znode_path; From 36aeb597059ac8a9f782d535d4d52a8ababe7385 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 18 May 2021 11:03:38 +0300 Subject: [PATCH 14/46] Fix lazy test --- .../PartMovesBetweenShardsOrchestrator.h | 2 +- .../test_part_moves_between_shards/test.py | 68 ++++++++----------- 2 files changed, 31 insertions(+), 39 deletions(-) diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h index b7bb0ad1b55..a5ea09050a6 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h @@ -101,7 +101,7 @@ public: private: /// Transient value for CAS. - uint32_t version; + uint32_t version = 0; String znode_path; diff --git a/tests/integration/test_part_moves_between_shards/test.py b/tests/integration/test_part_moves_between_shards/test.py index f6d3942a4c4..00407f95389 100644 --- a/tests/integration/test_part_moves_between_shards/test.py +++ b/tests/integration/test_part_moves_between_shards/test.py @@ -42,27 +42,26 @@ def started_cluster(): def test_move(started_cluster): for shard_ix, rs in enumerate([[s0r0, s0r1], [s1r0, s1r1]]): for replica_ix, r in enumerate(rs): - r.query("DROP TABLE IF EXISTS t SYNC") r.query(""" - CREATE TABLE t(v UInt64) - ENGINE ReplicatedMergeTree('/clickhouse/shard_{}/tables/t', '{}') + CREATE TABLE test_move(v UInt64) + ENGINE ReplicatedMergeTree('/clickhouse/shard_{}/tables/test_move', '{}') ORDER BY tuple() """.format(shard_ix, replica_ix)) - s0r0.query("SYSTEM STOP MERGES t") + s0r0.query("SYSTEM STOP MERGES test_move") - s0r0.query("INSERT INTO t VALUES (1)") - s0r0.query("INSERT INTO t VALUES (2)") + s0r0.query("INSERT INTO test_move VALUES (1)") + s0r0.query("INSERT INTO test_move VALUES (2)") - assert "2" == s0r0.query("SELECT count() FROM t").strip() - assert "0" == s1r0.query("SELECT count() FROM t").strip() + assert "2" == s0r0.query("SELECT count() FROM test_move").strip() + assert "0" == s1r0.query("SELECT count() FROM test_move").strip() - s0r0.query("ALTER TABLE t MOVE PART 'all_0_0_0' TO SHARD '/clickhouse/shard_1/tables/t'") + s0r0.query("ALTER TABLE test_move MOVE PART 'all_0_0_0' TO SHARD '/clickhouse/shard_1/tables/test_move'") print(s0r0.query("SELECT * FROM system.part_moves_between_shards")) - s0r0.query("SYSTEM START MERGES t") - s0r0.query("OPTIMIZE TABLE t FINAL") + s0r0.query("SYSTEM START MERGES test_move") + s0r0.query("OPTIMIZE TABLE test_move FINAL") while True: time.sleep(3) @@ -70,17 +69,17 @@ def test_move(started_cluster): print(s0r0.query("SELECT * FROM system.part_moves_between_shards")) # Eventually. - if "DONE" == s0r0.query("SELECT state FROM system.part_moves_between_shards").strip(): + if "DONE" == s0r0.query("SELECT state FROM system.part_moves_between_shards WHERE table = 'test_move'").strip(): break for n in [s0r0, s0r1]: - assert "1" == n.query("SELECT count() FROM t").strip() + assert "1" == n.query("SELECT count() FROM test_move").strip() for n in [s1r0, s1r1]: - assert "1" == n.query("SELECT count() FROM t").strip() + assert "1" == n.query("SELECT count() FROM test_move").strip() # Move part back - s1r0.query("ALTER TABLE t MOVE PART 'all_0_0_0' TO SHARD '/clickhouse/shard_0/tables/t'") + s1r0.query("ALTER TABLE test_move MOVE PART 'all_0_0_0' TO SHARD '/clickhouse/shard_0/tables/test_move'") while True: time.sleep(3) @@ -88,48 +87,41 @@ def test_move(started_cluster): print(s1r0.query("SELECT * FROM system.part_moves_between_shards")) # Eventually. - if "DONE" == s1r0.query("SELECT state FROM system.part_moves_between_shards").strip(): + if "DONE" == s1r0.query("SELECT state FROM system.part_moves_between_shards WHERE table = 'test_move'").strip(): break for n in [s0r0, s0r1]: - assert "2" == n.query("SELECT count() FROM t").strip() + assert "2" == n.query("SELECT count() FROM test_move").strip() for n in [s1r0, s1r1]: - assert "0" == n.query("SELECT count() FROM t").strip() - - # Cleanup. - for n in started_cluster.instances.values(): - n.query("DROP TABLE t SYNC") + assert "0" == n.query("SELECT count() FROM test_move").strip() def test_deduplication_while_move(started_cluster): for shard_ix, rs in enumerate([[s0r0, s0r1], [s1r0, s1r1]]): for replica_ix, r in enumerate(rs): - r.query("DROP TABLE IF EXISTS t") - r.query("DROP TABLE IF EXISTS t_d") - r.query(""" - CREATE TABLE t(v UInt64) - ENGINE ReplicatedMergeTree('/clickhouse/shard_{}/tables/t', '{}') + CREATE TABLE test_deduplication(v UInt64) + ENGINE ReplicatedMergeTree('/clickhouse/shard_{}/tables/test_deduplication', '{}') ORDER BY tuple() """.format(shard_ix, replica_ix)) r.query(""" - CREATE TABLE t_d AS t - ENGINE Distributed('test_cluster', '', t) + CREATE TABLE t_d AS test_deduplication + ENGINE Distributed('test_cluster', '', test_deduplication) """) - s0r0.query("SYSTEM STOP MERGES t") + s0r0.query("SYSTEM STOP MERGES test_deduplication") - s0r0.query("INSERT INTO t VALUES (1)") - s0r0.query("INSERT INTO t VALUES (2)") - s0r1.query("SYSTEM SYNC REPLICA t") + s0r0.query("INSERT INTO test_deduplication VALUES (1)") + s0r0.query("INSERT INTO test_deduplication VALUES (2)") + s0r1.query("SYSTEM SYNC REPLICA test_deduplication", timeout=20) - assert "2" == s0r0.query("SELECT count() FROM t").strip() - assert "0" == s1r0.query("SELECT count() FROM t").strip() + assert "2" == s0r0.query("SELECT count() FROM test_deduplication").strip() + assert "0" == s1r0.query("SELECT count() FROM test_deduplication").strip() - s0r0.query("ALTER TABLE t MOVE PART 'all_0_0_0' TO SHARD '/clickhouse/shard_1/tables/t'") - s0r0.query("SYSTEM START MERGES t") + s0r0.query("ALTER TABLE test_deduplication MOVE PART 'all_0_0_0' TO SHARD '/clickhouse/shard_1/tables/test_deduplication'") + s0r0.query("SYSTEM START MERGES test_deduplication") expected = """ 1 @@ -137,7 +129,7 @@ def test_deduplication_while_move(started_cluster): """ # Verify that we get consisntent result at all times while the part is moving from one shard to another. - while "DONE" != s0r0.query("SELECT state FROM system.part_moves_between_shards ORDER BY create_time DESC LIMIT 1").strip(): + while "DONE" != s0r0.query("SELECT state FROM system.part_moves_between_shards WHERE table = 'test_deduplication' ORDER BY create_time DESC LIMIT 1").strip(): n = random.choice(list(started_cluster.instances.values())) assert TSV(n.query("SELECT * FROM t_d ORDER BY v", settings={ From 20a34e951a4c8fa3ada14442226974faae118e3a Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 21 May 2021 12:30:49 +0300 Subject: [PATCH 15/46] Split fetch part into two stages, don't use replicated block output stream --- .../PartMovesBetweenShardsOrchestrator.cpp | 51 ++++++++++++++++++- .../PartMovesBetweenShardsOrchestrator.h | 3 ++ .../ReplicatedMergeTreeBlockOutputStream.cpp | 4 +- .../ReplicatedMergeTreeBlockOutputStream.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 32 +++++++++--- src/Storages/StorageReplicatedMergeTree.h | 4 +- 6 files changed, 83 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp index 12160b0e578..7a4c3d530a7 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp @@ -246,7 +246,6 @@ void PartMovesBetweenShardsOrchestrator::stepEntry(const Entry & entry, zkutil:: case EntryState::DESTINATION_FETCH: { - /// There is a chance that attach on destination will fail and this task will be left in the queue forever. /// Make sure table structure doesn't change when there are part movements in progress. { Coordination::Requests ops; @@ -259,7 +258,55 @@ void PartMovesBetweenShardsOrchestrator::stepEntry(const Entry & entry, zkutil:: log_entry.new_part_name = entry.part_name; log_entry.source_replica = storage.replica_name; log_entry.source_shard = zookeeper_path; - log_entry.block_id = toString(entry.task_uuid); + ops.emplace_back(zkutil::makeSetRequest(entry.to_shard + "/log", "", -1)); + ops.emplace_back(zkutil::makeCreateRequest( + entry.to_shard + "/log/log-", log_entry.toString(), zkutil::CreateMode::PersistentSequential)); + + Coordination::Responses responses; + Coordination::Error rc = zk->tryMulti(ops, responses); + zkutil::KeeperMultiException::check(rc, ops, responses); + + String log_znode_path = dynamic_cast(*responses.back()).path_created; + log_entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); + + storage.waitForAllTableReplicasToProcessLogEntry(entry.to_shard, log_entry, true); + } + + { + /// State transition. + Entry entry_copy = entry; + entry_copy.state = EntryState::DESTINATION_ATTACH; + entry_copy.update_time = std::time(nullptr); + zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); + } + } + break; + + case EntryState::DESTINATION_ATTACH: + { + /// There is a chance that attach on destination will fail and this task will be left in the queue forever. + { + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCheckRequest(entry.znode_path, entry.version)); + + auto part = storage.getActiveContainingPart(entry.part_name); + /// Allocating block number in other replicas zookeeper path + /// TODO Maybe we can do better. + auto block_number_lock = storage.allocateBlockNumber(part->info.partition_id, zk, "", entry.to_shard); + auto block_number = block_number_lock->getNumber(); + + auto part_info = part->info; + part_info.min_block = block_number; + part_info.max_block = block_number; + part_info.level = 0; + part_info.mutation = 0; + + /// Attach log entry (all replicas already fetched part) + ReplicatedMergeTreeLogEntryData log_entry; + log_entry.type = ReplicatedMergeTreeLogEntryData::ATTACH_PART; + log_entry.part_checksum = part->checksums.getTotalChecksumHex(); + log_entry.create_time = std::time(nullptr); + log_entry.new_part_name = part_info.getPartName(); ops.emplace_back(zkutil::makeSetRequest(entry.to_shard + "/log", "", -1)); ops.emplace_back(zkutil::makeCreateRequest( entry.to_shard + "/log/log-", log_entry.toString(), zkutil::CreateMode::PersistentSequential)); diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h index a5ea09050a6..9e54ae8a8ed 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h @@ -30,6 +30,7 @@ public: SYNC_SOURCE, SYNC_DESTINATION, DESTINATION_FETCH, + DESTINATION_ATTACH, SOURCE_DROP_PRE_DELAY, SOURCE_DROP, SOURCE_DROP_POST_DELAY, @@ -51,6 +52,7 @@ public: case SYNC_SOURCE: return "SYNC_SOURCE"; case SYNC_DESTINATION: return "SYNC_DESTINATION"; case DESTINATION_FETCH: return "DESTINATION_FETCH"; + case DESTINATION_ATTACH: return "DESTINATION_ATTACH"; case SOURCE_DROP_PRE_DELAY: return "SOURCE_DROP_PRE_DELAY"; case SOURCE_DROP: return "SOURCE_DROP"; case SOURCE_DROP_POST_DELAY: return "SOURCE_DROP_POST_DELAY"; @@ -68,6 +70,7 @@ public: else if (in == "SYNC_SOURCE") return SYNC_SOURCE; else if (in == "SYNC_DESTINATION") return SYNC_DESTINATION; else if (in == "DESTINATION_FETCH") return DESTINATION_FETCH; + else if (in == "DESTINATION_ATTACH") return DESTINATION_ATTACH; else if (in == "SOURCE_DROP_PRE_DELAY") return SOURCE_DROP_PRE_DELAY; else if (in == "SOURCE_DROP") return SOURCE_DROP; else if (in == "SOURCE_DROP_POST_DELAY") return SOURCE_DROP_POST_DELAY; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index c4f281c2e78..51a24606dde 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -183,7 +183,7 @@ void ReplicatedMergeTreeBlockOutputStream::write(const Block & block) } -void ReplicatedMergeTreeBlockOutputStream::writeExistingPart(MergeTreeData::MutableDataPartPtr & part, const String & block_id) +void ReplicatedMergeTreeBlockOutputStream::writeExistingPart(MergeTreeData::MutableDataPartPtr & part) { last_block_is_duplicate = false; @@ -199,7 +199,7 @@ void ReplicatedMergeTreeBlockOutputStream::writeExistingPart(MergeTreeData::Muta try { - commitPart(zookeeper, part, block_id); + commitPart(zookeeper, part, ""); PartLog::addNewPart(storage.getContext(), part, watch.elapsed()); } catch (...) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h index 4b88a082cf1..a3fce65a840 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h @@ -40,7 +40,7 @@ public: void write(const Block & block) override; /// For ATTACHing existing data on filesystem. - void writeExistingPart(MergeTreeData::MutableDataPartPtr & part, const String & block_id = ""); + void writeExistingPart(MergeTreeData::MutableDataPartPtr & part); /// For proper deduplication in MaterializedViews bool lastBlockIsDuplicate() const diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 18dd006b15d..3adf3026b23 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2583,7 +2583,18 @@ void StorageReplicatedMergeTree::executeClonePartFromShard(const LogEntry & entr Strings replicas = zookeeper->getChildren(entry.source_shard + "/replicas"); std::shuffle(replicas.begin(), replicas.end(), thread_local_rng); - String replica = replicas.front(); + String replica; + for (const String & candidate : replicas) + { + if (zookeeper->exists(entry.source_shard + "/replicas/" + candidate + "/is_active")) + { + replica = candidate; + break; + } + } + + if (replica.empty()) + throw Exception(ErrorCodes::NO_REPLICA_HAS_PART, "Not found active replica on shard {} to clone part {}", entry.source_shard, entry.new_part_name); LOG_INFO(log, "Will clone part from shard " + entry.source_shard + " and replica " + replica); @@ -2611,9 +2622,10 @@ void StorageReplicatedMergeTree::executeClonePartFromShard(const LogEntry & entr }; part = get_part(); - - ReplicatedMergeTreeBlockOutputStream output(*this, metadata_snapshot, 0, 0, 0, false, false, getContext(), false); - output.writeExistingPart(part, "clone_part_from_shard_" + entry.block_id); + // The fetched part is valuable and should not be cleaned like a temp part. + part->is_temp = false; + part->renameTo("detached/" + entry.new_part_name, true); + LOG_INFO(log, "Cloned part {} to detached directory", part->name); } } @@ -5175,8 +5187,14 @@ bool StorageReplicatedMergeTree::existsNodeCached(const std::string & path) cons std::optional StorageReplicatedMergeTree::allocateBlockNumber( - const String & partition_id, const zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_block_id_path) const + const String & partition_id, const zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_block_id_path, const String & zookeeper_path_prefix) const { + String zookeeper_table_path; + if (zookeeper_path_prefix.empty()) + zookeeper_table_path = zookeeper_path; + else + zookeeper_table_path = zookeeper_path_prefix; + /// Lets check for duplicates in advance, to avoid superfluous block numbers allocation Coordination::Requests deduplication_check_ops; if (!zookeeper_block_id_path.empty()) @@ -5185,7 +5203,7 @@ StorageReplicatedMergeTree::allocateBlockNumber( deduplication_check_ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_block_id_path, -1)); } - String block_numbers_path = zookeeper_path + "/block_numbers"; + String block_numbers_path = zookeeper_table_path + "/block_numbers"; String partition_path = block_numbers_path + "/" + partition_id; if (!existsNodeCached(partition_path)) @@ -5208,7 +5226,7 @@ StorageReplicatedMergeTree::allocateBlockNumber( try { lock = EphemeralLockInZooKeeper( - partition_path + "/block-", zookeeper_path + "/temp", *zookeeper, &deduplication_check_ops); + partition_path + "/block-", zookeeper_table_path + "/temp", *zookeeper, &deduplication_check_ops); } catch (const zkutil::KeeperMultiException & e) { diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 886dc7b75c3..4e697f2d1f2 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -595,9 +595,11 @@ private: bool partIsInsertingWithParallelQuorum(const MergeTreePartInfo & part_info) const; /// Creates new block number if block with such block_id does not exist + /// If zookeeper_path_prefix specified then allocate block number on this path + /// (can be used if we want to allocate blocks on other replicas) std::optional allocateBlockNumber( const String & partition_id, const zkutil::ZooKeeperPtr & zookeeper, - const String & zookeeper_block_id_path = "") const; + const String & zookeeper_block_id_path = "", const String & zookeeper_path_prefix = "") const; /** Wait until all replicas, including this, execute the specified action from the log. * If replicas are added at the same time, it can not wait the added replica . From 0f0a8640575a50c8f22058a40f9d37da2f5ab9da Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 21 May 2021 13:22:19 +0300 Subject: [PATCH 16/46] Fix flkay 01033_quota_dcl --- tests/queries/0_stateless/01033_quota_dcl.reference | 1 - tests/queries/0_stateless/01033_quota_dcl.sql | 1 - 2 files changed, 2 deletions(-) diff --git a/tests/queries/0_stateless/01033_quota_dcl.reference b/tests/queries/0_stateless/01033_quota_dcl.reference index 2ad3c3ad784..7478adac441 100644 --- a/tests/queries/0_stateless/01033_quota_dcl.reference +++ b/tests/queries/0_stateless/01033_quota_dcl.reference @@ -1,2 +1 @@ -default CREATE QUOTA default KEYED BY user_name FOR INTERVAL 1 hour TRACKING ONLY TO default, readonly diff --git a/tests/queries/0_stateless/01033_quota_dcl.sql b/tests/queries/0_stateless/01033_quota_dcl.sql index 8832846a945..a7796402bc2 100644 --- a/tests/queries/0_stateless/01033_quota_dcl.sql +++ b/tests/queries/0_stateless/01033_quota_dcl.sql @@ -1,2 +1 @@ -SHOW QUOTAS; SHOW CREATE QUOTA default; From 8c63f0f8e20a6996f7d0b18ea5e8026f359a63ab Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 22 May 2021 10:46:12 +0300 Subject: [PATCH 17/46] Fix race condition on keeper shutdown --- src/Coordination/KeeperStorageDispatcher.cpp | 4 ++++ src/Server/KeeperTCPHandler.cpp | 16 ++++++++-------- 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/src/Coordination/KeeperStorageDispatcher.cpp b/src/Coordination/KeeperStorageDispatcher.cpp index 14a44ee6f3f..7da7c22c8a3 100644 --- a/src/Coordination/KeeperStorageDispatcher.cpp +++ b/src/Coordination/KeeperStorageDispatcher.cpp @@ -226,6 +226,10 @@ bool KeeperStorageDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr request_info.session_id = session_id; std::lock_guard lock(push_request_mutex); + + if (shutdown_called) + return false; + /// Put close requests without timeouts if (request->getOpNum() == Coordination::OpNum::Close) requests_queue->push(std::move(request_info)); diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 1dadd3437f7..2d5f41fe666 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -194,7 +194,7 @@ KeeperTCPHandler::KeeperTCPHandler(IServer & server_, const Poco::Net::StreamSoc , server(server_) , log(&Poco::Logger::get("NuKeeperTCPHandler")) , global_context(Context::createCopy(server.context())) - , nu_keeper_storage_dispatcher(global_context->getKeeperStorageDispatcher()) + , keeper_dispatcher(global_context->getKeeperStorageDispatcher()) , operation_timeout(0, global_context->getConfigRef().getUInt("test_keeper_server.operation_timeout_ms", Coordination::DEFAULT_OPERATION_TIMEOUT_MS) * 1000) , session_timeout(0, global_context->getConfigRef().getUInt("test_keeper_server.session_timeout_ms", Coordination::DEFAULT_SESSION_TIMEOUT_MS) * 1000) , poll_wrapper(std::make_unique(socket_)) @@ -286,12 +286,12 @@ void KeeperTCPHandler::runImpl() return; } - if (nu_keeper_storage_dispatcher->hasLeader()) + if (keeper_dispatcher->hasLeader()) { try { LOG_INFO(log, "Requesting session ID for the new client"); - session_id = nu_keeper_storage_dispatcher->getSessionID(session_timeout.totalMilliseconds()); + session_id = keeper_dispatcher->getSessionID(session_timeout.totalMilliseconds()); LOG_INFO(log, "Received session ID {}", session_id); } catch (const Exception & e) @@ -318,7 +318,7 @@ void KeeperTCPHandler::runImpl() UInt8 single_byte = 1; [[maybe_unused]] int result = write(response_fd, &single_byte, sizeof(single_byte)); }; - nu_keeper_storage_dispatcher->registerSession(session_id, response_callback); + keeper_dispatcher->registerSession(session_id, response_callback); session_stopwatch.start(); bool close_received = false; @@ -368,7 +368,7 @@ void KeeperTCPHandler::runImpl() if (response->error == Coordination::Error::ZSESSIONEXPIRED) { LOG_DEBUG(log, "Session #{} expired because server shutting down or quorum is not alive", session_id); - nu_keeper_storage_dispatcher->finishSession(session_id); + keeper_dispatcher->finishSession(session_id); return; } @@ -381,7 +381,7 @@ void KeeperTCPHandler::runImpl() if (session_stopwatch.elapsedMicroseconds() > static_cast(session_timeout.totalMicroseconds())) { LOG_DEBUG(log, "Session #{} expired", session_id); - nu_keeper_storage_dispatcher->finishSession(session_id); + keeper_dispatcher->finishSession(session_id); break; } } @@ -389,7 +389,7 @@ void KeeperTCPHandler::runImpl() catch (const Exception & ex) { LOG_INFO(log, "Got exception processing session #{}: {}", session_id, getExceptionMessage(ex, true)); - nu_keeper_storage_dispatcher->finishSession(session_id); + keeper_dispatcher->finishSession(session_id); } } @@ -407,7 +407,7 @@ std::pair KeeperTCPHandler::receiveReque request->xid = xid; request->readImpl(*in); - if (!nu_keeper_storage_dispatcher->putRequest(request, session_id)) + if (!keeper_dispatcher->putRequest(request, session_id)) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Session {} already disconnected", session_id); return std::make_pair(opnum, xid); } From 8c06f811303878ab6a5bd9ff2aaf5023feb29951 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 22 May 2021 10:50:23 +0300 Subject: [PATCH 18/46] Followup fix --- src/Coordination/KeeperStorageDispatcher.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Coordination/KeeperStorageDispatcher.cpp b/src/Coordination/KeeperStorageDispatcher.cpp index 7da7c22c8a3..e95a6940baa 100644 --- a/src/Coordination/KeeperStorageDispatcher.cpp +++ b/src/Coordination/KeeperStorageDispatcher.cpp @@ -320,6 +320,8 @@ void KeeperStorageDispatcher::shutdown() break; } } + + std::lock_guard lock(session_to_response_callback_mutex); session_to_response_callback.clear(); } catch (...) From 3165db2799a11093adf99df86ce6d81442c39801 Mon Sep 17 00:00:00 2001 From: BoloniniD Date: Sat, 22 May 2021 00:48:10 +0300 Subject: [PATCH 19/46] Wrote first version of docs for YAML configuration --- docs/en/operations/configuration-files.md | 68 ++++++++++++++++++++++- 1 file changed, 65 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/configuration-files.md b/docs/en/operations/configuration-files.md index 9864efd648a..32239b0dc80 100644 --- a/docs/en/operations/configuration-files.md +++ b/docs/en/operations/configuration-files.md @@ -5,9 +5,9 @@ toc_title: Configuration Files # Configuration Files {#configuration_files} -ClickHouse supports multi-file configuration management. The main server configuration file is `/etc/clickhouse-server/config.xml`. Other files must be in the `/etc/clickhouse-server/config.d` directory. +ClickHouse supports multi-file configuration management. The main server configuration file is `/etc/clickhouse-server/config.xml` or `/etc/clickhouse-server/config.yaml`. Other files must be in the `/etc/clickhouse-server/config.d` directory. Note, that any configuration file can be written either in XML or YAML, but mixing formats in one file is not supported. -All the configuration files should be in XML format. Also, they should have the same root element, usually ``. +All the configuration files should be in XML or YAML formats. All XML files should have the same root element, usually ``. As for YAML, writing `yandex:` is not needed, parser will insert it automatically. ## Override {#override} @@ -32,7 +32,7 @@ Users configuration can be splitted into separate files similar to `config.xml` Directory name is defined as `users_config` setting without `.xml` postfix concatenated with `.d`. Directory `users.d` is used by default, as `users_config` defaults to `users.xml`. -## Example {#example} +## XML example {#example} For example, you can have separate config file for each user like this: @@ -55,6 +55,68 @@ $ cat /etc/clickhouse-server/users.d/alice.xml ``` +## YAML examples {#example} + +There are some differences between YAML and XML formats in terms of ClickHouse configurations. Here are some tips for writing a configuration in YAML format. + +You should use a Scalar node to write a key-value pair: +``` yaml +key: value +``` + +To create a node, containing other nodes you should use a Map: +``` yaml +map_key: + key1: val1 + key2: val2 + key3: val3 +``` + +To create a list of values or nodes assigned to one tag you should use a Sequence: +``` yaml +seq_key: + - val1 + - val2 + - key1: val3 + - map: + key2: val4 + key3: val5 +``` + +If you want to write an attribute for a Sequence or Map node, you should use a @ prefix before the attribute key. Note, that @ is reserved by YAML standard, so you should also to wrap it into double quotes: + +``` yaml +map: + "@attr1": value1 + "@attr2": value2 + key: 123 +``` + +From that Map we will get these XML nodes: + +``` xml + + 123 + +``` + +You can also set attributes for Sequence: + +``` yaml +seq: + - "@attr1": value1 + - "@attr2": value2 + - 123 + - abc +``` + +So, we can get YAML config equal to this XML one: + +``` xml +123 +abc +``` + ## Implementation Details {#implementation-details} For each config file, the server also generates `file-preprocessed.xml` files when starting. These files contain all the completed substitutions and overrides, and they are intended for informational use. If ZooKeeper substitutions were used in the config files but ZooKeeper is not available on the server start, the server loads the configuration from the preprocessed file. From 53ce22ae65c739fc3680cbb83d5e851186caadef Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 22 May 2021 09:38:31 +0000 Subject: [PATCH 20/46] Exponencial backoff if queues are empty --- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 16 ++++++++++++++-- src/Storages/RabbitMQ/StorageRabbitMQ.h | 2 ++ 2 files changed, 16 insertions(+), 2 deletions(-) diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index bbb2a85d37a..60f0ce313f8 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -43,6 +43,7 @@ static const auto RETRIES_MAX = 20; static const uint32_t QUEUE_SIZE = 100000; static const auto MAX_FAILED_READ_ATTEMPTS = 10; static const auto RESCHEDULE_MS = 500; +static const auto BACKOFF_TRESHOLD = 32000; static const auto MAX_THREAD_WORK_DURATION_MS = 60000; namespace ErrorCodes @@ -100,6 +101,7 @@ StorageRabbitMQ::StorageRabbitMQ( , semaphore(0, num_consumers) , unique_strbase(getRandomName()) , queue_size(std::max(QUEUE_SIZE, static_cast(getMaxBlockSize()))) + , milliseconds_to_wait(RESCHEDULE_MS) { event_handler = std::make_shared(loop.getLoop(), log); restoreConnection(false); @@ -852,7 +854,17 @@ void StorageRabbitMQ::streamingToViewsFunc() LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); if (streamToViews()) + { + /// Reschedule with backoff. + if (milliseconds_to_wait < BACKOFF_TRESHOLD) + milliseconds_to_wait *= 2; + event_handler->updateLoopState(Loop::STOP); break; + } + else + { + milliseconds_to_wait = RESCHEDULE_MS; + } auto end_time = std::chrono::steady_clock::now(); auto duration = std::chrono::duration_cast(end_time - start_time); @@ -871,9 +883,8 @@ void StorageRabbitMQ::streamingToViewsFunc() } } - /// Wait for attached views if (!stream_cancelled) - streaming_task->scheduleAfter(RESCHEDULE_MS); + streaming_task->scheduleAfter(milliseconds_to_wait); } @@ -1019,6 +1030,7 @@ bool StorageRabbitMQ::streamToViews() looping_task->activateAndSchedule(); } + /// Do not reschedule, do not stop event loop. return false; } diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index c605b0dc0bd..6f1724c6c2e 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -136,6 +136,8 @@ private: BackgroundSchedulePool::TaskHolder looping_task; BackgroundSchedulePool::TaskHolder connection_task; + uint64_t milliseconds_to_wait; + std::atomic stream_cancelled{false}; size_t read_attempts = 0; mutable bool drop_table = false; From f07f133be51ba6a61ab51d73aff430f80b152871 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 20 May 2021 07:37:50 +0300 Subject: [PATCH 21/46] Fix typo in 01800_log_nested --- tests/queries/0_stateless/01800_log_nested.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01800_log_nested.sql b/tests/queries/0_stateless/01800_log_nested.sql index 6fe2b257905..eb74b8707e4 100644 --- a/tests/queries/0_stateless/01800_log_nested.sql +++ b/tests/queries/0_stateless/01800_log_nested.sql @@ -1,11 +1,11 @@ --- TinyTinyLog +-- TinyLog DROP TABLE IF EXISTS nested_01800_tiny_log; CREATE TABLE nested_01800_tiny_log (`column` Nested(name String, names Array(String), types Array(Enum8('PU' = 1, 'US' = 2, 'OTHER' = 3)))) ENGINE = TinyLog; INSERT INTO nested_01800_tiny_log VALUES (['Hello', 'World'], [['a'], ['b', 'c']], [['PU', 'US'], ['OTHER']]); SELECT 10 FROM nested_01800_tiny_log FORMAT Null; DROP TABLE nested_01800_tiny_log; --- StripeStripeLog +-- StripeLog DROP TABLE IF EXISTS nested_01800_stripe_log; CREATE TABLE nested_01800_stripe_log (`column` Nested(name String, names Array(String), types Array(Enum8('PU' = 1, 'US' = 2, 'OTHER' = 3)))) ENGINE = StripeLog; INSERT INTO nested_01800_stripe_log VALUES (['Hello', 'World'], [['a'], ['b', 'c']], [['PU', 'US'], ['OTHER']]); From 1ad78a39ed43c9721ded0eb0721b4c716cb6c3ef Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 22 May 2021 09:25:39 +0300 Subject: [PATCH 22/46] check-style: improve current_database check --- utils/check-style/check-style | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index db6b33a569b..bd70334110e 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -94,7 +94,9 @@ tests_with_query_log=( $( xargs grep --with-filename -e system.query_log -e system.query_thread_log | cut -d: -f1 | sort -u ) ) for test_case in "${tests_with_query_log[@]}"; do - grep -qE current_database.*currentDatabase "$test_case" || echo "Queries to system.query_log/system.query_thread_log does not have current_database = currentDatabase() condition in $test_case" + grep -qE current_database.*currentDatabase "$test_case" || { + grep -qE 'current_database.*\$CLICKHOUSE_DATABASE' "$test_case" + } || echo "Queries to system.query_log/system.query_thread_log does not have current_database = currentDatabase() condition in $test_case" done # Queries with ReplicatedMergeTree From 03e28e1cdf20eb1da19e6b6ef370054802d40288 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 22 May 2021 09:16:24 +0300 Subject: [PATCH 23/46] check-style: database filter for system.{tables,parts,parts_columns,columns} --- utils/check-style/check-style | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index bd70334110e..551368e5fd8 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -99,6 +99,25 @@ for test_case in "${tests_with_query_log[@]}"; do } || echo "Queries to system.query_log/system.query_thread_log does not have current_database = currentDatabase() condition in $test_case" done +# Queries to system.tables/system.parts/system.detached_parts/system.parts_columns/system.columns should have database = currentDatabase() condition +# NOTE: it is not that accuate, but at least something. +tests_with_database_column=( $( + find $ROOT_PATH/tests/queries -iname '*.sql' -or -iname '*.sh' -or -iname '*.py' | + grep -vP $EXCLUDE_DIRS | + grep -v -x -e $ROOT_PATH/tests/queries/query_test.py | + xargs grep --with-filename -e system.tables -e system.parts -e system.detached_parts -e system.parts_columns -e system.columns | cut -d: -f1 | sort -u +) ) +for test_case in "${tests_with_database_column[@]}"; do + grep -qE database.*currentDatabase "$test_case" || { + grep -qE 'database.*\$CLICKHOUSE_DATABASE' "$test_case" + } || { + # explicit database + grep -qE "database[ ]*=[ ]*'" "$test_case" + } || { + echo "Queries to system.tables/system.parts/system.detached_parts/system.parts_columns/system.columns does not have database = currentDatabase()/\$CLICKHOUSE_DATABASE condition in $test_case" + } +done + # Queries with ReplicatedMergeTree # NOTE: it is not that accuate, but at least something. tests_with_replicated_merge_tree=( $( From 12cc3fc4af63804a65789b3c5120e2714c226d7d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 22 May 2021 09:41:47 +0300 Subject: [PATCH 24/46] clickhouse-test: fix stdout reference w/o --database argument It was appended, but never truncated. --- tests/clickhouse-test | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index d0866dd3bbe..c1878d4c52d 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -185,9 +185,10 @@ def run_single_test(args, ext, server_logs_level, client_options, case_file, std 'stderr': stderr_file, } - # >> append to stdout and stderr, because there are also output of per test database creation + # >> append to stderr (but not stdout since it is not used there), + # because there are also output of per test database creation if not args.database: - pattern = '{test} >> {stdout} 2>> {stderr}' + pattern = '{test} > {stdout} 2>> {stderr}' else: pattern = '{test} > {stdout} 2> {stderr}' From 8058166d5596f6da70fd0bc1a5288f3dcd4c7ea3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 22 May 2021 08:43:24 +0300 Subject: [PATCH 25/46] tests: fix flakiness with system.{parts,detached_parts,columns,parts_clumns,tables} --- .../00080_show_tables_and_system_tables.sql | 2 +- ...5_storage_file_and_clickhouse-local_app.sh | 3 +- ...3_max_block_size_system_tables_columns.sql | 2 + .../00696_system_columns_limit.sql | 2 + ...00753_system_columns_and_system_tables.sql | 50 ++++++++----------- ...00821_distributed_storage_with_join_on.sql | 2 + ...ong_concurrent_select_and_drop_deadlock.sh | 2 + ...ookeeper_test_alter_compression_codecs.sql | 16 +++--- .../0_stateless/00938_fix_rwlock_segfault.sh | 1 + .../00941_system_columns_race_condition.sh | 1 + .../00991_system_parts_race_condition.sh | 1 + ...tem_parts_race_condition_zookeeper_long.sh | 1 + ...tem_parts_race_condition_drop_zookeeper.sh | 3 +- .../0_stateless/01004_rename_deadlock.sh | 1 + .../0_stateless/01007_r1r2_w_r2r1_deadlock.sh | 6 ++- ...arallel_alter_add_drop_column_zookeeper.sh | 12 ++--- .../01098_temporary_and_external_tables.sh | 1 + .../0_stateless/01109_exchange_tables.sql | 1 + .../01213_alter_rename_column_zookeeper.sh | 4 +- .../01445_create_table_as_table_function.sh | 1 + .../0_stateless/01451_detach_drop_part.sql | 6 +-- .../01451_replicated_detach_drop_part.sql | 6 +-- ...18_cast_nullable_virtual_system_column.sql | 3 +- .../01533_optimize_skip_merged_partitions.sql | 2 +- .../0_stateless/01560_optimize_on_insert.sql | 2 +- .../0_stateless/01600_parts_states_metrics.sh | 1 + .../0_stateless/01600_parts_types_metrics.sh | 1 + ...676_long_clickhouse_client_autocomplete.sh | 2 + .../0_stateless/01821_table_comment.sql | 2 +- .../1_stateful/00076_system_columns_bytes.sql | 1 + 30 files changed, 79 insertions(+), 59 deletions(-) diff --git a/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql b/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql index b3a3ddd63c8..efb7ccf5f71 100644 --- a/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql +++ b/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql @@ -25,6 +25,6 @@ DROP DATABASE IF EXISTS test_DatabaseMemory; CREATE DATABASE test_DatabaseMemory ENGINE = Memory; CREATE TABLE test_DatabaseMemory.A (A UInt8) ENGINE = Null; -SELECT sum(ignore(*, metadata_modification_time, engine_full, create_table_query)) FROM system.tables; +SELECT sum(ignore(*, metadata_modification_time, engine_full, create_table_query)) FROM system.tables WHERE database = 'test_DatabaseMemory'; DROP DATABASE test_DatabaseMemory; diff --git a/tests/queries/0_stateless/00385_storage_file_and_clickhouse-local_app.sh b/tests/queries/0_stateless/00385_storage_file_and_clickhouse-local_app.sh index ef0ec1ae842..a09e9a4adb2 100755 --- a/tests/queries/0_stateless/00385_storage_file_and_clickhouse-local_app.sh +++ b/tests/queries/0_stateless/00385_storage_file_and_clickhouse-local_app.sh @@ -57,7 +57,8 @@ ${CLICKHOUSE_LOCAL} -q "CREATE TABLE sophisticated_default a UInt8 DEFAULT 3, b UInt8 ALIAS a + 5, c UInt8 -) ENGINE = Memory; SELECT count() FROM system.tables WHERE name='sophisticated_default';" +) ENGINE = Memory; +SELECT count() FROM system.tables WHERE name='sophisticated_default' AND database = currentDatabase();" # Help is not skipped [[ $(${CLICKHOUSE_LOCAL} --help | wc -l) -gt 100 ]] diff --git a/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.sql b/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.sql index 9415952e0b3..1d6abf7b252 100644 --- a/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.sql +++ b/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.sql @@ -1,3 +1,5 @@ +-- NOTE: database = currentDatabase() is not mandatory + SELECT avg(blockSize()) <= 10 FROM system.tables SETTINGS max_block_size = 10; SELECT avg(blockSize()) <= 10 FROM system.tables LIMIT 10 SETTINGS max_block_size = 10; SELECT (SELECT count() FROM system.tables SETTINGS max_block_size = 10) = (SELECT count() FROM system.tables SETTINGS max_block_size = 9); diff --git a/tests/queries/0_stateless/00696_system_columns_limit.sql b/tests/queries/0_stateless/00696_system_columns_limit.sql index 007fac06053..c7bb53f4cb7 100644 --- a/tests/queries/0_stateless/00696_system_columns_limit.sql +++ b/tests/queries/0_stateless/00696_system_columns_limit.sql @@ -1,3 +1,5 @@ +-- NOTE: database = currentDatabase() is not mandatory + SELECT count() > 0 FROM (SELECT * FROM system.columns LIMIT 0); SELECT count() > 0 FROM (SELECT * FROM system.columns LIMIT 1); SELECT count() > 0 FROM (SELECT * FROM system.columns LIMIT 2); diff --git a/tests/queries/0_stateless/00753_system_columns_and_system_tables.sql b/tests/queries/0_stateless/00753_system_columns_and_system_tables.sql index 862e3693f0e..1037b5a6381 100644 --- a/tests/queries/0_stateless/00753_system_columns_and_system_tables.sql +++ b/tests/queries/0_stateless/00753_system_columns_and_system_tables.sql @@ -13,17 +13,15 @@ CREATE TABLE check_system_tables SETTINGS min_bytes_for_wide_part = 0; SELECT name, partition_key, sorting_key, primary_key, sampling_key, storage_policy, total_rows -FROM system.tables -WHERE name = 'check_system_tables' +FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase() FORMAT PrettyCompactNoEscapes; SELECT name, is_in_partition_key, is_in_sorting_key, is_in_primary_key, is_in_sampling_key -FROM system.columns -WHERE table = 'check_system_tables' +FROM system.columns WHERE table = 'check_system_tables' AND database = currentDatabase() FORMAT PrettyCompactNoEscapes; INSERT INTO check_system_tables VALUES (1, 1, 1); -SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); DROP TABLE IF EXISTS check_system_tables; @@ -39,13 +37,11 @@ CREATE TABLE check_system_tables ORDER BY date; SELECT name, partition_key, sorting_key, primary_key, sampling_key -FROM system.tables -WHERE name = 'check_system_tables' +FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase() FORMAT PrettyCompactNoEscapes; SELECT name, is_in_partition_key, is_in_sorting_key, is_in_primary_key, is_in_sampling_key -FROM system.columns -WHERE table = 'check_system_tables' +FROM system.columns WHERE table = 'check_system_tables' AND database = currentDatabase() FORMAT PrettyCompactNoEscapes; DROP TABLE IF EXISTS check_system_tables; @@ -59,29 +55,27 @@ CREATE TABLE check_system_tables ) ENGINE = MergeTree(Event, intHash32(UserId), (Counter, Event, intHash32(UserId)), 8192); SELECT name, partition_key, sorting_key, primary_key, sampling_key -FROM system.tables -WHERE name = 'check_system_tables' +FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase() FORMAT PrettyCompactNoEscapes; SELECT name, is_in_partition_key, is_in_sorting_key, is_in_primary_key, is_in_sampling_key -FROM system.columns -WHERE table = 'check_system_tables' +FROM system.columns WHERE table = 'check_system_tables' AND database = currentDatabase() FORMAT PrettyCompactNoEscapes; DROP TABLE IF EXISTS check_system_tables; SELECT 'Check total_bytes/total_rows for TinyLog'; CREATE TABLE check_system_tables (key UInt8) ENGINE = TinyLog(); -SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); INSERT INTO check_system_tables VALUES (1); -SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); DROP TABLE check_system_tables; SELECT 'Check total_bytes/total_rows for Memory'; CREATE TABLE check_system_tables (key UInt16) ENGINE = Memory(); -SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); INSERT INTO check_system_tables VALUES (1); -SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); DROP TABLE check_system_tables; SELECT 'Check total_bytes/total_rows for Buffer'; @@ -96,33 +90,33 @@ CREATE TABLE check_system_tables (key UInt16) ENGINE = Buffer( 100, 100, /* min_rows /max_rows */ 0, 1e6 /* min_bytes/max_bytes */ ); -SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); INSERT INTO check_system_tables SELECT * FROM numbers_mt(50); -SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); SELECT 'Check lifetime_bytes/lifetime_rows for Buffer'; -SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); OPTIMIZE TABLE check_system_tables; -- flush -SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); INSERT INTO check_system_tables SELECT * FROM numbers_mt(50); -SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); OPTIMIZE TABLE check_system_tables; -- flush -SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); INSERT INTO check_system_tables SELECT * FROM numbers_mt(101); -- direct block write (due to min_rows exceeded) -SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); DROP TABLE check_system_tables; DROP TABLE check_system_tables_null; SELECT 'Check total_bytes/total_rows for Set'; CREATE TABLE check_system_tables Engine=Set() AS SELECT * FROM numbers(50); -SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); INSERT INTO check_system_tables SELECT number+50 FROM numbers(50); -SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); DROP TABLE check_system_tables; SELECT 'Check total_bytes/total_rows for Join'; CREATE TABLE check_system_tables Engine=Join(ANY, LEFT, number) AS SELECT * FROM numbers(50); -SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); INSERT INTO check_system_tables SELECT number+50 FROM numbers(50); -SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); DROP TABLE check_system_tables; diff --git a/tests/queries/0_stateless/00821_distributed_storage_with_join_on.sql b/tests/queries/0_stateless/00821_distributed_storage_with_join_on.sql index 9e87c739928..f84c20feaea 100644 --- a/tests/queries/0_stateless/00821_distributed_storage_with_join_on.sql +++ b/tests/queries/0_stateless/00821_distributed_storage_with_join_on.sql @@ -1,3 +1,5 @@ +-- NOTE: database = currentDatabase() is not mandatory + DROP TABLE IF EXISTS table1; DROP TABLE IF EXISTS table2; diff --git a/tests/queries/0_stateless/00840_long_concurrent_select_and_drop_deadlock.sh b/tests/queries/0_stateless/00840_long_concurrent_select_and_drop_deadlock.sh index 0a68225a31a..448e7ad076c 100755 --- a/tests/queries/0_stateless/00840_long_concurrent_select_and_drop_deadlock.sh +++ b/tests/queries/0_stateless/00840_long_concurrent_select_and_drop_deadlock.sh @@ -1,5 +1,7 @@ #!/usr/bin/env bash +# NOTE: database = $CLICKHOUSE_DATABASE is unwanted + set -e CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) diff --git a/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs.sql b/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs.sql index 0108bb68743..e8fb073ba7a 100644 --- a/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs.sql +++ b/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs.sql @@ -25,8 +25,8 @@ ALTER TABLE alter_compression_codec1 ADD COLUMN alter_column String DEFAULT 'def SYSTEM SYNC REPLICA alter_compression_codec1; SYSTEM SYNC REPLICA alter_compression_codec2; -SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec1' AND name = 'alter_column'; -SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec2' AND name = 'alter_column'; +SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec1' AND name = 'alter_column' AND database = currentDatabase(); +SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec2' AND name = 'alter_column' AND database = currentDatabase(); INSERT INTO alter_compression_codec1 VALUES('2018-01-01', 3, '3'); INSERT INTO alter_compression_codec1 VALUES('2018-01-01', 4, '4'); @@ -37,8 +37,8 @@ SELECT * FROM alter_compression_codec1 ORDER BY id; SELECT * FROM alter_compression_codec2 ORDER BY id; ALTER TABLE alter_compression_codec1 MODIFY COLUMN alter_column CODEC(NONE); -SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec1' AND name = 'alter_column'; -SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec2' AND name = 'alter_column'; +SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec1' AND name = 'alter_column' AND database = currentDatabase(); +SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec2' AND name = 'alter_column' AND database = currentDatabase(); INSERT INTO alter_compression_codec2 VALUES('2018-01-01', 5, '5'); INSERT INTO alter_compression_codec2 VALUES('2018-01-01', 6, '6'); @@ -50,8 +50,8 @@ SET allow_suspicious_codecs = 1; ALTER TABLE alter_compression_codec1 MODIFY COLUMN alter_column CODEC(ZSTD, LZ4HC, LZ4, LZ4, NONE); SYSTEM SYNC REPLICA alter_compression_codec1; SYSTEM SYNC REPLICA alter_compression_codec2; -SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec1' AND name = 'alter_column'; -SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec2' AND name = 'alter_column'; +SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec1' AND name = 'alter_column' AND database = currentDatabase(); +SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec2' AND name = 'alter_column' AND database = currentDatabase(); INSERT INTO alter_compression_codec1 VALUES('2018-01-01', 7, '7'); INSERT INTO alter_compression_codec2 VALUES('2018-01-01', 8, '8'); @@ -62,8 +62,8 @@ SELECT * FROM alter_compression_codec2 ORDER BY id; ALTER TABLE alter_compression_codec1 MODIFY COLUMN alter_column FixedString(100); SYSTEM SYNC REPLICA alter_compression_codec2; -SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec1' AND name = 'alter_column'; -SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec2' AND name = 'alter_column'; +SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec1' AND name = 'alter_column' AND database = currentDatabase(); +SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec2' AND name = 'alter_column' AND database = currentDatabase(); DROP TABLE IF EXISTS alter_compression_codec1; DROP TABLE IF EXISTS alter_compression_codec2; diff --git a/tests/queries/0_stateless/00938_fix_rwlock_segfault.sh b/tests/queries/0_stateless/00938_fix_rwlock_segfault.sh index 5c4253e682b..01519f7a598 100755 --- a/tests/queries/0_stateless/00938_fix_rwlock_segfault.sh +++ b/tests/queries/0_stateless/00938_fix_rwlock_segfault.sh @@ -8,6 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e +# NOTE: database = $CLICKHOUSE_DATABASE is unwanted for _ in {1..100}; do \ $CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables UNION ALL SELECT name FROM system.columns format Null"; done diff --git a/tests/queries/0_stateless/00941_system_columns_race_condition.sh b/tests/queries/0_stateless/00941_system_columns_race_condition.sh index 0a3fc7f3b3f..1b34abc50d7 100755 --- a/tests/queries/0_stateless/00941_system_columns_race_condition.sh +++ b/tests/queries/0_stateless/00941_system_columns_race_condition.sh @@ -13,6 +13,7 @@ $CLICKHOUSE_CLIENT -q "CREATE TABLE alter_table (a UInt8, b Int16, c Float32, d function thread1() { + # NOTE: database = $CLICKHOUSE_DATABASE is unwanted while true; do $CLICKHOUSE_CLIENT --query "SELECT name FROM system.columns UNION ALL SELECT name FROM system.columns FORMAT Null"; done } diff --git a/tests/queries/0_stateless/00991_system_parts_race_condition.sh b/tests/queries/0_stateless/00991_system_parts_race_condition.sh index 55ff4d97149..f1b12a319c0 100755 --- a/tests/queries/0_stateless/00991_system_parts_race_condition.sh +++ b/tests/queries/0_stateless/00991_system_parts_race_condition.sh @@ -15,6 +15,7 @@ $CLICKHOUSE_CLIENT -q "CREATE TABLE alter_table (a UInt8, b Int16, c Float32, d function thread1() { + # NOTE: database = $CLICKHOUSE_DATABASE is unwanted while true; do $CLICKHOUSE_CLIENT --query "SELECT * FROM system.parts FORMAT Null"; done } diff --git a/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh b/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh index fe6246e02f6..793fc8e9575 100755 --- a/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh +++ b/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh @@ -16,6 +16,7 @@ $CLICKHOUSE_CLIENT -n -q " function thread1() { + # NOTE: database = $CLICKHOUSE_DATABASE is unwanted while true; do $CLICKHOUSE_CLIENT --query "SELECT * FROM system.parts FORMAT Null"; done } diff --git a/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh b/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh index d960d8ff91d..32fe31f68c6 100755 --- a/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh +++ b/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh @@ -8,7 +8,8 @@ set -e function thread1() { - while true; do + # NOTE: database = $CLICKHOUSE_DATABASE is unwanted + while true; do $CLICKHOUSE_CLIENT --query "SELECT * FROM system.parts FORMAT Null"; done } diff --git a/tests/queries/0_stateless/01004_rename_deadlock.sh b/tests/queries/0_stateless/01004_rename_deadlock.sh index aa9e6f8a5bc..2bde9256bba 100755 --- a/tests/queries/0_stateless/01004_rename_deadlock.sh +++ b/tests/queries/0_stateless/01004_rename_deadlock.sh @@ -28,6 +28,7 @@ function thread2() function thread3() { while true; do + # NOTE: database = $CLICKHOUSE_DATABASE is unwanted $CLICKHOUSE_CLIENT --query "SELECT * FROM system.tables" --format Null done } diff --git a/tests/queries/0_stateless/01007_r1r2_w_r2r1_deadlock.sh b/tests/queries/0_stateless/01007_r1r2_w_r2r1_deadlock.sh index 8773a180822..8936d8da775 100755 --- a/tests/queries/0_stateless/01007_r1r2_w_r2r1_deadlock.sh +++ b/tests/queries/0_stateless/01007_r1r2_w_r2r1_deadlock.sh @@ -15,14 +15,16 @@ $CLICKHOUSE_CLIENT --query "CREATE TABLE b (x UInt8) ENGINE = MergeTree ORDER BY function thread1() { - while true; do + while true; do + # NOTE: database = $CLICKHOUSE_DATABASE is unwanted seq 1 100 | awk '{ print "SELECT x FROM a WHERE x IN (SELECT toUInt8(count()) FROM system.tables);" }' | $CLICKHOUSE_CLIENT -n done } function thread2() { - while true; do + while true; do + # NOTE: database = $CLICKHOUSE_DATABASE is unwanted seq 1 100 | awk '{ print "SELECT x FROM b WHERE x IN (SELECT toUInt8(count()) FROM system.tables);" }' | $CLICKHOUSE_CLIENT -n done } diff --git a/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.sh b/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.sh index 7a3e3cf155f..fd0b53cf122 100755 --- a/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.sh +++ b/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.sh @@ -82,14 +82,14 @@ wait echo "Finishing alters" -columns1=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_1'" 2> /dev/null) -columns2=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_2'" 2> /dev/null) -columns3=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_3'" 2> /dev/null) +columns1=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_1' and database='$CLICKHOUSE_DATABASE'" 2> /dev/null) +columns2=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_2' and database='$CLICKHOUSE_DATABASE'" 2> /dev/null) +columns3=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_3' and database='$CLICKHOUSE_DATABASE'" 2> /dev/null) while [ "$columns1" != "$columns2" ] || [ "$columns2" != "$columns3" ]; do - columns1=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_1'" 2> /dev/null) - columns2=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_2'" 2> /dev/null) - columns3=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_3'" 2> /dev/null) + columns1=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_1' and database='$CLICKHOUSE_DATABASE'" 2> /dev/null) + columns2=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_2' and database='$CLICKHOUSE_DATABASE'" 2> /dev/null) + columns3=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_3' and database='$CLICKHOUSE_DATABASE'" 2> /dev/null) sleep 1 done diff --git a/tests/queries/0_stateless/01098_temporary_and_external_tables.sh b/tests/queries/0_stateless/01098_temporary_and_external_tables.sh index bdac3c6fae3..17683852f2b 100755 --- a/tests/queries/0_stateless/01098_temporary_and_external_tables.sh +++ b/tests/queries/0_stateless/01098_temporary_and_external_tables.sh @@ -10,6 +10,7 @@ url="${url_without_session}session_id=test_01098" ${CLICKHOUSE_CURL} -m 30 -sSk "$url" --data "DROP TEMPORARY TABLE IF EXISTS tmp_table" ${CLICKHOUSE_CURL} -m 30 -sSk "$url" --data "CREATE TEMPORARY TABLE tmp_table AS SELECT number AS n FROM numbers(42)" +# NOTE: database = $CLICKHOUSE_DATABASE is unwanted id=$(echo "SELECT uuid FROM system.tables WHERE name='tmp_table' AND is_temporary" | ${CLICKHOUSE_CURL} -m 31 -sSgk "$url" -d @-) internal_table_name="_temporary_and_external_tables.\`_tmp_$id\`" diff --git a/tests/queries/0_stateless/01109_exchange_tables.sql b/tests/queries/0_stateless/01109_exchange_tables.sql index 7fbb36e8ce9..f2b8e4adb1d 100644 --- a/tests/queries/0_stateless/01109_exchange_tables.sql +++ b/tests/queries/0_stateless/01109_exchange_tables.sql @@ -4,6 +4,7 @@ CREATE DATABASE test_01109 ENGINE=Atomic; USE test_01109; CREATE TABLE t0 ENGINE=MergeTree() ORDER BY tuple() AS SELECT rowNumberInAllBlocks(), * FROM (SELECT toLowCardinality(arrayJoin(['exchange', 'tables']))); +-- NOTE: database = currentDatabase() is not mandatory CREATE TABLE t1 ENGINE=Log() AS SELECT * FROM system.tables AS t JOIN system.databases AS d ON t.database=d.name; CREATE TABLE t2 ENGINE=MergeTree() ORDER BY tuple() AS SELECT rowNumberInAllBlocks() + (SELECT count() FROM t0), * FROM (SELECT arrayJoin(['hello', 'world'])); diff --git a/tests/queries/0_stateless/01213_alter_rename_column_zookeeper.sh b/tests/queries/0_stateless/01213_alter_rename_column_zookeeper.sh index 5da8de70c46..b82b132be05 100755 --- a/tests/queries/0_stateless/01213_alter_rename_column_zookeeper.sh +++ b/tests/queries/0_stateless/01213_alter_rename_column_zookeeper.sh @@ -32,11 +32,11 @@ $CLICKHOUSE_CLIENT --query "SHOW CREATE TABLE table_for_rename_replicated;" $CLICKHOUSE_CLIENT --query "ALTER TABLE table_for_rename_replicated RENAME COLUMN value1 to renamed_value1" --replication_alter_partitions_sync=0 -while [[ -z $($CLICKHOUSE_CLIENT --query "SELECT name FROM system.columns WHERE name = 'renamed_value1' and table = 'table_for_rename_replicated'" 2>/dev/null) ]]; do +while [[ -z $($CLICKHOUSE_CLIENT --query "SELECT name FROM system.columns WHERE name = 'renamed_value1' and table = 'table_for_rename_replicated' AND database = '$CLICKHOUSE_DATABASE'" 2>/dev/null) ]]; do sleep 0.5 done -$CLICKHOUSE_CLIENT --query "SELECT name FROM system.columns WHERE name = 'renamed_value1' and table = 'table_for_rename_replicated'" +$CLICKHOUSE_CLIENT --query "SELECT name FROM system.columns WHERE name = 'renamed_value1' and table = 'table_for_rename_replicated' AND database = '$CLICKHOUSE_DATABASE'" # SHOW CREATE TABLE takes query from .sql file on disk. # previous select take metadata from memory. So, when previous select says, that return renamed_value1 already exists in table, it's still can have old version on disk. diff --git a/tests/queries/0_stateless/01445_create_table_as_table_function.sh b/tests/queries/0_stateless/01445_create_table_as_table_function.sh index f963c700779..6df5b7d70d1 100755 --- a/tests/queries/0_stateless/01445_create_table_as_table_function.sh +++ b/tests/queries/0_stateless/01445_create_table_as_table_function.sh @@ -6,6 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh +# NOTE: database = $CLICKHOUSE_DATABASE is unwanted ${CLICKHOUSE_CLIENT} --query "CREATE TABLE system.columns AS numbers(10);" 2>&1 | grep -F "Code: 57" > /dev/null && echo 'OK' || echo 'FAIL' ${CLICKHOUSE_CLIENT} --query "CREATE TABLE system.columns engine=Memory AS numbers(10);" 2>&1 | grep -F "Code: 62" > /dev/null && echo 'OK' || echo 'FAIL' ${CLICKHOUSE_CLIENT} --query "CREATE TABLE system.columns AS numbers(10) engine=Memory;" 2>&1 | grep -F "Code: 62" > /dev/null && echo 'OK' || echo 'FAIL' diff --git a/tests/queries/0_stateless/01451_detach_drop_part.sql b/tests/queries/0_stateless/01451_detach_drop_part.sql index d70f4e37de4..a285730e45f 100644 --- a/tests/queries/0_stateless/01451_detach_drop_part.sql +++ b/tests/queries/0_stateless/01451_detach_drop_part.sql @@ -15,13 +15,13 @@ ALTER TABLE mt_01451 DETACH PART 'all_2_2_0'; SELECT v FROM mt_01451 ORDER BY v; -SELECT name FROM system.detached_parts WHERE table = 'mt_01451'; +SELECT name FROM system.detached_parts WHERE table = 'mt_01451' AND database = currentDatabase(); ALTER TABLE mt_01451 ATTACH PART 'all_2_2_0'; SELECT v FROM mt_01451 ORDER BY v; -SELECT name FROM system.detached_parts WHERE table = 'mt_01451'; +SELECT name FROM system.detached_parts WHERE table = 'mt_01451' AND database = currentDatabase(); SELECT '-- drop part --'; @@ -37,6 +37,6 @@ OPTIMIZE TABLE mt_01451 FINAL; SELECT v FROM mt_01451 ORDER BY v; -SELECT name FROM system.parts WHERE table = 'mt_01451' AND active; +SELECT name FROM system.parts WHERE table = 'mt_01451' AND active AND database = currentDatabase(); DROP TABLE mt_01451; diff --git a/tests/queries/0_stateless/01451_replicated_detach_drop_part.sql b/tests/queries/0_stateless/01451_replicated_detach_drop_part.sql index 3cd9fc7bc7e..7cefa9bd7ca 100644 --- a/tests/queries/0_stateless/01451_replicated_detach_drop_part.sql +++ b/tests/queries/0_stateless/01451_replicated_detach_drop_part.sql @@ -19,14 +19,14 @@ ALTER TABLE replica2 DETACH PART 'all_1_1_0'; SELECT v FROM replica1 ORDER BY v; -SELECT name FROM system.detached_parts WHERE table = 'replica2'; +SELECT name FROM system.detached_parts WHERE table = 'replica2' AND database = currentDatabase(); ALTER TABLE replica2 ATTACH PART 'all_1_1_0'; SYSTEM SYNC REPLICA replica1; SELECT v FROM replica1 ORDER BY v; -SELECT name FROM system.detached_parts WHERE table = 'replica2'; +SELECT name FROM system.detached_parts WHERE table = 'replica2' AND database = currentDatabase(); SELECT '-- drop part --'; @@ -43,7 +43,7 @@ OPTIMIZE TABLE replica1 FINAL; SELECT v FROM replica1 ORDER BY v; -SELECT name FROM system.parts WHERE table = 'replica2' AND active; +SELECT name FROM system.parts WHERE table = 'replica2' AND active AND database = currentDatabase(); DROP TABLE replica1; DROP TABLE replica2; diff --git a/tests/queries/0_stateless/01518_cast_nullable_virtual_system_column.sql b/tests/queries/0_stateless/01518_cast_nullable_virtual_system_column.sql index 6366bae7b37..9f4ab03e941 100644 --- a/tests/queries/0_stateless/01518_cast_nullable_virtual_system_column.sql +++ b/tests/queries/0_stateless/01518_cast_nullable_virtual_system_column.sql @@ -1,7 +1,8 @@ +-- NOTE: database = currentDatabase() is not mandatory + SELECT database FROM system.tables WHERE database LIKE '%' format Null; SELECT database AS db FROM system.tables WHERE db LIKE '%' format Null; SELECT CAST(database, 'String') AS db FROM system.tables WHERE db LIKE '%' format Null; SELECT CAST('a string', 'Nullable(String)') AS str WHERE str LIKE '%' format Null; SELECT CAST(database, 'Nullable(String)') AS ndb FROM system.tables WHERE ndb LIKE '%' format Null; SELECT 'all tests passed'; - diff --git a/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql b/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql index 63bbbd2d721..3e9e5cda23e 100644 --- a/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql +++ b/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql @@ -14,7 +14,7 @@ INSERT INTO optimize_final SELECT toDate('2000-01-01'), number + 5 FROM numbers( OPTIMIZE TABLE optimize_final FINAL; -SELECT table, partition, active, level from system.parts where table = 'optimize_final' and active = 1; +SELECT table, partition, active, level from system.parts where table = 'optimize_final' and database = currentDatabase() and active = 1; DROP TABLE optimize_final; diff --git a/tests/queries/0_stateless/01560_optimize_on_insert.sql b/tests/queries/0_stateless/01560_optimize_on_insert.sql index f64f4c75cfe..1c6418a50df 100644 --- a/tests/queries/0_stateless/01560_optimize_on_insert.sql +++ b/tests/queries/0_stateless/01560_optimize_on_insert.sql @@ -38,5 +38,5 @@ DROP TABLE IF EXISTS empty; CREATE TABLE empty (key UInt32, val UInt32, date Datetime) ENGINE=SummingMergeTree(val) PARTITION BY date ORDER BY key; INSERT INTO empty VALUES (1, 1, '2020-01-01'), (1, 1, '2020-01-01'), (1, -2, '2020-01-01'); SELECT * FROM empty ORDER BY key; -SELECT table, partition, active FROM system.parts where table = 'empty' and active = 1; +SELECT table, partition, active FROM system.parts where table = 'empty' and active = 1 and database = currentDatabase(); DROP TABLE empty; diff --git a/tests/queries/0_stateless/01600_parts_states_metrics.sh b/tests/queries/0_stateless/01600_parts_states_metrics.sh index e32210633bb..634cb403f85 100755 --- a/tests/queries/0_stateless/01600_parts_states_metrics.sh +++ b/tests/queries/0_stateless/01600_parts_states_metrics.sh @@ -4,6 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +# NOTE: database = $CLICKHOUSE_DATABASE is unwanted verify_sql="SELECT (SELECT sumIf(value, metric = 'PartsCommitted'), sumIf(value, metric = 'PartsOutdated') FROM system.metrics) = (SELECT sum(active), sum(NOT active) FROM system.parts)" diff --git a/tests/queries/0_stateless/01600_parts_types_metrics.sh b/tests/queries/0_stateless/01600_parts_types_metrics.sh index 1a1942be32a..67d2c0c306d 100755 --- a/tests/queries/0_stateless/01600_parts_types_metrics.sh +++ b/tests/queries/0_stateless/01600_parts_types_metrics.sh @@ -7,6 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e set -o pipefail +# NOTE: database = $CLICKHOUSE_DATABASE is unwanted verify_sql="SELECT (SELECT sumIf(value, metric = 'PartsInMemory'), sumIf(value, metric = 'PartsCompact'), sumIf(value, metric = 'PartsWide') FROM system.metrics) = (SELECT countIf(part_type == 'InMemory'), countIf(part_type == 'Compact'), countIf(part_type == 'Wide') FROM system.parts)" diff --git a/tests/queries/0_stateless/01676_long_clickhouse_client_autocomplete.sh b/tests/queries/0_stateless/01676_long_clickhouse_client_autocomplete.sh index 1ed5c6be272..3bb9da92362 100755 --- a/tests/queries/0_stateless/01676_long_clickhouse_client_autocomplete.sh +++ b/tests/queries/0_stateless/01676_long_clickhouse_client_autocomplete.sh @@ -4,6 +4,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +# NOTE: database = $CLICKHOUSE_DATABASE is superfluous + function test_completion_word() { local w=$1 && shift diff --git a/tests/queries/0_stateless/01821_table_comment.sql b/tests/queries/0_stateless/01821_table_comment.sql index c09f121459c..6d9218d5a42 100644 --- a/tests/queries/0_stateless/01821_table_comment.sql +++ b/tests/queries/0_stateless/01821_table_comment.sql @@ -28,7 +28,7 @@ SELECT name, comment FROM system.tables -WHERE name IN ('t1', 't2', 't3') order by name; +WHERE name IN ('t1', 't2', 't3') AND database = currentDatabase() order by name; SHOW CREATE TABLE t1; diff --git a/tests/queries/1_stateful/00076_system_columns_bytes.sql b/tests/queries/1_stateful/00076_system_columns_bytes.sql index 434054b6980..dc0858e3de6 100644 --- a/tests/queries/1_stateful/00076_system_columns_bytes.sql +++ b/tests/queries/1_stateful/00076_system_columns_bytes.sql @@ -1 +1,2 @@ +-- NOTE: database = currentDatabase() is not mandatory SELECT sum(data_compressed_bytes) > 0, sum(data_uncompressed_bytes) > 0, sum(marks_bytes) > 0 FROM system.columns; From 3a355ccff23c018bfceea4c31648b0caa285e862 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 22 May 2021 11:15:16 +0300 Subject: [PATCH 26/46] Mark some tests as long --- ...0385_storage_file_and_clickhouse-local_app_long.reference} | 0 ...sh => 00385_storage_file_and_clickhouse-local_app_long.sh} | 0 ... => 00753_system_columns_and_system_tables_long.reference} | 0 ...es.sql => 00753_system_columns_and_system_tables_long.sql} | 0 ...10_zookeeper_test_alter_compression_codecs_long.reference} | 0 ...=> 00910_zookeeper_test_alter_compression_codecs_long.sql} | 0 ...ult.reference => 00938_fix_rwlock_segfault_long.reference} | 0 ...x_rwlock_segfault.sh => 00938_fix_rwlock_segfault_long.sh} | 0 ...rence => 00991_system_parts_race_condition_long.reference} | 0 ...condition.sh => 00991_system_parts_race_condition_long.sh} | 0 ...nce => 01213_alter_rename_column_zookeeper_long.reference} | 4 ++-- ...okeeper.sh => 01213_alter_rename_column_zookeeper_long.sh} | 0 ...rence => 01451_replicated_detach_drop_part_long.reference} | 0 ...op_part.sql => 01451_replicated_detach_drop_part_long.sql} | 0 ...sert.reference => 01560_optimize_on_insert_long.reference} | 0 ...timize_on_insert.sql => 01560_optimize_on_insert_long.sql} | 0 ...cs.reference => 01600_parts_states_metrics_long.reference} | 0 ...s_states_metrics.sh => 01600_parts_states_metrics_long.sh} | 0 ...ics.reference => 01600_parts_types_metrics_long.reference} | 0 ...rts_types_metrics.sh => 01600_parts_types_metrics_long.sh} | 0 20 files changed, 2 insertions(+), 2 deletions(-) rename tests/queries/0_stateless/{00385_storage_file_and_clickhouse-local_app.reference => 00385_storage_file_and_clickhouse-local_app_long.reference} (100%) rename tests/queries/0_stateless/{00385_storage_file_and_clickhouse-local_app.sh => 00385_storage_file_and_clickhouse-local_app_long.sh} (100%) rename tests/queries/0_stateless/{00753_system_columns_and_system_tables.reference => 00753_system_columns_and_system_tables_long.reference} (100%) rename tests/queries/0_stateless/{00753_system_columns_and_system_tables.sql => 00753_system_columns_and_system_tables_long.sql} (100%) rename tests/queries/0_stateless/{00910_zookeeper_test_alter_compression_codecs.reference => 00910_zookeeper_test_alter_compression_codecs_long.reference} (100%) rename tests/queries/0_stateless/{00910_zookeeper_test_alter_compression_codecs.sql => 00910_zookeeper_test_alter_compression_codecs_long.sql} (100%) rename tests/queries/0_stateless/{00938_fix_rwlock_segfault.reference => 00938_fix_rwlock_segfault_long.reference} (100%) rename tests/queries/0_stateless/{00938_fix_rwlock_segfault.sh => 00938_fix_rwlock_segfault_long.sh} (100%) rename tests/queries/0_stateless/{00991_system_parts_race_condition.reference => 00991_system_parts_race_condition_long.reference} (100%) rename tests/queries/0_stateless/{00991_system_parts_race_condition.sh => 00991_system_parts_race_condition_long.sh} (100%) rename tests/queries/0_stateless/{01213_alter_rename_column_zookeeper.reference => 01213_alter_rename_column_zookeeper_long.reference} (59%) rename tests/queries/0_stateless/{01213_alter_rename_column_zookeeper.sh => 01213_alter_rename_column_zookeeper_long.sh} (100%) rename tests/queries/0_stateless/{01451_replicated_detach_drop_part.reference => 01451_replicated_detach_drop_part_long.reference} (100%) rename tests/queries/0_stateless/{01451_replicated_detach_drop_part.sql => 01451_replicated_detach_drop_part_long.sql} (100%) rename tests/queries/0_stateless/{01560_optimize_on_insert.reference => 01560_optimize_on_insert_long.reference} (100%) rename tests/queries/0_stateless/{01560_optimize_on_insert.sql => 01560_optimize_on_insert_long.sql} (100%) rename tests/queries/0_stateless/{01600_parts_states_metrics.reference => 01600_parts_states_metrics_long.reference} (100%) rename tests/queries/0_stateless/{01600_parts_states_metrics.sh => 01600_parts_states_metrics_long.sh} (100%) rename tests/queries/0_stateless/{01600_parts_types_metrics.reference => 01600_parts_types_metrics_long.reference} (100%) rename tests/queries/0_stateless/{01600_parts_types_metrics.sh => 01600_parts_types_metrics_long.sh} (100%) diff --git a/tests/queries/0_stateless/00385_storage_file_and_clickhouse-local_app.reference b/tests/queries/0_stateless/00385_storage_file_and_clickhouse-local_app_long.reference similarity index 100% rename from tests/queries/0_stateless/00385_storage_file_and_clickhouse-local_app.reference rename to tests/queries/0_stateless/00385_storage_file_and_clickhouse-local_app_long.reference diff --git a/tests/queries/0_stateless/00385_storage_file_and_clickhouse-local_app.sh b/tests/queries/0_stateless/00385_storage_file_and_clickhouse-local_app_long.sh similarity index 100% rename from tests/queries/0_stateless/00385_storage_file_and_clickhouse-local_app.sh rename to tests/queries/0_stateless/00385_storage_file_and_clickhouse-local_app_long.sh diff --git a/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference b/tests/queries/0_stateless/00753_system_columns_and_system_tables_long.reference similarity index 100% rename from tests/queries/0_stateless/00753_system_columns_and_system_tables.reference rename to tests/queries/0_stateless/00753_system_columns_and_system_tables_long.reference diff --git a/tests/queries/0_stateless/00753_system_columns_and_system_tables.sql b/tests/queries/0_stateless/00753_system_columns_and_system_tables_long.sql similarity index 100% rename from tests/queries/0_stateless/00753_system_columns_and_system_tables.sql rename to tests/queries/0_stateless/00753_system_columns_and_system_tables_long.sql diff --git a/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs.reference b/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs_long.reference similarity index 100% rename from tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs.reference rename to tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs_long.reference diff --git a/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs.sql b/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs_long.sql similarity index 100% rename from tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs.sql rename to tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs_long.sql diff --git a/tests/queries/0_stateless/00938_fix_rwlock_segfault.reference b/tests/queries/0_stateless/00938_fix_rwlock_segfault_long.reference similarity index 100% rename from tests/queries/0_stateless/00938_fix_rwlock_segfault.reference rename to tests/queries/0_stateless/00938_fix_rwlock_segfault_long.reference diff --git a/tests/queries/0_stateless/00938_fix_rwlock_segfault.sh b/tests/queries/0_stateless/00938_fix_rwlock_segfault_long.sh similarity index 100% rename from tests/queries/0_stateless/00938_fix_rwlock_segfault.sh rename to tests/queries/0_stateless/00938_fix_rwlock_segfault_long.sh diff --git a/tests/queries/0_stateless/00991_system_parts_race_condition.reference b/tests/queries/0_stateless/00991_system_parts_race_condition_long.reference similarity index 100% rename from tests/queries/0_stateless/00991_system_parts_race_condition.reference rename to tests/queries/0_stateless/00991_system_parts_race_condition_long.reference diff --git a/tests/queries/0_stateless/00991_system_parts_race_condition.sh b/tests/queries/0_stateless/00991_system_parts_race_condition_long.sh similarity index 100% rename from tests/queries/0_stateless/00991_system_parts_race_condition.sh rename to tests/queries/0_stateless/00991_system_parts_race_condition_long.sh diff --git a/tests/queries/0_stateless/01213_alter_rename_column_zookeeper.reference b/tests/queries/0_stateless/01213_alter_rename_column_zookeeper_long.reference similarity index 59% rename from tests/queries/0_stateless/01213_alter_rename_column_zookeeper.reference rename to tests/queries/0_stateless/01213_alter_rename_column_zookeeper_long.reference index 35385731ad3..059dcebbb37 100644 --- a/tests/queries/0_stateless/01213_alter_rename_column_zookeeper.reference +++ b/tests/queries/0_stateless/01213_alter_rename_column_zookeeper_long.reference @@ -1,7 +1,7 @@ 1 -CREATE TABLE default.table_for_rename_replicated\n(\n `date` Date,\n `key` UInt64,\n `value1` String,\n `value2` String,\n `value3` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01213_alter_rename_column_zookeeper_default/table_for_rename_replicated\', \'1\')\nPARTITION BY date\nORDER BY key\nSETTINGS index_granularity = 8192 +CREATE TABLE default.table_for_rename_replicated\n(\n `date` Date,\n `key` UInt64,\n `value1` String,\n `value2` String,\n `value3` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01213_alter_rename_column_zookeeper_long_default/table_for_rename_replicated\', \'1\')\nPARTITION BY date\nORDER BY key\nSETTINGS index_granularity = 8192 renamed_value1 -CREATE TABLE default.table_for_rename_replicated\n(\n `date` Date,\n `key` UInt64,\n `renamed_value1` String,\n `value2` String,\n `value3` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01213_alter_rename_column_zookeeper_default/table_for_rename_replicated\', \'1\')\nPARTITION BY date\nORDER BY key\nSETTINGS index_granularity = 8192 +CREATE TABLE default.table_for_rename_replicated\n(\n `date` Date,\n `key` UInt64,\n `renamed_value1` String,\n `value2` String,\n `value3` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01213_alter_rename_column_zookeeper_long_default/table_for_rename_replicated\', \'1\')\nPARTITION BY date\nORDER BY key\nSETTINGS index_granularity = 8192 1 date key renamed_value1 value2 value3 2019-10-02 1 1 1 1 diff --git a/tests/queries/0_stateless/01213_alter_rename_column_zookeeper.sh b/tests/queries/0_stateless/01213_alter_rename_column_zookeeper_long.sh similarity index 100% rename from tests/queries/0_stateless/01213_alter_rename_column_zookeeper.sh rename to tests/queries/0_stateless/01213_alter_rename_column_zookeeper_long.sh diff --git a/tests/queries/0_stateless/01451_replicated_detach_drop_part.reference b/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.reference similarity index 100% rename from tests/queries/0_stateless/01451_replicated_detach_drop_part.reference rename to tests/queries/0_stateless/01451_replicated_detach_drop_part_long.reference diff --git a/tests/queries/0_stateless/01451_replicated_detach_drop_part.sql b/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql similarity index 100% rename from tests/queries/0_stateless/01451_replicated_detach_drop_part.sql rename to tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql diff --git a/tests/queries/0_stateless/01560_optimize_on_insert.reference b/tests/queries/0_stateless/01560_optimize_on_insert_long.reference similarity index 100% rename from tests/queries/0_stateless/01560_optimize_on_insert.reference rename to tests/queries/0_stateless/01560_optimize_on_insert_long.reference diff --git a/tests/queries/0_stateless/01560_optimize_on_insert.sql b/tests/queries/0_stateless/01560_optimize_on_insert_long.sql similarity index 100% rename from tests/queries/0_stateless/01560_optimize_on_insert.sql rename to tests/queries/0_stateless/01560_optimize_on_insert_long.sql diff --git a/tests/queries/0_stateless/01600_parts_states_metrics.reference b/tests/queries/0_stateless/01600_parts_states_metrics_long.reference similarity index 100% rename from tests/queries/0_stateless/01600_parts_states_metrics.reference rename to tests/queries/0_stateless/01600_parts_states_metrics_long.reference diff --git a/tests/queries/0_stateless/01600_parts_states_metrics.sh b/tests/queries/0_stateless/01600_parts_states_metrics_long.sh similarity index 100% rename from tests/queries/0_stateless/01600_parts_states_metrics.sh rename to tests/queries/0_stateless/01600_parts_states_metrics_long.sh diff --git a/tests/queries/0_stateless/01600_parts_types_metrics.reference b/tests/queries/0_stateless/01600_parts_types_metrics_long.reference similarity index 100% rename from tests/queries/0_stateless/01600_parts_types_metrics.reference rename to tests/queries/0_stateless/01600_parts_types_metrics_long.reference diff --git a/tests/queries/0_stateless/01600_parts_types_metrics.sh b/tests/queries/0_stateless/01600_parts_types_metrics_long.sh similarity index 100% rename from tests/queries/0_stateless/01600_parts_types_metrics.sh rename to tests/queries/0_stateless/01600_parts_types_metrics_long.sh From 46162c292c1cf2e149f9d928eb63f2a887539767 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 22 May 2021 11:20:45 +0300 Subject: [PATCH 27/46] Use unique zk path in tests with ReplicatedMergeTree --- .../00910_zookeeper_test_alter_compression_codecs_long.sql | 4 ++-- .../0_stateless/01451_replicated_detach_drop_part_long.sql | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs_long.sql b/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs_long.sql index e8fb073ba7a..085a79485fb 100644 --- a/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs_long.sql +++ b/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs_long.sql @@ -7,12 +7,12 @@ DROP TABLE IF EXISTS alter_compression_codec2; CREATE TABLE alter_compression_codec1 ( somedate Date CODEC(LZ4), id UInt64 CODEC(NONE) -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/alter_compression_codecs', '1') PARTITION BY somedate ORDER BY id; +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/'||currentDatabase()||'alter_compression_codecs', '1') PARTITION BY somedate ORDER BY id; CREATE TABLE alter_compression_codec2 ( somedate Date CODEC(LZ4), id UInt64 CODEC(NONE) -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/alter_compression_codecs', '2') PARTITION BY somedate ORDER BY id; +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/'||currentDatabase()||'alter_compression_codecs', '2') PARTITION BY somedate ORDER BY id; INSERT INTO alter_compression_codec1 VALUES('2018-01-01', 1); INSERT INTO alter_compression_codec1 VALUES('2018-01-01', 2); diff --git a/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql b/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql index 7cefa9bd7ca..cd8267ce59a 100644 --- a/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql +++ b/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql @@ -3,8 +3,8 @@ SET replication_alter_partitions_sync = 2; DROP TABLE IF EXISTS replica1; DROP TABLE IF EXISTS replica2; -CREATE TABLE replica1 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/01451/attach', 'r1') order by tuple() settings max_replicated_merges_in_queue = 0; -CREATE TABLE replica2 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/01451/attach', 'r2') order by tuple() settings max_replicated_merges_in_queue = 0; +CREATE TABLE replica1 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/'||currentDatabase()||'test/01451/attach', 'r1') order by tuple() settings max_replicated_merges_in_queue = 0; +CREATE TABLE replica2 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/'||currentDatabase()||'test/01451/attach', 'r2') order by tuple() settings max_replicated_merges_in_queue = 0; INSERT INTO replica1 VALUES (0); INSERT INTO replica1 VALUES (1); From ef2335d835e0d84b5bc755f49a55797d9babf0fc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 22 May 2021 11:22:51 +0300 Subject: [PATCH 28/46] Increase timeout in 01676_long_clickhouse_client_autocomplete https://clickhouse-test-reports.s3.yandex.net/24411/84dd2d472812d226655bd0a4420863bbf291d601/functional_stateless_tests_flaky_check_(address)/test_run.txt.out.log --- .../0_stateless/01676_long_clickhouse_client_autocomplete.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01676_long_clickhouse_client_autocomplete.sh b/tests/queries/0_stateless/01676_long_clickhouse_client_autocomplete.sh index 3bb9da92362..8e13f2dcaee 100755 --- a/tests/queries/0_stateless/01676_long_clickhouse_client_autocomplete.sh +++ b/tests/queries/0_stateless/01676_long_clickhouse_client_autocomplete.sh @@ -15,7 +15,7 @@ function test_completion_word() local compword_end=${w:$((w_len-3))} # NOTE: here and below you should escape variables of the expect. - timeout 22s expect << EOF + timeout 60s expect << EOF log_user 0 set timeout 3 match_max 100000 From 42e91df363b29746673dc7269c32458e3ceb4b27 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 22 May 2021 15:22:38 +0300 Subject: [PATCH 29/46] Updated submodule to llvm12 --- CMakeLists.txt | 1 - cmake/find/llvm.cmake | 95 ++++------------------- cmake/find/termcap.cmake | 17 ---- contrib/CMakeLists.txt | 5 +- contrib/llvm | 2 +- src/Core/Settings.h | 2 +- src/DataTypes/Native.h | 11 +-- src/Interpreters/JIT/compileFunction.cpp | 14 ++-- src/Interpreters/examples/jit_example.cpp | 2 +- utils/ci/build-normal.sh | 5 -- utils/ci/default-config | 1 - utils/ci/install-libraries.sh | 4 - utils/ci/jobs/quick-build/run.sh | 1 - 13 files changed, 31 insertions(+), 129 deletions(-) delete mode 100644 cmake/find/termcap.cmake diff --git a/CMakeLists.txt b/CMakeLists.txt index 866d9f542e1..d4a92b91830 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -504,7 +504,6 @@ include (cmake/find/libuv.cmake) # for amqpcpp and cassandra include (cmake/find/amqpcpp.cmake) include (cmake/find/capnp.cmake) include (cmake/find/llvm.cmake) -include (cmake/find/termcap.cmake) # for external static llvm include (cmake/find/h3.cmake) include (cmake/find/libxml2.cmake) include (cmake/find/brotli.cmake) diff --git a/cmake/find/llvm.cmake b/cmake/find/llvm.cmake index e08f45b9932..da1f2122fc6 100644 --- a/cmake/find/llvm.cmake +++ b/cmake/find/llvm.cmake @@ -1,98 +1,31 @@ -if (APPLE OR SPLIT_SHARED_LIBRARIES OR NOT ARCH_AMD64) +if (APPLE OR SPLIT_SHARED_LIBRARIES OR NOT ARCH_AMD64 OR SANITIZE STREQUAL "undefined") set (ENABLE_EMBEDDED_COMPILER OFF CACHE INTERNAL "") endif() option (ENABLE_EMBEDDED_COMPILER "Enable support for 'compile_expressions' option for query execution" ON) -# Broken in macos. TODO: update clang, re-test, enable on Apple -if (ENABLE_EMBEDDED_COMPILER AND NOT SPLIT_SHARED_LIBRARIES AND ARCH_AMD64 AND NOT (SANITIZE STREQUAL "undefined")) - option (USE_INTERNAL_LLVM_LIBRARY "Use bundled or system LLVM library." ${NOT_UNBUNDLED}) -endif() if (NOT ENABLE_EMBEDDED_COMPILER) - if(USE_INTERNAL_LLVM_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal LLVM library with ENABLE_EMBEDDED_COMPILER=OFF") - endif() + set (USE_EMBEDDED_COMPILER 0) return() endif() if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/llvm/llvm/CMakeLists.txt") - if (USE_INTERNAL_LLVM_LIBRARY) - message (WARNING "submodule contrib/llvm is missing. to fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't fidd internal LLVM library") - endif() - set (MISSING_INTERNAL_LLVM_LIBRARY 1) + message (${RECONFIGURE_MESSAGE_LEVEL} "submodule /contrib/llvm is missing. to fix try run: \n git submodule update --init --recursive") endif () -if (NOT USE_INTERNAL_LLVM_LIBRARY) - set (LLVM_PATHS "/usr/local/lib/llvm" "/usr/lib/llvm") +set (USE_EMBEDDED_COMPILER 1) - foreach(llvm_v 11.1 11) - if (NOT LLVM_FOUND) - find_package (LLVM ${llvm_v} CONFIG PATHS ${LLVM_PATHS}) - endif () - endforeach () +set (LLVM_FOUND 1) +set (LLVM_VERSION "12.0.0bundled") +set (LLVM_INCLUDE_DIRS + "${ClickHouse_SOURCE_DIR}/contrib/llvm/llvm/include" + "${ClickHouse_BINARY_DIR}/contrib/llvm/llvm/include" +) +set (LLVM_LIBRARY_DIRS "${ClickHouse_BINARY_DIR}/contrib/llvm/llvm") - if (LLVM_FOUND) - # Remove dynamically-linked zlib and libedit from LLVM's dependencies: - set_target_properties(LLVMSupport PROPERTIES INTERFACE_LINK_LIBRARIES "-lpthread;LLVMDemangle;${ZLIB_LIBRARIES}") - set_target_properties(LLVMLineEditor PROPERTIES INTERFACE_LINK_LIBRARIES "LLVMSupport") - - option(LLVM_HAS_RTTI "Enable if LLVM was build with RTTI enabled" ON) - set (USE_EMBEDDED_COMPILER 1) - else() - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system LLVM") - set (USE_EMBEDDED_COMPILER 0) - endif() - - if (LLVM_FOUND AND OS_LINUX AND USE_LIBCXX AND NOT FORCE_LLVM_WITH_LIBCXX) - message(WARNING "Option USE_INTERNAL_LLVM_LIBRARY is not set but the LLVM library from OS packages " - "in Linux is incompatible with libc++ ABI. LLVM Will be disabled. Force: -DFORCE_LLVM_WITH_LIBCXX=ON") - message (${RECONFIGURE_MESSAGE_LEVEL} "Unsupported LLVM configuration, cannot enable LLVM") - set (LLVM_FOUND 0) - set (USE_EMBEDDED_COMPILER 0) - endif () -endif() - -if(NOT LLVM_FOUND AND NOT MISSING_INTERNAL_LLVM_LIBRARY) - if (CMAKE_CURRENT_SOURCE_DIR STREQUAL CMAKE_CURRENT_BINARY_DIR) - message(WARNING "Option ENABLE_EMBEDDED_COMPILER is set but internal LLVM library cannot build if build directory is the same as source directory.") - set (LLVM_FOUND 0) - set (USE_EMBEDDED_COMPILER 0) - elseif (SPLIT_SHARED_LIBRARIES) - # llvm-tablegen cannot find shared libraries that we build. Probably can be easily fixed. - message(WARNING "Option USE_INTERNAL_LLVM_LIBRARY is not compatible with SPLIT_SHARED_LIBRARIES. Build of LLVM will be disabled.") - set (LLVM_FOUND 0) - set (USE_EMBEDDED_COMPILER 0) - elseif (NOT ARCH_AMD64) - # It's not supported yet, but you can help. - message(WARNING "Option USE_INTERNAL_LLVM_LIBRARY is only available for x86_64. Build of LLVM will be disabled.") - set (LLVM_FOUND 0) - set (USE_EMBEDDED_COMPILER 0) - elseif (SANITIZE STREQUAL "undefined") - # llvm-tblgen, that is used during LLVM build, doesn't work with UBSan. - message(WARNING "Option USE_INTERNAL_LLVM_LIBRARY does not work with UBSan, because 'llvm-tblgen' tool from LLVM has undefined behaviour. Build of LLVM will be disabled.") - set (LLVM_FOUND 0) - set (USE_EMBEDDED_COMPILER 0) - else () - set (USE_INTERNAL_LLVM_LIBRARY ON) - set (LLVM_FOUND 1) - set (USE_EMBEDDED_COMPILER 1) - set (LLVM_VERSION "9.0.0bundled") - set (LLVM_INCLUDE_DIRS - "${ClickHouse_SOURCE_DIR}/contrib/llvm/llvm/include" - "${ClickHouse_BINARY_DIR}/contrib/llvm/llvm/include" - ) - set (LLVM_LIBRARY_DIRS "${ClickHouse_BINARY_DIR}/contrib/llvm/llvm") - endif() -endif() - -if (LLVM_FOUND) - message(STATUS "LLVM include Directory: ${LLVM_INCLUDE_DIRS}") - message(STATUS "LLVM library Directory: ${LLVM_LIBRARY_DIRS}") - message(STATUS "LLVM C++ compiler flags: ${LLVM_CXXFLAGS}") -else() - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't enable LLVM") -endif() +message(STATUS "LLVM include Directory: ${LLVM_INCLUDE_DIRS}") +message(STATUS "LLVM library Directory: ${LLVM_LIBRARY_DIRS}") +message(STATUS "LLVM C++ compiler flags: ${LLVM_CXXFLAGS}") # This list was generated by listing all LLVM libraries, compiling the binary and removing all libraries while it still compiles. set (REQUIRED_LLVM_LIBRARIES diff --git a/cmake/find/termcap.cmake b/cmake/find/termcap.cmake deleted file mode 100644 index 58454165785..00000000000 --- a/cmake/find/termcap.cmake +++ /dev/null @@ -1,17 +0,0 @@ -if (ENABLE_EMBEDDED_COMPILER AND NOT USE_INTERNAL_LLVM_LIBRARY AND USE_STATIC_LIBRARIES) - find_library (TERMCAP_LIBRARY tinfo) - if (NOT TERMCAP_LIBRARY) - find_library (TERMCAP_LIBRARY ncurses) - endif() - if (NOT TERMCAP_LIBRARY) - find_library (TERMCAP_LIBRARY termcap) - endif() - - if (NOT TERMCAP_LIBRARY) - message (FATAL_ERROR "Statically Linking external LLVM requires termcap") - endif() - - target_link_libraries(LLVMSupport INTERFACE ${TERMCAP_LIBRARY}) - - message (STATUS "Using termcap: ${TERMCAP_LIBRARY}") -endif() diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index a9438aa4b76..c499da9d087 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -209,11 +209,12 @@ elseif(GTEST_SRC_DIR) target_compile_definitions(gtest INTERFACE GTEST_HAS_POSIX_RE=0) endif() -if (USE_EMBEDDED_COMPILER AND USE_INTERNAL_LLVM_LIBRARY) +if (USE_EMBEDDED_COMPILER) # ld: unknown option: --color-diagnostics if (APPLE) set (LINKER_SUPPORTS_COLOR_DIAGNOSTICS 0 CACHE INTERNAL "") endif () + set (LLVM_ENABLE_EH 1 CACHE INTERNAL "") set (LLVM_ENABLE_RTTI 1 CACHE INTERNAL "") set (LLVM_ENABLE_PIC 0 CACHE INTERNAL "") @@ -228,8 +229,6 @@ if (USE_EMBEDDED_COMPILER AND USE_INTERNAL_LLVM_LIBRARY) set (CMAKE_CXX_STANDARD ${CMAKE_CXX_STANDARD_bak}) unset (CMAKE_CXX_STANDARD_bak) - - target_include_directories(LLVMSupport SYSTEM BEFORE PRIVATE ${ZLIB_INCLUDE_DIR}) endif () if (USE_INTERNAL_LIBGSASL_LIBRARY) diff --git a/contrib/llvm b/contrib/llvm index cfaf365cf96..a7198805de6 160000 --- a/contrib/llvm +++ b/contrib/llvm @@ -1 +1 @@ -Subproject commit cfaf365cf96918999d09d976ec736b4518cf5d02 +Subproject commit a7198805de67374eb3fb4c6b89797fa2d1cd7e50 diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 125879486ab..737385237e9 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -102,7 +102,7 @@ class IColumn; \ M(Bool, allow_suspicious_low_cardinality_types, false, "In CREATE TABLE statement allows specifying LowCardinality modifier for types of small fixed size (8 or less). Enabling this may increase merge times and memory consumption.", 0) \ M(Bool, compile_expressions, true, "Compile some scalar functions and operators to native code.", 0) \ - M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \ + M(UInt64, min_count_to_compile_expression, 0, "The number of identical expressions before they are JIT-compiled", 0) \ M(UInt64, group_by_two_level_threshold, 100000, "From what number of keys, a two-level aggregation starts. 0 - the threshold is not set.", 0) \ M(UInt64, group_by_two_level_threshold_bytes, 50000000, "From what size of the aggregation state in bytes, a two-level aggregation begins to be used. 0 - the threshold is not set. Two-level aggregation is used when at least one of the thresholds is triggered.", 0) \ M(Bool, distributed_aggregation_memory_efficient, true, "Is the memory-saving mode of distributed aggregation enabled.", 0) \ diff --git a/src/DataTypes/Native.h b/src/DataTypes/Native.h index a31b0c7483c..a1d07b54c8e 100644 --- a/src/DataTypes/Native.h +++ b/src/DataTypes/Native.h @@ -9,7 +9,6 @@ # include # include -# include # include # include # pragma GCC diagnostic push @@ -41,7 +40,8 @@ static inline llvm::Type * toNativeType(llvm::IRBuilderBase & builder, const IDa { const auto & data_type_nullable = static_cast(type); auto * wrapped = toNativeType(builder, *data_type_nullable.getNestedType()); - return wrapped ? llvm::StructType::get(wrapped, /* is null = */ builder.getInt1Ty()) : nullptr; + auto * is_null_type = builder.getInt1Ty(); + return wrapped ? llvm::StructType::get(wrapped, is_null_type) : nullptr; } /// LLVM doesn't have unsigned types, it has unsigned instructions. @@ -57,11 +57,6 @@ static inline llvm::Type * toNativeType(llvm::IRBuilderBase & builder, const IDa return builder.getFloatTy(); else if (data_type.isFloat64()) return builder.getDoubleTy(); - else if (data_type.isFixedString()) - { - const auto & data_type_fixed_string = static_cast(type); - return llvm::VectorType::get(builder.getInt8Ty(), data_type_fixed_string.getN()); - } return nullptr; } @@ -76,7 +71,7 @@ static inline bool canBeNativeType(const IDataType & type) return canBeNativeType(*data_type_nullable.getNestedType()); } - return data_type.isNativeInt() || data_type.isNativeUInt() || data_type.isFloat() || data_type.isFixedString() || data_type.isDate(); + return data_type.isNativeInt() || data_type.isNativeUInt() || data_type.isFloat() || data_type.isDate(); } static inline llvm::Type * toNativeType(llvm::IRBuilderBase & builder, const DataTypePtr & type) diff --git a/src/Interpreters/JIT/compileFunction.cpp b/src/Interpreters/JIT/compileFunction.cpp index abf184b7872..c643722c1cd 100644 --- a/src/Interpreters/JIT/compileFunction.cpp +++ b/src/Interpreters/JIT/compileFunction.cpp @@ -87,7 +87,7 @@ static void compileFunction(llvm::Module & module, const IFunctionBase & functio for (size_t i = 0; i <= arg_types.size(); ++i) { const auto & type = i == arg_types.size() ? function.getResultType() : arg_types[i]; - auto * data = b.CreateLoad(b.CreateConstInBoundsGEP1_32(data_type, columns_arg, i)); + auto * data = b.CreateLoad(data_type, b.CreateConstInBoundsGEP1_32(data_type, columns_arg, i)); columns[i].data_init = b.CreatePointerCast(b.CreateExtractValue(data, {0}), toNativeType(b, removeNullable(type))->getPointerTo()); columns[i].null_init = type->isNullable() ? b.CreateExtractValue(data, {1}) : nullptr; } @@ -122,15 +122,19 @@ static void compileFunction(llvm::Module & module, const IFunctionBase & functio auto & column = columns[i]; auto type = arg_types[i]; - auto * value = b.CreateLoad(column.data); - if (!column.null) + auto * value = b.CreateLoad(toNativeType(b, removeNullable(type)), column.data); + if (!type->isNullable()) { arguments.emplace_back(value); continue; } - auto * is_null = b.CreateICmpNE(b.CreateLoad(column.null), b.getInt8(0)); - auto * nullable_unitilized = llvm::Constant::getNullValue(toNativeType(b, type)); + auto * nullable_type_untyped = toNativeType(b, type); + llvm::StructType * nullable_type = static_cast(nullable_type_untyped); + llvm::Type * is_null_type = nullable_type->getElementType(1); + + auto * is_null = b.CreateICmpNE(b.CreateLoad(is_null_type, column.null), b.getInt8(0)); + auto * nullable_unitilized = llvm::Constant::getNullValue(nullable_type_untyped); auto * nullable_value = b.CreateInsertValue(b.CreateInsertValue(nullable_unitilized, value, {0}), is_null, {1}); arguments.emplace_back(nullable_value); } diff --git a/src/Interpreters/examples/jit_example.cpp b/src/Interpreters/examples/jit_example.cpp index 92215429bfc..db02d843f28 100644 --- a/src/Interpreters/examples/jit_example.cpp +++ b/src/Interpreters/examples/jit_example.cpp @@ -38,7 +38,7 @@ int main(int argc, char **argv) // b.CreateCall(func_declaration); - // auto * load_argument = b.CreateLoad(argument); + // auto * load_argument = b.CreateLoad(value_type, argument); // auto * value = b.CreateAdd(load_argument, load_argument); // b.CreateRet(value); // }); diff --git a/utils/ci/build-normal.sh b/utils/ci/build-normal.sh index b937269c8a3..328bd2c9f51 100755 --- a/utils/ci/build-normal.sh +++ b/utils/ci/build-normal.sh @@ -8,11 +8,6 @@ source default-config mkdir -p "${WORKSPACE}/build" pushd "${WORKSPACE}/build" -if [[ "${ENABLE_EMBEDDED_COMPILER}" == 1 ]]; then - [[ "$USE_LLVM_LIBRARIES_FROM_SYSTEM" == 0 ]] && CMAKE_FLAGS="$CMAKE_FLAGS -DUSE_INTERNAL_LLVM_LIBRARY=1" - [[ "$USE_LLVM_LIBRARIES_FROM_SYSTEM" != 0 ]] && CMAKE_FLAGS="$CMAKE_FLAGS -DUSE_INTERNAL_LLVM_LIBRARY=0" -fi - cmake -DCMAKE_BUILD_TYPE=${BUILD_TYPE} -DENABLE_EMBEDDED_COMPILER=${ENABLE_EMBEDDED_COMPILER} $CMAKE_FLAGS ../sources [[ "$BUILD_TARGETS" != 'all' ]] && BUILD_TARGETS_STRING="--target $BUILD_TARGETS" diff --git a/utils/ci/default-config b/utils/ci/default-config index cd6f25ecf9b..b66121cc757 100644 --- a/utils/ci/default-config +++ b/utils/ci/default-config @@ -27,7 +27,6 @@ CLANG_SOURCES_BRANCH=trunk # or tags/RELEASE_600/final GCC_SOURCES_VERSION=latest # or gcc-7.1.0 # install-libraries -USE_LLVM_LIBRARIES_FROM_SYSTEM=0 # 0 or 1 ENABLE_EMBEDDED_COMPILER=1 # build diff --git a/utils/ci/install-libraries.sh b/utils/ci/install-libraries.sh index d7fb856dbed..7615375fbc1 100755 --- a/utils/ci/install-libraries.sh +++ b/utils/ci/install-libraries.sh @@ -5,7 +5,3 @@ source default-config ./install-os-packages.sh libicu-dev ./install-os-packages.sh libreadline-dev - -if [[ "$ENABLE_EMBEDDED_COMPILER" == 1 && "$USE_LLVM_LIBRARIES_FROM_SYSTEM" == 1 ]]; then - ./install-os-packages.sh llvm-libs-5.0 -fi diff --git a/utils/ci/jobs/quick-build/run.sh b/utils/ci/jobs/quick-build/run.sh index 3d755625c8d..af977d14465 100755 --- a/utils/ci/jobs/quick-build/run.sh +++ b/utils/ci/jobs/quick-build/run.sh @@ -15,7 +15,6 @@ SOURCES_METHOD=local COMPILER=clang COMPILER_INSTALL_METHOD=packages COMPILER_PACKAGE_VERSION=6.0 -USE_LLVM_LIBRARIES_FROM_SYSTEM=0 BUILD_METHOD=normal BUILD_TARGETS=clickhouse BUILD_TYPE=Debug From e4ce6faab6f1652b1b539673e431de1559a2bff6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 22 May 2021 15:32:53 +0300 Subject: [PATCH 30/46] Fix build --- src/Server/KeeperTCPHandler.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Server/KeeperTCPHandler.h b/src/Server/KeeperTCPHandler.h index 6c3929198c0..76371ed1a0c 100644 --- a/src/Server/KeeperTCPHandler.h +++ b/src/Server/KeeperTCPHandler.h @@ -38,7 +38,7 @@ private: IServer & server; Poco::Logger * log; ContextPtr global_context; - std::shared_ptr nu_keeper_storage_dispatcher; + std::shared_ptr keeper_dispatcher; Poco::Timespan operation_timeout; Poco::Timespan session_timeout; int64_t session_id{-1}; From 8e6fabd67466ba4e9baebf23b8d2c2bcead94454 Mon Sep 17 00:00:00 2001 From: BoloniniD Date: Sat, 22 May 2021 19:42:09 +0300 Subject: [PATCH 31/46] Fixes for .md, preparing to add config-example.yaml --- docs/en/operations/configuration-files.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/configuration-files.md b/docs/en/operations/configuration-files.md index 32239b0dc80..0000bf6815b 100644 --- a/docs/en/operations/configuration-files.md +++ b/docs/en/operations/configuration-files.md @@ -5,9 +5,9 @@ toc_title: Configuration Files # Configuration Files {#configuration_files} -ClickHouse supports multi-file configuration management. The main server configuration file is `/etc/clickhouse-server/config.xml` or `/etc/clickhouse-server/config.yaml`. Other files must be in the `/etc/clickhouse-server/config.d` directory. Note, that any configuration file can be written either in XML or YAML, but mixing formats in one file is not supported. +ClickHouse supports multi-file configuration management. The main server configuration file is `/etc/clickhouse-server/config.xml` or `/etc/clickhouse-server/config.yaml`. Other files must be in the `/etc/clickhouse-server/config.d` directory. Note, that any configuration file can be written either in XML or YAML, but mixing formats in one file is not supported. For example, you can have main configs as `config.xml` and `users.xml` and write additional files in `config.d` and `users.d` directories in `.yaml`. -All the configuration files should be in XML or YAML formats. All XML files should have the same root element, usually ``. As for YAML, writing `yandex:` is not needed, parser will insert it automatically. +All the configuration files should be in XML or YAML formats. All XML files should have the same root element, usually ``. As for YAML, `yandex:` should not be present, the parser will insert it automatically. ## Override {#override} From c220260a1dabc28ba57b6eb4262d79d9e6b042b0 Mon Sep 17 00:00:00 2001 From: BoloniniD Date: Sat, 22 May 2021 21:27:46 +0300 Subject: [PATCH 32/46] Added default config in YAML to programs/server --- programs/server/config-example.yaml | 980 ++++++++++++++++++++++++++++ 1 file changed, 980 insertions(+) create mode 100644 programs/server/config-example.yaml diff --git a/programs/server/config-example.yaml b/programs/server/config-example.yaml new file mode 100644 index 00000000000..49e5f7ee853 --- /dev/null +++ b/programs/server/config-example.yaml @@ -0,0 +1,980 @@ +# NOTE: User and query level settings are set up in "users.xml" file. +# If you have accidentally specified user-level settings here, server won't start. +# You can either move the settings to the right place inside "users.xml" file +# or add skip_check_for_incorrect_settings: 1 here. +logger: +# Possible levels [1]: +# - none (turns off logging) +# - fatal +# - critical +# - error +# - warning +# - notice +# - information +# - debug +# - trace +# +# [1]: https://github.com/pocoproject/poco/blob/poco-1.9.4-release/Foundation/include/Poco/Logger.h#L105-L114 + level: trace + log: /var/log/clickhouse-server/clickhouse-server.log + errorlog: /var/log/clickhouse-server/clickhouse-server.err.log +# Rotation policy +# See https://github.com/pocoproject/poco/blob/poco-1.9.4-release/Foundation/include/Poco/FileChannel.h#L54-L85 + size: 1000M + count: 10 +# console: 1 +# Default behavior is autodetection (log to console if not daemon mode and is tty) +# Per level overrides (legacy): +# For example to suppress logging of the ConfigReloader you can use: +# NOTE: levels.logger is reserved, see below. + +# levels: +# ConfigReloader: none + +# Per level overrides: +# For example to suppress logging of the RBAC for default user you can use: +# (But please note that the logger name maybe changed from version to version, even after minor upgrade) +# +# levels: +# - logger: +# name: 'ContextAccess (default)' +# level: none +# - logger: +# name: 'DatabaseOrdinary (test)' +# level: none + +# It is the name that will be shown in the clickhouse-client. +# By default, anything with "production" will be highlighted in red in query prompt. +# +# display_name: production + +# Port for HTTP API. See also 'https_port' for secure connections. +# This interface is also used by ODBC and JDBC drivers (DataGrip, Dbeaver, ...) +# and by most of web interfaces (embedded UI, Grafana, Redash, ...). +# +http_port: 8123 +# Port for interaction by native protocol with: +# - clickhouse-client and other native ClickHouse tools (clickhouse-benchmark, clickhouse-copier); +# - clickhouse-server with other clickhouse-servers for distributed query processing; +# - ClickHouse drivers and applications supporting native protocol +# (this protocol is also informally called as "the TCP protocol"); +# See also 'tcp_port_secure' for secure connections. +# +tcp_port: 9000 +# Compatibility with MySQL protocol. +# ClickHouse will pretend to be MySQL for applications connecting to this port. +# +mysql_port: 9004 +# Compatibility with PostgreSQL protocol. +# ClickHouse will pretend to be PostgreSQL for applications connecting to this port. +# +postgresql_port: 9005 +# HTTP API with TLS (HTTPS). +# You have to configure certificate to enable this interface. +# See the openSSL section below. +# +# https_port: 8443 + +# Native interface with TLS. +# You have to configure certificate to enable this interface. +# See the openSSL section below. +# +# tcp_port_secure: 9440 + +# Native interface wrapped with PROXYv1 protocol +# PROXYv1 header sent for every connection. +# ClickHouse will extract information about proxy-forwarded client address from the header. +# +# tcp_with_proxy_port: 9011 + +# Port for communication between replicas. Used for data exchange. +# It provides low-level data access between servers. +# This port should not be accessible from untrusted networks. +# See also 'interserver_http_credentials'. +# Data transferred over connections to this port should not go through untrusted networks. +# See also 'interserver_https_port'. +# +interserver_http_port: 9009 +# Port for communication between replicas with TLS. +# You have to configure certificate to enable this interface. +# See the openSSL section below. +# See also 'interserver_http_credentials'. +# +# interserver_https_port: 9010 + +# Hostname that is used by other replicas to request this server. +# If not specified, than it is determined analogous to 'hostname -f' command. +# This setting could be used to switch replication to another network interface +# (the server may be connected to multiple networks via multiple addresses) +# +# +# interserver_http_host: example.yandex.ru +# + +# You can specify credentials for authenthication between replicas. +# This is required when interserver_https_port is accessible from untrusted networks, +# and also recommended to avoid SSRF attacks from possibly compromised services in your network. +# +# interserver_http_credentials: +# user: interserver +# password: '' + +# Listen specified address. +# Use :: (wildcard IPv6 address), if you want to accept connections both with IPv4 and IPv6 from everywhere. +# Notes: +# If you open connections from wildcard address, make sure that at least one of the following measures applied: +# - server is protected by firewall and not accessible from untrusted networks; +# - all users are restricted to subset of network addresses (see users.xml); +# - all users have strong passwords, only secure (TLS) interfaces are accessible, or connections are only made via TLS interfaces. +# - users without password have readonly access. +# See also: https://www.shodan.io/search?query=clickhouse +# +# listen_host: '::' + +# Same for hosts without support for IPv6: +# listen_host: 0.0.0.0 + +# Default values - try listen localhost on IPv4 and IPv6. +# +# listen_host: '::1' +# listen_host: 127.0.0.1 +# + +# Don't exit if IPv6 or IPv4 networks are unavailable while trying to listen. +# listen_try: 0 + +# Allow multiple servers to listen on the same address:port. This is not recommended. +# +# listen_reuse_port: 0 + +# listen_backlog: 64 +max_connections: 4096 +# For 'Connection: keep-alive' in HTTP 1.1 +keep_alive_timeout: 3 +# gRPC protocol (see src/Server/grpc_protos/clickhouse_grpc.proto for the API) +# grpc_port: 9100 +grpc: + enable_ssl: false +# The following two files are used only if enable_ssl=1 + ssl_cert_file: /path/to/ssl_cert_file + ssl_key_file: /path/to/ssl_key_file +# Whether server will request client for a certificate + ssl_require_client_auth: false +# The following file is used only if ssl_require_client_auth=1 + ssl_ca_cert_file: /path/to/ssl_ca_cert_file +# Default compression algorithm (applied if client doesn't specify another algorithm). +# Supported algorithms: none, deflate, gzip, stream_gzip + compression: deflate +# Default compression level (applied if client doesn't specify another level). +# Supported levels: none, low, medium, high + compression_level: medium +# Send/receive message size limits in bytes. -1 means unlimited + max_send_message_size: -1 + max_receive_message_size: -1 +# Enable if you want very detailed logs + verbose_logs: false +# Used with https_port and tcp_port_secure. Full ssl options list: https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h#L71 +openSSL: + server: + # Used for https server AND secure tcp port + # openssl req -subj "/CN=localhost" -new -newkey rsa:2048 -days 365 -nodes -x509 -keyout /etc/clickhouse-server/server.key -out /etc/clickhouse-server/server.crt + certificateFile: /etc/clickhouse-server/server.crt + privateKeyFile: /etc/clickhouse-server/server.key + # dhparams are optional. You can delete the element. + # To generate dhparams, use the following command: + # openssl dhparam -out /etc/clickhouse-server/dhparam.pem 4096 + # Only file format with BEGIN DH PARAMETERS is supported. + # + dhParamsFile: /etc/clickhouse-server/dhparam.pem + verificationMode: none + loadDefaultCAFile: true + cacheSessions: true + disableProtocols: 'sslv2,sslv3' + preferServerCiphers: true + client: + # Used for connecting to https dictionary source and secured Zookeeper communication + loadDefaultCAFile: true + cacheSessions: true + disableProtocols: 'sslv2,sslv3' + preferServerCiphers: true + # Use for self-signed: none + invalidCertificateHandler: + # Use for self-signed: AcceptCertificateHandler + name: RejectCertificateHandler +# Default root page on http[s] server. For example load UI from https://tabix.io/ when opening http://localhost:8123 +# +#
]]>
+# + +# Maximum number of concurrent queries. +max_concurrent_queries: 100 +# Maximum memory usage (resident set size) for server process. +# Zero value or unset means default. Default is "max_server_memory_usage_to_ram_ratio" of available physical RAM. +# If the value is larger than "max_server_memory_usage_to_ram_ratio" of available physical RAM, it will be cut down. + +# The constraint is checked on query execution time. +# If a query tries to allocate memory and the current memory usage plus allocation is greater +# than specified threshold, exception will be thrown. + +# It is not practical to set this constraint to small values like just a few gigabytes, +# because memory allocator will keep this amount of memory in caches and the server will deny service of queries. +# +max_server_memory_usage: 0 +# Maximum number of threads in the Global thread pool. +# This will default to a maximum of 10000 threads if not specified. +# This setting will be useful in scenarios where there are a large number +# of distributed queries that are running concurrently but are idling most +# of the time, in which case a higher number of threads might be required. +# +max_thread_pool_size: 10000 +# On memory constrained environments you may have to set this to value larger than 1. +max_server_memory_usage_to_ram_ratio: 0.9 +# Simple server-wide memory profiler. Collect a stack trace at every peak allocation step (in bytes). +# Data will be stored in system.trace_log table with query_id = empty string. +# Zero means disabled. +# +total_memory_profiler_step: 4194304 +# Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. +# The probability is for every alloc/free regardless to the size of the allocation. +# Note that sampling happens only when the amount of untracked memory exceeds the untracked memory limit, +# which is 4 MiB by default but can be lowered if 'total_memory_profiler_step' is lowered. +# You may want to set 'total_memory_profiler_step' to 1 for extra fine grained sampling. +# +total_memory_tracker_sample_probability: 0 +# Set limit on number of open files (default: maximum). This setting makes sense on Mac OS X because getrlimit() fails to retrieve +# correct maximum value. +# max_open_files: 262144 + +# Size of cache of uncompressed blocks of data, used in tables of MergeTree family. +# In bytes. Cache is single for server. Memory is allocated only on demand. +# Cache is used when 'use_uncompressed_cache' user setting turned on (off by default). +# Uncompressed cache is advantageous only for very short queries and in rare cases. + +# Note: uncompressed cache can be pointless for lz4, because memory bandwidth +# is slower than multi-core decompression on some server configurations. +# Enabling it can sometimes paradoxically make queries slower. +# +uncompressed_cache_size: 8589934592 +# Approximate size of mark cache, used in tables of MergeTree family. +# In bytes. Cache is single for server. Memory is allocated only on demand. +# You should not lower this value. +# +mark_cache_size: 5368709120 +# If you enable the `min_bytes_to_use_mmap_io` setting, +# the data in MergeTree tables can be read with mmap to avoid copying from kernel to userspace. +# It makes sense only for large files and helps only if data reside in page cache. +# To avoid frequent open/mmap/munmap/close calls (which are very expensive due to consequent page faults) +# and to reuse mappings from several threads and queries, +# the cache of mapped files is maintained. Its size is the number of mapped regions (usually equal to the number of mapped files). +# The amount of data in mapped files can be monitored +# in system.metrics, system.metric_log by the MMappedFiles, MMappedFileBytes metrics +# and in system.asynchronous_metrics, system.asynchronous_metrics_log by the MMapCacheCells metric, +# and also in system.events, system.processes, system.query_log, system.query_thread_log by the +# CreatedReadBufferMMap, CreatedReadBufferMMapFailed, MMappedFileCacheHits, MMappedFileCacheMisses events. +# Note that the amount of data in mapped files does not consume memory directly and is not accounted +# in query or server memory usage - because this memory can be discarded similar to OS page cache. +# The cache is dropped (the files are closed) automatically on removal of old parts in MergeTree, +# also it can be dropped manually by the SYSTEM DROP MMAP CACHE query. +# +mmap_cache_size: 1000 +# Cache size for compiled expressions. +compiled_expression_cache_size: 1073741824 +# Path to data directory, with trailing slash. +path: /var/lib/clickhouse/ +# Path to temporary data for processing hard queries. +tmp_path: /var/lib/clickhouse/tmp/ +# Policy from the for the temporary files. +# If not set is used, otherwise is ignored. + +# Notes: +# - move_factor is ignored +# - keep_free_space_bytes is ignored +# - max_data_part_size_bytes is ignored +# - you must have exactly one volume in that policy +# +# tmp_policy: tmp + +# Directory with user provided files that are accessible by 'file' table function. +user_files_path: /var/lib/clickhouse/user_files/ +# LDAP server definitions. +ldap_servers: '' +# List LDAP servers with their connection parameters here to later 1) use them as authenticators for dedicated local users, +# who have 'ldap' authentication mechanism specified instead of 'password', or to 2) use them as remote user directories. +# Parameters: +# host - LDAP server hostname or IP, this parameter is mandatory and cannot be empty. +# port - LDAP server port, default is 636 if enable_tls is set to true, 389 otherwise. +# bind_dn - template used to construct the DN to bind to. +# The resulting DN will be constructed by replacing all '{user_name}' substrings of the template with the actual +# user name during each authentication attempt. +# user_dn_detection - section with LDAP search parameters for detecting the actual user DN of the bound user. +# This is mainly used in search filters for further role mapping when the server is Active Directory. The +# resulting user DN will be used when replacing '{user_dn}' substrings wherever they are allowed. By default, +# user DN is set equal to bind DN, but once search is performed, it will be updated with to the actual detected +# user DN value. +# base_dn - template used to construct the base DN for the LDAP search. +# The resulting DN will be constructed by replacing all '{user_name}' and '{bind_dn}' substrings +# of the template with the actual user name and bind DN during the LDAP search. +# scope - scope of the LDAP search. +# Accepted values are: 'base', 'one_level', 'children', 'subtree' (the default). +# search_filter - template used to construct the search filter for the LDAP search. +# The resulting filter will be constructed by replacing all '{user_name}', '{bind_dn}', and '{base_dn}' +# substrings of the template with the actual user name, bind DN, and base DN during the LDAP search. +# Note, that the special characters must be escaped properly in XML. +# verification_cooldown - a period of time, in seconds, after a successful bind attempt, during which a user will be assumed +# to be successfully authenticated for all consecutive requests without contacting the LDAP server. +# Specify 0 (the default) to disable caching and force contacting the LDAP server for each authentication request. +# enable_tls - flag to trigger use of secure connection to the LDAP server. +# Specify 'no' for plain text (ldap://) protocol (not recommended). +# Specify 'yes' for LDAP over SSL/TLS (ldaps://) protocol (recommended, the default). +# Specify 'starttls' for legacy StartTLS protocol (plain text (ldap://) protocol, upgraded to TLS). +# tls_minimum_protocol_version - the minimum protocol version of SSL/TLS. +# Accepted values are: 'ssl2', 'ssl3', 'tls1.0', 'tls1.1', 'tls1.2' (the default). +# tls_require_cert - SSL/TLS peer certificate verification behavior. +# Accepted values are: 'never', 'allow', 'try', 'demand' (the default). +# tls_cert_file - path to certificate file. +# tls_key_file - path to certificate key file. +# tls_ca_cert_file - path to CA certificate file. +# tls_ca_cert_dir - path to the directory containing CA certificates. +# tls_cipher_suite - allowed cipher suite (in OpenSSL notation). +# Example: +# my_ldap_server: +# host: localhost +# port: 636 +# bind_dn: 'uid={user_name},ou=users,dc=example,dc=com' +# verification_cooldown: 300 +# enable_tls: yes +# tls_minimum_protocol_version: tls1.2 +# tls_require_cert: demand +# tls_cert_file: /path/to/tls_cert_file +# tls_key_file: /path/to/tls_key_file +# tls_ca_cert_file: /path/to/tls_ca_cert_file +# tls_ca_cert_dir: /path/to/tls_ca_cert_dir +# tls_cipher_suite: ECDHE-ECDSA-AES256-GCM-SHA384:ECDHE-RSA-AES256-GCM-SHA384:AES256-GCM-SHA384 +# +# Example (typical Active Directory with configured user DN detection for further role mapping): +# my_ad_server: +# host: localhost +# port: 389 +# bind_dn: 'EXAMPLE\{user_name}' +# user_dn_detection: +# base_dn: CN=Users,DC=example,DC=com +# search_filter: '(&(objectClass=user)(sAMAccountName={user_name}))' +# enable_tls: no +# + +# To enable Kerberos authentication support for HTTP requests (GSS-SPNEGO), for those users who are explicitly configured +# to authenticate via Kerberos, define a single 'kerberos' section here. +# Parameters: +# principal - canonical service principal name, that will be acquired and used when accepting security contexts. +# This parameter is optional, if omitted, the default principal will be used. +# This parameter cannot be specified together with 'realm' parameter. +# realm - a realm, that will be used to restrict authentication to only those requests whose initiator's realm matches it. +# This parameter is optional, if omitted, no additional filtering by realm will be applied. +# This parameter cannot be specified together with 'principal' parameter. +# Example: +# kerberos: '' +# +# Example: +# kerberos: +# principal: HTTP/clickhouse.example.com@EXAMPLE.COM +# +# Example: +# kerberos: +# realm: EXAMPLE.COM +# + +# Sources to read users, roles, access rights, profiles of settings, quotas. +user_directories: + users_xml: + # Path to configuration file with predefined users. + path: users.yaml + local_directory: + # Path to folder where users created by SQL commands are stored. + path: /var/lib/clickhouse/access/ +# To add an LDAP server as a remote user directory of users that are not defined locally, define a single 'ldap' section +# with the following parameters: +# server - one of LDAP server names defined in 'ldap_servers' config section above. +# This parameter is mandatory and cannot be empty. +# roles - section with a list of locally defined roles that will be assigned to each user retrieved from the LDAP server. +# If no roles are specified here or assigned during role mapping (below), user will not be able to perform any +# actions after authentication. +# role_mapping - section with LDAP search parameters and mapping rules. +# When a user authenticates, while still bound to LDAP, an LDAP search is performed using search_filter and the +# name of the logged in user. For each entry found during that search, the value of the specified attribute is +# extracted. For each attribute value that has the specified prefix, the prefix is removed, and the rest of the +# value becomes the name of a local role defined in ClickHouse, which is expected to be created beforehand by +# CREATE ROLE command. +# There can be multiple 'role_mapping' sections defined inside the same 'ldap' section. All of them will be +# applied. +# base_dn - template used to construct the base DN for the LDAP search. +# The resulting DN will be constructed by replacing all '{user_name}', '{bind_dn}', and '{user_dn}' +# substrings of the template with the actual user name, bind DN, and user DN during each LDAP search. +# scope - scope of the LDAP search. +# Accepted values are: 'base', 'one_level', 'children', 'subtree' (the default). +# search_filter - template used to construct the search filter for the LDAP search. +# The resulting filter will be constructed by replacing all '{user_name}', '{bind_dn}', '{user_dn}', and +# '{base_dn}' substrings of the template with the actual user name, bind DN, user DN, and base DN during +# each LDAP search. +# Note, that the special characters must be escaped properly in XML. +# attribute - attribute name whose values will be returned by the LDAP search. 'cn', by default. +# prefix - prefix, that will be expected to be in front of each string in the original list of strings returned by +# the LDAP search. Prefix will be removed from the original strings and resulting strings will be treated +# as local role names. Empty, by default. +# Example: +# +# ldap: +# server: my_ldap_server +# roles: +# my_local_role1: '' +# my_local_role2: '' +# role_mapping: +# base_dn: 'ou=groups,dc=example,dc=com' +# scope: subtree +# search_filter: '(&(objectClass=groupOfNames)(member={bind_dn}))' +# attribute: cn +# prefix: clickhouse_ +# +# Example (typical Active Directory with role mapping that relies on the detected user DN): +# ldap: +# server: my_ad_server +# role_mapping: +# base_dn: 'CN=Users,DC=example,DC=com' +# attribute: CN +# scope: subtree +# search_filter: '(&(objectClass=group)(member={user_dn}))' +# prefix: clickhouse_ +# + +# Default profile of settings. +default_profile: default +# Comma-separated list of prefixes for user-defined settings. +# custom_settings_prefixes: '' +# System profile of settings. This settings are used by internal processes (Distributed DDL worker and so on). +# system_profile: default + +# Buffer profile of settings. +# This settings are used by Buffer storage to flush data to the underlying table. +# Default: used from system_profile directive. +# +# buffer_profile: default + +# Default database. +default_database: default +# Server time zone could be set here. + +# Time zone is used when converting between String and DateTime types, +# when printing DateTime in text formats and parsing DateTime from text, +# it is used in date and time related functions, if specific time zone was not passed as an argument. + +# Time zone is specified as identifier from IANA time zone database, like UTC or Africa/Abidjan. +# If not specified, system time zone at server startup is used. + +# Please note, that server could display time zone alias instead of specified name. +# Example: W-SU is an alias for Europe/Moscow and Zulu is an alias for UTC. +# +# timezone: Europe/Moscow + +# You can specify umask here (see "man umask"). Server will apply it on startup. +# Number is always parsed as octal. Default umask is 027 (other users cannot read logs, data files, etc; group can only read). +# +# umask: 022 + +# Perform mlockall after startup to lower first queries latency +# and to prevent clickhouse executable from being paged out under high IO load. +# Enabling this option is recommended but will lead to increased startup time for up to a few seconds. +# +mlock_executable: true +# Reallocate memory for machine code ("text") using huge pages. Highly experimental. +remap_executable: false +# Uncomment below in order to use JDBC table engine and function. +# To install and run JDBC bridge in background: +# * [Debian/Ubuntu] +# export MVN_URL=https://repo1.maven.org/maven2/ru/yandex/clickhouse/clickhouse-jdbc-bridge +# export PKG_VER=$(curl -sL $MVN_URL/maven-metadata.xml | grep '' | sed -e 's|.*>\(.*\)<.*|\1|') +# wget https://github.com/ClickHouse/clickhouse-jdbc-bridge/releases/download/v$PKG_VER/clickhouse-jdbc-bridge_$PKG_VER-1_all.deb +# apt install --no-install-recommends -f ./clickhouse-jdbc-bridge_$PKG_VER-1_all.deb +# clickhouse-jdbc-bridge & +# * [CentOS/RHEL] +# export MVN_URL=https://repo1.maven.org/maven2/ru/yandex/clickhouse/clickhouse-jdbc-bridge +# export PKG_VER=$(curl -sL $MVN_URL/maven-metadata.xml | grep '' | sed -e 's|.*>\(.*\)<.*|\1|') +# wget https://github.com/ClickHouse/clickhouse-jdbc-bridge/releases/download/v$PKG_VER/clickhouse-jdbc-bridge-$PKG_VER-1.noarch.rpm +# yum localinstall -y clickhouse-jdbc-bridge-$PKG_VER-1.noarch.rpm +# clickhouse-jdbc-bridge & +# Please refer to https://github.com/ClickHouse/clickhouse-jdbc-bridge#usage for more information. + +# jdbc_bridge: +# host: 127.0.0.1 +# port: 9019 + +# Configuration of clusters that could be used in Distributed tables. +# https://clickhouse.tech/docs/en/operations/table_engines/distributed/ +# +remote_servers: +# Test only shard config for testing distributed storage + test_shard_localhost: +# Inter-server per-cluster secret for Distributed queries +# default: no secret (no authentication will be performed) + +# If set, then Distributed queries will be validated on shards, so at least: +# - such cluster should exist on the shard, +# - such cluster should have the same secret. + +# And also (and which is more important), the initial_user will +# be used as current user for the query. + +# Right now the protocol is pretty simple and it only takes into account: +# - cluster name +# - query + +# Also it will be nice if the following will be implemented: +# - source hostname (see interserver_http_host), but then it will depends from DNS, +# it can use IP address instead, but then the you need to get correct on the initiator node. +# - target hostname / ip address (same notes as for source hostname) +# - time-based security tokens +# +# secret: '' + shard: + # Optional. Whether to write data to just one of the replicas. Default: false (write data to all replicas). + # internal_replication: false + # Optional. Shard weight when writing data. Default: 1. + # weight: 1 + replica: + host: localhost + port: 9000 + # Optional. Priority of the replica for load_balancing. Default: 1 (less value has more priority). + # priority: 1 + test_cluster_two_shards_localhost: + shard: + - replica: + host: localhost + port: 9000 + - replica: + host: localhost + port: 9000 + test_cluster_two_shards: + shard: + - replica: + host: 127.0.0.1 + port: 9000 + - replica: + host: 127.0.0.2 + port: 9000 + test_cluster_two_shards_internal_replication: + shard: + - internal_replication: true + replica: + host: 127.0.0.1 + port: 9000 + - internal_replication: true + replica: + host: 127.0.0.2 + port: 9000 + test_shard_localhost_secure: + shard: + replica: + host: localhost + port: 9440 + secure: 1 + test_unavailable_shard: + shard: + - replica: + host: localhost + port: 9000 + - replica: + host: localhost + port: 1 +# The list of hosts allowed to use in URL-related storage engines and table functions. +# If this section is not present in configuration, all hosts are allowed. +# +# remote_url_allow_hosts: +# +# Host should be specified exactly as in URL. The name is checked before DNS resolution. +# Example: "yandex.ru", "yandex.ru." and "www.yandex.ru" are different hosts. +# If port is explicitly specified in URL, the host:port is checked as a whole. +# If host specified here without port, any port with this host allowed. +# "yandex.ru" -> "yandex.ru:443", "yandex.ru:80" etc. is allowed, but "yandex.ru:80" -> only "yandex.ru:80" is allowed. +# If the host is specified as IP address, it is checked as specified in URL. Example: "[2a02:6b8:a::a]". +# If there are redirects and support for redirects is enabled, every redirect (the Location field) is checked. +# + +# Regular expression can be specified. RE2 engine is used for regexps. +# Regexps are not aligned: don't forget to add ^ and $. Also don't forget to escape dot (.) metacharacter +# (forgetting to do so is a common source of error). +# + +# If element has 'incl' attribute, then for it's value will be used corresponding substitution from another file. +# By default, path to file with substitutions is /etc/metrika.xml. It could be changed in config in 'include_from' element. +# Values for substitutions are specified in /yandex/name_of_substitution elements in that file. +# + +# ZooKeeper is used to store metadata about replicas, when using Replicated tables. +# Optional. If you don't use replicated tables, you could omit that. + +# See https://clickhouse.tech/docs/en/engines/table-engines/mergetree-family/replication/ +# + +# +# zookeeper: +# - node: +# host: example1 +# port: 2181 +# - node: +# host: example2 +# port: 2181 +# - node: +# host: example3 +# port: 2181 +# + +# Substitutions for parameters of replicated tables. +# Optional. If you don't use replicated tables, you could omit that. + +# See https://clickhouse.tech/docs/en/engines/table-engines/mergetree-family/replication/#creating-replicated-tables +# +# +# macros: +# shard: 01 +# replica: example01-01-1 +# + +# Reloading interval for embedded dictionaries, in seconds. Default: 3600. +builtin_dictionaries_reload_interval: 3600 +# Maximum session timeout, in seconds. Default: 3600. +max_session_timeout: 3600 +# Default session timeout, in seconds. Default: 60. +default_session_timeout: 60 +# Sending data to Graphite for monitoring. Several sections can be defined. +# +# interval - send every X second +# root_path - prefix for keys +# hostname_in_path - append hostname to root_path (default = true) +# metrics - send data from table system.metrics +# events - send data from table system.events +# asynchronous_metrics - send data from table system.asynchronous_metrics +# +# +# graphite: +# host: localhost +# port: 42000 +# timeout: 0.1 +# interval: 60 +# root_path: one_min +# hostname_in_path: true + +# metrics: true +# events: true +# events_cumulative: false +# asynchronous_metrics: true +# +# graphite: +# host: localhost +# port: 42000 +# timeout: 0.1 +# interval: 1 +# root_path: one_sec + +# metrics: true +# events: true +# events_cumulative: false +# asynchronous_metrics: false +# + +# Serve endpoint for Prometheus monitoring. +# +#endpoint - mertics path (relative to root, statring with "/") +#port - port to setup server. If not defined or 0 than http_port used +#metrics - send data from table system.metrics +#events - send data from table system.events +#asynchronous_metrics - send data from table system.asynchronous_metrics +#status_info - send data from different component from CH, ex: Dictionaries status +# +# +# prometheus: +# endpoint: /metrics +# port: 9363 + +# metrics: true +# events: true +# asynchronous_metrics: true +# status_info: true +# + +# Query log. Used only for queries with setting log_queries = 1. +query_log: +# What table to insert data. If table is not exist, it will be created. +# When query log structure is changed after system update, +# then old table will be renamed and new table will be created automatically. +# + database: system + table: query_log +# +# PARTITION BY expr: https://clickhouse.yandex/docs/en/table_engines/mergetree-family/custom_partitioning_key/ +# Example: +# event_date +# toMonday(event_date) +# toYYYYMM(event_date) +# toStartOfHour(event_time) +# + partition_by: toYYYYMM(event_date) +# +# Table TTL specification: https://clickhouse.tech/docs/en/engines/table-engines/mergetree-family/mergetree/#mergetree-table-ttl +# Example: +# event_date + INTERVAL 1 WEEK +# event_date + INTERVAL 7 DAY DELETE +# event_date + INTERVAL 2 WEEK TO DISK 'bbb' + +# ttl: 'event_date + INTERVAL 30 DAY DELETE' +# + +# Instead of partition_by, you can provide full engine expression (starting with ENGINE = ) with parameters, +# Example: engine: 'ENGINE = MergeTree PARTITION BY toYYYYMM(event_date) ORDER BY (event_date, event_time) SETTINGS index_granularity = 1024' +# + +# Interval of flushing data. + flush_interval_milliseconds: 7500 +# Trace log. Stores stack traces collected by query profilers. +# See query_profiler_real_time_period_ns and query_profiler_cpu_time_period_ns settings. +trace_log: + database: system + table: trace_log + partition_by: toYYYYMM(event_date) + flush_interval_milliseconds: 7500 +# Query thread log. Has information about all threads participated in query execution. +# Used only for queries with setting log_query_threads = 1. +query_thread_log: + database: system + table: query_thread_log + partition_by: toYYYYMM(event_date) + flush_interval_milliseconds: 7500 +# Uncomment if use part log. +# Part log contains information about all actions with parts in MergeTree tables (creation, deletion, merges, downloads). +# part_log: +# database: system +# table: part_log +# flush_interval_milliseconds: 7500 +# + +# Uncomment to write text log into table. +# Text log contains all information from usual server log but stores it in structured and efficient way. +# The level of the messages that goes to the table can be limited (), if not specified all messages will go to the table. +# text_log: +# database: system +# table: text_log +# flush_interval_milliseconds: 7500 +# level: '' +# + +# Metric log contains rows with current values of ProfileEvents, CurrentMetrics collected with "collect_interval_milliseconds" interval. +metric_log: + database: system + table: metric_log + flush_interval_milliseconds: 7500 + collect_interval_milliseconds: 1000 +# +#Asynchronous metric log contains values of metrics from +#system.asynchronous_metrics. +# +asynchronous_metric_log: + database: system + table: asynchronous_metric_log +# +# Asynchronous metrics are updated once a minute, so there is +# no need to flush more often. +# + flush_interval_milliseconds: 60000 + +# +#OpenTelemetry log contains OpenTelemetry trace spans. +# +opentelemetry_span_log: +# +# The default table creation code is insufficient, this spec +# is a workaround. There is no 'event_time' for this log, but two times, +# start and finish. It is sorted by finish time, to avoid inserting +# data too far away in the past (probably we can sometimes insert a span +# that is seconds earlier than the last span in the table, due to a race +# between several spans inserted in parallel). This gives the spans a +# global order that we can use to e.g. retry insertion into some external +# system. +# + engine: |- + engine MergeTree + partition by toYYYYMM(finish_date) + order by (finish_date, finish_time_us, trace_id) + database: system + table: opentelemetry_span_log + flush_interval_milliseconds: 7500 +# Crash log. Stores stack traces for fatal errors. +# This table is normally empty. +crash_log: + database: system + table: crash_log + partition_by: '' + flush_interval_milliseconds: 1000 +# Parameters for embedded dictionaries, used in Yandex.Metrica. +# See https://clickhouse.yandex/docs/en/dicts/internal_dicts/ +# + +# Path to file with region hierarchy. +# path_to_regions_hierarchy_file: /opt/geo/regions_hierarchy.txt + +# Path to directory with files containing names of regions +# path_to_regions_names_files: /opt/geo/ + + +# top_level_domains_path: /var/lib/clickhouse/top_level_domains/ +# Custom TLD lists. +# Format: name: /path/to/file + +# Changes will not be applied w/o server restart. +# Path to the list is under top_level_domains_path (see above). +# +top_level_domains_lists: '' +# +# public_suffix_list: /path/to/public_suffix_list.dat +# + +# Configuration of external dictionaries. See: +# https://clickhouse.tech/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts +# +dictionaries_config: '*_dictionary.xml' +# Uncomment if you want data to be compressed 30-100% better. +# Don't do that if you just started using ClickHouse. +# +# +# compression: + # Set of variants. Checked in order. Last matching case wins. If nothing matches, lz4 will be used. +# case: + # Conditions. All must be satisfied. Some conditions may be omitted. +# min_part_size: 10000000000 # Min part size in bytes. +# min_part_size_ratio: 0.01 # Min size of part relative to whole table size. + + # What compression method to use. +# method: zstd +# + +# Allow to execute distributed DDL queries (CREATE, DROP, ALTER, RENAME) on cluster. +# Works only if ZooKeeper is enabled. Comment it if such functionality isn't required. +distributed_ddl: +# Path in ZooKeeper to queue with DDL queries + path: /clickhouse/task_queue/ddl +# Settings from this profile will be used to execute DDL queries +# profile: default + +# Controls how much ON CLUSTER queries can be run simultaneously. +# pool_size: 1 + +# +# Cleanup settings (active tasks will not be removed) +# + +# Controls task TTL (default 1 week) +# task_max_lifetime: 604800 + +# Controls how often cleanup should be performed (in seconds) +# cleanup_delay_period: 60 + +# Controls how many tasks could be in the queue +# max_tasks_in_queue: 1000 + +# Settings to fine tune MergeTree tables. See documentation in source code, in MergeTreeSettings.h +# +# merge_tree: +# max_suspicious_broken_parts: 5 +# + +# Protection from accidental DROP. +# If size of a MergeTree table is greater than max_table_size_to_drop (in bytes) than table could not be dropped with any DROP query. +# If you want do delete one table and don't want to change clickhouse-server config, you could create special file /flags/force_drop_table and make DROP once. +# By default max_table_size_to_drop is 50GB; max_table_size_to_drop=0 allows to DROP any tables. +# The same for max_partition_size_to_drop. +# Uncomment to disable protection. +# +# max_table_size_to_drop: 0 +# max_partition_size_to_drop: 0 + +# Example of parameters for GraphiteMergeTree table engine +graphite_rollup_example: + pattern: + regexp: click_cost + function: any + retention: + - age: 0 + precision: 3600 + - age: 86400 + precision: 60 + default: + function: max + retention: + - age: 0 + precision: 60 + - age: 3600 + precision: 300 + - age: 86400 + precision: 3600 +# Directory in containing schema files for various input formats. +# The directory will be created if it doesn't exist. +# +format_schema_path: /var/lib/clickhouse/format_schemas/ +# Default query masking rules, matching lines would be replaced with something else in the logs +# (both text logs and system.query_log). +# name - name for the rule (optional) +# regexp - RE2 compatible regular expression (mandatory) +# replace - substitution string for sensitive data (optional, by default - six asterisks) +# +query_masking_rules: + rule: + name: hide encrypt/decrypt arguments + regexp: '((?:aes_)?(?:encrypt|decrypt)(?:_mysql)?)\s*\(\s*(?:''(?:\\''|.)+''|.*?)\s*\)' + # or more secure, but also more invasive: + # (aes_\w+)\s*\(.*\) + # + replace: \1(???) +# Uncomment to use custom http handlers. +# rules are checked from top to bottom, first match runs the handler +# url - to match request URL, you can use 'regex:' prefix to use regex match(optional) +# methods - to match request method, you can use commas to separate multiple method matches(optional) +# headers - to match request headers, match each child element(child element name is header name), you can use 'regex:' prefix to use regex match(optional) +# handler is request handler +# type - supported types: static, dynamic_query_handler, predefined_query_handler +# query - use with predefined_query_handler type, executes query when the handler is called +# query_param_name - use with dynamic_query_handler type, extracts and executes the value corresponding to the value in HTTP request params +# status - use with static type, response status code +# content_type - use with static type, response content-type +# response_content - use with static type, Response content sent to client, when using the prefix 'file://' or 'config://', find the content from the file or configuration send to client. + +# http_handlers: +# - rule: +# url: / +# methods: POST,GET +# headers: +# pragma: no-cache +# handler: +# type: dynamic_query_handler +# query_param_name: query +# - rule: +# url: /predefined_query +# methods: POST,GET +# handler: +# type: predefined_query_handler +# query: 'SELECT * FROM system.settings' +# - rule: +# handler: +# type: static +# status: 200 +# content_type: 'text/plain; charset=UTF-8' +# response_content: config://http_server_default_response +# +send_crash_reports: +# Changing to true allows sending crash reports to +# the ClickHouse core developers team via Sentry https://sentry.io +# Doing so at least in pre-production environments is highly appreciated + enabled: false +# Change to true if you don't feel comfortable attaching the server hostname to the crash report + anonymize: false +# Default endpoint should be changed to different Sentry DSN only if you have +# some in-house engineers or hired consultants who're going to debug ClickHouse issues for you + endpoint: 'https://6f33034cfe684dd7a3ab9875e57b1c8d@o388870.ingest.sentry.io/5226277' +# Uncomment to disable ClickHouse internal DNS caching. +# disable_internal_dns_cache: 1 + From 6c2e02461d06847ed086f546afbc891809ee7bd1 Mon Sep 17 00:00:00 2001 From: BoloniniD Date: Sat, 22 May 2021 21:34:19 +0300 Subject: [PATCH 33/46] Added link to config-example.yaml --- docs/en/operations/configuration-files.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/operations/configuration-files.md b/docs/en/operations/configuration-files.md index 0000bf6815b..eeed344f970 100644 --- a/docs/en/operations/configuration-files.md +++ b/docs/en/operations/configuration-files.md @@ -57,6 +57,8 @@ $ cat /etc/clickhouse-server/users.d/alice.xml ## YAML examples {#example} +Here you can see default config written in YAML: [config-example.yaml](../../../programs/server/config-example.yaml). + There are some differences between YAML and XML formats in terms of ClickHouse configurations. Here are some tips for writing a configuration in YAML format. You should use a Scalar node to write a key-value pair: From 518552c866b6326fa01e33cd7f156aac4abfc82c Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 23 May 2021 01:37:54 +0300 Subject: [PATCH 34/46] Fixed tests --- src/Interpreters/JIT/compileFunction.cpp | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/JIT/compileFunction.cpp b/src/Interpreters/JIT/compileFunction.cpp index c643722c1cd..43f914c715a 100644 --- a/src/Interpreters/JIT/compileFunction.cpp +++ b/src/Interpreters/JIT/compileFunction.cpp @@ -129,12 +129,8 @@ static void compileFunction(llvm::Module & module, const IFunctionBase & functio continue; } - auto * nullable_type_untyped = toNativeType(b, type); - llvm::StructType * nullable_type = static_cast(nullable_type_untyped); - llvm::Type * is_null_type = nullable_type->getElementType(1); - - auto * is_null = b.CreateICmpNE(b.CreateLoad(is_null_type, column.null), b.getInt8(0)); - auto * nullable_unitilized = llvm::Constant::getNullValue(nullable_type_untyped); + auto * is_null = b.CreateICmpNE(b.CreateLoad(b.getInt8Ty(), column.null), b.getInt8(0)); + auto * nullable_unitilized = llvm::Constant::getNullValue(toNativeType(b, type)); auto * nullable_value = b.CreateInsertValue(b.CreateInsertValue(nullable_unitilized, value, {0}), is_null, {1}); arguments.emplace_back(nullable_value); } From c704837a4a855583f581bbf8844f42e0d4b7f2d3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 23 May 2021 02:28:50 +0300 Subject: [PATCH 35/46] Switch message level to WARNING for FAIL_ON_UNSUPPORTED_OPTIONS_COMBINATION=OFF STATUS (as before this patch) is too quite, WARNING will at least log this with a stacktrace. --- CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 866d9f542e1..753690273a6 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -36,7 +36,7 @@ option(FAIL_ON_UNSUPPORTED_OPTIONS_COMBINATION if(FAIL_ON_UNSUPPORTED_OPTIONS_COMBINATION) set(RECONFIGURE_MESSAGE_LEVEL FATAL_ERROR) else() - set(RECONFIGURE_MESSAGE_LEVEL STATUS) + set(RECONFIGURE_MESSAGE_LEVEL WARNING) endif() enable_language(C CXX ASM) From c343f85b641c7c9325455b31ebe39574add5de1d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 23 May 2021 02:26:40 +0300 Subject: [PATCH 36/46] Do not built clickhouse-keeper w/o NuRaft Otherwise it will fail because of unused argument and unreachable code. --- programs/CMakeLists.txt | 28 ++++++++++++++++++++++++++-- programs/keeper/Keeper.cpp | 8 +------- 2 files changed, 27 insertions(+), 9 deletions(-) diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index 6fd4c2050b4..2af0331c70b 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -49,6 +49,12 @@ option (ENABLE_CLICKHOUSE_GIT_IMPORT "A tool to analyze Git repositories" option (ENABLE_CLICKHOUSE_KEEPER "ClickHouse alternative to ZooKeeper" ${ENABLE_CLICKHOUSE_ALL}) +if (NOT USE_NURAFT) + # RECONFIGURE_MESSAGE_LEVEL should not be used here, + # since USE_NURAFT is set to OFF for FreeBSD and Darwin. + message (STATUS "clickhouse-keeper will not be built (lack of NuRaft)") + set(ENABLE_CLICKHOUSE_KEEPER OFF) +endif() if (CLICKHOUSE_SPLIT_BINARY) option(ENABLE_CLICKHOUSE_INSTALL "Install ClickHouse without .deb/.rpm/.tgz packages (having the binary only)" OFF) @@ -259,7 +265,10 @@ add_subdirectory (obfuscator) add_subdirectory (install) add_subdirectory (git-import) add_subdirectory (bash-completion) -add_subdirectory (keeper) + +if (ENABLE_CLICKHOUSE_KEEPER) + add_subdirectory (keeper) +endif() if (ENABLE_CLICKHOUSE_ODBC_BRIDGE) add_subdirectory (odbc-bridge) @@ -278,7 +287,18 @@ if (CLICKHOUSE_ONE_SHARED) endif() if (CLICKHOUSE_SPLIT_BINARY) - set (CLICKHOUSE_ALL_TARGETS clickhouse-server clickhouse-client clickhouse-local clickhouse-benchmark clickhouse-extract-from-config clickhouse-compressor clickhouse-format clickhouse-obfuscator clickhouse-git-import clickhouse-copier clickhouse-keeper) + set (CLICKHOUSE_ALL_TARGETS + clickhouse-server + clickhouse-client + clickhouse-local + clickhouse-benchmark + clickhouse-extract-from-config + clickhouse-compressor + clickhouse-format + clickhouse-obfuscator + clickhouse-git-import + clickhouse-copier + ) if (ENABLE_CLICKHOUSE_ODBC_BRIDGE) list (APPEND CLICKHOUSE_ALL_TARGETS clickhouse-odbc-bridge) @@ -288,6 +308,10 @@ if (CLICKHOUSE_SPLIT_BINARY) list (APPEND CLICKHOUSE_ALL_TARGETS clickhouse-library-bridge) endif () + if (ENABLE_CLICKHOUSE_KEEPER) + list (APPEND CLICKHOUSE_ALL_TARGETS clickhouse-keeper) + endif () + set_target_properties(${CLICKHOUSE_ALL_TARGETS} PROPERTIES RUNTIME_OUTPUT_DIRECTORY ..) add_custom_target (clickhouse-bundle ALL DEPENDS ${CLICKHOUSE_ALL_TARGETS}) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index b9d87ba7fdb..8b35ec12850 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -30,9 +30,7 @@ # include #endif -#if USE_NURAFT -# include -#endif +#include #if defined(OS_LINUX) # include @@ -357,7 +355,6 @@ int Keeper::main(const std::vector & /*args*/) auto servers = std::make_shared>(); -#if USE_NURAFT /// Initialize test keeper RAFT. Do nothing if no nu_keeper_server in config. global_context->initializeKeeperStorageDispatcher(); for (const auto & listen_host : listen_hosts) @@ -398,9 +395,6 @@ int Keeper::main(const std::vector & /*args*/) #endif }); } -#else - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "ClickHouse keeper built without NuRaft library. Cannot use coordination."); -#endif for (auto & server : *servers) server.start(); From 32859fd9016bde4370573e119eb730f206f07c81 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 23 May 2021 12:43:29 +0300 Subject: [PATCH 37/46] Update min_count_to_compile_expression --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 737385237e9..125879486ab 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -102,7 +102,7 @@ class IColumn; \ M(Bool, allow_suspicious_low_cardinality_types, false, "In CREATE TABLE statement allows specifying LowCardinality modifier for types of small fixed size (8 or less). Enabling this may increase merge times and memory consumption.", 0) \ M(Bool, compile_expressions, true, "Compile some scalar functions and operators to native code.", 0) \ - M(UInt64, min_count_to_compile_expression, 0, "The number of identical expressions before they are JIT-compiled", 0) \ + M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \ M(UInt64, group_by_two_level_threshold, 100000, "From what number of keys, a two-level aggregation starts. 0 - the threshold is not set.", 0) \ M(UInt64, group_by_two_level_threshold_bytes, 50000000, "From what size of the aggregation state in bytes, a two-level aggregation begins to be used. 0 - the threshold is not set. Two-level aggregation is used when at least one of the thresholds is triggered.", 0) \ M(Bool, distributed_aggregation_memory_efficient, true, "Is the memory-saving mode of distributed aggregation enabled.", 0) \ From b949b4384b3a20ca332aca1c6c5e75e60ef55bcc Mon Sep 17 00:00:00 2001 From: BoloniniD Date: Sun, 23 May 2021 13:20:19 +0300 Subject: [PATCH 38/46] Fixes after review --- docs/en/operations/configuration-files.md | 2 +- programs/server/config-example.yaml | 1000 ++++++++++----------- 2 files changed, 484 insertions(+), 518 deletions(-) diff --git a/docs/en/operations/configuration-files.md b/docs/en/operations/configuration-files.md index eeed344f970..a52d82f21d0 100644 --- a/docs/en/operations/configuration-files.md +++ b/docs/en/operations/configuration-files.md @@ -57,7 +57,7 @@ $ cat /etc/clickhouse-server/users.d/alice.xml ## YAML examples {#example} -Here you can see default config written in YAML: [config-example.yaml](../../../programs/server/config-example.yaml). +Here you can see default config written in YAML: [config-example.yaml](https://github.com/ClickHouse/ClickHouse/blob/master/programs/server/config-example.yaml). There are some differences between YAML and XML formats in terms of ClickHouse configurations. Here are some tips for writing a configuration in YAML format. diff --git a/programs/server/config-example.yaml b/programs/server/config-example.yaml index 49e5f7ee853..48973fe6fc0 100644 --- a/programs/server/config-example.yaml +++ b/programs/server/config-example.yaml @@ -3,88 +3,82 @@ # You can either move the settings to the right place inside "users.xml" file # or add skip_check_for_incorrect_settings: 1 here. logger: -# Possible levels [1]: -# - none (turns off logging) -# - fatal -# - critical -# - error -# - warning -# - notice -# - information -# - debug -# - trace -# -# [1]: https://github.com/pocoproject/poco/blob/poco-1.9.4-release/Foundation/include/Poco/Logger.h#L105-L114 - level: trace - log: /var/log/clickhouse-server/clickhouse-server.log - errorlog: /var/log/clickhouse-server/clickhouse-server.err.log -# Rotation policy -# See https://github.com/pocoproject/poco/blob/poco-1.9.4-release/Foundation/include/Poco/FileChannel.h#L54-L85 - size: 1000M - count: 10 -# console: 1 -# Default behavior is autodetection (log to console if not daemon mode and is tty) -# Per level overrides (legacy): -# For example to suppress logging of the ConfigReloader you can use: -# NOTE: levels.logger is reserved, see below. + # Possible levels [1]: + # - none (turns off logging) + # - fatal + # - critical + # - error + # - warning + # - notice + # - information + # - debug + # - trace + # [1]: https://github.com/pocoproject/poco/blob/poco-1.9.4-release/Foundation/include/Poco/Logger.h#L105-L114 + level: trace + log: /var/log/clickhouse-server/clickhouse-server.log + errorlog: /var/log/clickhouse-server/clickhouse-server.err.log + # Rotation policy + # See https://github.com/pocoproject/poco/blob/poco-1.9.4-release/Foundation/include/Poco/FileChannel.h#L54-L85 + size: 1000M + count: 10 + # console: 1 + # Default behavior is autodetection (log to console if not daemon mode and is tty) -# levels: -# ConfigReloader: none + # Per level overrides (legacy): + # For example to suppress logging of the ConfigReloader you can use: + # NOTE: levels.logger is reserved, see below. + # levels: + # ConfigReloader: none -# Per level overrides: -# For example to suppress logging of the RBAC for default user you can use: -# (But please note that the logger name maybe changed from version to version, even after minor upgrade) -# -# levels: -# - logger: -# name: 'ContextAccess (default)' -# level: none -# - logger: -# name: 'DatabaseOrdinary (test)' -# level: none + # Per level overrides: + # For example to suppress logging of the RBAC for default user you can use: + # (But please note that the logger name maybe changed from version to version, even after minor upgrade) + # levels: + # - logger: + # name: 'ContextAccess (default)' + # level: none + # - logger: + # name: 'DatabaseOrdinary (test)' + # level: none # It is the name that will be shown in the clickhouse-client. # By default, anything with "production" will be highlighted in red in query prompt. -# # display_name: production # Port for HTTP API. See also 'https_port' for secure connections. # This interface is also used by ODBC and JDBC drivers (DataGrip, Dbeaver, ...) # and by most of web interfaces (embedded UI, Grafana, Redash, ...). -# http_port: 8123 + # Port for interaction by native protocol with: # - clickhouse-client and other native ClickHouse tools (clickhouse-benchmark, clickhouse-copier); # - clickhouse-server with other clickhouse-servers for distributed query processing; # - ClickHouse drivers and applications supporting native protocol # (this protocol is also informally called as "the TCP protocol"); # See also 'tcp_port_secure' for secure connections. -# tcp_port: 9000 + # Compatibility with MySQL protocol. # ClickHouse will pretend to be MySQL for applications connecting to this port. -# mysql_port: 9004 + # Compatibility with PostgreSQL protocol. # ClickHouse will pretend to be PostgreSQL for applications connecting to this port. -# postgresql_port: 9005 + # HTTP API with TLS (HTTPS). # You have to configure certificate to enable this interface. # See the openSSL section below. -# # https_port: 8443 # Native interface with TLS. # You have to configure certificate to enable this interface. # See the openSSL section below. -# # tcp_port_secure: 9440 # Native interface wrapped with PROXYv1 protocol # PROXYv1 header sent for every connection. # ClickHouse will extract information about proxy-forwarded client address from the header. -# # tcp_with_proxy_port: 9011 # Port for communication between replicas. Used for data exchange. @@ -93,31 +87,26 @@ postgresql_port: 9005 # See also 'interserver_http_credentials'. # Data transferred over connections to this port should not go through untrusted networks. # See also 'interserver_https_port'. -# interserver_http_port: 9009 + # Port for communication between replicas with TLS. # You have to configure certificate to enable this interface. # See the openSSL section below. # See also 'interserver_http_credentials'. -# # interserver_https_port: 9010 # Hostname that is used by other replicas to request this server. # If not specified, than it is determined analogous to 'hostname -f' command. # This setting could be used to switch replication to another network interface # (the server may be connected to multiple networks via multiple addresses) -# -# # interserver_http_host: example.yandex.ru -# # You can specify credentials for authenthication between replicas. # This is required when interserver_https_port is accessible from untrusted networks, # and also recommended to avoid SSRF attacks from possibly compromised services in your network. -# # interserver_http_credentials: -# user: interserver -# password: '' +# user: interserver +# password: '' # Listen specified address. # Use :: (wildcard IPv6 address), if you want to accept connections both with IPv4 and IPv6 from everywhere. @@ -128,86 +117,93 @@ interserver_http_port: 9009 # - all users have strong passwords, only secure (TLS) interfaces are accessible, or connections are only made via TLS interfaces. # - users without password have readonly access. # See also: https://www.shodan.io/search?query=clickhouse -# # listen_host: '::' # Same for hosts without support for IPv6: # listen_host: 0.0.0.0 # Default values - try listen localhost on IPv4 and IPv6. -# # listen_host: '::1' # listen_host: 127.0.0.1 -# # Don't exit if IPv6 or IPv4 networks are unavailable while trying to listen. # listen_try: 0 # Allow multiple servers to listen on the same address:port. This is not recommended. -# # listen_reuse_port: 0 # listen_backlog: 64 max_connections: 4096 + # For 'Connection: keep-alive' in HTTP 1.1 keep_alive_timeout: 3 + # gRPC protocol (see src/Server/grpc_protos/clickhouse_grpc.proto for the API) # grpc_port: 9100 grpc: - enable_ssl: false -# The following two files are used only if enable_ssl=1 - ssl_cert_file: /path/to/ssl_cert_file - ssl_key_file: /path/to/ssl_key_file -# Whether server will request client for a certificate - ssl_require_client_auth: false -# The following file is used only if ssl_require_client_auth=1 - ssl_ca_cert_file: /path/to/ssl_ca_cert_file -# Default compression algorithm (applied if client doesn't specify another algorithm). -# Supported algorithms: none, deflate, gzip, stream_gzip - compression: deflate -# Default compression level (applied if client doesn't specify another level). -# Supported levels: none, low, medium, high - compression_level: medium -# Send/receive message size limits in bytes. -1 means unlimited - max_send_message_size: -1 - max_receive_message_size: -1 -# Enable if you want very detailed logs - verbose_logs: false + enable_ssl: false + + # The following two files are used only if enable_ssl=1 + ssl_cert_file: /path/to/ssl_cert_file + ssl_key_file: /path/to/ssl_key_file + + # Whether server will request client for a certificate + ssl_require_client_auth: false + + # The following file is used only if ssl_require_client_auth=1 + ssl_ca_cert_file: /path/to/ssl_ca_cert_file + + # Default compression algorithm (applied if client doesn't specify another algorithm). + # Supported algorithms: none, deflate, gzip, stream_gzip + compression: deflate + + # Default compression level (applied if client doesn't specify another level). + # Supported levels: none, low, medium, high + compression_level: medium + + # Send/receive message size limits in bytes. -1 means unlimited + max_send_message_size: -1 + max_receive_message_size: -1 + + # Enable if you want very detailed logs + verbose_logs: false + # Used with https_port and tcp_port_secure. Full ssl options list: https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h#L71 openSSL: - server: - # Used for https server AND secure tcp port - # openssl req -subj "/CN=localhost" -new -newkey rsa:2048 -days 365 -nodes -x509 -keyout /etc/clickhouse-server/server.key -out /etc/clickhouse-server/server.crt - certificateFile: /etc/clickhouse-server/server.crt - privateKeyFile: /etc/clickhouse-server/server.key - # dhparams are optional. You can delete the element. - # To generate dhparams, use the following command: - # openssl dhparam -out /etc/clickhouse-server/dhparam.pem 4096 - # Only file format with BEGIN DH PARAMETERS is supported. - # - dhParamsFile: /etc/clickhouse-server/dhparam.pem - verificationMode: none - loadDefaultCAFile: true - cacheSessions: true - disableProtocols: 'sslv2,sslv3' - preferServerCiphers: true - client: - # Used for connecting to https dictionary source and secured Zookeeper communication - loadDefaultCAFile: true - cacheSessions: true - disableProtocols: 'sslv2,sslv3' - preferServerCiphers: true - # Use for self-signed: none - invalidCertificateHandler: - # Use for self-signed: AcceptCertificateHandler - name: RejectCertificateHandler + server: + # Used for https server AND secure tcp port + # openssl req -subj "/CN=localhost" -new -newkey rsa:2048 -days 365 -nodes -x509 -keyout /etc/clickhouse-server/server.key -out /etc/clickhouse-server/server.crt + certificateFile: /etc/clickhouse-server/server.crt + privateKeyFile: /etc/clickhouse-server/server.key + + # dhparams are optional. You can delete the dhParamsFile: element. + # To generate dhparams, use the following command: + # openssl dhparam -out /etc/clickhouse-server/dhparam.pem 4096 + # Only file format with BEGIN DH PARAMETERS is supported. + dhParamsFile: /etc/clickhouse-server/dhparam.pem + verificationMode: none + loadDefaultCAFile: true + cacheSessions: true + disableProtocols: 'sslv2,sslv3' + preferServerCiphers: true + client: + # Used for connecting to https dictionary source and secured Zookeeper communication + loadDefaultCAFile: true + cacheSessions: true + disableProtocols: 'sslv2,sslv3' + preferServerCiphers: true + + # Use for self-signed: verificationMode: none + invalidCertificateHandler: + # Use for self-signed: name: AcceptCertificateHandler + name: RejectCertificateHandler + # Default root page on http[s] server. For example load UI from https://tabix.io/ when opening http://localhost:8123 -# #
]]>
-# # Maximum number of concurrent queries. max_concurrent_queries: 100 + # Maximum memory usage (resident set size) for server process. # Zero value or unset means default. Default is "max_server_memory_usage_to_ram_ratio" of available physical RAM. # If the value is larger than "max_server_memory_usage_to_ram_ratio" of available physical RAM, it will be cut down. @@ -218,29 +214,30 @@ max_concurrent_queries: 100 # It is not practical to set this constraint to small values like just a few gigabytes, # because memory allocator will keep this amount of memory in caches and the server will deny service of queries. -# max_server_memory_usage: 0 + # Maximum number of threads in the Global thread pool. # This will default to a maximum of 10000 threads if not specified. # This setting will be useful in scenarios where there are a large number # of distributed queries that are running concurrently but are idling most # of the time, in which case a higher number of threads might be required. -# max_thread_pool_size: 10000 + # On memory constrained environments you may have to set this to value larger than 1. max_server_memory_usage_to_ram_ratio: 0.9 + # Simple server-wide memory profiler. Collect a stack trace at every peak allocation step (in bytes). # Data will be stored in system.trace_log table with query_id = empty string. # Zero means disabled. -# total_memory_profiler_step: 4194304 + # Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. # The probability is for every alloc/free regardless to the size of the allocation. # Note that sampling happens only when the amount of untracked memory exceeds the untracked memory limit, # which is 4 MiB by default but can be lowered if 'total_memory_profiler_step' is lowered. # You may want to set 'total_memory_profiler_step' to 1 for extra fine grained sampling. -# total_memory_tracker_sample_probability: 0 + # Set limit on number of open files (default: maximum). This setting makes sense on Mac OS X because getrlimit() fails to retrieve # correct maximum value. # max_open_files: 262144 @@ -253,13 +250,13 @@ total_memory_tracker_sample_probability: 0 # Note: uncompressed cache can be pointless for lz4, because memory bandwidth # is slower than multi-core decompression on some server configurations. # Enabling it can sometimes paradoxically make queries slower. -# uncompressed_cache_size: 8589934592 + # Approximate size of mark cache, used in tables of MergeTree family. # In bytes. Cache is single for server. Memory is allocated only on demand. # You should not lower this value. -# mark_cache_size: 5368709120 + # If you enable the `min_bytes_to_use_mmap_io` setting, # the data in MergeTree tables can be read with mmap to avoid copying from kernel to userspace. # It makes sense only for large files and helps only if data reside in page cache. @@ -275,29 +272,33 @@ mark_cache_size: 5368709120 # in query or server memory usage - because this memory can be discarded similar to OS page cache. # The cache is dropped (the files are closed) automatically on removal of old parts in MergeTree, # also it can be dropped manually by the SYSTEM DROP MMAP CACHE query. -# mmap_cache_size: 1000 + # Cache size for compiled expressions. compiled_expression_cache_size: 1073741824 + # Path to data directory, with trailing slash. path: /var/lib/clickhouse/ + # Path to temporary data for processing hard queries. tmp_path: /var/lib/clickhouse/tmp/ + # Policy from the for the temporary files. # If not set is used, otherwise is ignored. # Notes: -# - move_factor is ignored +# - move_factor is ignored # - keep_free_space_bytes is ignored # - max_data_part_size_bytes is ignored # - you must have exactly one volume in that policy -# # tmp_policy: tmp # Directory with user provided files that are accessible by 'file' table function. user_files_path: /var/lib/clickhouse/user_files/ + # LDAP server definitions. ldap_servers: '' + # List LDAP servers with their connection parameters here to later 1) use them as authenticators for dedicated local users, # who have 'ldap' authentication mechanism specified instead of 'password', or to 2) use them as remote user directories. # Parameters: @@ -338,29 +339,28 @@ ldap_servers: '' # tls_cipher_suite - allowed cipher suite (in OpenSSL notation). # Example: # my_ldap_server: -# host: localhost -# port: 636 -# bind_dn: 'uid={user_name},ou=users,dc=example,dc=com' -# verification_cooldown: 300 -# enable_tls: yes -# tls_minimum_protocol_version: tls1.2 -# tls_require_cert: demand -# tls_cert_file: /path/to/tls_cert_file -# tls_key_file: /path/to/tls_key_file -# tls_ca_cert_file: /path/to/tls_ca_cert_file -# tls_ca_cert_dir: /path/to/tls_ca_cert_dir -# tls_cipher_suite: ECDHE-ECDSA-AES256-GCM-SHA384:ECDHE-RSA-AES256-GCM-SHA384:AES256-GCM-SHA384 -# +# host: localhost +# port: 636 +# bind_dn: 'uid={user_name},ou=users,dc=example,dc=com' +# verification_cooldown: 300 +# enable_tls: yes +# tls_minimum_protocol_version: tls1.2 +# tls_require_cert: demand +# tls_cert_file: /path/to/tls_cert_file +# tls_key_file: /path/to/tls_key_file +# tls_ca_cert_file: /path/to/tls_ca_cert_file +# tls_ca_cert_dir: /path/to/tls_ca_cert_dir +# tls_cipher_suite: ECDHE-ECDSA-AES256-GCM-SHA384:ECDHE-RSA-AES256-GCM-SHA384:AES256-GCM-SHA384 + # Example (typical Active Directory with configured user DN detection for further role mapping): # my_ad_server: -# host: localhost -# port: 389 -# bind_dn: 'EXAMPLE\{user_name}' -# user_dn_detection: -# base_dn: CN=Users,DC=example,DC=com -# search_filter: '(&(objectClass=user)(sAMAccountName={user_name}))' -# enable_tls: no -# +# host: localhost +# port: 389 +# bind_dn: 'EXAMPLE\{user_name}' +# user_dn_detection: +# base_dn: CN=Users,DC=example,DC=com +# search_filter: '(&(objectClass=user)(sAMAccountName={user_name}))' +# enable_tls: no # To enable Kerberos authentication support for HTTP requests (GSS-SPNEGO), for those users who are explicitly configured # to authenticate via Kerberos, define a single 'kerberos' section here. @@ -373,80 +373,78 @@ ldap_servers: '' # This parameter cannot be specified together with 'principal' parameter. # Example: # kerberos: '' -# + # Example: # kerberos: -# principal: HTTP/clickhouse.example.com@EXAMPLE.COM -# +# principal: HTTP/clickhouse.example.com@EXAMPLE.COM + # Example: # kerberos: -# realm: EXAMPLE.COM -# +# realm: EXAMPLE.COM # Sources to read users, roles, access rights, profiles of settings, quotas. user_directories: - users_xml: - # Path to configuration file with predefined users. - path: users.yaml - local_directory: - # Path to folder where users created by SQL commands are stored. - path: /var/lib/clickhouse/access/ -# To add an LDAP server as a remote user directory of users that are not defined locally, define a single 'ldap' section -# with the following parameters: -# server - one of LDAP server names defined in 'ldap_servers' config section above. -# This parameter is mandatory and cannot be empty. -# roles - section with a list of locally defined roles that will be assigned to each user retrieved from the LDAP server. -# If no roles are specified here or assigned during role mapping (below), user will not be able to perform any -# actions after authentication. -# role_mapping - section with LDAP search parameters and mapping rules. -# When a user authenticates, while still bound to LDAP, an LDAP search is performed using search_filter and the -# name of the logged in user. For each entry found during that search, the value of the specified attribute is -# extracted. For each attribute value that has the specified prefix, the prefix is removed, and the rest of the -# value becomes the name of a local role defined in ClickHouse, which is expected to be created beforehand by -# CREATE ROLE command. -# There can be multiple 'role_mapping' sections defined inside the same 'ldap' section. All of them will be -# applied. -# base_dn - template used to construct the base DN for the LDAP search. -# The resulting DN will be constructed by replacing all '{user_name}', '{bind_dn}', and '{user_dn}' -# substrings of the template with the actual user name, bind DN, and user DN during each LDAP search. -# scope - scope of the LDAP search. -# Accepted values are: 'base', 'one_level', 'children', 'subtree' (the default). -# search_filter - template used to construct the search filter for the LDAP search. -# The resulting filter will be constructed by replacing all '{user_name}', '{bind_dn}', '{user_dn}', and -# '{base_dn}' substrings of the template with the actual user name, bind DN, user DN, and base DN during -# each LDAP search. -# Note, that the special characters must be escaped properly in XML. -# attribute - attribute name whose values will be returned by the LDAP search. 'cn', by default. -# prefix - prefix, that will be expected to be in front of each string in the original list of strings returned by -# the LDAP search. Prefix will be removed from the original strings and resulting strings will be treated -# as local role names. Empty, by default. -# Example: -# -# ldap: -# server: my_ldap_server -# roles: -# my_local_role1: '' -# my_local_role2: '' -# role_mapping: -# base_dn: 'ou=groups,dc=example,dc=com' -# scope: subtree -# search_filter: '(&(objectClass=groupOfNames)(member={bind_dn}))' -# attribute: cn -# prefix: clickhouse_ -# -# Example (typical Active Directory with role mapping that relies on the detected user DN): -# ldap: -# server: my_ad_server -# role_mapping: -# base_dn: 'CN=Users,DC=example,DC=com' -# attribute: CN -# scope: subtree -# search_filter: '(&(objectClass=group)(member={user_dn}))' -# prefix: clickhouse_ -# + users_xml: + # Path to configuration file with predefined users. + path: users.yaml + local_directory: + # Path to folder where users created by SQL commands are stored. + path: /var/lib/clickhouse/access/ + + # To add an LDAP server as a remote user directory of users that are not defined locally, define a single 'ldap' section + # with the following parameters: + # server - one of LDAP server names defined in 'ldap_servers' config section above. + # This parameter is mandatory and cannot be empty. + # roles - section with a list of locally defined roles that will be assigned to each user retrieved from the LDAP server. + # If no roles are specified here or assigned during role mapping (below), user will not be able to perform any + # actions after authentication. + # role_mapping - section with LDAP search parameters and mapping rules. + # When a user authenticates, while still bound to LDAP, an LDAP search is performed using search_filter and the + # name of the logged in user. For each entry found during that search, the value of the specified attribute is + # extracted. For each attribute value that has the specified prefix, the prefix is removed, and the rest of the + # value becomes the name of a local role defined in ClickHouse, which is expected to be created beforehand by + # CREATE ROLE command. + # There can be multiple 'role_mapping' sections defined inside the same 'ldap' section. All of them will be + # applied. + # base_dn - template used to construct the base DN for the LDAP search. + # The resulting DN will be constructed by replacing all '{user_name}', '{bind_dn}', and '{user_dn}' + # substrings of the template with the actual user name, bind DN, and user DN during each LDAP search. + # scope - scope of the LDAP search. + # Accepted values are: 'base', 'one_level', 'children', 'subtree' (the default). + # search_filter - template used to construct the search filter for the LDAP search. + # The resulting filter will be constructed by replacing all '{user_name}', '{bind_dn}', '{user_dn}', and + # '{base_dn}' substrings of the template with the actual user name, bind DN, user DN, and base DN during + # each LDAP search. + # Note, that the special characters must be escaped properly in XML. + # attribute - attribute name whose values will be returned by the LDAP search. 'cn', by default. + # prefix - prefix, that will be expected to be in front of each string in the original list of strings returned by + # the LDAP search. Prefix will be removed from the original strings and resulting strings will be treated + # as local role names. Empty, by default. + # Example: + # ldap: + # server: my_ldap_server + # roles: + # my_local_role1: '' + # my_local_role2: '' + # role_mapping: + # base_dn: 'ou=groups,dc=example,dc=com' + # scope: subtree + # search_filter: '(&(objectClass=groupOfNames)(member={bind_dn}))' + # attribute: cn + # prefix: clickhouse_ + # Example (typical Active Directory with role mapping that relies on the detected user DN): + # ldap: + # server: my_ad_server + # role_mapping: + # base_dn: 'CN=Users,DC=example,DC=com' + # attribute: CN + # scope: subtree + # search_filter: '(&(objectClass=group)(member={user_dn}))' + # prefix: clickhouse_ # Default profile of settings. default_profile: default + # Comma-separated list of prefixes for user-defined settings. # custom_settings_prefixes: '' # System profile of settings. This settings are used by internal processes (Distributed DDL worker and so on). @@ -455,11 +453,11 @@ default_profile: default # Buffer profile of settings. # This settings are used by Buffer storage to flush data to the underlying table. # Default: used from system_profile directive. -# # buffer_profile: default # Default database. default_database: default + # Server time zone could be set here. # Time zone is used when converting between String and DateTime types, @@ -471,21 +469,20 @@ default_database: default # Please note, that server could display time zone alias instead of specified name. # Example: W-SU is an alias for Europe/Moscow and Zulu is an alias for UTC. -# # timezone: Europe/Moscow # You can specify umask here (see "man umask"). Server will apply it on startup. # Number is always parsed as octal. Default umask is 027 (other users cannot read logs, data files, etc; group can only read). -# # umask: 022 # Perform mlockall after startup to lower first queries latency # and to prevent clickhouse executable from being paged out under high IO load. # Enabling this option is recommended but will lead to increased startup time for up to a few seconds. -# mlock_executable: true + # Reallocate memory for machine code ("text") using huge pages. Highly experimental. remap_executable: false + # Uncomment below in order to use JDBC table engine and function. # To install and run JDBC bridge in background: # * [Debian/Ubuntu] @@ -503,91 +500,89 @@ remap_executable: false # Please refer to https://github.com/ClickHouse/clickhouse-jdbc-bridge#usage for more information. # jdbc_bridge: -# host: 127.0.0.1 -# port: 9019 +# host: 127.0.0.1 +# port: 9019 # Configuration of clusters that could be used in Distributed tables. # https://clickhouse.tech/docs/en/operations/table_engines/distributed/ -# remote_servers: -# Test only shard config for testing distributed storage - test_shard_localhost: -# Inter-server per-cluster secret for Distributed queries -# default: no secret (no authentication will be performed) + # Test only shard config for testing distributed storage + test_shard_localhost: + # Inter-server per-cluster secret for Distributed queries + # default: no secret (no authentication will be performed) -# If set, then Distributed queries will be validated on shards, so at least: -# - such cluster should exist on the shard, -# - such cluster should have the same secret. + # If set, then Distributed queries will be validated on shards, so at least: + # - such cluster should exist on the shard, + # - such cluster should have the same secret. -# And also (and which is more important), the initial_user will -# be used as current user for the query. + # And also (and which is more important), the initial_user will + # be used as current user for the query. -# Right now the protocol is pretty simple and it only takes into account: -# - cluster name -# - query + # Right now the protocol is pretty simple and it only takes into account: + # - cluster name + # - query + + # Also it will be nice if the following will be implemented: + # - source hostname (see interserver_http_host), but then it will depends from DNS, + # it can use IP address instead, but then the you need to get correct on the initiator node. + # - target hostname / ip address (same notes as for source hostname) + # - time-based security tokens + # secret: '' + shard: + # Optional. Whether to write data to just one of the replicas. Default: false (write data to all replicas). + # internal_replication: false + # Optional. Shard weight when writing data. Default: 1. + # weight: 1 + replica: + host: localhost + port: 9000 + # Optional. Priority of the replica for load_balancing. Default: 1 (less value has more priority). + # priority: 1 + test_cluster_two_shards_localhost: + shard: + - replica: + host: localhost + port: 9000 + - replica: + host: localhost + port: 9000 + test_cluster_two_shards: + shard: + - replica: + host: 127.0.0.1 + port: 9000 + - replica: + host: 127.0.0.2 + port: 9000 + test_cluster_two_shards_internal_replication: + shard: + - internal_replication: true + replica: + host: 127.0.0.1 + port: 9000 + - internal_replication: true + replica: + host: 127.0.0.2 + port: 9000 + test_shard_localhost_secure: + shard: + replica: + host: localhost + port: 9440 + secure: 1 + test_unavailable_shard: + shard: + - replica: + host: localhost + port: 9000 + - replica: + host: localhost + port: 1 -# Also it will be nice if the following will be implemented: -# - source hostname (see interserver_http_host), but then it will depends from DNS, -# it can use IP address instead, but then the you need to get correct on the initiator node. -# - target hostname / ip address (same notes as for source hostname) -# - time-based security tokens -# -# secret: '' - shard: - # Optional. Whether to write data to just one of the replicas. Default: false (write data to all replicas). - # internal_replication: false - # Optional. Shard weight when writing data. Default: 1. - # weight: 1 - replica: - host: localhost - port: 9000 - # Optional. Priority of the replica for load_balancing. Default: 1 (less value has more priority). - # priority: 1 - test_cluster_two_shards_localhost: - shard: - - replica: - host: localhost - port: 9000 - - replica: - host: localhost - port: 9000 - test_cluster_two_shards: - shard: - - replica: - host: 127.0.0.1 - port: 9000 - - replica: - host: 127.0.0.2 - port: 9000 - test_cluster_two_shards_internal_replication: - shard: - - internal_replication: true - replica: - host: 127.0.0.1 - port: 9000 - - internal_replication: true - replica: - host: 127.0.0.2 - port: 9000 - test_shard_localhost_secure: - shard: - replica: - host: localhost - port: 9440 - secure: 1 - test_unavailable_shard: - shard: - - replica: - host: localhost - port: 9000 - - replica: - host: localhost - port: 1 # The list of hosts allowed to use in URL-related storage engines and table functions. # If this section is not present in configuration, all hosts are allowed. -# # remote_url_allow_hosts: -# + # Host should be specified exactly as in URL. The name is checked before DNS resolution. # Example: "yandex.ru", "yandex.ru." and "www.yandex.ru" are different hosts. # If port is explicitly specified in URL, the host:port is checked as a whole. @@ -595,225 +590,204 @@ remote_servers: # "yandex.ru" -> "yandex.ru:443", "yandex.ru:80" etc. is allowed, but "yandex.ru:80" -> only "yandex.ru:80" is allowed. # If the host is specified as IP address, it is checked as specified in URL. Example: "[2a02:6b8:a::a]". # If there are redirects and support for redirects is enabled, every redirect (the Location field) is checked. -# # Regular expression can be specified. RE2 engine is used for regexps. # Regexps are not aligned: don't forget to add ^ and $. Also don't forget to escape dot (.) metacharacter # (forgetting to do so is a common source of error). -# # If element has 'incl' attribute, then for it's value will be used corresponding substitution from another file. # By default, path to file with substitutions is /etc/metrika.xml. It could be changed in config in 'include_from' element. # Values for substitutions are specified in /yandex/name_of_substitution elements in that file. -# # ZooKeeper is used to store metadata about replicas, when using Replicated tables. # Optional. If you don't use replicated tables, you could omit that. - # See https://clickhouse.tech/docs/en/engines/table-engines/mergetree-family/replication/ -# -# # zookeeper: -# - node: -# host: example1 -# port: 2181 -# - node: -# host: example2 -# port: 2181 -# - node: -# host: example3 -# port: 2181 -# +# - node: +# host: example1 +# port: 2181 +# - node: +# host: example2 +# port: 2181 +# - node: +# host: example3 +# port: 2181 # Substitutions for parameters of replicated tables. # Optional. If you don't use replicated tables, you could omit that. - # See https://clickhouse.tech/docs/en/engines/table-engines/mergetree-family/replication/#creating-replicated-tables -# -# # macros: -# shard: 01 -# replica: example01-01-1 -# +# shard: 01 +# replica: example01-01-1 # Reloading interval for embedded dictionaries, in seconds. Default: 3600. builtin_dictionaries_reload_interval: 3600 + # Maximum session timeout, in seconds. Default: 3600. max_session_timeout: 3600 + # Default session timeout, in seconds. Default: 60. default_session_timeout: 60 + # Sending data to Graphite for monitoring. Several sections can be defined. -# # interval - send every X second # root_path - prefix for keys # hostname_in_path - append hostname to root_path (default = true) # metrics - send data from table system.metrics # events - send data from table system.events # asynchronous_metrics - send data from table system.asynchronous_metrics -# -# -# graphite: -# host: localhost -# port: 42000 -# timeout: 0.1 -# interval: 60 -# root_path: one_min -# hostname_in_path: true -# metrics: true -# events: true -# events_cumulative: false -# asynchronous_metrics: true -# # graphite: -# host: localhost -# port: 42000 -# timeout: 0.1 -# interval: 1 -# root_path: one_sec +# host: localhost +# port: 42000 +# timeout: 0.1 +# interval: 60 +# root_path: one_min +# hostname_in_path: true -# metrics: true -# events: true -# events_cumulative: false -# asynchronous_metrics: false -# +# metrics: true +# events: true +# events_cumulative: false +# asynchronous_metrics: true + +# graphite: +# host: localhost +# port: 42000 +# timeout: 0.1 +# interval: 1 +# root_path: one_sec + +# metrics: true +# events: true +# events_cumulative: false +# asynchronous_metrics: false # Serve endpoint for Prometheus monitoring. -# -#endpoint - mertics path (relative to root, statring with "/") -#port - port to setup server. If not defined or 0 than http_port used -#metrics - send data from table system.metrics -#events - send data from table system.events -#asynchronous_metrics - send data from table system.asynchronous_metrics -#status_info - send data from different component from CH, ex: Dictionaries status -# -# -# prometheus: -# endpoint: /metrics -# port: 9363 +# endpoint - mertics path (relative to root, statring with "/") +# port - port to setup server. If not defined or 0 than http_port used +# metrics - send data from table system.metrics +# events - send data from table system.events +# asynchronous_metrics - send data from table system.asynchronous_metrics +# status_info - send data from different component from CH, ex: Dictionaries status -# metrics: true -# events: true -# asynchronous_metrics: true -# status_info: true -# +# prometheus: +# endpoint: /metrics +# port: 9363 + +# metrics: true +# events: true +# asynchronous_metrics: true +# status_info: true # Query log. Used only for queries with setting log_queries = 1. query_log: -# What table to insert data. If table is not exist, it will be created. -# When query log structure is changed after system update, -# then old table will be renamed and new table will be created automatically. -# - database: system - table: query_log -# -# PARTITION BY expr: https://clickhouse.yandex/docs/en/table_engines/mergetree-family/custom_partitioning_key/ -# Example: -# event_date -# toMonday(event_date) -# toYYYYMM(event_date) -# toStartOfHour(event_time) -# - partition_by: toYYYYMM(event_date) -# -# Table TTL specification: https://clickhouse.tech/docs/en/engines/table-engines/mergetree-family/mergetree/#mergetree-table-ttl -# Example: -# event_date + INTERVAL 1 WEEK -# event_date + INTERVAL 7 DAY DELETE -# event_date + INTERVAL 2 WEEK TO DISK 'bbb' + # What table to insert data. If table is not exist, it will be created. + # When query log structure is changed after system update, + # then old table will be renamed and new table will be created automatically. + database: system + table: query_log -# ttl: 'event_date + INTERVAL 30 DAY DELETE' -# + # PARTITION BY expr: https://clickhouse.yandex/docs/en/table_engines/mergetree-family/custom_partitioning_key/ + # Example: + # event_date + # toMonday(event_date) + # toYYYYMM(event_date) + # toStartOfHour(event_time) + partition_by: toYYYYMM(event_date) -# Instead of partition_by, you can provide full engine expression (starting with ENGINE = ) with parameters, -# Example: engine: 'ENGINE = MergeTree PARTITION BY toYYYYMM(event_date) ORDER BY (event_date, event_time) SETTINGS index_granularity = 1024' -# + # Table TTL specification: https://clickhouse.tech/docs/en/engines/table-engines/mergetree-family/mergetree/#mergetree-table-ttl + # Example: + # event_date + INTERVAL 1 WEEK + # event_date + INTERVAL 7 DAY DELETE + # event_date + INTERVAL 2 WEEK TO DISK 'bbb' + + # ttl: 'event_date + INTERVAL 30 DAY DELETE' + + # Instead of partition_by, you can provide full engine expression (starting with ENGINE = ) with parameters, + # Example: engine: 'ENGINE = MergeTree PARTITION BY toYYYYMM(event_date) ORDER BY (event_date, event_time) SETTINGS index_granularity = 1024' + + # Interval of flushing data. + flush_interval_milliseconds: 7500 -# Interval of flushing data. - flush_interval_milliseconds: 7500 # Trace log. Stores stack traces collected by query profilers. # See query_profiler_real_time_period_ns and query_profiler_cpu_time_period_ns settings. trace_log: - database: system - table: trace_log - partition_by: toYYYYMM(event_date) - flush_interval_milliseconds: 7500 + database: system + table: trace_log + partition_by: toYYYYMM(event_date) + flush_interval_milliseconds: 7500 + # Query thread log. Has information about all threads participated in query execution. # Used only for queries with setting log_query_threads = 1. query_thread_log: - database: system - table: query_thread_log - partition_by: toYYYYMM(event_date) - flush_interval_milliseconds: 7500 + database: system + table: query_thread_log + partition_by: toYYYYMM(event_date) + flush_interval_milliseconds: 7500 + # Uncomment if use part log. # Part log contains information about all actions with parts in MergeTree tables (creation, deletion, merges, downloads). # part_log: -# database: system -# table: part_log -# flush_interval_milliseconds: 7500 -# +# database: system +# table: part_log +# flush_interval_milliseconds: 7500 # Uncomment to write text log into table. # Text log contains all information from usual server log but stores it in structured and efficient way. # The level of the messages that goes to the table can be limited (), if not specified all messages will go to the table. # text_log: -# database: system -# table: text_log -# flush_interval_milliseconds: 7500 -# level: '' -# +# database: system +# table: text_log +# flush_interval_milliseconds: 7500 +# level: '' # Metric log contains rows with current values of ProfileEvents, CurrentMetrics collected with "collect_interval_milliseconds" interval. metric_log: - database: system - table: metric_log - flush_interval_milliseconds: 7500 - collect_interval_milliseconds: 1000 -# -#Asynchronous metric log contains values of metrics from -#system.asynchronous_metrics. -# -asynchronous_metric_log: - database: system - table: asynchronous_metric_log -# -# Asynchronous metrics are updated once a minute, so there is -# no need to flush more often. -# - flush_interval_milliseconds: 60000 + database: system + table: metric_log + flush_interval_milliseconds: 7500 + collect_interval_milliseconds: 1000 -# -#OpenTelemetry log contains OpenTelemetry trace spans. -# +# Asynchronous metric log contains values of metrics from +# system.asynchronous_metrics. +asynchronous_metric_log: + database: system + table: asynchronous_metric_log + + # Asynchronous metrics are updated once a minute, so there is + # no need to flush more often. + flush_interval_milliseconds: 60000 + +# OpenTelemetry log contains OpenTelemetry trace spans. opentelemetry_span_log: -# -# The default table creation code is insufficient, this spec -# is a workaround. There is no 'event_time' for this log, but two times, -# start and finish. It is sorted by finish time, to avoid inserting -# data too far away in the past (probably we can sometimes insert a span -# that is seconds earlier than the last span in the table, due to a race -# between several spans inserted in parallel). This gives the spans a -# global order that we can use to e.g. retry insertion into some external -# system. -# - engine: |- - engine MergeTree - partition by toYYYYMM(finish_date) - order by (finish_date, finish_time_us, trace_id) - database: system - table: opentelemetry_span_log - flush_interval_milliseconds: 7500 + + # The default table creation code is insufficient, this spec + # is a workaround. There is no 'event_time' for this log, but two times, + # start and finish. It is sorted by finish time, to avoid inserting + # data too far away in the past (probably we can sometimes insert a span + # that is seconds earlier than the last span in the table, due to a race + # between several spans inserted in parallel). This gives the spans a + # global order that we can use to e.g. retry insertion into some external + # system. + engine: |- + engine MergeTree + partition by toYYYYMM(finish_date) + order by (finish_date, finish_time_us, trace_id) + database: system + table: opentelemetry_span_log + flush_interval_milliseconds: 7500 + # Crash log. Stores stack traces for fatal errors. # This table is normally empty. crash_log: - database: system - table: crash_log - partition_by: '' - flush_interval_milliseconds: 1000 + database: system + table: crash_log + partition_by: '' + flush_interval_milliseconds: 1000 + # Parameters for embedded dictionaries, used in Yandex.Metrica. # See https://clickhouse.yandex/docs/en/dicts/internal_dicts/ -# # Path to file with region hierarchy. # path_to_regions_hierarchy_file: /opt/geo/regions_hierarchy.txt @@ -828,60 +802,53 @@ crash_log: # Changes will not be applied w/o server restart. # Path to the list is under top_level_domains_path (see above). -# top_level_domains_lists: '' -# + # public_suffix_list: /path/to/public_suffix_list.dat -# # Configuration of external dictionaries. See: # https://clickhouse.tech/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts -# dictionaries_config: '*_dictionary.xml' + # Uncomment if you want data to be compressed 30-100% better. # Don't do that if you just started using ClickHouse. -# -# + # compression: # Set of variants. Checked in order. Last matching case wins. If nothing matches, lz4 will be used. -# case: - # Conditions. All must be satisfied. Some conditions may be omitted. -# min_part_size: 10000000000 # Min part size in bytes. -# min_part_size_ratio: 0.01 # Min size of part relative to whole table size. + # case: + # Conditions. All must be satisfied. Some conditions may be omitted. + # min_part_size: 10000000000 # Min part size in bytes. + # min_part_size_ratio: 0.01 # Min size of part relative to whole table size. - # What compression method to use. -# method: zstd -# + # What compression method to use. + # method: zstd # Allow to execute distributed DDL queries (CREATE, DROP, ALTER, RENAME) on cluster. # Works only if ZooKeeper is enabled. Comment it if such functionality isn't required. distributed_ddl: -# Path in ZooKeeper to queue with DDL queries - path: /clickhouse/task_queue/ddl -# Settings from this profile will be used to execute DDL queries -# profile: default + # Path in ZooKeeper to queue with DDL queries + path: /clickhouse/task_queue/ddl -# Controls how much ON CLUSTER queries can be run simultaneously. -# pool_size: 1 + # Settings from this profile will be used to execute DDL queries + # profile: default -# -# Cleanup settings (active tasks will not be removed) -# + # Controls how much ON CLUSTER queries can be run simultaneously. + # pool_size: 1 -# Controls task TTL (default 1 week) -# task_max_lifetime: 604800 + # Cleanup settings (active tasks will not be removed) -# Controls how often cleanup should be performed (in seconds) -# cleanup_delay_period: 60 + # Controls task TTL (default 1 week) + # task_max_lifetime: 604800 -# Controls how many tasks could be in the queue -# max_tasks_in_queue: 1000 + # Controls how often cleanup should be performed (in seconds) + # cleanup_delay_period: 60 + + # Controls how many tasks could be in the queue + # max_tasks_in_queue: 1000 # Settings to fine tune MergeTree tables. See documentation in source code, in MergeTreeSettings.h -# # merge_tree: -# max_suspicious_broken_parts: 5 -# +# max_suspicious_broken_parts: 5 # Protection from accidental DROP. # If size of a MergeTree table is greater than max_table_size_to_drop (in bytes) than table could not be dropped with any DROP query. @@ -889,47 +856,47 @@ distributed_ddl: # By default max_table_size_to_drop is 50GB; max_table_size_to_drop=0 allows to DROP any tables. # The same for max_partition_size_to_drop. # Uncomment to disable protection. -# + # max_table_size_to_drop: 0 # max_partition_size_to_drop: 0 # Example of parameters for GraphiteMergeTree table engine graphite_rollup_example: - pattern: - regexp: click_cost - function: any - retention: - - age: 0 - precision: 3600 - - age: 86400 - precision: 60 - default: - function: max - retention: - - age: 0 - precision: 60 - - age: 3600 - precision: 300 - - age: 86400 - precision: 3600 + pattern: + regexp: click_cost + function: any + retention: + - age: 0 + precision: 3600 + - age: 86400 + precision: 60 + default: + function: max + retention: + - age: 0 + precision: 60 + - age: 3600 + precision: 300 + - age: 86400 + precision: 3600 + # Directory in containing schema files for various input formats. # The directory will be created if it doesn't exist. -# format_schema_path: /var/lib/clickhouse/format_schemas/ + # Default query masking rules, matching lines would be replaced with something else in the logs # (both text logs and system.query_log). # name - name for the rule (optional) # regexp - RE2 compatible regular expression (mandatory) # replace - substitution string for sensitive data (optional, by default - six asterisks) -# query_masking_rules: - rule: - name: hide encrypt/decrypt arguments - regexp: '((?:aes_)?(?:encrypt|decrypt)(?:_mysql)?)\s*\(\s*(?:''(?:\\''|.)+''|.*?)\s*\)' - # or more secure, but also more invasive: - # (aes_\w+)\s*\(.*\) - # - replace: \1(???) + rule: + name: hide encrypt/decrypt arguments + regexp: '((?:aes_)?(?:encrypt|decrypt)(?:_mysql)?)\s*\(\s*(?:''(?:\\''|.)+''|.*?)\s*\)' + # or more secure, but also more invasive: + # (aes_\w+)\s*\(.*\) + replace: \1(???) + # Uncomment to use custom http handlers. # rules are checked from top to bottom, first match runs the handler # url - to match request URL, you can use 'regex:' prefix to use regex match(optional) @@ -944,37 +911,36 @@ query_masking_rules: # response_content - use with static type, Response content sent to client, when using the prefix 'file://' or 'config://', find the content from the file or configuration send to client. # http_handlers: -# - rule: -# url: / -# methods: POST,GET -# headers: -# pragma: no-cache -# handler: -# type: dynamic_query_handler -# query_param_name: query -# - rule: -# url: /predefined_query -# methods: POST,GET -# handler: -# type: predefined_query_handler -# query: 'SELECT * FROM system.settings' -# - rule: -# handler: -# type: static -# status: 200 -# content_type: 'text/plain; charset=UTF-8' -# response_content: config://http_server_default_response -# -send_crash_reports: -# Changing to true allows sending crash reports to -# the ClickHouse core developers team via Sentry https://sentry.io -# Doing so at least in pre-production environments is highly appreciated - enabled: false -# Change to true if you don't feel comfortable attaching the server hostname to the crash report - anonymize: false -# Default endpoint should be changed to different Sentry DSN only if you have -# some in-house engineers or hired consultants who're going to debug ClickHouse issues for you - endpoint: 'https://6f33034cfe684dd7a3ab9875e57b1c8d@o388870.ingest.sentry.io/5226277' -# Uncomment to disable ClickHouse internal DNS caching. -# disable_internal_dns_cache: 1 +# - rule: +# url: / +# methods: POST,GET +# headers: +# pragma: no-cache +# handler: +# type: dynamic_query_handler +# query_param_name: query +# - rule: +# url: /predefined_query +# methods: POST,GET +# handler: +# type: predefined_query_handler +# query: 'SELECT * FROM system.settings' +# - rule: +# handler: +# type: static +# status: 200 +# content_type: 'text/plain; charset=UTF-8' +# response_content: config://http_server_default_response +send_crash_reports: + # Changing to true allows sending crash reports to + # the ClickHouse core developers team via Sentry https://sentry.io + # Doing so at least in pre-production environments is highly appreciated + enabled: false + # Change to true if you don't feel comfortable attaching the server hostname to the crash report + anonymize: false + # Default endpoint should be changed to different Sentry DSN only if you have + # some in-house engineers or hired consultants who're going to debug ClickHouse issues for you + endpoint: 'https://6f33034cfe684dd7a3ab9875e57b1c8d@o388870.ingest.sentry.io/5226277' + # Uncomment to disable ClickHouse internal DNS caching. + # disable_internal_dns_cache: 1 From 82c25cb9611a861089c2fd277088a912f91f6a53 Mon Sep 17 00:00:00 2001 From: Dmitry Ukolov Date: Sun, 23 May 2021 18:11:56 +0400 Subject: [PATCH 39/46] Update index.md --- docs/ru/sql-reference/dictionaries/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/dictionaries/index.md b/docs/ru/sql-reference/dictionaries/index.md index 59c7518d0c5..bcb54a07005 100644 --- a/docs/ru/sql-reference/dictionaries/index.md +++ b/docs/ru/sql-reference/dictionaries/index.md @@ -13,5 +13,5 @@ ClickHouse поддерживает специальные функции для ClickHouse поддерживает: - [Встроенные словари](internal-dicts.md#internal_dicts) со специфическим [набором функций](../../sql-reference/dictionaries/external-dictionaries/index.md). -- [Подключаемые (внешние) словари](external-dictionaries/external-dicts.md#dicts-external-dicts) с [набором функций](../../sql-reference/dictionaries/external-dictionaries/index.md). +- [Подключаемые (внешние) словари](external-dictionaries/external-dicts.md#dicts-external-dicts) с [набором функций](../../sql-reference/dictionaries/ext-dict-functions/). From d7e0e3fe0236a72399ff4d0e15cd98a9dc5625e1 Mon Sep 17 00:00:00 2001 From: huzhichengdd <83016994+huzhichengdd@users.noreply.github.com> Date: Sun, 23 May 2021 22:12:43 +0800 Subject: [PATCH 40/46] Update index.md --- docs/zh/sql-reference/dictionaries/index.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/zh/sql-reference/dictionaries/index.md b/docs/zh/sql-reference/dictionaries/index.md index 7e8f5e83aa7..092afd5bac1 100644 --- a/docs/zh/sql-reference/dictionaries/index.md +++ b/docs/zh/sql-reference/dictionaries/index.md @@ -8,15 +8,15 @@ toc_title: "\u5BFC\u8A00" # 字典 {#dictionaries} -字典是一个映射 (`key -> attributes`)这是方便各种类型的参考清单。 +字典是一个映射 (`键 -> 属性`), 是方便各种类型的参考清单。 -ClickHouse支持使用可用于查询的字典的特殊功能。 这是更容易和更有效地使用字典与功能比 `JOIN` 与参考表。 +ClickHouse支持一些特殊函数配合字典在查询中使用。 将字典与函数结合使用比将 `JOIN` 操作与引用表结合使用更简单、更有效。 [NULL](../../sql-reference/syntax.md#null-literal) 值不能存储在字典中。 ClickHouse支持: -- [内置字典](internal-dicts.md#internal_dicts) 具有特定的 [功能集](../../sql-reference/functions/ym-dict-functions.md). -- [插件(外部)字典](external-dictionaries/external-dicts.md#dicts-external-dicts) 用一个 [功能集](../../sql-reference/functions/ext-dict-functions.md). +- [内置字典](internal-dicts.md#internal_dicts) ,这些字典具有特定的 [函数集](../../sql-reference/functions/ym-dict-functions.md). +- [插件(外部)字典](external-dictionaries/external-dicts.md#dicts-external-dicts) ,这些字典拥有一个 [函数集](../../sql-reference/functions/ext-dict-functions.md). [原始文章](https://clickhouse.tech/docs/en/query_language/dicts/) From 2d6a3648565571103700b717c189c1ffaf3d78f1 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 23 May 2021 17:38:23 +0300 Subject: [PATCH 41/46] Update index.md --- docs/ru/sql-reference/dictionaries/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/dictionaries/index.md b/docs/ru/sql-reference/dictionaries/index.md index bcb54a07005..ee6c3e6f72a 100644 --- a/docs/ru/sql-reference/dictionaries/index.md +++ b/docs/ru/sql-reference/dictionaries/index.md @@ -12,6 +12,6 @@ ClickHouse поддерживает специальные функции для ClickHouse поддерживает: -- [Встроенные словари](internal-dicts.md#internal_dicts) со специфическим [набором функций](../../sql-reference/dictionaries/external-dictionaries/index.md). +- [Встроенные словари](internal-dicts.md#internal_dicts) со специфическим [набором функций](../../sql-reference/dictionaries/ext-dict-functions/). - [Подключаемые (внешние) словари](external-dictionaries/external-dicts.md#dicts-external-dicts) с [набором функций](../../sql-reference/dictionaries/ext-dict-functions/). From ab686628a9ebe9753f1690cb6549ae7b390b8005 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 24 May 2021 00:06:08 +0300 Subject: [PATCH 42/46] Fix dictionary functions documentation --- docs/ru/sql-reference/dictionaries/index.md | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/docs/ru/sql-reference/dictionaries/index.md b/docs/ru/sql-reference/dictionaries/index.md index ee6c3e6f72a..84c6f1a3c13 100644 --- a/docs/ru/sql-reference/dictionaries/index.md +++ b/docs/ru/sql-reference/dictionaries/index.md @@ -12,6 +12,5 @@ ClickHouse поддерживает специальные функции для ClickHouse поддерживает: -- [Встроенные словари](internal-dicts.md#internal_dicts) со специфическим [набором функций](../../sql-reference/dictionaries/ext-dict-functions/). -- [Подключаемые (внешние) словари](external-dictionaries/external-dicts.md#dicts-external-dicts) с [набором функций](../../sql-reference/dictionaries/ext-dict-functions/). - +- [Встроенные словари](internal-dicts.md#internal_dicts) со специфическим [набором функций](../../sql-reference/functions/ext-dict-functions.md). +- [Подключаемые (внешние) словари](external-dictionaries/external-dicts.md#dicts-external-dicts) с [набором функций](../../sql-reference/functions/ext-dict-functions.md). \ No newline at end of file From e5354f8f3685b4df8938e8e385b620bfca04f26d Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 23 May 2021 19:42:55 -0300 Subject: [PATCH 43/46] PREWHERE + FINAL --- docs/ru/sql-reference/statements/select/prewhere.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/ru/sql-reference/statements/select/prewhere.md b/docs/ru/sql-reference/statements/select/prewhere.md index c2a02b1a436..5ba25e6fa6e 100644 --- a/docs/ru/sql-reference/statements/select/prewhere.md +++ b/docs/ru/sql-reference/statements/select/prewhere.md @@ -16,6 +16,9 @@ Prewhere — это оптимизация для более эффективн Если значение параметра `optimize_move_to_prewhere` равно 0, эвристика по автоматическому перемещнию части выражений из `WHERE` к `PREWHERE` отключается. +!!! note "Внимание" + Секция `PREWHERE` выполняется до `FINAL`, поэтому результаты запросов `FROM FINAL` могут исказится при использовании `PREWHERE` с полями не входящями в `ORDER BY` таблицы. + ## Ограничения {#limitations} `PREWHERE` поддерживается только табличными движками из семейства `*MergeTree`. From 12a05637413032e33b09a7c149afa1d080c89819 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 23 May 2021 19:47:29 -0300 Subject: [PATCH 44/46] PREWHERE+FINAL --- docs/en/sql-reference/statements/select/prewhere.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/en/sql-reference/statements/select/prewhere.md b/docs/en/sql-reference/statements/select/prewhere.md index fc43d1de0a1..663b84f2d48 100644 --- a/docs/en/sql-reference/statements/select/prewhere.md +++ b/docs/en/sql-reference/statements/select/prewhere.md @@ -16,6 +16,9 @@ A query may simultaneously specify `PREWHERE` and `WHERE`. In this case, `PREWHE If the `optimize_move_to_prewhere` setting is set to 0, heuristics to automatically move parts of expressions from `WHERE` to `PREWHERE` are disabled. +!!! note "Attention" + The `PREWHERE` section is executed before` FINAL`, so the results of `FROM FINAL` queries may be skewed when using` PREWHERE` with fields not in the `ORDER BY` section of a table. + ## Limitations {#limitations} `PREWHERE` is only supported by tables from the `*MergeTree` family. From 5c815e4c7fa2c962dd8cf7996373bb5efc15bca5 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 24 May 2021 03:51:37 +0300 Subject: [PATCH 45/46] Update config-example.yaml --- programs/server/config-example.yaml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/programs/server/config-example.yaml b/programs/server/config-example.yaml index 48973fe6fc0..a2f1ff352d8 100644 --- a/programs/server/config-example.yaml +++ b/programs/server/config-example.yaml @@ -199,7 +199,8 @@ openSSL: name: RejectCertificateHandler # Default root page on http[s] server. For example load UI from https://tabix.io/ when opening http://localhost:8123 -#
]]>
+# http_server_default_response: |- +#
# Maximum number of concurrent queries. max_concurrent_queries: 100 From f80e6535b0fbe1c204e7473774b9073d4bc45619 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 May 2021 05:55:03 +0300 Subject: [PATCH 46/46] Maybe fix YAML linter --- programs/server/config-example.yaml | 24 +++--- programs/server/config-example.yaml.disabled | 86 -------------------- 2 files changed, 11 insertions(+), 99 deletions(-) delete mode 100644 programs/server/config-example.yaml.disabled diff --git a/programs/server/config-example.yaml b/programs/server/config-example.yaml index a2f1ff352d8..af62455ccef 100644 --- a/programs/server/config-example.yaml +++ b/programs/server/config-example.yaml @@ -391,7 +391,6 @@ user_directories: local_directory: # Path to folder where users created by SQL commands are stored. path: /var/lib/clickhouse/access/ - # To add an LDAP server as a remote user directory of users that are not defined locally, define a single 'ldap' section # with the following parameters: # server - one of LDAP server names defined in 'ldap_servers' config section above. @@ -574,11 +573,11 @@ remote_servers: test_unavailable_shard: shard: - replica: - host: localhost - port: 9000 + host: localhost + port: 9000 - replica: - host: localhost - port: 1 + host: localhost + port: 1 # The list of hosts allowed to use in URL-related storage engines and table functions. # If this section is not present in configuration, all hosts are allowed. @@ -815,14 +814,13 @@ dictionaries_config: '*_dictionary.xml' # Don't do that if you just started using ClickHouse. # compression: - # Set of variants. Checked in order. Last matching case wins. If nothing matches, lz4 will be used. - # case: - # Conditions. All must be satisfied. Some conditions may be omitted. - # min_part_size: 10000000000 # Min part size in bytes. - # min_part_size_ratio: 0.01 # Min size of part relative to whole table size. - - # What compression method to use. - # method: zstd +# # Set of variants. Checked in order. Last matching case wins. If nothing matches, lz4 will be used. +# case: +# Conditions. All must be satisfied. Some conditions may be omitted. +# # min_part_size: 10000000000 # Min part size in bytes. +# # min_part_size_ratio: 0.01 # Min size of part relative to whole table size. +# # What compression method to use. +# method: zstd # Allow to execute distributed DDL queries (CREATE, DROP, ALTER, RENAME) on cluster. # Works only if ZooKeeper is enabled. Comment it if such functionality isn't required. diff --git a/programs/server/config-example.yaml.disabled b/programs/server/config-example.yaml.disabled deleted file mode 100644 index a83acf50de0..00000000000 --- a/programs/server/config-example.yaml.disabled +++ /dev/null @@ -1,86 +0,0 @@ -# We can use 3 main node types in YAML: Scalar, Map and Sequence. - - - -# A Scalar is a simple key-value pair: - -scalar: 123 - -# Here we have a key "scalar" and value "123" -# If we rewrite this in XML, we will get 123 - -# We can also represent an empty value with '': - -key: '' - - - -# A Map is a node, which contains other nodes: - -map: - key1: value1 - key2: value2 - small_map: - key3: value3 - -# This map can be converted into: -# -# value1 -# value2 -# -# value3 -# -# - - - -# A Sequence is a node, which contains also other nodes. -# The main difference from Map is that Sequence can also contain simple values. - -sequence: - - val1 - - val2 - - key: 123 - - map: - mkey1: foo - mkey2: bar - -# We can represent it in XML this way: -# val1 -# val2 -# -# 123 -# -# -# -# foo -# bar -# -# - - - -# YAML does not have direct support for structures like XML attributes. -# We represent them as nodes with @ prefix in key. Note, that @ is reserved by YAML standard, -# so you will need to write double quotes around the key. Both Map and Sequence can have -# attributes as children nodes - -map: - "@attr1": value1 - "@attr2": value2 - key: 123 - -# This gives us: -# -# 123 -# - -sequence: - - "@attr1": value1 - - "@attr2": value2 - - 123 - - abc - -# And this gives us: -# 123 -# abc