From 2695aa13c4a67dcbefb31c246effb4b33852f4ba Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 16 Dec 2019 18:51:15 +0300 Subject: [PATCH 01/53] Add ability to execute mutations asynchronously --- dbms/src/Core/Settings.h | 1 + dbms/src/Storages/StorageMergeTree.cpp | 31 ++++- dbms/src/Storages/StorageMergeTree.h | 6 + .../Storages/StorageReplicatedMergeTree.cpp | 120 +++++++++++++++++- .../src/Storages/StorageReplicatedMergeTree.h | 9 ++ ..._zookeeper_synchronous_mutations.reference | 10 ++ .../01049_zookeeper_synchronous_mutations.sql | 48 +++++++ 7 files changed, 220 insertions(+), 5 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.reference create mode 100644 dbms/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.sql diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 00d7efb4a5b..74199925bec 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -388,6 +388,7 @@ struct Settings : public SettingsCollection \ M(SettingBool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.", 0) \ M(SettingBool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \ + M(SettingUInt64, mutation_synchronous_wait_timeout, 0, "Seconds to wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). After execute asynchronously. 0 - execute asynchronously from the begging.", 0) \ \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 41c9335de1e..f93a99bf89b 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -42,6 +42,7 @@ namespace ErrorCodes extern const int PART_IS_TEMPORARILY_LOCKED; extern const int UNKNOWN_SETTING; extern const int TOO_BIG_AST; + extern const int UNFINISHED; } namespace ActionLocks @@ -425,17 +426,18 @@ public: }; -void StorageMergeTree::mutate(const MutationCommands & commands, const Context &) +void StorageMergeTree::mutate(const MutationCommands & commands, const Context & query_context) { /// Choose any disk, because when we load mutations we search them at each disk /// where storage can be placed. See loadMutations(). auto disk = storage_policy->getAnyDisk(); MergeTreeMutationEntry entry(commands, getFullPathOnDisk(disk), insert_increment.get()); String file_name; + Int64 version; { std::lock_guard lock(currently_processing_in_background_mutex); - Int64 version = increment.get(); + version = increment.get(); entry.commit(version); file_name = entry.file_name; auto insertion = current_mutations_by_id.emplace(file_name, std::move(entry)); @@ -444,6 +446,17 @@ void StorageMergeTree::mutate(const MutationCommands & commands, const Context & LOG_INFO(log, "Added mutation: " << file_name); merging_mutating_task_handle->wake(); + + size_t timeout = query_context.getSettingsRef().mutation_synchronous_wait_timeout; + /// If timeout is set, than we can wait + if (timeout != 0) + { + LOG_INFO(log, "Waiting mutation: " << file_name << " for " << timeout << " seconds"); + auto check = [version, this]() { return isMutationDone(version); }; + std::unique_lock lock(mutation_wait_mutex); + if (!mutation_wait_event.wait_for(lock, std::chrono::seconds{timeout}, check)) + throw Exception("Mutation " + file_name + " is not finished. Will be done asynchronously", ErrorCodes::UNFINISHED); + } } namespace @@ -462,6 +475,17 @@ bool comparator(const PartVersionWithName & f, const PartVersionWithName & s) } +bool StorageMergeTree::isMutationDone(Int64 mutation_version) const +{ + std::lock_guard lock(currently_processing_in_background_mutex); + + auto data_parts = getDataPartsVector(); + for (const auto & data_part : data_parts) + if (data_part->info.getDataVersion() < mutation_version) + return false; + return true; +} + std::vector StorageMergeTree::getMutationsStatus() const { std::lock_guard lock(currently_processing_in_background_mutex); @@ -771,6 +795,9 @@ bool StorageMergeTree::tryMutatePart() renameTempPartAndReplace(new_part); tagger->is_successful = true; write_part_log({}); + + /// Notify all, who wait for this or previous mutations + mutation_wait_event.notify_all(); } catch (...) { diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index dfef9261145..5972717e980 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -79,6 +79,10 @@ public: private: + /// Mutex and condvar for synchronous mutations wait + std::mutex mutation_wait_mutex; + std::condition_variable mutation_wait_event; + MergeTreeDataSelectExecutor reader; MergeTreeDataWriter writer; MergeTreeDataMergerMutator merger_mutator; @@ -138,6 +142,8 @@ private: void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context); bool partIsAssignedToBackgroundOperation(const DataPartPtr & part) const override; + /// Just checks versions of each active data part + bool isMutationDone(Int64 mutation_version) const; friend class MergeTreeBlockOutputStream; friend class MergeTreeData; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index b3f69d463f2..be8124b05ca 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -54,6 +54,7 @@ #include #include +#include namespace ProfileEvents { @@ -309,6 +310,92 @@ bool StorageReplicatedMergeTree::checkFixedGranualrityInZookeeper() } +void StorageReplicatedMergeTree::waitForAllReplicasToStatisfyNodeCondition( + size_t timeout, const String & name_for_logging, + const String & replica_relative_node_path, CheckNodeCallback callback) const +{ + const auto operation_start = std::chrono::system_clock::now(); + std::chrono::milliseconds total_time{timeout * 1000}; + zkutil::EventPtr wait_event = std::make_shared(); + Strings replicas = getZooKeeper()->getChildren(zookeeper_path + "/replicas"); + std::set inactive_replicas; + std::set timed_out_replicas; + for (const String & replica : replicas) + { + LOG_DEBUG(log, "Waiting for " << replica << " to apply " + name_for_logging); + + bool operation_is_processed_by_relica = false; + while (!partial_shutdown_called) + { + auto zookeeper = getZooKeeper(); + /// Replica could be inactive. + if (!zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")) + { + LOG_WARNING(log, "Replica " << replica << " is not active during mutation query." + << name_for_logging << " will be done asynchronously when replica becomes active."); + + inactive_replicas.emplace(replica); + break; + } + + String node_for_check = zookeeper_path + "/replicas/" + replica + "/" + replica_relative_node_path; + std::string node_for_check_value; + Coordination::Stat stat; + /// Replica could be removed + if (!zookeeper->tryGet(node_for_check, node_for_check_value, &stat, wait_event)) + { + LOG_WARNING(log, replica << " was removed"); + operation_is_processed_by_relica = true; + break; + } + else /// in other case check required node + { + if (callback(node_for_check_value)) + { + operation_is_processed_by_relica = true; + break; /// operation is done + } + } + + std::chrono::milliseconds time_spent = + std::chrono::duration_cast(std::chrono::system_clock::now() - operation_start); + std::chrono::milliseconds time_left = total_time - time_spent; + + /// We have some time to wait + if (time_left.count() > 0) + wait_event->tryWait(time_left.count()); + else /// Otherwise time is up + break; + } + + if (partial_shutdown_called) + throw Exception(name_for_logging + " is not finished because table shutdown was called. " + name_for_logging + " will be done after table restart.", + ErrorCodes::UNFINISHED); + + if (!operation_is_processed_by_relica && !inactive_replicas.count(replica)) + timed_out_replicas.emplace(replica); + } + + if (!inactive_replicas.empty() || !timed_out_replicas.empty()) + { + std::stringstream exception_message; + exception_message << name_for_logging << " is not finished because"; + + if (!inactive_replicas.empty()) + exception_message << " some replicas are inactive right now: " << boost::algorithm::join(inactive_replicas, ", "); + + if (!timed_out_replicas.empty() && !inactive_replicas.empty()) + exception_message << " and"; + + if (!timed_out_replicas.empty()) + exception_message << " timeout when waiting for some replicas: " << boost::algorithm::join(timed_out_replicas, ", "); + + exception_message << ". " << name_for_logging << " will be done asynchronously"; + + throw Exception(exception_message.str(), ErrorCodes::UNFINISHED); + } +} + void StorageReplicatedMergeTree::createNewZooKeeperNodes() { auto zookeeper = getZooKeeper(); @@ -3200,6 +3287,7 @@ void StorageReplicatedMergeTree::alter( int32_t new_version = -1; /// Initialization is to suppress (useless) false positive warning found by cppcheck. }; + /// /columns and /metadata nodes std::vector changed_nodes; { @@ -3294,6 +3382,10 @@ void StorageReplicatedMergeTree::alter( time_t replication_alter_columns_timeout = query_context.getSettingsRef().replication_alter_columns_timeout; + /// This code is quite similar with waitForAllReplicasToStatisfyNodeCondition + /// but contains more complicated details (versions manipulations, multiple nodes, etc.). + /// It will be removed soon in favor of alter-modify implementation on top of mutations. + /// TODO (alesap) for (const String & replica : replicas) { LOG_DEBUG(log, "Waiting for " << replica << " to apply changes"); @@ -3396,8 +3488,16 @@ void StorageReplicatedMergeTree::alter( if (replica_nodes_changed_concurrently) continue; - /// Now wait for replica nodes to change. - + /// alter_query_event subscribed with zookeeper watch callback to /repliacs/{replica}/metadata + /// and /replicas/{replica}/columns nodes for current relica + shared nodes /columns and /metadata, + /// which is common for all replicas. If changes happen with this nodes (delete, set and create) + /// than event will be notified and wait will be interrupted. + /// + /// ReplicatedMergeTreeAlterThread responsible for local /replicas/{replica}/metadata and + /// /replicas/{replica}/columns changes. Shared /columns and /metadata nodes can be changed by *newer* + /// concurrent alter from other replica. First of all it will update shared nodes and we will have no + /// ability to identify, that our *current* alter finshed. So we cannot do anything better than just + /// return from *current* alter with success result. if (!replication_alter_columns_timeout) { alter_query_event->wait(); @@ -4399,7 +4499,7 @@ void StorageReplicatedMergeTree::fetchPartition(const ASTPtr & partition, const } -void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, const Context &) +void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, const Context & query_context) { /// Overview of the mutation algorithm. /// @@ -4502,6 +4602,20 @@ void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, const else throw Coordination::Exception("Unable to create a mutation znode", rc); } + + if (query_context.getSettingsRef().mutation_synchronous_wait_timeout != 0) /// some timeout specified + { + auto check_callback = [mutation_number = entry.znode_name](const String & zk_value) + { + /// Maybe we already processed more fresh mutation + /// We can compare their znode names (numbers like 0000000000 and 0000000001). + return zk_value >= mutation_number; + }; + + waitForAllReplicasToStatisfyNodeCondition( + query_context.getSettingsRef().mutation_synchronous_wait_timeout, "Mutation", "mutation_pointer", check_callback); + } + } std::vector StorageReplicatedMergeTree::getMutationsStatus() const diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 3727d5de0d8..55957439a1c 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -532,6 +532,15 @@ private: /// return true if it's fixed bool checkFixedGranualrityInZookeeper(); + using CheckNodeCallback = std::function; + + /// Wait for timeout seconds when condition became true for node + /// /replicas/{replica}/replica_replative_node_path value for all replicas. + /// operation_name_for_logging used for logging about errors. + void waitForAllReplicasToStatisfyNodeCondition( + size_t timeout, const String & operaton_name_for_logging, + const String & replica_relative_node_path, CheckNodeCallback condition) const; + protected: /** If not 'attach', either creates a new table in ZK, or adds a replica to an existing table. */ diff --git a/dbms/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.reference b/dbms/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.reference new file mode 100644 index 00000000000..d65c89faf0e --- /dev/null +++ b/dbms/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.reference @@ -0,0 +1,10 @@ +Replicated +1 +1 +1 +1 +Normal +1 +1 +1 +1 diff --git a/dbms/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.sql b/dbms/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.sql new file mode 100644 index 00000000000..ede0ee22a50 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.sql @@ -0,0 +1,48 @@ +DROP TABLE IF EXISTS table_for_synchronous_mutations1; +DROP TABLE IF EXISTS table_for_synchronous_mutations2; + +SELECT 'Replicated'; + +CREATE TABLE table_for_synchronous_mutations1(k UInt32, v1 UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/table_for_synchronous_mutations', '1') ORDER BY k PARTITION BY modulo(k, 2); + +CREATE TABLE table_for_synchronous_mutations2(k UInt32, v1 UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/table_for_synchronous_mutations', '2') ORDER BY k PARTITION BY modulo(k, 2); + +INSERT INTO table_for_synchronous_mutations1 select number, number from numbers(100000); + +SYSTEM SYNC REPLICA table_for_synchronous_mutations2; + +ALTER TABLE table_for_synchronous_mutations1 UPDATE v1 = v1 + 1 WHERE 1 SETTINGS mutation_synchronous_wait_timeout = 10; + +SELECT is_done FROM system.mutations where table = 'table_for_synchronous_mutations1'; + +ALTER TABLE table_for_synchronous_mutations1 UPDATE v1 = 1 WHERE ignore(sleep(3)) SETTINGS mutation_synchronous_wait_timeout = 2; --{serverError 341} + +-- Another mutation, just to be sure, that previous finished +ALTER TABLE table_for_synchronous_mutations1 UPDATE v1 = v1 + 1 WHERE 1 SETTINGS mutation_synchronous_wait_timeout = 10; + +SELECT is_done FROM system.mutations where table = 'table_for_synchronous_mutations1'; + +DROP TABLE IF EXISTS table_for_synchronous_mutations1; +DROP TABLE IF EXISTS table_for_synchronous_mutations2; + +SELECT 'Normal'; + +DROP TABLE IF EXISTS table_for_synchronous_mutations_no_replication; + +CREATE TABLE table_for_synchronous_mutations_no_replication(k UInt32, v1 UInt64) ENGINE MergeTree ORDER BY k PARTITION BY modulo(k, 2); + +INSERT INTO table_for_synchronous_mutations_no_replication select number, number from numbers(100000); + +ALTER TABLE table_for_synchronous_mutations_no_replication UPDATE v1 = v1 + 1 WHERE 1 SETTINGS mutation_synchronous_wait_timeout = 10; + +SELECT is_done FROM system.mutations where table = 'table_for_synchronous_mutations_no_replication'; + +ALTER TABLE table_for_synchronous_mutations_no_replication UPDATE v1 = 1 WHERE ignore(sleep(3)) SETTINGS mutation_synchronous_wait_timeout = 2; --{serverError 341} + +-- Another mutation, just to be sure, that previous finished +ALTER TABLE table_for_synchronous_mutations_no_replication UPDATE v1 = v1 + 1 WHERE 1 SETTINGS mutation_synchronous_wait_timeout = 10; + +SELECT is_done FROM system.mutations where table = 'table_for_synchronous_mutations_no_replication'; + + +DROP TABLE IF EXISTS table_for_synchronous_mutations_no_replication; From 9e7033cc68ae46b2934cd0822012868dbdee8763 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 16 Dec 2019 22:56:17 +0300 Subject: [PATCH 02/53] Simplify test and sligtly increase timeouts --- .../01049_zookeeper_synchronous_mutations.sql | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.sql b/dbms/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.sql index ede0ee22a50..3b4bb145f1a 100644 --- a/dbms/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.sql +++ b/dbms/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.sql @@ -3,9 +3,9 @@ DROP TABLE IF EXISTS table_for_synchronous_mutations2; SELECT 'Replicated'; -CREATE TABLE table_for_synchronous_mutations1(k UInt32, v1 UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/table_for_synchronous_mutations', '1') ORDER BY k PARTITION BY modulo(k, 2); +CREATE TABLE table_for_synchronous_mutations1(k UInt32, v1 UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/table_for_synchronous_mutations', '1') ORDER BY k; -CREATE TABLE table_for_synchronous_mutations2(k UInt32, v1 UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/table_for_synchronous_mutations', '2') ORDER BY k PARTITION BY modulo(k, 2); +CREATE TABLE table_for_synchronous_mutations2(k UInt32, v1 UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/table_for_synchronous_mutations', '2') ORDER BY k; INSERT INTO table_for_synchronous_mutations1 select number, number from numbers(100000); @@ -18,7 +18,7 @@ SELECT is_done FROM system.mutations where table = 'table_for_synchronous_mutati ALTER TABLE table_for_synchronous_mutations1 UPDATE v1 = 1 WHERE ignore(sleep(3)) SETTINGS mutation_synchronous_wait_timeout = 2; --{serverError 341} -- Another mutation, just to be sure, that previous finished -ALTER TABLE table_for_synchronous_mutations1 UPDATE v1 = v1 + 1 WHERE 1 SETTINGS mutation_synchronous_wait_timeout = 10; +ALTER TABLE table_for_synchronous_mutations1 UPDATE v1 = v1 + 1 WHERE 1 SETTINGS mutation_synchronous_wait_timeout = 15; SELECT is_done FROM system.mutations where table = 'table_for_synchronous_mutations1'; @@ -29,7 +29,7 @@ SELECT 'Normal'; DROP TABLE IF EXISTS table_for_synchronous_mutations_no_replication; -CREATE TABLE table_for_synchronous_mutations_no_replication(k UInt32, v1 UInt64) ENGINE MergeTree ORDER BY k PARTITION BY modulo(k, 2); +CREATE TABLE table_for_synchronous_mutations_no_replication(k UInt32, v1 UInt64) ENGINE MergeTree ORDER BY k; INSERT INTO table_for_synchronous_mutations_no_replication select number, number from numbers(100000); @@ -40,7 +40,7 @@ SELECT is_done FROM system.mutations where table = 'table_for_synchronous_mutati ALTER TABLE table_for_synchronous_mutations_no_replication UPDATE v1 = 1 WHERE ignore(sleep(3)) SETTINGS mutation_synchronous_wait_timeout = 2; --{serverError 341} -- Another mutation, just to be sure, that previous finished -ALTER TABLE table_for_synchronous_mutations_no_replication UPDATE v1 = v1 + 1 WHERE 1 SETTINGS mutation_synchronous_wait_timeout = 10; +ALTER TABLE table_for_synchronous_mutations_no_replication UPDATE v1 = v1 + 1 WHERE 1 SETTINGS mutation_synchronous_wait_timeout = 15; SELECT is_done FROM system.mutations where table = 'table_for_synchronous_mutations_no_replication'; From 8116351f09b8bf9da25be79a885faf496b75eb7f Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 18 Dec 2019 12:20:33 +0300 Subject: [PATCH 03/53] Fix setting description --- dbms/src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 0b1e5e76be7..c4ac2925feb 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -388,7 +388,7 @@ struct Settings : public SettingsCollection \ M(SettingBool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.", 0) \ M(SettingBool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \ - M(SettingUInt64, mutation_synchronous_wait_timeout, 0, "Seconds to wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). After execute asynchronously. 0 - execute asynchronously from the begging.", 0) \ + M(SettingUInt64, mutation_synchronous_wait_timeout, 0, "Seconds to wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). After execute asynchronously. 0 - execute asynchronously from the start.", 0) \ \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ From 9b3bc2f0b35bc4996dc8b3be836a62aab10a4d14 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 18 Dec 2019 16:00:20 +0300 Subject: [PATCH 04/53] Fix flapping test --- .../00834_kill_mutation_replicated_zookeeper.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh b/dbms/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh index 59c9882a388..570e4ea785b 100755 --- a/dbms/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh +++ b/dbms/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh @@ -17,9 +17,9 @@ ${CLICKHOUSE_CLIENT} --query="INSERT INTO test.kill_mutation_r1 VALUES ('2001-01 ${CLICKHOUSE_CLIENT} --query="SELECT '*** Create and kill a single invalid mutation ***'" -${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.kill_mutation_r1 DELETE WHERE toUInt32(s) = 1" +# wrong mutation +${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.kill_mutation_r1 DELETE WHERE toUInt32(s) = 1 SETTINGS mutation_synchronous_wait_timeout=2" 2>/dev/null -sleep 1 ${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, latest_failed_part IN ('20000101_0_0_0', '20010101_0_0_0'), latest_fail_time != 0, substr(latest_fail_reason, 1, 8) FROM system.mutations WHERE database = 'test' AND table = 'kill_mutation_r1'" ${CLICKHOUSE_CLIENT} --query="KILL MUTATION WHERE database = 'test' AND table = 'kill_mutation_r1'" @@ -31,9 +31,9 @@ ${CLICKHOUSE_CLIENT} --query="SELECT '*** Create and kill invalid mutation that ${CLICKHOUSE_CLIENT} --query="SYSTEM SYNC REPLICA test.kill_mutation_r1" ${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.kill_mutation_r1 DELETE WHERE toUInt32(s) = 1" -${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.kill_mutation_r1 DELETE WHERE x = 1" -sleep 1 +${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.kill_mutation_r1 DELETE WHERE x = 1 SETTINGS mutation_synchronous_wait_timeout=2" 2>/dev/null + ${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, latest_failed_part IN ('20000101_0_0_0_1', '20010101_0_0_0_1'), latest_fail_time != 0, substr(latest_fail_reason, 1, 8) FROM system.mutations WHERE database = 'test' AND table = 'kill_mutation_r1' AND mutation_id = '0000000001'" ${CLICKHOUSE_CLIENT} --query="KILL MUTATION WHERE database = 'test' AND table = 'kill_mutation_r1' AND mutation_id = '0000000001'" From 9fea941009e88f59124c14cebfa532dc277989d5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 19 Dec 2019 18:27:56 +0300 Subject: [PATCH 05/53] Make wait endless --- dbms/src/Core/Settings.h | 2 +- .../MergeTree/MergeTreeMutationEntry.h | 1 + dbms/src/Storages/StorageMergeTree.cpp | 16 ++-- .../Storages/StorageReplicatedMergeTree.cpp | 94 +++++++++---------- .../src/Storages/StorageReplicatedMergeTree.h | 11 +-- .../0_stateless/00834_kill_mutation.sh | 8 +- ...ll_mutation_replicated_zookeeper.reference | 1 + ...0834_kill_mutation_replicated_zookeeper.sh | 33 ++++++- ..._zookeeper_synchronous_mutations.reference | 2 - .../01049_zookeeper_synchronous_mutations.sql | 13 +-- 10 files changed, 95 insertions(+), 86 deletions(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index c4ac2925feb..e64440961a4 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -388,7 +388,7 @@ struct Settings : public SettingsCollection \ M(SettingBool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.", 0) \ M(SettingBool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \ - M(SettingUInt64, mutation_synchronous_wait_timeout, 0, "Seconds to wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). After execute asynchronously. 0 - execute asynchronously from the start.", 0) \ + M(SettingUInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \ \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ diff --git a/dbms/src/Storages/MergeTree/MergeTreeMutationEntry.h b/dbms/src/Storages/MergeTree/MergeTreeMutationEntry.h index 2b3bde72552..f5c512617d3 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeMutationEntry.h +++ b/dbms/src/Storages/MergeTree/MergeTreeMutationEntry.h @@ -25,6 +25,7 @@ struct MergeTreeMutationEntry MergeTreePartInfo latest_failed_part_info; time_t latest_fail_time = 0; String latest_fail_reason; + int latest_fail_error_code = 0; /// Create a new entry and write it to a temporary file. MergeTreeMutationEntry(MutationCommands commands_, const String & path_prefix_, Int64 tmp_number); diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index f93a99bf89b..1ae67e8a5a8 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -447,15 +447,14 @@ void StorageMergeTree::mutate(const MutationCommands & commands, const Context & LOG_INFO(log, "Added mutation: " << file_name); merging_mutating_task_handle->wake(); - size_t timeout = query_context.getSettingsRef().mutation_synchronous_wait_timeout; - /// If timeout is set, than we can wait - if (timeout != 0) + /// We have to wait mutation end + if (query_context.getSettingsRef().mutations_sync > 0) { - LOG_INFO(log, "Waiting mutation: " << file_name << " for " << timeout << " seconds"); + LOG_INFO(log, "Waiting mutation: " << file_name); auto check = [version, this]() { return isMutationDone(version); }; std::unique_lock lock(mutation_wait_mutex); - if (!mutation_wait_event.wait_for(lock, std::chrono::seconds{timeout}, check)) - throw Exception("Mutation " + file_name + " is not finished. Will be done asynchronously", ErrorCodes::UNFINISHED); + mutation_wait_event.wait(lock, check); + } } @@ -479,6 +478,10 @@ bool StorageMergeTree::isMutationDone(Int64 mutation_version) const { std::lock_guard lock(currently_processing_in_background_mutex); + /// Killed + if (!current_mutations_by_version.count(mutation_version)) + return true; + auto data_parts = getDataPartsVector(); for (const auto & data_part : data_parts) if (data_part->info.getDataVersion() < mutation_version) @@ -559,6 +562,7 @@ CancellationCode StorageMergeTree::killMutation(const String & mutation_id) global_context.getMergeList().cancelPartMutations({}, to_kill->block_number); to_kill->removeFile(); LOG_TRACE(log, "Cancelled part mutations and removed mutation file " << mutation_id); + mutation_wait_event.notify_all(); /// Maybe there is another mutation that was blocked by the killed one. Try to execute it immediately. merging_mutating_task_handle->wake(); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index be8124b05ca..1bdd506dee9 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -310,87 +310,73 @@ bool StorageReplicatedMergeTree::checkFixedGranualrityInZookeeper() } -void StorageReplicatedMergeTree::waitForAllReplicasToStatisfyNodeCondition( - size_t timeout, const String & name_for_logging, - const String & replica_relative_node_path, CheckNodeCallback callback) const +void StorageReplicatedMergeTree::waitMutationToFinishOnReplicas( + const Strings & replicas, const String & mutation_id) const { - const auto operation_start = std::chrono::system_clock::now(); - std::chrono::milliseconds total_time{timeout * 1000}; + if (replicas.empty()) + return; + zkutil::EventPtr wait_event = std::make_shared(); - Strings replicas = getZooKeeper()->getChildren(zookeeper_path + "/replicas"); + + std::set inactive_replicas; - std::set timed_out_replicas; for (const String & replica : replicas) { - LOG_DEBUG(log, "Waiting for " << replica << " to apply " + name_for_logging); - bool operation_is_processed_by_relica = false; + LOG_DEBUG(log, "Waiting for " << replica << " to apply mutation " + mutation_id); + while (!partial_shutdown_called) { + /// Mutation maybe killed or whole replica was deleted. + /// Wait event will unblock at this moment. + Coordination::Stat exists_stat; + if (!getZooKeeper()->exists(zookeeper_path + "/mutations/" + mutation_id, &exists_stat, wait_event)) + { + LOG_WARNING(log, "Mutation " << mutation_id << " was killed or manually removed. Nothing to wait."); + return; + } + auto zookeeper = getZooKeeper(); /// Replica could be inactive. if (!zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")) { - LOG_WARNING(log, "Replica " << replica << " is not active during mutation query." - << name_for_logging << " will be done asynchronously when replica becomes active."); + LOG_WARNING(log, "Replica " << replica << " is not active during mutation. " + "Mutation will be done asynchronously when replica becomes active."); inactive_replicas.emplace(replica); break; } - String node_for_check = zookeeper_path + "/replicas/" + replica + "/" + replica_relative_node_path; - std::string node_for_check_value; - Coordination::Stat stat; + String mutation_pointer = zookeeper_path + "/replicas/" + replica + "/mutation_pointer"; + std::string mutation_pointer_value; + Coordination::Stat get_stat; /// Replica could be removed - if (!zookeeper->tryGet(node_for_check, node_for_check_value, &stat, wait_event)) + if (!zookeeper->tryGet(mutation_pointer, mutation_pointer_value, &get_stat, wait_event)) { LOG_WARNING(log, replica << " was removed"); - operation_is_processed_by_relica = true; break; } - else /// in other case check required node - { - if (callback(node_for_check_value)) - { - operation_is_processed_by_relica = true; - break; /// operation is done - } - } + else if (mutation_pointer_value >= mutation_id) /// Maybe we already processed more fresh mutation + break; /// (numbers like 0000000000 and 0000000001) - std::chrono::milliseconds time_spent = - std::chrono::duration_cast(std::chrono::system_clock::now() - operation_start); - std::chrono::milliseconds time_left = total_time - time_spent; - - /// We have some time to wait - if (time_left.count() > 0) - wait_event->tryWait(time_left.count()); - else /// Otherwise time is up - break; + /// We wait without timeout. + wait_event->wait(); } if (partial_shutdown_called) - throw Exception(name_for_logging + " is not finished because table shutdown was called. " + name_for_logging + " will be done after table restart.", + throw Exception("Mutation is not finished because table shutdown was called. It will be done after table restart.", ErrorCodes::UNFINISHED); - - if (!operation_is_processed_by_relica && !inactive_replicas.count(replica)) - timed_out_replicas.emplace(replica); } - if (!inactive_replicas.empty() || !timed_out_replicas.empty()) + if (!inactive_replicas.empty()) { std::stringstream exception_message; - exception_message << name_for_logging << " is not finished because"; + exception_message << "Mutation is not finished because"; if (!inactive_replicas.empty()) exception_message << " some replicas are inactive right now: " << boost::algorithm::join(inactive_replicas, ", "); - if (!timed_out_replicas.empty() && !inactive_replicas.empty()) - exception_message << " and"; - - if (!timed_out_replicas.empty()) - exception_message << " timeout when waiting for some replicas: " << boost::algorithm::join(timed_out_replicas, ", "); - - exception_message << ". " << name_for_logging << " will be done asynchronously"; + exception_message << ". Mutation will be done asynchronously"; throw Exception(exception_message.str(), ErrorCodes::UNFINISHED); } @@ -3382,7 +3368,7 @@ void StorageReplicatedMergeTree::alter( time_t replication_alter_columns_timeout = query_context.getSettingsRef().replication_alter_columns_timeout; - /// This code is quite similar with waitForAllReplicasToStatisfyNodeCondition + /// This code is quite similar with waitMutationToFinishOnReplicas /// but contains more complicated details (versions manipulations, multiple nodes, etc.). /// It will be removed soon in favor of alter-modify implementation on top of mutations. /// TODO (alesap) @@ -4603,17 +4589,21 @@ void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, const throw Coordination::Exception("Unable to create a mutation znode", rc); } - if (query_context.getSettingsRef().mutation_synchronous_wait_timeout != 0) /// some timeout specified + /// we have to wait + if (query_context.getSettingsRef().mutations_sync != 0) { auto check_callback = [mutation_number = entry.znode_name](const String & zk_value) { - /// Maybe we already processed more fresh mutation - /// We can compare their znode names (numbers like 0000000000 and 0000000001). return zk_value >= mutation_number; }; - waitForAllReplicasToStatisfyNodeCondition( - query_context.getSettingsRef().mutation_synchronous_wait_timeout, "Mutation", "mutation_pointer", check_callback); + Strings replicas; + if (query_context.getSettingsRef().mutations_sync == 2) /// wait for all replicas + replicas = getZooKeeper()->getChildren(zookeeper_path + "/replicas"); + else if (query_context.getSettingsRef().mutations_sync == 1) /// just wait for ourself + replicas.push_back(replica_path); + + waitMutationToFinishOnReplicas(replicas, entry.znode_name); } } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 55957439a1c..90b18a07eec 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -532,14 +532,9 @@ private: /// return true if it's fixed bool checkFixedGranualrityInZookeeper(); - using CheckNodeCallback = std::function; - - /// Wait for timeout seconds when condition became true for node - /// /replicas/{replica}/replica_replative_node_path value for all replicas. - /// operation_name_for_logging used for logging about errors. - void waitForAllReplicasToStatisfyNodeCondition( - size_t timeout, const String & operaton_name_for_logging, - const String & replica_relative_node_path, CheckNodeCallback condition) const; + /// Wait for timeout seconds mutation is finished on replicas + void waitMutationToFinishOnReplicas( + const Strings & replicas, const String & mutation_id) const; protected: /** If not 'attach', either creates a new table in ZK, or adds a replica to an existing table. diff --git a/dbms/tests/queries/0_stateless/00834_kill_mutation.sh b/dbms/tests/queries/0_stateless/00834_kill_mutation.sh index 03369dbff7a..726764c654b 100755 --- a/dbms/tests/queries/0_stateless/00834_kill_mutation.sh +++ b/dbms/tests/queries/0_stateless/00834_kill_mutation.sh @@ -14,27 +14,29 @@ ${CLICKHOUSE_CLIENT} --query="INSERT INTO test.kill_mutation VALUES ('2001-01-01 ${CLICKHOUSE_CLIENT} --query="SELECT '*** Create and kill a single invalid mutation ***'" -${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.kill_mutation DELETE WHERE toUInt32(s) = 1" +${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.kill_mutation DELETE WHERE toUInt32(s) = 1 SETTINGS mutations_sync = 1" & sleep 0.1 ${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, latest_failed_part IN ('20000101_1_1_0', '20010101_2_2_0'), latest_fail_time != 0, substr(latest_fail_reason, 1, 8) FROM system.mutations WHERE database = 'test' AND table = 'kill_mutation'" ${CLICKHOUSE_CLIENT} --query="KILL MUTATION WHERE database = 'test' AND table = 'kill_mutation'" +wait + ${CLICKHOUSE_CLIENT} --query="SELECT mutation_id FROM system.mutations WHERE database = 'test' AND table = 'kill_mutation'" ${CLICKHOUSE_CLIENT} --query="SELECT '*** Create and kill invalid mutation that blocks another mutation ***'" ${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.kill_mutation DELETE WHERE toUInt32(s) = 1" -${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.kill_mutation DELETE WHERE x = 1" +${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.kill_mutation DELETE WHERE x = 1 SETTINGS mutations_sync = 1" & ${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, latest_failed_part IN ('20000101_1_1_0', '20010101_2_2_0'), latest_fail_time != 0, substr(latest_fail_reason, 1, 8) FROM system.mutations WHERE database = 'test' AND table = 'kill_mutation' AND mutation_id = 'mutation_4.txt'" sleep 0.1 ${CLICKHOUSE_CLIENT} --query="KILL MUTATION WHERE database = 'test' AND table = 'kill_mutation' AND mutation_id = 'mutation_4.txt'" -wait_for_mutation "kill_mutation" "mutation_5.txt" "test" +wait ${CLICKHOUSE_CLIENT} --query="SELECT * FROM test.kill_mutation" diff --git a/dbms/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.reference b/dbms/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.reference index 3db1b92953c..9f1aa8e599d 100644 --- a/dbms/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.reference +++ b/dbms/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.reference @@ -1,6 +1,7 @@ *** Create and kill a single invalid mutation *** 0000000000 1 1 Code: 6, waiting test kill_mutation_r1 0000000000 +Mutation 0000000000 was killed *** Create and kill invalid mutation that blocks another mutation *** 0000000001 1 1 Code: 6, waiting test kill_mutation_r1 0000000001 diff --git a/dbms/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh b/dbms/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh index 570e4ea785b..3d0887c71b3 100755 --- a/dbms/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh +++ b/dbms/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh @@ -18,12 +18,24 @@ ${CLICKHOUSE_CLIENT} --query="INSERT INTO test.kill_mutation_r1 VALUES ('2001-01 ${CLICKHOUSE_CLIENT} --query="SELECT '*** Create and kill a single invalid mutation ***'" # wrong mutation -${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.kill_mutation_r1 DELETE WHERE toUInt32(s) = 1 SETTINGS mutation_synchronous_wait_timeout=2" 2>/dev/null +${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.kill_mutation_r1 DELETE WHERE toUInt32(s) = 1 SETTINGS mutations_sync=2" 2>&1 | grep -o "Mutation 0000000000 was killed" & -${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, latest_failed_part IN ('20000101_0_0_0', '20010101_0_0_0'), latest_fail_time != 0, substr(latest_fail_reason, 1, 8) FROM system.mutations WHERE database = 'test' AND table = 'kill_mutation_r1'" +check_query1="SELECT mutation_id, latest_failed_part IN ('20000101_0_0_0', '20010101_0_0_0'), latest_fail_time != 0, substr(latest_fail_reason, 1, 8) FROM system.mutations WHERE database = 'test' AND table = 'kill_mutation_r1'" + +query_result=`$CLICKHOUSE_CLIENT --query="$check_query1" 2>&1` + +while [ -z "$query_result" ] +do + query_result=`$CLICKHOUSE_CLIENT --query="$check_query1" 2>&1` + sleep 0.1 +done + +$CLICKHOUSE_CLIENT --query="$check_query1" ${CLICKHOUSE_CLIENT} --query="KILL MUTATION WHERE database = 'test' AND table = 'kill_mutation_r1'" +wait + ${CLICKHOUSE_CLIENT} --query="SELECT mutation_id FROM system.mutations WHERE database = 'test' AND table = 'kill_mutation_r1'" @@ -32,13 +44,24 @@ ${CLICKHOUSE_CLIENT} --query="SELECT '*** Create and kill invalid mutation that ${CLICKHOUSE_CLIENT} --query="SYSTEM SYNC REPLICA test.kill_mutation_r1" ${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.kill_mutation_r1 DELETE WHERE toUInt32(s) = 1" -${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.kill_mutation_r1 DELETE WHERE x = 1 SETTINGS mutation_synchronous_wait_timeout=2" 2>/dev/null +# good mutation, but blocked with wrong mutation +${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.kill_mutation_r1 DELETE WHERE x = 1 SETTINGS mutations_sync=2" & -${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, latest_failed_part IN ('20000101_0_0_0_1', '20010101_0_0_0_1'), latest_fail_time != 0, substr(latest_fail_reason, 1, 8) FROM system.mutations WHERE database = 'test' AND table = 'kill_mutation_r1' AND mutation_id = '0000000001'" +check_query2="SELECT mutation_id, latest_failed_part IN ('20000101_0_0_0_1', '20010101_0_0_0_1'), latest_fail_time != 0, substr(latest_fail_reason, 1, 8) FROM system.mutations WHERE database = 'test' AND table = 'kill_mutation_r1' AND mutation_id = '0000000001'" + +query_result=`$CLICKHOUSE_CLIENT --query="$check_query2" 2>&1` + +while [ -z "$query_result" ] +do + query_result=`$CLICKHOUSE_CLIENT --query="$check_query2" 2>&1` + sleep 0.1 +done + +$CLICKHOUSE_CLIENT --query="$check_query2" ${CLICKHOUSE_CLIENT} --query="KILL MUTATION WHERE database = 'test' AND table = 'kill_mutation_r1' AND mutation_id = '0000000001'" -wait_for_mutation "kill_mutation_r2" "0000000002" "test" +wait ${CLICKHOUSE_CLIENT} --query="SELECT * FROM test.kill_mutation_r2" diff --git a/dbms/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.reference b/dbms/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.reference index d65c89faf0e..1b9855f14a9 100644 --- a/dbms/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.reference +++ b/dbms/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.reference @@ -2,9 +2,7 @@ Replicated 1 1 1 -1 Normal 1 1 1 -1 diff --git a/dbms/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.sql b/dbms/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.sql index 3b4bb145f1a..269b568b5f6 100644 --- a/dbms/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.sql +++ b/dbms/tests/queries/0_stateless/01049_zookeeper_synchronous_mutations.sql @@ -11,14 +11,12 @@ INSERT INTO table_for_synchronous_mutations1 select number, number from numbers( SYSTEM SYNC REPLICA table_for_synchronous_mutations2; -ALTER TABLE table_for_synchronous_mutations1 UPDATE v1 = v1 + 1 WHERE 1 SETTINGS mutation_synchronous_wait_timeout = 10; +ALTER TABLE table_for_synchronous_mutations1 UPDATE v1 = v1 + 1 WHERE 1 SETTINGS mutations_sync = 2; SELECT is_done FROM system.mutations where table = 'table_for_synchronous_mutations1'; -ALTER TABLE table_for_synchronous_mutations1 UPDATE v1 = 1 WHERE ignore(sleep(3)) SETTINGS mutation_synchronous_wait_timeout = 2; --{serverError 341} - -- Another mutation, just to be sure, that previous finished -ALTER TABLE table_for_synchronous_mutations1 UPDATE v1 = v1 + 1 WHERE 1 SETTINGS mutation_synchronous_wait_timeout = 15; +ALTER TABLE table_for_synchronous_mutations1 UPDATE v1 = v1 + 1 WHERE 1 SETTINGS mutations_sync = 2; SELECT is_done FROM system.mutations where table = 'table_for_synchronous_mutations1'; @@ -33,16 +31,13 @@ CREATE TABLE table_for_synchronous_mutations_no_replication(k UInt32, v1 UInt64) INSERT INTO table_for_synchronous_mutations_no_replication select number, number from numbers(100000); -ALTER TABLE table_for_synchronous_mutations_no_replication UPDATE v1 = v1 + 1 WHERE 1 SETTINGS mutation_synchronous_wait_timeout = 10; +ALTER TABLE table_for_synchronous_mutations_no_replication UPDATE v1 = v1 + 1 WHERE 1 SETTINGS mutations_sync = 2; SELECT is_done FROM system.mutations where table = 'table_for_synchronous_mutations_no_replication'; -ALTER TABLE table_for_synchronous_mutations_no_replication UPDATE v1 = 1 WHERE ignore(sleep(3)) SETTINGS mutation_synchronous_wait_timeout = 2; --{serverError 341} - -- Another mutation, just to be sure, that previous finished -ALTER TABLE table_for_synchronous_mutations_no_replication UPDATE v1 = v1 + 1 WHERE 1 SETTINGS mutation_synchronous_wait_timeout = 15; +ALTER TABLE table_for_synchronous_mutations_no_replication UPDATE v1 = v1 + 1 WHERE 1 SETTINGS mutations_sync = 2; SELECT is_done FROM system.mutations where table = 'table_for_synchronous_mutations_no_replication'; - DROP TABLE IF EXISTS table_for_synchronous_mutations_no_replication; From 0cc7ccfc2ed1f57ae69f14fc47244a3fffcb41a7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 19 Dec 2019 18:30:52 +0300 Subject: [PATCH 06/53] Remove unintended diff --- dbms/src/Storages/MergeTree/MergeTreeMutationEntry.h | 1 - dbms/src/Storages/StorageMergeTree.cpp | 1 - 2 files changed, 2 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeMutationEntry.h b/dbms/src/Storages/MergeTree/MergeTreeMutationEntry.h index f5c512617d3..2b3bde72552 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeMutationEntry.h +++ b/dbms/src/Storages/MergeTree/MergeTreeMutationEntry.h @@ -25,7 +25,6 @@ struct MergeTreeMutationEntry MergeTreePartInfo latest_failed_part_info; time_t latest_fail_time = 0; String latest_fail_reason; - int latest_fail_error_code = 0; /// Create a new entry and write it to a temporary file. MergeTreeMutationEntry(MutationCommands commands_, const String & path_prefix_, Int64 tmp_number); diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 1ae67e8a5a8..a4ce04b7356 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -42,7 +42,6 @@ namespace ErrorCodes extern const int PART_IS_TEMPORARILY_LOCKED; extern const int UNKNOWN_SETTING; extern const int TOO_BIG_AST; - extern const int UNFINISHED; } namespace ActionLocks From ee851113b81b6485c85e8b2b37e31f1be00c73ba Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 20 Dec 2019 12:18:48 +0300 Subject: [PATCH 07/53] Event better test --- .../00834_kill_mutation_replicated_zookeeper.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh b/dbms/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh index 3d0887c71b3..19c3fc4eed3 100755 --- a/dbms/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh +++ b/dbms/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh @@ -20,7 +20,7 @@ ${CLICKHOUSE_CLIENT} --query="SELECT '*** Create and kill a single invalid mutat # wrong mutation ${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.kill_mutation_r1 DELETE WHERE toUInt32(s) = 1 SETTINGS mutations_sync=2" 2>&1 | grep -o "Mutation 0000000000 was killed" & -check_query1="SELECT mutation_id, latest_failed_part IN ('20000101_0_0_0', '20010101_0_0_0'), latest_fail_time != 0, substr(latest_fail_reason, 1, 8) FROM system.mutations WHERE database = 'test' AND table = 'kill_mutation_r1'" +check_query1="SELECT substr(latest_fail_reason, 1, 8) as ErrorCode FROM system.mutations WHERE database = 'test' AND table = 'kill_mutation_r1' AND ErrorCode != ''" query_result=`$CLICKHOUSE_CLIENT --query="$check_query1" 2>&1` @@ -30,7 +30,7 @@ do sleep 0.1 done -$CLICKHOUSE_CLIENT --query="$check_query1" +$CLICKHOUSE_CLIENT --query="SELECT mutation_id, latest_failed_part IN ('20000101_0_0_0', '20010101_0_0_0'), latest_fail_time != 0, substr(latest_fail_reason, 1, 8) FROM system.mutations WHERE database = 'test' AND table = 'kill_mutation_r1'" ${CLICKHOUSE_CLIENT} --query="KILL MUTATION WHERE database = 'test' AND table = 'kill_mutation_r1'" @@ -47,7 +47,7 @@ ${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.kill_mutation_r1 DELETE WHERE toU # good mutation, but blocked with wrong mutation ${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.kill_mutation_r1 DELETE WHERE x = 1 SETTINGS mutations_sync=2" & -check_query2="SELECT mutation_id, latest_failed_part IN ('20000101_0_0_0_1', '20010101_0_0_0_1'), latest_fail_time != 0, substr(latest_fail_reason, 1, 8) FROM system.mutations WHERE database = 'test' AND table = 'kill_mutation_r1' AND mutation_id = '0000000001'" +check_query2="SELECT substr(latest_fail_reason, 1, 8) as ErrorCode FROM system.mutations WHERE database = 'test' AND table = 'kill_mutation_r1' AND mutation_id = '0000000001' AND ErrorCode != ''" query_result=`$CLICKHOUSE_CLIENT --query="$check_query2" 2>&1` @@ -57,7 +57,7 @@ do sleep 0.1 done -$CLICKHOUSE_CLIENT --query="$check_query2" +$CLICKHOUSE_CLIENT --query="SELECT mutation_id, latest_failed_part IN ('20000101_0_0_0_1', '20010101_0_0_0_1'), latest_fail_time != 0, substr(latest_fail_reason, 1, 8) FROM system.mutations WHERE database = 'test' AND table = 'kill_mutation_r1' AND mutation_id = '0000000001'" ${CLICKHOUSE_CLIENT} --query="KILL MUTATION WHERE database = 'test' AND table = 'kill_mutation_r1' AND mutation_id = '0000000001'" From 10ad22faea898ec17c3785b7c2dcba997fc7af01 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 16 Dec 2019 21:10:35 +0300 Subject: [PATCH 08/53] refactor InterpreterCreateQuery::createTable(...) --- dbms/src/Databases/DatabaseOrdinary.cpp | 12 +- .../Interpreters/InterpreterCreateQuery.cpp | 273 +++++++++--------- .../src/Interpreters/InterpreterCreateQuery.h | 17 +- dbms/src/Parsers/IAST.h | 9 + dbms/src/Storages/StorageFactory.cpp | 2 +- dbms/src/Storages/StorageFactory.h | 2 +- 6 files changed, 157 insertions(+), 158 deletions(-) diff --git a/dbms/src/Databases/DatabaseOrdinary.cpp b/dbms/src/Databases/DatabaseOrdinary.cpp index b4a900399a9..a03d0a62f5a 100644 --- a/dbms/src/Databases/DatabaseOrdinary.cpp +++ b/dbms/src/Databases/DatabaseOrdinary.cpp @@ -344,16 +344,8 @@ void DatabaseOrdinary::alterTable( ASTPtr new_constraints = InterpreterCreateQuery::formatConstraints(constraints); ast_create_query.columns_list->replace(ast_create_query.columns_list->columns, new_columns); - - if (ast_create_query.columns_list->indices) - ast_create_query.columns_list->replace(ast_create_query.columns_list->indices, new_indices); - else - ast_create_query.columns_list->set(ast_create_query.columns_list->indices, new_indices); - - if (ast_create_query.columns_list->constraints) - ast_create_query.columns_list->replace(ast_create_query.columns_list->constraints, new_constraints); - else - ast_create_query.columns_list->set(ast_create_query.columns_list->constraints, new_constraints); + ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->indices, new_indices); + ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->constraints, new_constraints); if (storage_modifier) storage_modifier(*ast_create_query.storage); diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 15fd8dfd3f9..a6f56546005 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -21,7 +21,6 @@ #include #include -#include #include #include @@ -36,7 +35,6 @@ #include #include #include -#include #include #include @@ -396,84 +394,97 @@ ConstraintsDescription InterpreterCreateQuery::getConstraintsDescription(const A } -ColumnsDescription InterpreterCreateQuery::setProperties( - ASTCreateQuery & create, const Block & as_select_sample, const StoragePtr & as_storage) const +InterpreterCreateQuery::TableProperties InterpreterCreateQuery::setProperties(ASTCreateQuery & create) const { - ColumnsDescription columns; - IndicesDescription indices; - ConstraintsDescription constraints; + TableProperties properties; + TableStructureReadLockHolder as_storage_lock; if (create.columns_list) { if (create.columns_list->columns) - columns = getColumnsDescription(*create.columns_list->columns, context); + properties.columns = getColumnsDescription(*create.columns_list->columns, context); if (create.columns_list->indices) for (const auto & index : create.columns_list->indices->children) - indices.indices.push_back( + properties.indices.indices.push_back( std::dynamic_pointer_cast(index->clone())); - if (create.columns_list->constraints) - for (const auto & constraint : create.columns_list->constraints->children) - constraints.constraints.push_back( - std::dynamic_pointer_cast(constraint->clone())); + properties.constraints = getConstraintsDescription(create.columns_list->constraints); } else if (!create.as_table.empty()) { - columns = as_storage->getColumns(); + String as_database_name = create.as_database.empty() ? context.getCurrentDatabase() : create.as_database; + StoragePtr as_storage = context.getTable(as_database_name, create.as_table); + + /// as_storage->getColumns() and setEngine(...) must be called under structure lock of other_table for CREATE ... AS other_table. + as_storage_lock = as_storage->lockStructureForShare(false, context.getCurrentQueryId()); + properties.columns = as_storage->getColumns(); /// Secondary indices make sense only for MergeTree family of storage engines. /// We should not copy them for other storages. if (create.storage && endsWith(create.storage->engine->name, "MergeTree")) - indices = as_storage->getIndices(); + properties.indices = as_storage->getIndices(); - constraints = as_storage->getConstraints(); + properties.constraints = as_storage->getConstraints(); } else if (create.select) { - columns = ColumnsDescription(as_select_sample.getNamesAndTypesList()); + Block as_select_sample = InterpreterSelectWithUnionQuery::getSampleBlock(create.select->clone(), context); + properties.columns = ColumnsDescription(as_select_sample.getNamesAndTypesList()); } + else if (create.as_table_function) + return {}; else throw Exception("Incorrect CREATE query: required list of column descriptions or AS section or SELECT.", ErrorCodes::INCORRECT_QUERY); + /// Even if query has list of columns, canonicalize it (unfold Nested columns). - ASTPtr new_columns = formatColumns(columns); - ASTPtr new_indices = formatIndices(indices); - ASTPtr new_constraints = formatConstraints(constraints); - if (!create.columns_list) - { - auto new_columns_list = std::make_shared(); - create.set(create.columns_list, new_columns_list); - } + create.set(create.columns_list, std::make_shared()); - if (create.columns_list->columns) - create.columns_list->replace(create.columns_list->columns, new_columns); - else - create.columns_list->set(create.columns_list->columns, new_columns); + ASTPtr new_columns = formatColumns(properties.columns); + ASTPtr new_indices = formatIndices(properties.indices); + ASTPtr new_constraints = formatConstraints(properties.constraints); - if (new_indices && create.columns_list->indices) - create.columns_list->replace(create.columns_list->indices, new_indices); - else if (new_indices) - create.columns_list->set(create.columns_list->indices, new_indices); + create.columns_list->setOrReplace(create.columns_list->columns, new_columns); + create.columns_list->setOrReplace(create.columns_list->indices, new_indices); + create.columns_list->setOrReplace(create.columns_list->constraints, new_constraints); - if (new_constraints && create.columns_list->constraints) - create.columns_list->replace(create.columns_list->constraints, new_constraints); - else if (new_constraints) - create.columns_list->set(create.columns_list->constraints, new_constraints); + validateTableStructure(create, properties); + /// Set the table engine if it was not specified explicitly. + setEngine(create); + return properties; +} + +void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & create, + const InterpreterCreateQuery::TableProperties & properties) const +{ /// Check for duplicates std::set all_columns; - for (const auto & column : columns) + for (const auto & column : properties.columns) { if (!all_columns.emplace(column.name).second) throw Exception("Column " + backQuoteIfNeed(column.name) + " already exists", ErrorCodes::DUPLICATE_COLUMN); } - return columns; + /// Check low cardinality types in creating table if it was not allowed in setting + if (!create.attach && !context.getSettingsRef().allow_suspicious_low_cardinality_types && !create.is_materialized_view) + { + for (const auto & name_and_type_pair : properties.columns.getAllPhysical()) + { + if (const auto * current_type_ptr = typeid_cast(name_and_type_pair.type.get())) + { + if (!isStringOrFixedString(*removeNullable(current_type_ptr->getDictionaryType()))) + throw Exception("Creating columns of type " + current_type_ptr->getName() + " is prohibited by default " + "due to expected negative impact on performance. " + "It can be enabled with the \"allow_suspicious_low_cardinality_types\" setting.", + ErrorCodes::SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY); + } + } + } } - void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const { if (create.storage) @@ -535,12 +546,10 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) throw Exception("Temporary tables cannot be inside a database. You should not specify a database for a temporary table.", ErrorCodes::BAD_DATABASE_FOR_TEMPORARY_TABLE); - String path = context.getPath(); String current_database = context.getCurrentDatabase(); String database_name = create.database.empty() ? current_database : create.database; String table_name = create.table; - String table_name_escaped = escapeForFileName(table_name); // If this is a stub ATTACH query, read the query definition from the database if (create.attach && !create.storage && !create.columns_list) @@ -560,26 +569,63 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) visitor.visit(*create.select); } - Block as_select_sample; - if (create.select && (!create.attach || !create.columns_list)) - as_select_sample = InterpreterSelectWithUnionQuery::getSampleBlock(create.select->clone(), context); + /// Set and retrieve list of columns, indices and constraints. Set table engine if needed. Rewrite query in canonical way. + TableProperties properties = setProperties(create); - String as_database_name = create.as_database.empty() ? current_database : create.as_database; - String as_table_name = create.as_table; + /// Actually creates table + bool created = doCreateTable(create, properties, database_name); + if (!created) + return {}; - StoragePtr as_storage; - TableStructureReadLockHolder as_storage_lock; + return fillTableIfNeeded(create, database_name); +} - if (!as_table_name.empty()) +bool InterpreterCreateQuery::doCreateTable(const ASTCreateQuery & create, + const InterpreterCreateQuery::TableProperties & properties, + const String & database_name) +{ + std::unique_ptr guard; + + String data_path; + DatabasePtr database; + + const String & table_name = create.table; + bool need_add_to_database = !create.temporary || create.is_live_view; + if (need_add_to_database) { - as_storage = context.getTable(as_database_name, as_table_name); - as_storage_lock = as_storage->lockStructureForShare(false, context.getCurrentQueryId()); + database = context.getDatabase(database_name); + data_path = database->getDataPath(); + + /** If the request specifies IF NOT EXISTS, we allow concurrent CREATE queries (which do nothing). + * If table doesnt exist, one thread is creating table, while others wait in DDLGuard. + */ + guard = context.getDDLGuard(database_name, table_name); + + /// Table can be created before or it can be created concurrently in another thread, while we were waiting in DDLGuard. + if (database->isTableExist(context, table_name)) + { + /// TODO Check structure of table + if (create.if_not_exists) + return false; + else if (create.replace_view) + { + /// when executing CREATE OR REPLACE VIEW, drop current existing view + auto drop_ast = std::make_shared(); + drop_ast->database = database_name; + drop_ast->table = table_name; + drop_ast->no_ddl_lock = true; + + InterpreterDropQuery interpreter(drop_ast, context); + interpreter.execute(); + } + else + throw Exception("Table " + database_name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); + } } + else if (context.tryGetExternalTable(table_name) && create.if_not_exists) + return false; - ColumnsDescription columns; - ConstraintsDescription constraints; StoragePtr res; - if (create.as_table_function) { const auto & table_function = create.as_table_function->as(); @@ -588,99 +634,38 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) } else { - /// Set and retrieve list of columns. - columns = setProperties(create, as_select_sample, as_storage); - constraints = getConstraintsDescription(create.columns_list->constraints); - - /// Check low cardinality types in creating table if it was not allowed in setting - if (!create.attach && !context.getSettingsRef().allow_suspicious_low_cardinality_types && !create.is_materialized_view) - { - for (const auto & name_and_type_pair : columns.getAllPhysical()) - { - if (const auto * current_type_ptr = typeid_cast(name_and_type_pair.type.get())) - { - if (!isStringOrFixedString(*removeNullable(current_type_ptr->getDictionaryType()))) - throw Exception("Creating columns of type " + current_type_ptr->getName() + " is prohibited by default due to expected negative impact on performance. It can be enabled with the \"allow_suspicious_low_cardinality_types\" setting.", - ErrorCodes::SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY); - } - } - } - - /// Set the table engine if it was not specified explicitly. - setEngine(create); + res = StorageFactory::instance().get(create, + data_path, + table_name, + database_name, + context, + context.getGlobalContext(), + properties.columns, + properties.constraints, + create.attach, + false); } - { - std::unique_ptr guard; + if (need_add_to_database) + database->createTable(context, table_name, res, query_ptr); + else + context.getSessionContext().addExternalTable(table_name, res, query_ptr); - String data_path; - DatabasePtr database; + /// We must call "startup" and "shutdown" while holding DDLGuard. + /// Because otherwise method "shutdown" (from InterpreterDropQuery) can be called before startup + /// (in case when table was created and instantly dropped before started up) + /// + /// Method "startup" may create background tasks and method "shutdown" will wait for them. + /// But if "shutdown" is called before "startup", it will exit early, because there are no background tasks to wait. + /// Then background task is created by "startup" method. And when destructor of a table object is called, background task is still active, + /// and the task will use references to freed data. - if (!create.temporary || create.is_live_view) - { - database = context.getDatabase(database_name); - data_path = database->getDataPath(); - - /** If the request specifies IF NOT EXISTS, we allow concurrent CREATE queries (which do nothing). - * If table doesnt exist, one thread is creating table, while others wait in DDLGuard. - */ - guard = context.getDDLGuard(database_name, table_name); - - /// Table can be created before or it can be created concurrently in another thread, while we were waiting in DDLGuard. - if (database->isTableExist(context, table_name)) - { - /// TODO Check structure of table - if (create.if_not_exists) - return {}; - else if (create.replace_view) - { - /// when executing CREATE OR REPLACE VIEW, drop current existing view - auto drop_ast = std::make_shared(); - drop_ast->database = database_name; - drop_ast->table = table_name; - drop_ast->no_ddl_lock = true; - - InterpreterDropQuery interpreter(drop_ast, context); - interpreter.execute(); - } - else - throw Exception("Table " + database_name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); - } - } - else if (context.tryGetExternalTable(table_name) && create.if_not_exists) - return {}; - - if (!create.as_table_function) - { - res = StorageFactory::instance().get(create, - data_path, - table_name, - database_name, - context, - context.getGlobalContext(), - columns, - constraints, - create.attach, - false); - } - - if (create.temporary && !create.is_live_view) - context.getSessionContext().addExternalTable(table_name, res, query_ptr); - else - database->createTable(context, table_name, res, query_ptr); - - /// We must call "startup" and "shutdown" while holding DDLGuard. - /// Because otherwise method "shutdown" (from InterpreterDropQuery) can be called before startup - /// (in case when table was created and instantly dropped before started up) - /// - /// Method "startup" may create background tasks and method "shutdown" will wait for them. - /// But if "shutdown" is called before "startup", it will exit early, because there are no background tasks to wait. - /// Then background task is created by "startup" method. And when destructor of a table object is called, background task is still active, - /// and the task will use references to freed data. - - res->startup(); - } + res->startup(); + return true; +} +BlockIO InterpreterCreateQuery::fillTableIfNeeded(const ASTCreateQuery & create, const String & database_name) +{ /// If the query is a CREATE SELECT, insert the data into the table. if (create.select && !create.attach && !create.is_view && !create.is_live_view && (!create.is_materialized_view || create.is_populate)) @@ -690,7 +675,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) if (!create.temporary) insert->database = database_name; - insert->table = table_name; + insert->table = create.table; insert->select = create.select->clone(); if (create.temporary && !context.getSessionContext().hasQueryContext()) diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.h b/dbms/src/Interpreters/InterpreterCreateQuery.h index fe32f44b84e..fa0ea4380b9 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.h +++ b/dbms/src/Interpreters/InterpreterCreateQuery.h @@ -49,15 +49,28 @@ public: static ConstraintsDescription getConstraintsDescription(const ASTExpressionList * constraints); private: + struct TableProperties + { + ColumnsDescription columns; + IndicesDescription indices; + ConstraintsDescription constraints; + }; + BlockIO createDatabase(ASTCreateQuery & create); BlockIO createTable(ASTCreateQuery & create); BlockIO createDictionary(ASTCreateQuery & create); - /// Calculate list of columns, constraints, indices, etc... of table and return columns. - ColumnsDescription setProperties(ASTCreateQuery & create, const Block & as_select_sample, const StoragePtr & as_storage) const; + /// Calculate list of columns, constraints, indices, etc... of table. Rewrite query in canonical way. + TableProperties setProperties(ASTCreateQuery & create) const; + void validateTableStructure(const ASTCreateQuery & create, const TableProperties & properties) const; void setEngine(ASTCreateQuery & create) const; void checkAccess(const ASTCreateQuery & create); + /// Create IStorage and add it to database. If table already exists and IF NOT EXISTS specified, do nothing and return false. + bool doCreateTable(const ASTCreateQuery & create, const TableProperties & properties, const String & database_name); + /// Inserts data in created table if it's CREATE ... SELECT + BlockIO fillTableIfNeeded(const ASTCreateQuery & create, const String & database_name); + ASTPtr query_ptr; Context & context; diff --git a/dbms/src/Parsers/IAST.h b/dbms/src/Parsers/IAST.h index d7c56d80a21..246b88f8c24 100644 --- a/dbms/src/Parsers/IAST.h +++ b/dbms/src/Parsers/IAST.h @@ -146,6 +146,15 @@ public: throw Exception("AST subtree not found in children", ErrorCodes::LOGICAL_ERROR); } + template + void setOrReplace(T * & field, const ASTPtr & child) + { + if (field) + replace(field, child); + else + set(field, child); + } + /// Convert to a string. /// Format settings. diff --git a/dbms/src/Storages/StorageFactory.cpp b/dbms/src/Storages/StorageFactory.cpp index d3796735a07..6e0e630efea 100644 --- a/dbms/src/Storages/StorageFactory.cpp +++ b/dbms/src/Storages/StorageFactory.cpp @@ -39,7 +39,7 @@ void StorageFactory::registerStorage(const std::string & name, Creator creator) StoragePtr StorageFactory::get( - ASTCreateQuery & query, + const ASTCreateQuery & query, const String & data_path, const String & table_name, const String & database_name, diff --git a/dbms/src/Storages/StorageFactory.h b/dbms/src/Storages/StorageFactory.h index 013ce4e82e2..00b756b1115 100644 --- a/dbms/src/Storages/StorageFactory.h +++ b/dbms/src/Storages/StorageFactory.h @@ -47,7 +47,7 @@ public: using Creator = std::function; StoragePtr get( - ASTCreateQuery & query, + const ASTCreateQuery & query, const String & data_path, const String & table_name, const String & database_name, From 9041977bb62d68a833a84cae0f1f4e919c0dba69 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 16 Dec 2019 21:29:18 +0300 Subject: [PATCH 09/53] InterpreterCreateQuery code cleanup --- .../Interpreters/InterpreterCreateQuery.cpp | 65 +++++++++---------- .../src/Interpreters/InterpreterCreateQuery.h | 4 +- docker/test/performance/run.sh | 2 +- docker/test/stateful/Dockerfile | 2 +- docker/test/stateful_with_coverage/run.sh | 2 +- docker/test/stress/Dockerfile | 2 +- 6 files changed, 37 insertions(+), 40 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index a6f56546005..d5bbe74e98a 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -95,29 +95,20 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) throw Exception("Database " + database_name + " already exists.", ErrorCodes::DATABASE_ALREADY_EXISTS); } - String database_engine_name; if (!create.storage) { - database_engine_name = "Ordinary"; /// Default database engine. auto engine = std::make_shared(); - engine->name = database_engine_name; auto storage = std::make_shared(); + engine->name = "Ordinary"; storage->set(storage->engine, engine); create.set(create.storage, storage); } - else + else if ((create.columns_list && create.columns_list->indices && !create.columns_list->indices->children.empty())) { - const ASTStorage & storage = *create.storage; - const ASTFunction & engine = *storage.engine; /// Currently, there are no database engines, that support any arguments. - if ((create.columns_list && create.columns_list->indices && !create.columns_list->indices->children.empty())) - { - std::stringstream ostr; - formatAST(storage, ostr, false, false); - throw Exception("Unknown database engine: " + ostr.str(), ErrorCodes::UNKNOWN_DATABASE_ENGINE); - } - - database_engine_name = engine.name; + std::stringstream ostr; + formatAST(*create.storage, ostr, false, false); + throw Exception("Unknown database engine: " + ostr.str(), ErrorCodes::UNKNOWN_DATABASE_ENGINE); } String database_name_escaped = escapeForFileName(database_name); @@ -153,19 +144,27 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) out.close(); } + bool added = false; + bool renamed = false; try { context.addDatabase(database_name, database); + added = true; if (need_write_metadata) + { Poco::File(metadata_file_tmp_path).renameTo(metadata_file_path); + renamed = true; + } database->loadStoredObjects(context, has_force_restore_data_flag); } catch (...) { - if (need_write_metadata) + if (renamed) Poco::File(metadata_file_tmp_path).remove(); + if (added) + context.detachDatabase(database_name); throw; } @@ -546,21 +545,19 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) throw Exception("Temporary tables cannot be inside a database. You should not specify a database for a temporary table.", ErrorCodes::BAD_DATABASE_FOR_TEMPORARY_TABLE); - String current_database = context.getCurrentDatabase(); - - String database_name = create.database.empty() ? current_database : create.database; - String table_name = create.table; - // If this is a stub ATTACH query, read the query definition from the database if (create.attach && !create.storage && !create.columns_list) { // Table SQL definition is available even if the table is detached - auto query = context.getCreateTableQuery(database_name, table_name); + auto query = context.getCreateTableQuery(create.database, create.table); create = query->as(); // Copy the saved create query, but use ATTACH instead of CREATE create.attach = true; } - if (create.to_database.empty()) + String current_database = context.getCurrentDatabase(); + if (!create.temporary && create.database.empty()) + create.database = current_database; + if (!create.to_table.empty() && create.to_database.empty()) create.to_database = current_database; if (create.select && (create.is_view || create.is_materialized_view || create.is_live_view)) @@ -573,16 +570,16 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) TableProperties properties = setProperties(create); /// Actually creates table - bool created = doCreateTable(create, properties, database_name); - if (!created) + bool created = doCreateTable(create, properties); + + if (!created) /// Table already exists return {}; - return fillTableIfNeeded(create, database_name); + return fillTableIfNeeded(create); } bool InterpreterCreateQuery::doCreateTable(const ASTCreateQuery & create, - const InterpreterCreateQuery::TableProperties & properties, - const String & database_name) + const InterpreterCreateQuery::TableProperties & properties) { std::unique_ptr guard; @@ -593,13 +590,13 @@ bool InterpreterCreateQuery::doCreateTable(const ASTCreateQuery & create, bool need_add_to_database = !create.temporary || create.is_live_view; if (need_add_to_database) { - database = context.getDatabase(database_name); + database = context.getDatabase(create.database); data_path = database->getDataPath(); /** If the request specifies IF NOT EXISTS, we allow concurrent CREATE queries (which do nothing). * If table doesnt exist, one thread is creating table, while others wait in DDLGuard. */ - guard = context.getDDLGuard(database_name, table_name); + guard = context.getDDLGuard(create.database, table_name); /// Table can be created before or it can be created concurrently in another thread, while we were waiting in DDLGuard. if (database->isTableExist(context, table_name)) @@ -611,7 +608,7 @@ bool InterpreterCreateQuery::doCreateTable(const ASTCreateQuery & create, { /// when executing CREATE OR REPLACE VIEW, drop current existing view auto drop_ast = std::make_shared(); - drop_ast->database = database_name; + drop_ast->database = create.database; drop_ast->table = table_name; drop_ast->no_ddl_lock = true; @@ -619,7 +616,7 @@ bool InterpreterCreateQuery::doCreateTable(const ASTCreateQuery & create, interpreter.execute(); } else - throw Exception("Table " + database_name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); + throw Exception("Table " + create.database + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); } } else if (context.tryGetExternalTable(table_name) && create.if_not_exists) @@ -637,7 +634,7 @@ bool InterpreterCreateQuery::doCreateTable(const ASTCreateQuery & create, res = StorageFactory::instance().get(create, data_path, table_name, - database_name, + create.database, context, context.getGlobalContext(), properties.columns, @@ -664,7 +661,7 @@ bool InterpreterCreateQuery::doCreateTable(const ASTCreateQuery & create, return true; } -BlockIO InterpreterCreateQuery::fillTableIfNeeded(const ASTCreateQuery & create, const String & database_name) +BlockIO InterpreterCreateQuery::fillTableIfNeeded(const ASTCreateQuery & create) { /// If the query is a CREATE SELECT, insert the data into the table. if (create.select && !create.attach @@ -673,7 +670,7 @@ BlockIO InterpreterCreateQuery::fillTableIfNeeded(const ASTCreateQuery & create, auto insert = std::make_shared(); if (!create.temporary) - insert->database = database_name; + insert->database = create.database; insert->table = create.table; insert->select = create.select->clone(); diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.h b/dbms/src/Interpreters/InterpreterCreateQuery.h index fa0ea4380b9..55b857554b9 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.h +++ b/dbms/src/Interpreters/InterpreterCreateQuery.h @@ -67,9 +67,9 @@ private: void checkAccess(const ASTCreateQuery & create); /// Create IStorage and add it to database. If table already exists and IF NOT EXISTS specified, do nothing and return false. - bool doCreateTable(const ASTCreateQuery & create, const TableProperties & properties, const String & database_name); + bool doCreateTable(const ASTCreateQuery & create, const TableProperties & properties); /// Inserts data in created table if it's CREATE ... SELECT - BlockIO fillTableIfNeeded(const ASTCreateQuery & create, const String & database_name); + BlockIO fillTableIfNeeded(const ASTCreateQuery & create); ASTPtr query_ptr; Context & context; diff --git a/docker/test/performance/run.sh b/docker/test/performance/run.sh index 2e149264ca0..54b80aab5b1 100755 --- a/docker/test/performance/run.sh +++ b/docker/test/performance/run.sh @@ -11,7 +11,7 @@ install_packages() { } download_data() { - clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets" + clickhouse-client --query "ATTACH DATABASE IF NOT EXISTS datasets ENGINE = Ordinary" clickhouse-client --query "CREATE DATABASE IF NOT EXISTS test" /s3downloader --dataset-names $OPEN_DATASETS /s3downloader --dataset-names $PRIVATE_DATASETS --url 'https://s3.mds.yandex.net/clickhouse-private-datasets' diff --git a/docker/test/stateful/Dockerfile b/docker/test/stateful/Dockerfile index ba9d86c5ad4..59e3c037265 100644 --- a/docker/test/stateful/Dockerfile +++ b/docker/test/stateful/Dockerfile @@ -39,7 +39,7 @@ CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \ && /s3downloader --dataset-names $DATASETS \ && chmod 777 -R /var/lib/clickhouse \ && clickhouse-client --query "SHOW DATABASES" \ - && clickhouse-client --query "CREATE DATABASE datasets" \ + && clickhouse-client --query "ATTACH DATABASE datasets ENGINE = Ordinary" \ && clickhouse-client --query "CREATE DATABASE test" \ && service clickhouse-server restart && sleep 5 \ && clickhouse-client --query "SHOW TABLES FROM datasets" \ diff --git a/docker/test/stateful_with_coverage/run.sh b/docker/test/stateful_with_coverage/run.sh index f91b51b033e..0c7e3e50cb9 100755 --- a/docker/test/stateful_with_coverage/run.sh +++ b/docker/test/stateful_with_coverage/run.sh @@ -81,7 +81,7 @@ while /bin/true; do done & LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-client --query "SHOW DATABASES" -LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-client --query "CREATE DATABASE datasets" +LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-client --query "ATTACH DATABASE datasets ENGINE = Ordinary" LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-client --query "CREATE DATABASE test" kill_clickhouse diff --git a/docker/test/stress/Dockerfile b/docker/test/stress/Dockerfile index b0b94ccc579..116f4ec03f2 100644 --- a/docker/test/stress/Dockerfile +++ b/docker/test/stress/Dockerfile @@ -39,7 +39,7 @@ CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \ service clickhouse-server start && sleep 5 \ && /s3downloader --dataset-names $DATASETS \ && chmod 777 -R /var/lib/clickhouse \ - && clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets" \ + && clickhouse-client --query "ATTACH DATABASE IF NOT EXISTS datasets ENGINE = Ordinary" \ && clickhouse-client --query "CREATE DATABASE IF NOT EXISTS test" \ && service clickhouse-server restart && sleep 5 \ && clickhouse-client --query "SHOW TABLES FROM datasets" \ From 2e4174a54cd1e0e3b1736348bbc0a0220c3494cb Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 22 Dec 2019 22:27:06 +0700 Subject: [PATCH 10/53] Fix checks if a client host is allowed in case it's the localhost. --- dbms/src/Access/AllowedClientHosts.cpp | 132 ++++++++++++++----------- dbms/src/Access/AllowedClientHosts.h | 2 +- 2 files changed, 78 insertions(+), 56 deletions(-) diff --git a/dbms/src/Access/AllowedClientHosts.cpp b/dbms/src/Access/AllowedClientHosts.cpp index 735411c5657..de720df1fe4 100644 --- a/dbms/src/Access/AllowedClientHosts.cpp +++ b/dbms/src/Access/AllowedClientHosts.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include @@ -23,29 +24,64 @@ namespace ErrorCodes namespace { using IPAddress = Poco::Net::IPAddress; + using IPSubnet = AllowedClientHosts::IPSubnet; + const IPSubnet ALL_ADDRESSES{IPAddress{IPAddress::IPv6}, IPAddress{IPAddress::IPv6}}; - const AllowedClientHosts::IPSubnet ALL_ADDRESSES = AllowedClientHosts::IPSubnet{IPAddress{IPAddress::IPv6}, IPAddress{IPAddress::IPv6}}; - - IPAddress toIPv6(const IPAddress & addr) + const IPAddress & getIPV6Loopback() { - if (addr.family() == IPAddress::IPv6) - return addr; - - if (addr.isLoopback()) - return IPAddress("::1"); - - return IPAddress("::FFFF:" + addr.toString()); + static const IPAddress ip("::1"); + return ip; } - IPAddress maskToIPv6(const IPAddress & mask) + bool isIPV4LoopbackMappedToIPV6(const IPAddress & ip) { - if (mask.family() == IPAddress::IPv6) - return mask; - - return IPAddress(96, IPAddress::IPv6) | toIPv6(mask); + static const IPAddress prefix("::ffff:127.0.0.0"); + /// 104 == 128 - 24, we have to reset the lowest 24 bits of 128 before comparing with `prefix` + /// (IPv4 loopback means any IP from 127.0.0.0 to 127.255.255.255). + return (ip & IPAddress(104, IPAddress::IPv6)) == prefix; } + /// Converts an address to IPv6. + /// The loopback address "127.0.0.1" (or any "127.x.y.z") is converted to "::1". + IPAddress toIPv6(const IPAddress & ip) + { + IPAddress v6; + if (ip.family() == IPAddress::IPv6) + v6 = ip; + else + v6 = IPAddress("::ffff:" + ip.toString()); + // ::ffff:127.XX.XX.XX -> ::1 + if (isIPV4LoopbackMappedToIPV6(v6)) + v6 = getIPV6Loopback(); + + return v6; + } + + /// Converts a subnet to IPv6. + IPSubnet toIPv6(const IPSubnet & subnet) + { + IPSubnet v6; + if (subnet.prefix.family() == IPAddress::IPv6) + v6.prefix = subnet.prefix; + else + v6.prefix = IPAddress("::ffff:" + subnet.prefix.toString()); + + if (subnet.mask.family() == IPAddress::IPv6) + v6.mask = subnet.mask; + else + v6.mask = IPAddress(96, IPAddress::IPv6) | IPAddress("::ffff:" + subnet.mask.toString()); + + v6.prefix = v6.prefix & v6.mask; + + // ::ffff:127.XX.XX.XX -> ::1 + if (isIPV4LoopbackMappedToIPV6(v6.prefix)) + v6 = {getIPV6Loopback(), IPAddress(128, IPAddress::IPv6)}; + + return v6; + } + + /// Helper function for isAddressOfHost(). bool isAddressOfHostImpl(const IPAddress & address, const String & host) { IPAddress addr_v6 = toIPv6(address); @@ -93,15 +129,15 @@ namespace return false; } - - /// Cached version of isAddressOfHostImpl(). We need to cache DNS requests. + /// Whether a specified address is one of the addresses of a specified host. bool isAddressOfHost(const IPAddress & address, const String & host) { + /// We need to cache DNS requests. static SimpleCache cache; return cache(address, host); } - + /// Helper function for isAddressOfLocalhost(). std::vector getAddressesOfLocalhostImpl() { std::vector addresses; @@ -114,7 +150,7 @@ namespace int err = getifaddrs(&ifa_begin); if (err) - return {IPAddress{"127.0.0.1"}, IPAddress{"::1"}}; + return {getIPV6Loopback()}; for (const ifaddrs * ifa = ifa_begin; ifa; ifa = ifa->ifa_next) { @@ -134,15 +170,15 @@ namespace return addresses; } - - /// Checks if a specified address pointers to the localhost. - bool isLocalAddress(const IPAddress & address) + /// Whether a specified address is one of the addresses of the localhost. + bool isAddressOfLocalhost(const IPAddress & address) { + /// We need to cache DNS requests. static const std::vector local_addresses = getAddressesOfLocalhostImpl(); - return boost::range::find(local_addresses, address) != local_addresses.end(); + return boost::range::find(local_addresses, toIPv6(address)) != local_addresses.end(); } - + /// Helper function for getHostByAddress(). String getHostByAddressImpl(const IPAddress & address) { Poco::Net::SocketAddress sock_addr(address, 0); @@ -160,10 +196,10 @@ namespace return host; } - - /// Cached version of getHostByAddressImpl(). We need to cache DNS requests. + /// Returns the host name by its address. String getHostByAddress(const IPAddress & address) { + /// We need to cache DNS requests. static SimpleCache cache; return cache(address); } @@ -203,7 +239,7 @@ AllowedClientHosts::AllowedClientHosts(const AllowedClientHosts & src) AllowedClientHosts & AllowedClientHosts::operator =(const AllowedClientHosts & src) { addresses = src.addresses; - loopback = src.loopback; + localhost = src.localhost; subnets = src.subnets; host_names = src.host_names; host_regexps = src.host_regexps; @@ -212,28 +248,14 @@ AllowedClientHosts & AllowedClientHosts::operator =(const AllowedClientHosts & s } -AllowedClientHosts::AllowedClientHosts(AllowedClientHosts && src) -{ - *this = src; -} - - -AllowedClientHosts & AllowedClientHosts::operator =(AllowedClientHosts && src) -{ - addresses = std::move(src.addresses); - loopback = src.loopback; - subnets = std::move(src.subnets); - host_names = std::move(src.host_names); - host_regexps = std::move(src.host_regexps); - compiled_host_regexps = std::move(src.compiled_host_regexps); - return *this; -} +AllowedClientHosts::AllowedClientHosts(AllowedClientHosts && src) = default; +AllowedClientHosts & AllowedClientHosts::operator =(AllowedClientHosts && src) = default; void AllowedClientHosts::clear() { addresses.clear(); - loopback = false; + localhost = false; subnets.clear(); host_names.clear(); host_regexps.clear(); @@ -250,10 +272,11 @@ bool AllowedClientHosts::empty() const void AllowedClientHosts::addAddress(const IPAddress & address) { IPAddress addr_v6 = toIPv6(address); - if (boost::range::find(addresses, addr_v6) == addresses.end()) - addresses.push_back(addr_v6); + if (boost::range::find(addresses, addr_v6) != addresses.end()) + return; + addresses.push_back(addr_v6); if (addr_v6.isLoopback()) - loopback = true; + localhost = true; } @@ -265,9 +288,7 @@ void AllowedClientHosts::addAddress(const String & address) void AllowedClientHosts::addSubnet(const IPSubnet & subnet) { - IPSubnet subnet_v6; - subnet_v6.prefix = toIPv6(subnet.prefix); - subnet_v6.mask = maskToIPv6(subnet.mask); + IPSubnet subnet_v6 = toIPv6(subnet); if (subnet_v6.mask == IPAddress(128, IPAddress::IPv6)) { @@ -275,8 +296,6 @@ void AllowedClientHosts::addSubnet(const IPSubnet & subnet) return; } - subnet_v6.prefix = subnet_v6.prefix & subnet_v6.mask; - if (boost::range::find(subnets, subnet_v6) == subnets.end()) subnets.push_back(subnet_v6); } @@ -314,8 +333,11 @@ void AllowedClientHosts::addSubnet(const String & subnet) void AllowedClientHosts::addHostName(const String & host_name) { - if (boost::range::find(host_names, host_name) == host_names.end()) - host_names.push_back(host_name); + if (boost::range::find(host_names, host_name) != host_names.end()) + return; + host_names.push_back(host_name); + if (boost::iequals(host_name, "localhost")) + localhost = true; } @@ -360,7 +382,7 @@ bool AllowedClientHosts::contains(const IPAddress & address) const if (boost::range::find(addresses, addr_v6) != addresses.end()) return true; - if (loopback && isLocalAddress(addr_v6)) + if (localhost && isAddressOfLocalhost(addr_v6)) return true; /// Check `ip_subnets`. diff --git a/dbms/src/Access/AllowedClientHosts.h b/dbms/src/Access/AllowedClientHosts.h index 17f8be878a1..34abd22c3bf 100644 --- a/dbms/src/Access/AllowedClientHosts.h +++ b/dbms/src/Access/AllowedClientHosts.h @@ -94,7 +94,7 @@ private: void compileRegexps() const; std::vector addresses; - bool loopback = false; + bool localhost = false; std::vector subnets; std::vector host_names; std::vector host_regexps; From e0000bef989a7fff327f22e8cf4e4443e0e45dff Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 22 Dec 2019 20:20:33 +0300 Subject: [PATCH 11/53] Added "system.stack_trace" table (development) --- dbms/src/Common/ErrorCodes.cpp | 1 + dbms/src/Common/PipeFDs.cpp | 110 +++++++++++++ dbms/src/Common/PipeFDs.h | 35 +++++ dbms/src/Common/QueryProfiler.cpp | 6 +- dbms/src/Common/ShellCommand.cpp | 8 +- dbms/src/Common/TaskStatsInfoGetter.h | 2 +- dbms/src/Common/TraceCollector.cpp | 37 +---- .../System/StorageSystemStackTrace.cpp | 145 ++++++++++++++++++ .../Storages/System/StorageSystemStackTrace.h | 30 ++++ .../Storages/System/attachSystemTables.cpp | 2 + libs/libcommon/CMakeLists.txt | 2 - libs/libcommon/include/common/Pipe.h | 34 ---- libs/libcommon/src/Pipe.cpp | 45 ------ libs/libdaemon/include/daemon/BaseDaemon.h | 8 + libs/libdaemon/src/BaseDaemon.cpp | 66 +++++--- 15 files changed, 389 insertions(+), 142 deletions(-) create mode 100644 dbms/src/Common/PipeFDs.cpp create mode 100644 dbms/src/Common/PipeFDs.h create mode 100644 dbms/src/Storages/System/StorageSystemStackTrace.cpp create mode 100644 dbms/src/Storages/System/StorageSystemStackTrace.h delete mode 100644 libs/libcommon/include/common/Pipe.h delete mode 100644 libs/libcommon/src/Pipe.cpp diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index 25a3bb7ba91..25d1b015a03 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -476,6 +476,7 @@ namespace ErrorCodes extern const int S3_ERROR = 499; extern const int CANNOT_CREATE_DICTIONARY_FROM_METADATA = 500; extern const int CANNOT_CREATE_DATABASE = 501; + extern const int CANNOT_SIGQUEUE = 502; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Common/PipeFDs.cpp b/dbms/src/Common/PipeFDs.cpp new file mode 100644 index 00000000000..463897f2c08 --- /dev/null +++ b/dbms/src/Common/PipeFDs.cpp @@ -0,0 +1,110 @@ +#include +#include +#include + +#include + +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_PIPE; + extern const int CANNOT_FCNTL; + extern const int LOGICAL_ERROR; +} + +void LazyPipeFDs::open() +{ + for (int & fd : fds_rw) + if (fd >= 0) + throw Exception("Pipe is already opened", ErrorCodes::LOGICAL_ERROR); + +#ifndef __APPLE__ + if (0 != pipe2(fds_rw, O_CLOEXEC)) + throwFromErrno("Cannot create pipe", ErrorCodes::CANNOT_PIPE); +#else + if (0 != pipe(fds_rw)) + throwFromErrno("Cannot create pipe", ErrorCodes::CANNOT_PIPE); + if (0 != fcntl(fds_rw[0], F_SETFD, FD_CLOEXEC)) + throwFromErrno("Cannot setup auto-close on exec for read end of pipe", ErrorCodes::CANNOT_FCNTL); + if (0 != fcntl(fds_rw[1], F_SETFD, FD_CLOEXEC)) + throwFromErrno("Cannot setup auto-close on exec for write end of pipe", ErrorCodes::CANNOT_FCNTL); +#endif +} + +void LazyPipeFDs::close() +{ + for (int & fd : fds_rw) + { + if (fd < 0) + continue; + if (0 != ::close(fd)) + throwFromErrno("Cannot close pipe", ErrorCodes::CANNOT_PIPE); + fd = -1; + } +} + +PipeFDs::PipeFDs() +{ + open(); +} + +LazyPipeFDs::~LazyPipeFDs() +{ + try + { + close(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} + + +void LazyPipeFDs::setNonBlocking() +{ + int flags = fcntl(fds_rw[1], F_GETFL, 0); + if (-1 == flags) + throwFromErrno("Cannot get file status flags of pipe", ErrorCodes::CANNOT_FCNTL); + if (-1 == fcntl(fds_rw[1], F_SETFL, flags | O_NONBLOCK)) + throwFromErrno("Cannot set non-blocking mode of pipe", ErrorCodes::CANNOT_FCNTL); +} + +void LazyPipeFDs::tryIncreaseSize(int desired_size) +{ +#if defined(OS_LINUX) + Poco::Logger * log = &Poco::Logger::get("Pipe"); + + /** Increase pipe size to avoid slowdown during fine-grained trace collection. + */ + int pipe_size = fcntl(fds_rw[1], F_GETPIPE_SZ); + if (-1 == pipe_size) + { + if (errno == EINVAL) + { + LOG_INFO(log, "Cannot get pipe capacity, " << errnoToString(ErrorCodes::CANNOT_FCNTL) << ". Very old Linux kernels have no support for this fcntl."); + /// It will work nevertheless. + } + else + throwFromErrno("Cannot get pipe capacity", ErrorCodes::CANNOT_FCNTL); + } + else + { + for (errno = 0; errno != EPERM && pipe_size < desired_size; pipe_size *= 2) + if (-1 == fcntl(fds_rw[1], F_SETPIPE_SZ, pipe_size * 2) && errno != EPERM) + throwFromErrno("Cannot increase pipe capacity to " + std::to_string(pipe_size * 2), ErrorCodes::CANNOT_FCNTL); + + LOG_TRACE(log, "Pipe capacity is " << formatReadableSizeWithBinarySuffix(std::min(pipe_size, desired_size))); + } +#endif +} + +} diff --git a/dbms/src/Common/PipeFDs.h b/dbms/src/Common/PipeFDs.h new file mode 100644 index 00000000000..fe76740da70 --- /dev/null +++ b/dbms/src/Common/PipeFDs.h @@ -0,0 +1,35 @@ +#pragma once + +#include + + +namespace DB +{ + +/** Struct containing a pipe with lazy initialization. + * Use `open` and `close` methods to manipulate pipe and `fds_rw` field to access + * pipe's file descriptors. + */ +struct LazyPipeFDs +{ + int fds_rw[2] = {-1, -1}; + + void open(); + void close(); + + void setNonBlocking(); + void tryIncreaseSize(int desired_size); + + ~LazyPipeFDs(); +}; + + +/** Struct which opens new pipe on creation and closes it on destruction. + * Use `fds_rw` field to access pipe's file descriptors. + */ +struct PipeFDs : public LazyPipeFDs +{ + PipeFDs(); +}; + +} diff --git a/dbms/src/Common/QueryProfiler.cpp b/dbms/src/Common/QueryProfiler.cpp index 34d6acc27b1..e142be2e4d9 100644 --- a/dbms/src/Common/QueryProfiler.cpp +++ b/dbms/src/Common/QueryProfiler.cpp @@ -1,12 +1,12 @@ #include "QueryProfiler.h" #include -#include #include #include -#include #include #include +#include +#include #include #include #include @@ -22,7 +22,7 @@ namespace ProfileEvents namespace DB { -extern LazyPipe trace_pipe; +extern LazyPipeFDs trace_pipe; namespace { diff --git a/dbms/src/Common/ShellCommand.cpp b/dbms/src/Common/ShellCommand.cpp index 8807d795a0d..9dbe3e2f074 100644 --- a/dbms/src/Common/ShellCommand.cpp +++ b/dbms/src/Common/ShellCommand.cpp @@ -4,11 +4,11 @@ #include #include #include +#include #include #include #include #include -#include namespace { @@ -66,9 +66,9 @@ std::unique_ptr ShellCommand::executeImpl(const char * filename, c if (!real_vfork) throwFromErrno("Cannot find symbol vfork in myself", ErrorCodes::CANNOT_DLSYM); - Pipe pipe_stdin; - Pipe pipe_stdout; - Pipe pipe_stderr; + PipeFDs pipe_stdin; + PipeFDs pipe_stdout; + PipeFDs pipe_stderr; pid_t pid = reinterpret_cast(real_vfork)(); diff --git a/dbms/src/Common/TaskStatsInfoGetter.h b/dbms/src/Common/TaskStatsInfoGetter.h index b3e35d65674..f3a581b2c78 100644 --- a/dbms/src/Common/TaskStatsInfoGetter.h +++ b/dbms/src/Common/TaskStatsInfoGetter.h @@ -24,7 +24,7 @@ public: /// Whether the current process has permissions (sudo or cap_net_admin capabilties) to get taskstats info static bool checkPermissions(); -#if defined(__linux__) +#if defined(OS_LINUX) private: int netlink_socket_fd = -1; UInt16 taskstats_family_id = 0; diff --git a/dbms/src/Common/TraceCollector.cpp b/dbms/src/Common/TraceCollector.cpp index bd06a200460..4b582c5ad4f 100644 --- a/dbms/src/Common/TraceCollector.cpp +++ b/dbms/src/Common/TraceCollector.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include @@ -19,13 +19,12 @@ namespace DB { -LazyPipe trace_pipe; +LazyPipeFDs trace_pipe; namespace ErrorCodes { extern const int NULL_POINTER_DEREFERENCE; extern const int THREAD_IS_NOT_JOINABLE; - extern const int CANNOT_FCNTL; } TraceCollector::TraceCollector(std::shared_ptr & trace_log_) @@ -40,36 +39,8 @@ TraceCollector::TraceCollector(std::shared_ptr & trace_log_) /** Turn write end of pipe to non-blocking mode to avoid deadlocks * when QueryProfiler is invoked under locks and TraceCollector cannot pull data from pipe. */ - int flags = fcntl(trace_pipe.fds_rw[1], F_GETFL, 0); - if (-1 == flags) - throwFromErrno("Cannot get file status flags of pipe", ErrorCodes::CANNOT_FCNTL); - if (-1 == fcntl(trace_pipe.fds_rw[1], F_SETFL, flags | O_NONBLOCK)) - throwFromErrno("Cannot set non-blocking mode of pipe", ErrorCodes::CANNOT_FCNTL); - -#if defined(OS_LINUX) - /** Increase pipe size to avoid slowdown during fine-grained trace collection. - */ - int pipe_size = fcntl(trace_pipe.fds_rw[1], F_GETPIPE_SZ); - if (-1 == pipe_size) - { - if (errno == EINVAL) - { - LOG_INFO(log, "Cannot get pipe capacity, " << errnoToString(ErrorCodes::CANNOT_FCNTL) << ". Very old Linux kernels have no support for this fcntl."); - /// It will work nevertheless. - } - else - throwFromErrno("Cannot get pipe capacity", ErrorCodes::CANNOT_FCNTL); - } - else - { - constexpr int max_pipe_capacity_to_set = 1048576; - for (errno = 0; errno != EPERM && pipe_size < max_pipe_capacity_to_set; pipe_size *= 2) - if (-1 == fcntl(trace_pipe.fds_rw[1], F_SETPIPE_SZ, pipe_size * 2) && errno != EPERM) - throwFromErrno("Cannot increase pipe capacity to " + toString(pipe_size * 2), ErrorCodes::CANNOT_FCNTL); - - LOG_TRACE(log, "Pipe capacity is " << formatReadableSizeWithBinarySuffix(std::min(pipe_size, max_pipe_capacity_to_set))); - } -#endif + trace_pipe.setNonBlocking(); + trace_pipe.tryIncreaseSize(1 << 20); thread = ThreadFromGlobalPool(&TraceCollector::run, this); } diff --git a/dbms/src/Storages/System/StorageSystemStackTrace.cpp b/dbms/src/Storages/System/StorageSystemStackTrace.cpp new file mode 100644 index 00000000000..97149fa8712 --- /dev/null +++ b/dbms/src/Storages/System/StorageSystemStackTrace.cpp @@ -0,0 +1,145 @@ +#include + +#include +#include +#include + +#include + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_SIGQUEUE; +} + + +NamesAndTypesList StorageSystemStackTrace::getNamesAndTypes() +{ + return + { + { "thread_number", std::make_shared() }, + { "query_id", std::make_shared() }, + { "trace", std::make_shared(std::make_shared()) } + }; +} + +namespace +{ + struct State + { + std::mutex mutex; + std::condition_variable condvar; + + size_t total_threads; + size_t threads_processed; + std::exception_ptr exception; + MutableColumns * columns_to_fill; + + State() { reset(); } + + void reset(MutableColumns * columns_to_fill_ = nullptr) + { + total_threads = 0; + threads_processed = 0; + exception = std::exception_ptr(); + columns_to_fill = columns_to_fill_; + } + + operator bool() + { + return columns_to_fill != nullptr; + } + }; + + State state; + + void callback(const siginfo_t &, const StackTrace & stack_trace, UInt32 thread_number) + { + std::lock_guard lock(state.mutex); + + std::cerr << thread_number << " !\n"; + + if (!state) + return; + + try + { + size_t stack_trace_size = stack_trace.getSize(); + size_t stack_trace_offset = stack_trace.getOffset(); + + Array arr; + arr.reserve(stack_trace_size - stack_trace_offset); + for (size_t i = stack_trace_offset; i < stack_trace_size; ++i) + arr.emplace_back(reinterpret_cast(stack_trace.getFrames()[i])); + + std::cerr << thread_number << " !!\n"; + + state.columns_to_fill->at(0)->insert(thread_number); + state.columns_to_fill->at(1)->insertDefault(); + state.columns_to_fill->at(2)->insert(arr); + + std::cerr << thread_number << " !!!\n"; + + ++state.threads_processed; + + std::cerr << state.threads_processed << ", " << state.total_threads << " !!!!\n"; + if (state.threads_processed >= state.total_threads) + state.condvar.notify_one(); + } + catch (...) + { + state.reset(); + state.exception = std::current_exception(); + state.condvar.notify_one(); + } + } +} + +void StorageSystemStackTrace::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const +{ + std::unique_lock lock(state.mutex); + + state.reset(&res_columns); + SCOPE_EXIT({ state.reset(); }); + + std::cerr << state.columns_to_fill->size() << "\n"; + + /// Send a signal to every thread + std::filesystem::directory_iterator end; + for (std::filesystem::directory_iterator it("/proc/self/task"); it != end; ++it) + { + sigval sig_value; + sig_value.sival_ptr = reinterpret_cast(&callback); + pid_t tid = parse(it->path().filename()); + if (0 == ::sigqueue(tid, SIGTSTP, sig_value)) + { + ++state.total_threads; + } + else + { + /// The thread may have been already finished. + if (ESRCH != errno) + throwFromErrno("Cannot send signal with sigqueue", ErrorCodes::CANNOT_SIGQUEUE); + } + } + + std::cerr << state.threads_processed << ", " << state.total_threads << " sent\n"; + + /// Timeout one second for the case the signal pipe will be full and messages will be dropped. + state.condvar.wait_for(lock, std::chrono::seconds(1), []{ return state.threads_processed >= state.total_threads || state.exception; }); + if (state.exception) + std::rethrow_exception(state.exception); +} + +} + diff --git a/dbms/src/Storages/System/StorageSystemStackTrace.h b/dbms/src/Storages/System/StorageSystemStackTrace.h new file mode 100644 index 00000000000..a402f56b420 --- /dev/null +++ b/dbms/src/Storages/System/StorageSystemStackTrace.h @@ -0,0 +1,30 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class Context; + + +/// Allows to introspect stack trace of all server threads. +/// It acts like an embedded debugger. +class StorageSystemStackTrace : public ext::shared_ptr_helper, public IStorageSystemOneBlock +{ + friend struct ext::shared_ptr_helper; +public: + String getName() const override { return "SystemStackTrace"; } + + static NamesAndTypesList getNamesAndTypes(); + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; +}; + +} + diff --git a/dbms/src/Storages/System/attachSystemTables.cpp b/dbms/src/Storages/System/attachSystemTables.cpp index 2b8e630cbed..cd224353acb 100644 --- a/dbms/src/Storages/System/attachSystemTables.cpp +++ b/dbms/src/Storages/System/attachSystemTables.cpp @@ -38,6 +38,7 @@ #include #include #include +#include namespace DB @@ -65,6 +66,7 @@ void attachSystemTablesLocal(IDatabase & system_database) system_database.attachTable("collations", StorageSystemCollations::create("collations")); system_database.attachTable("table_engines", StorageSystemTableEngines::create("table_engines")); system_database.attachTable("contributors", StorageSystemContributors::create("contributors")); + system_database.attachTable("stack_trace", StorageSystemStackTrace::create("stack_trace")); } void attachSystemTablesServer(IDatabase & system_database, bool has_zookeeper) diff --git a/libs/libcommon/CMakeLists.txt b/libs/libcommon/CMakeLists.txt index 3e58cba0164..f9d8ea696f3 100644 --- a/libs/libcommon/CMakeLists.txt +++ b/libs/libcommon/CMakeLists.txt @@ -23,7 +23,6 @@ add_library (common src/getThreadNumber.cpp src/sleep.cpp src/argsToConfig.cpp - src/Pipe.cpp src/phdr_cache.cpp include/common/SimpleCache.h @@ -46,7 +45,6 @@ add_library (common include/common/setTerminalEcho.h include/common/find_symbols.h include/common/constexpr_helpers.h - include/common/Pipe.h include/common/getThreadNumber.h include/common/sleep.h include/common/SimpleCache.h diff --git a/libs/libcommon/include/common/Pipe.h b/libs/libcommon/include/common/Pipe.h deleted file mode 100644 index 0137c3d97af..00000000000 --- a/libs/libcommon/include/common/Pipe.h +++ /dev/null @@ -1,34 +0,0 @@ -#pragma once - -#include -#include -#include - -/** - * Struct containing a pipe with lazy initialization. - * Use `open` and `close` methods to manipulate pipe and `fds_rw` field to access - * pipe's file descriptors. - */ -struct LazyPipe -{ - int fds_rw[2] = {-1, -1}; - - LazyPipe() = default; - - void open(); - - void close(); - - virtual ~LazyPipe() = default; -}; - -/** - * Struct which opens new pipe on creation and closes it on destruction. - * Use `fds_rw` field to access pipe's file descriptors. - */ -struct Pipe : public LazyPipe -{ - Pipe(); - - ~Pipe(); -}; diff --git a/libs/libcommon/src/Pipe.cpp b/libs/libcommon/src/Pipe.cpp deleted file mode 100644 index 83268b76ea6..00000000000 --- a/libs/libcommon/src/Pipe.cpp +++ /dev/null @@ -1,45 +0,0 @@ -#include "common/Pipe.h" - -void LazyPipe::open() -{ - for (int & fd : fds_rw) - { - if (fd >= 0) - { - throw std::logic_error("Pipe is already opened"); - } - } - -#ifndef __APPLE__ - if (0 != pipe2(fds_rw, O_CLOEXEC)) - throw std::runtime_error("Cannot create pipe"); -#else - if (0 != pipe(fds_rw)) - throw std::runtime_error("Cannot create pipe"); - if (0 != fcntl(fds_rw[0], F_SETFD, FD_CLOEXEC)) - throw std::runtime_error("Cannot setup auto-close on exec for read end of pipe"); - if (0 != fcntl(fds_rw[1], F_SETFD, FD_CLOEXEC)) - throw std::runtime_error("Cannot setup auto-close on exec for write end of pipe"); -#endif -} - -void LazyPipe::close() -{ - for (int fd : fds_rw) - { - if (fd >= 0) - { - ::close(fd); - } - } -} - -Pipe::Pipe() -{ - open(); -} - -Pipe::~Pipe() -{ - close(); -} diff --git a/libs/libdaemon/include/daemon/BaseDaemon.h b/libs/libdaemon/include/daemon/BaseDaemon.h index 9d323492c1f..9457d9cdbe0 100644 --- a/libs/libdaemon/include/daemon/BaseDaemon.h +++ b/libs/libdaemon/include/daemon/BaseDaemon.h @@ -19,10 +19,12 @@ #include #include #include +#include #include #include #include + namespace Poco { class TaskManager; } @@ -234,3 +236,9 @@ std::optional> BaseDaemon::tryGetInstance() else return {}; } + + +/// If you send TSTP signal with value (sigqueue) to a thread, it will make a callback +/// from a separate thread and you can call non signal-safe function from there. +using SignalCallback = void(const siginfo_t &, const StackTrace &, UInt32); + diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index 15b61c9b454..a3d56511508 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -1,5 +1,5 @@ #include -#include + #include #include #include @@ -12,19 +12,15 @@ #include #include -#include -#include -#include -#include #include #include #include #include #include #include + #include #include -#include #include #include #include @@ -36,16 +32,23 @@ #include #include #include -#include + +#include +#include +#include + #include #include #include #include #include +#include +#include +#include #include #include +#include #include -#include #ifdef __APPLE__ // ucontext is not available without _XOPEN_SOURCE @@ -54,7 +57,7 @@ #include -Pipe signal_pipe; +DB::PipeFDs signal_pipe; /** Reset signal handler to the default and send signal to itself. @@ -67,8 +70,13 @@ static void call_default_signal_handler(int sig) } -using ThreadNumber = decltype(getThreadNumber()); -static const size_t buf_size = sizeof(int) + sizeof(siginfo_t) + sizeof(ucontext_t) + sizeof(StackTrace) + sizeof(ThreadNumber); +/// Normally query_id is a UUID (string with a fixed length) but user can provide custom query_id. +/// Thus upper bound on query_id length should be introduced to avoid buffer overflow in signal handler. +constexpr size_t QUERY_ID_MAX_LEN = 1024; + +static const size_t buf_size = sizeof(int) + sizeof(siginfo_t) + sizeof(ucontext_t) + sizeof(StackTrace) + sizeof(UInt32) + + QUERY_ID_MAX_LEN + 2 /* varint encoding query_id length */; + using signal_function = void(int, siginfo_t*, void*); @@ -92,11 +100,12 @@ static void terminateRequestedSignalHandler(int sig, siginfo_t * info, void * co } -/** Handler for "fault" signals. Send data about fault to separate thread to write into log. +/** Handler for "fault" or diagnostic signals. Send data about fault to separate thread to write into log. */ -static void faultSignalHandler(int sig, siginfo_t * info, void * context) +static void signalHandler(int sig, siginfo_t * info, void * context) { char buf[buf_size]; + std::cerr << "Size of buffer: " << buf_size << "\n"; DB::WriteBufferFromFileDescriptorDiscardOnFailure out(signal_pipe.fds_rw[1], buf_size, buf); const ucontext_t signal_context = *reinterpret_cast(context); @@ -106,7 +115,7 @@ static void faultSignalHandler(int sig, siginfo_t * info, void * context) DB::writePODBinary(*info, out); DB::writePODBinary(signal_context, out); DB::writePODBinary(stack_trace, out); - DB::writeBinary(getThreadNumber(), out); + DB::writeBinary(UInt32(getThreadNumber()), out); out.next(); @@ -162,7 +171,7 @@ public: } else if (sig == Signals::StdTerminate) { - ThreadNumber thread_num; + UInt32 thread_num; std::string message; DB::readBinary(thread_num, in); @@ -181,13 +190,27 @@ public: siginfo_t info; ucontext_t context; StackTrace stack_trace(NoCapture{}); - ThreadNumber thread_num; + UInt32 thread_num; DB::readPODBinary(info, in); DB::readPODBinary(context, in); DB::readPODBinary(stack_trace, in); DB::readBinary(thread_num, in); + if (sig == SIGTSTP && info.si_value.sival_ptr) + { + /// TSTP signal with value is used to make a custom callback from this thread. + try + { + reinterpret_cast(info.si_value.sival_ptr)(info, stack_trace, thread_num); + continue; + } + catch (...) + { + /// Failed to process, will use 'onFault' function. + } + } + /// This allows to receive more signals if failure happens inside onFault function. /// Example: segfault while symbolizing stack trace. std::thread([=] { onFault(sig, info, context, stack_trace, thread_num); }).detach(); @@ -200,12 +223,12 @@ private: BaseDaemon & daemon; private: - void onTerminate(const std::string & message, ThreadNumber thread_num) const + void onTerminate(const std::string & message, UInt32 thread_num) const { LOG_FATAL(log, "(version " << VERSION_STRING << VERSION_OFFICIAL << ") (from thread " << thread_num << ") " << message); } - void onFault(int sig, const siginfo_t & info, const ucontext_t & context, const StackTrace & stack_trace, ThreadNumber thread_num) const + void onFault(int sig, const siginfo_t & info, const ucontext_t & context, const StackTrace & stack_trace, UInt32 thread_num) const { LOG_FATAL(log, "########################################"); LOG_FATAL(log, "(version " << VERSION_STRING << VERSION_OFFICIAL << ") (from thread " << thread_num << ") " @@ -264,7 +287,7 @@ static void terminate_handler() DB::WriteBufferFromFileDescriptor out(signal_pipe.fds_rw[1], buf_size, buf); DB::writeBinary(static_cast(SignalListener::StdTerminate), out); - DB::writeBinary(getThreadNumber(), out); + DB::writeBinary(UInt32(getThreadNumber()), out); DB::writeBinary(log_message, out); out.next(); @@ -721,7 +744,7 @@ void BaseDaemon::initializeTerminationAndSignalProcessing() /// SIGTSTP is added for debugging purposes. To output a stack trace of any running thread at anytime. - add_signal_handler({SIGABRT, SIGSEGV, SIGILL, SIGBUS, SIGSYS, SIGFPE, SIGPIPE, SIGTSTP}, faultSignalHandler); + add_signal_handler({SIGABRT, SIGSEGV, SIGILL, SIGBUS, SIGSYS, SIGFPE, SIGPIPE, SIGTSTP}, signalHandler); add_signal_handler({SIGHUP, SIGUSR1}, closeLogsSignalHandler); add_signal_handler({SIGINT, SIGQUIT, SIGTERM}, terminateRequestedSignalHandler); @@ -729,6 +752,9 @@ void BaseDaemon::initializeTerminationAndSignalProcessing() static KillingErrorHandler killing_error_handler; Poco::ErrorHandler::set(&killing_error_handler); + signal_pipe.setNonBlocking(); + signal_pipe.tryIncreaseSize(1 << 20); + signal_listener.reset(new SignalListener(*this)); signal_listener_thread.start(*signal_listener); } From 9c868c910a5e7c46178e247393787ff96dc48b3f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 22 Dec 2019 23:17:16 +0300 Subject: [PATCH 12/53] Simplification --- .../System/StorageSystemStackTrace.cpp | 225 +++++++++++------- .../Storages/System/StorageSystemStackTrace.h | 7 +- libs/libdaemon/src/BaseDaemon.cpp | 15 -- 3 files changed, 141 insertions(+), 106 deletions(-) diff --git a/dbms/src/Storages/System/StorageSystemStackTrace.cpp b/dbms/src/Storages/System/StorageSystemStackTrace.cpp index 97149fa8712..7f538a7c6de 100644 --- a/dbms/src/Storages/System/StorageSystemStackTrace.cpp +++ b/dbms/src/Storages/System/StorageSystemStackTrace.cpp @@ -1,7 +1,7 @@ #include +#include #include -#include #include #include @@ -10,8 +10,9 @@ #include #include #include -#include #include +#include +#include namespace DB @@ -20,6 +21,100 @@ namespace DB namespace ErrorCodes { extern const int CANNOT_SIGQUEUE; + extern const int CANNOT_MANIPULATE_SIGSET; + extern const int CANNOT_SET_SIGNAL_HANDLER; + extern const int CANNOT_READ_FROM_FILE_DESCRIPTOR; + extern const int LOGICAL_ERROR; +} + + +namespace +{ + const pid_t expected_pid = getpid(); + const int sig = SIGRTMIN; + UInt32 thread_number{0}; + std::optional stack_trace; + LazyPipeFDs notification_pipe; + + void signalHandler(int, siginfo_t * info, void * context) + { + /// In case malicious user is sending signals manually (for unknown reason). + /// If we don't check - it may break our synchronization. + if (info->si_pid != expected_pid) + return; + + /// All these methods are signal-safe. + const ucontext_t signal_context = *reinterpret_cast(context); + stack_trace.emplace(signal_context); + thread_number = getThreadNumber(); + + char buf = 0; + /// We cannot do anything if write failed. + (void)::write(notification_pipe.fds_rw[1], &buf, 1); + } + + /// Wait for data in pipe. + bool wait(int timeout_ms) + { + while (true) + { + int fd = notification_pipe.fds_rw[0]; + pollfd poll_fd{fd, POLLIN, 0}; + + int poll_res = poll(&poll_fd, 1, timeout_ms); + if (poll_res < 0) + { + if (errno == EINTR) + { + --timeout_ms; /// Quite a hacky way to update timeout. Just to make sure we avoid infinite waiting. + if (timeout_ms == 0) + return false; + continue; + } + + throwFromErrno("Cannot poll pipe", ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR); + } + if (poll_res == 0) + return false; + + char buf = 0; + ssize_t read_res = ::read(fd, &buf, 1); + if (read_res == 1) + return true; + + if (read_res < 0) + { + if (errno == EINTR) + continue; + + throwFromErrno("Cannot read from pipe", ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR); + } + + throw Exception("Logical error: read for one byte returned more than one byte", ErrorCodes::LOGICAL_ERROR); + } + } +} + + +StorageSystemStackTrace::StorageSystemStackTrace(const String & name) + : IStorageSystemOneBlock(name) +{ + notification_pipe.open(); + + /// Setup signal handler. + + struct sigaction sa{}; + sa.sa_sigaction = signalHandler; + sa.sa_flags = SA_SIGINFO; + + if (sigemptyset(&sa.sa_mask)) + throwFromErrno("Cannot set signal handler.", ErrorCodes::CANNOT_MANIPULATE_SIGSET); + + if (sigaddset(&sa.sa_mask, sig)) + throwFromErrno("Cannot set signal handler.", ErrorCodes::CANNOT_MANIPULATE_SIGSET); + + if (sigaction(sig, &sa, nullptr)) + throwFromErrno("Cannot set signal handler.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER); } @@ -33,112 +128,64 @@ NamesAndTypesList StorageSystemStackTrace::getNamesAndTypes() }; } -namespace + +void StorageSystemStackTrace::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const { - struct State + /// It shouldn't be possible to do concurrent reads from this table. + std::lock_guard lock(mutex); + + /// Send a signal to every thread and wait for result. + /// We must wait for every thread one by one sequentially, + /// because there is a limit on number of queued signals in OS and otherwise signals may get lost. + /// Also, non-RT signals are not delivered if previous signal is handled right now (by default; but we use RT signals). + + /// Obviously, results for different threads may be out of sync. + + /// There is no better way to enumerate threads in a process other than looking into procfs. + + std::filesystem::directory_iterator end; + for (std::filesystem::directory_iterator it("/proc/self/task"); it != end; ++it) { - std::mutex mutex; - std::condition_variable condvar; + sigval sig_value{}; + pid_t tid = parse(it->path().filename()); - size_t total_threads; - size_t threads_processed; - std::exception_ptr exception; - MutableColumns * columns_to_fill; + std::cerr << "Requested: " << tid << "\n"; - State() { reset(); } - - void reset(MutableColumns * columns_to_fill_ = nullptr) + if (0 != ::sigqueue(tid, sig, sig_value)) { - total_threads = 0; - threads_processed = 0; - exception = std::exception_ptr(); - columns_to_fill = columns_to_fill_; + /// The thread may has been already finished. + if (ESRCH == errno) + continue; + + throwFromErrno("Cannot send signal with sigqueue", ErrorCodes::CANNOT_SIGQUEUE); } - operator bool() + /// Just in case we will wait for pipe with timeout. In case signal didn't get processed. + if (wait(100)) { - return columns_to_fill != nullptr; - } - }; - - State state; - - void callback(const siginfo_t &, const StackTrace & stack_trace, UInt32 thread_number) - { - std::lock_guard lock(state.mutex); - - std::cerr << thread_number << " !\n"; - - if (!state) - return; - - try - { - size_t stack_trace_size = stack_trace.getSize(); - size_t stack_trace_offset = stack_trace.getOffset(); + size_t stack_trace_size = stack_trace->getSize(); + size_t stack_trace_offset = stack_trace->getOffset(); Array arr; arr.reserve(stack_trace_size - stack_trace_offset); for (size_t i = stack_trace_offset; i < stack_trace_size; ++i) - arr.emplace_back(reinterpret_cast(stack_trace.getFrames()[i])); + arr.emplace_back(reinterpret_cast(stack_trace->getFrames()[i])); - std::cerr << thread_number << " !!\n"; + std::cerr << tid << ", " << thread_number << " !!\n"; - state.columns_to_fill->at(0)->insert(thread_number); - state.columns_to_fill->at(1)->insertDefault(); - state.columns_to_fill->at(2)->insert(arr); - - std::cerr << thread_number << " !!!\n"; - - ++state.threads_processed; - - std::cerr << state.threads_processed << ", " << state.total_threads << " !!!!\n"; - if (state.threads_processed >= state.total_threads) - state.condvar.notify_one(); - } - catch (...) - { - state.reset(); - state.exception = std::current_exception(); - state.condvar.notify_one(); - } - } -} - -void StorageSystemStackTrace::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const -{ - std::unique_lock lock(state.mutex); - - state.reset(&res_columns); - SCOPE_EXIT({ state.reset(); }); - - std::cerr << state.columns_to_fill->size() << "\n"; - - /// Send a signal to every thread - std::filesystem::directory_iterator end; - for (std::filesystem::directory_iterator it("/proc/self/task"); it != end; ++it) - { - sigval sig_value; - sig_value.sival_ptr = reinterpret_cast(&callback); - pid_t tid = parse(it->path().filename()); - if (0 == ::sigqueue(tid, SIGTSTP, sig_value)) - { - ++state.total_threads; + res_columns[0]->insert(thread_number); + res_columns[1]->insertDefault(); + res_columns[2]->insert(arr); } else { - /// The thread may have been already finished. - if (ESRCH != errno) - throwFromErrno("Cannot send signal with sigqueue", ErrorCodes::CANNOT_SIGQUEUE); + /// Cannot obtain a stack trace. But create a record in result nevertheless. + + res_columns[0]->insert(tid); + res_columns[1]->insertDefault(); + res_columns[2]->insertDefault(); } } - - std::cerr << state.threads_processed << ", " << state.total_threads << " sent\n"; - - /// Timeout one second for the case the signal pipe will be full and messages will be dropped. - state.condvar.wait_for(lock, std::chrono::seconds(1), []{ return state.threads_processed >= state.total_threads || state.exception; }); - if (state.exception) - std::rethrow_exception(state.exception); } } diff --git a/dbms/src/Storages/System/StorageSystemStackTrace.h b/dbms/src/Storages/System/StorageSystemStackTrace.h index a402f56b420..161e7f2e2fb 100644 --- a/dbms/src/Storages/System/StorageSystemStackTrace.h +++ b/dbms/src/Storages/System/StorageSystemStackTrace.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include @@ -17,13 +18,15 @@ class StorageSystemStackTrace : public ext::shared_ptr_helper; public: String getName() const override { return "SystemStackTrace"; } - static NamesAndTypesList getNamesAndTypes(); + StorageSystemStackTrace(const String & name); + protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; + + mutable std::mutex mutex; }; } diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index a3d56511508..c09139257a9 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -105,7 +105,6 @@ static void terminateRequestedSignalHandler(int sig, siginfo_t * info, void * co static void signalHandler(int sig, siginfo_t * info, void * context) { char buf[buf_size]; - std::cerr << "Size of buffer: " << buf_size << "\n"; DB::WriteBufferFromFileDescriptorDiscardOnFailure out(signal_pipe.fds_rw[1], buf_size, buf); const ucontext_t signal_context = *reinterpret_cast(context); @@ -197,20 +196,6 @@ public: DB::readPODBinary(stack_trace, in); DB::readBinary(thread_num, in); - if (sig == SIGTSTP && info.si_value.sival_ptr) - { - /// TSTP signal with value is used to make a custom callback from this thread. - try - { - reinterpret_cast(info.si_value.sival_ptr)(info, stack_trace, thread_num); - continue; - } - catch (...) - { - /// Failed to process, will use 'onFault' function. - } - } - /// This allows to receive more signals if failure happens inside onFault function. /// Example: segfault while symbolizing stack trace. std::thread([=] { onFault(sig, info, context, stack_trace, thread_num); }).detach(); From 48d126e88a577881674e55d30acc7487c79bb73d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 22 Dec 2019 23:19:29 +0300 Subject: [PATCH 13/53] Removed unrelated changes --- libs/libdaemon/include/daemon/BaseDaemon.h | 6 ------ libs/libdaemon/src/BaseDaemon.cpp | 7 +------ 2 files changed, 1 insertion(+), 12 deletions(-) diff --git a/libs/libdaemon/include/daemon/BaseDaemon.h b/libs/libdaemon/include/daemon/BaseDaemon.h index 9457d9cdbe0..462cbb95418 100644 --- a/libs/libdaemon/include/daemon/BaseDaemon.h +++ b/libs/libdaemon/include/daemon/BaseDaemon.h @@ -236,9 +236,3 @@ std::optional> BaseDaemon::tryGetInstance() else return {}; } - - -/// If you send TSTP signal with value (sigqueue) to a thread, it will make a callback -/// from a separate thread and you can call non signal-safe function from there. -using SignalCallback = void(const siginfo_t &, const StackTrace &, UInt32); - diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index c09139257a9..233a11707a9 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -70,12 +70,7 @@ static void call_default_signal_handler(int sig) } -/// Normally query_id is a UUID (string with a fixed length) but user can provide custom query_id. -/// Thus upper bound on query_id length should be introduced to avoid buffer overflow in signal handler. -constexpr size_t QUERY_ID_MAX_LEN = 1024; - -static const size_t buf_size = sizeof(int) + sizeof(siginfo_t) + sizeof(ucontext_t) + sizeof(StackTrace) + sizeof(UInt32) - + QUERY_ID_MAX_LEN + 2 /* varint encoding query_id length */; +static const size_t buf_size = sizeof(int) + sizeof(siginfo_t) + sizeof(ucontext_t) + sizeof(StackTrace) + sizeof(UInt32); using signal_function = void(int, siginfo_t*, void*); From ca25e2f30fe723f9bd68687a54659c2cdba3a0d2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 22 Dec 2019 23:37:29 +0300 Subject: [PATCH 14/53] Added query_id to "system.stack_trace" --- .../System/StorageSystemStackTrace.cpp | 21 ++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/dbms/src/Storages/System/StorageSystemStackTrace.cpp b/dbms/src/Storages/System/StorageSystemStackTrace.cpp index 7f538a7c6de..8a932a3eb7c 100644 --- a/dbms/src/Storages/System/StorageSystemStackTrace.cpp +++ b/dbms/src/Storages/System/StorageSystemStackTrace.cpp @@ -32,8 +32,14 @@ namespace { const pid_t expected_pid = getpid(); const int sig = SIGRTMIN; + UInt32 thread_number{0}; std::optional stack_trace; + + static constexpr size_t max_query_id_size = 128; + char query_id_data[max_query_id_size]; + size_t query_id_size = 0; + LazyPipeFDs notification_pipe; void signalHandler(int, siginfo_t * info, void * context) @@ -48,12 +54,16 @@ namespace stack_trace.emplace(signal_context); thread_number = getThreadNumber(); + StringRef query_id = CurrentThread::getQueryId(); + query_id_size = std::min(query_id.size, max_query_id_size); + memcpy(query_id_data, query_id.data, query_id_size); + char buf = 0; /// We cannot do anything if write failed. (void)::write(notification_pipe.fds_rw[1], &buf, 1); } - /// Wait for data in pipe. + /// Wait for data in pipe and read it. bool wait(int timeout_ms) { while (true) @@ -149,8 +159,6 @@ void StorageSystemStackTrace::fillData(MutableColumns & res_columns, const Conte sigval sig_value{}; pid_t tid = parse(it->path().filename()); - std::cerr << "Requested: " << tid << "\n"; - if (0 != ::sigqueue(tid, sig, sig_value)) { /// The thread may has been already finished. @@ -161,6 +169,7 @@ void StorageSystemStackTrace::fillData(MutableColumns & res_columns, const Conte } /// Just in case we will wait for pipe with timeout. In case signal didn't get processed. + if (wait(100)) { size_t stack_trace_size = stack_trace->getSize(); @@ -171,17 +180,15 @@ void StorageSystemStackTrace::fillData(MutableColumns & res_columns, const Conte for (size_t i = stack_trace_offset; i < stack_trace_size; ++i) arr.emplace_back(reinterpret_cast(stack_trace->getFrames()[i])); - std::cerr << tid << ", " << thread_number << " !!\n"; - res_columns[0]->insert(thread_number); - res_columns[1]->insertDefault(); + res_columns[1]->insertData(query_id_data, query_id_size); res_columns[2]->insert(arr); } else { /// Cannot obtain a stack trace. But create a record in result nevertheless. - res_columns[0]->insert(tid); + res_columns[0]->insert(tid); /// TODO Replace all thread numbers to OS thread numbers. res_columns[1]->insertDefault(); res_columns[2]->insertDefault(); } From 1e83ebe5214df6fe0ddd301c0ff6ada71645caf9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 22 Dec 2019 23:50:00 +0300 Subject: [PATCH 15/53] Added test --- .../queries/0_stateless/01051_system_stack_trace.reference | 1 + dbms/tests/queries/0_stateless/01051_system_stack_trace.sql | 2 ++ 2 files changed, 3 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/01051_system_stack_trace.reference create mode 100644 dbms/tests/queries/0_stateless/01051_system_stack_trace.sql diff --git a/dbms/tests/queries/0_stateless/01051_system_stack_trace.reference b/dbms/tests/queries/0_stateless/01051_system_stack_trace.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01051_system_stack_trace.reference @@ -0,0 +1 @@ +1 diff --git a/dbms/tests/queries/0_stateless/01051_system_stack_trace.sql b/dbms/tests/queries/0_stateless/01051_system_stack_trace.sql new file mode 100644 index 00000000000..32d344fce7e --- /dev/null +++ b/dbms/tests/queries/0_stateless/01051_system_stack_trace.sql @@ -0,0 +1,2 @@ +-- at least this query should be present +SELECT count() > 0 FROM system.stack_trace WHERE query_id != ''; From c78443d9393781b7ca00e04673b16fa9522a42f4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 22 Dec 2019 23:50:33 +0300 Subject: [PATCH 16/53] Avoid using query_id from old threads --- dbms/src/Interpreters/ThreadStatusExt.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Interpreters/ThreadStatusExt.cpp b/dbms/src/Interpreters/ThreadStatusExt.cpp index 8c578422d6e..0bf1ac36d3d 100644 --- a/dbms/src/Interpreters/ThreadStatusExt.cpp +++ b/dbms/src/Interpreters/ThreadStatusExt.cpp @@ -197,6 +197,7 @@ void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) /// Must reset pointer to thread_group's memory_tracker, because it will be destroyed two lines below. memory_tracker.setParent(nullptr); + query_id.clear(); query_context = nullptr; thread_group.reset(); From 4d349e9ed0f24ec8335501cbce0c1d180f59aa19 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 23 Dec 2019 00:01:12 +0300 Subject: [PATCH 17/53] Added TODO --- dbms/src/Storages/System/StorageSystemStackTrace.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Storages/System/StorageSystemStackTrace.cpp b/dbms/src/Storages/System/StorageSystemStackTrace.cpp index 8a932a3eb7c..284c4241f48 100644 --- a/dbms/src/Storages/System/StorageSystemStackTrace.cpp +++ b/dbms/src/Storages/System/StorageSystemStackTrace.cpp @@ -169,6 +169,7 @@ void StorageSystemStackTrace::fillData(MutableColumns & res_columns, const Conte } /// Just in case we will wait for pipe with timeout. In case signal didn't get processed. + /// TODO How to deal with stale values in a pipe? TSan will also argue. if (wait(100)) { From 68305165d1c38cd063f6bc22ce8d8644bb25323a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 23 Dec 2019 12:25:58 +0300 Subject: [PATCH 18/53] Create query context for clickhouse dictionary source. --- dbms/src/Dictionaries/ClickHouseDictionarySource.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp b/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp index 027367e3847..3bf03ec28b1 100644 --- a/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -76,6 +76,9 @@ ClickHouseDictionarySource::ClickHouseDictionarySource( context.setUser(user, password, Poco::Net::SocketAddress("127.0.0.1", 0), {}); /// Processors are not supported here yet. context.getSettingsRef().experimental_use_processors = false; + /// Query context is needed because some code in executeQuery function may assume it exists. + /// Current example is Context::getSampleBlockCache from InterpreterSelectWithUnionQuery::getSampleBlock. + context.makeQueryContext(); } @@ -100,6 +103,7 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(const ClickHouseDictionar , pool{is_local ? nullptr : createPool(host, port, secure, db, user, password)} , load_all_query{other.load_all_query} { + context.makeQueryContext(); } std::string ClickHouseDictionarySource::getUpdateFieldAndDate() From 4f641b05d44016dd4554eec7e4ec2807a17bac5e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 23 Dec 2019 12:28:09 +0300 Subject: [PATCH 19/53] Added test. --- ...lickhouse_dict_source_with_subquery.reference | 1 + ...1050_clickhouse_dict_source_with_subquery.sql | 16 ++++++++++++++++ 2 files changed, 17 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/01050_clickhouse_dict_source_with_subquery.reference create mode 100644 dbms/tests/queries/0_stateless/01050_clickhouse_dict_source_with_subquery.sql diff --git a/dbms/tests/queries/0_stateless/01050_clickhouse_dict_source_with_subquery.reference b/dbms/tests/queries/0_stateless/01050_clickhouse_dict_source_with_subquery.reference new file mode 100644 index 00000000000..b7e2bb8cec1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01050_clickhouse_dict_source_with_subquery.reference @@ -0,0 +1 @@ +a_y_x diff --git a/dbms/tests/queries/0_stateless/01050_clickhouse_dict_source_with_subquery.sql b/dbms/tests/queries/0_stateless/01050_clickhouse_dict_source_with_subquery.sql new file mode 100644 index 00000000000..e3d360065c6 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01050_clickhouse_dict_source_with_subquery.sql @@ -0,0 +1,16 @@ +drop table if exists default.test_01051_d; +drop table if exists default.test_view_01051_d; +drop dictionary if exists default.test_dict_01051_d; + +create table default.test_01051_d (key UInt64, value String) engine = MergeTree order by key; +create view default.test_view_01051_d (key UInt64, value String) as select k2 + 1 as key, v2 || '_x' as value from (select key + 2 as k2, value || '_y' as v2 from default.test_01051_d); + +insert into default.test_01051_d values (1, 'a'); + +create dictionary default.test_dict_01051_d (key UInt64, value String) primary key key source(clickhouse(host 'localhost' port '9000' user 'default' password '' db 'default' table 'test_view_01051_d')) layout(flat()) lifetime(100500); + +select dictGet('default.test_dict_01051_d', 'value', toUInt64(4)); + +drop table if exists default.test_01051_d; +drop table if exists default.test_view_01051_d; +drop dictionary if exists default.test_dict_01051_d; From 73c9f0e71cdc812b85b9d307398adb91faf2116c Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 23 Dec 2019 17:46:00 +0300 Subject: [PATCH 20/53] Remove redundant code --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 1bdd506dee9..a931801dd0f 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -4592,11 +4592,6 @@ void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, const /// we have to wait if (query_context.getSettingsRef().mutations_sync != 0) { - auto check_callback = [mutation_number = entry.znode_name](const String & zk_value) - { - return zk_value >= mutation_number; - }; - Strings replicas; if (query_context.getSettingsRef().mutations_sync == 2) /// wait for all replicas replicas = getZooKeeper()->getChildren(zookeeper_path + "/replicas"); From 593ec1fd769a887783eb28b2789bad5edb5450e7 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Mon, 23 Dec 2019 19:29:42 +0300 Subject: [PATCH 21/53] Remove outdated stuff from en roadmap --- docs/en/roadmap.md | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/docs/en/roadmap.md b/docs/en/roadmap.md index 11f1f793235..f8cbf1c17c0 100644 --- a/docs/en/roadmap.md +++ b/docs/en/roadmap.md @@ -1,14 +1,7 @@ # Roadmap -## Q3 2019 +## Q1 2020 -- DDL for dictionaries -- Integration with S3-like object stores -- Multiple storages for hot/cold data, JBOD support - -## Q4 2019 - -- JOIN not limited by available memory - Resource pools for more precise distribution of cluster capacity between users - Fine-grained authorization - Integration with external authentication services From 25ecc386154cc87703c6014bb44e3a42eff6b507 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 23 Dec 2019 19:44:50 +0300 Subject: [PATCH 22/53] More correct separation between metadata and data alters for non replicated MergeTree. --- dbms/src/Storages/AlterCommands.cpp | 27 +++-- dbms/src/Storages/AlterCommands.h | 11 ++- dbms/src/Storages/IStorage.cpp | 2 +- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 20 ++-- dbms/src/Storages/StorageMergeTree.cpp | 98 ++++++++++--------- .../Storages/StorageReplicatedMergeTree.cpp | 4 + 6 files changed, 86 insertions(+), 76 deletions(-) diff --git a/dbms/src/Storages/AlterCommands.cpp b/dbms/src/Storages/AlterCommands.cpp index bcfd852a628..b24863a2afb 100644 --- a/dbms/src/Storages/AlterCommands.cpp +++ b/dbms/src/Storages/AlterCommands.cpp @@ -109,7 +109,7 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ if (ast_col_decl.comment) { const auto & ast_comment = ast_col_decl.comment->as(); - command.comment = ast_comment.value.get(); + command.comment.emplace(ast_comment.value.get()); } if (ast_col_decl.ttl) @@ -225,7 +225,7 @@ void AlterCommand::apply(ColumnsDescription & columns_description, IndicesDescri column.default_desc.kind = default_kind; column.default_desc.expression = default_expression; } - column.comment = comment; + column.comment = *comment; column.codec = codec; column.ttl = ttl; @@ -251,11 +251,8 @@ void AlterCommand::apply(ColumnsDescription & columns_description, IndicesDescri column.codec = codec; } - if (!isMutable()) - { - column.comment = comment; - return; - } + if (comment) + column.comment = *comment; if (ttl) column.ttl = ttl; @@ -279,7 +276,7 @@ void AlterCommand::apply(ColumnsDescription & columns_description, IndicesDescri } else if (type == COMMENT_COLUMN) { - columns_description.modify(column_name, [&](ColumnDescription & column) { column.comment = comment; }); + columns_description.modify(column_name, [&](ColumnDescription & column) { column.comment = *comment; }); } else if (type == ADD_INDEX) { @@ -390,13 +387,15 @@ void AlterCommand::apply(ColumnsDescription & columns_description, IndicesDescri throw Exception("Wrong parameter type in ALTER query", ErrorCodes::LOGICAL_ERROR); } -bool AlterCommand::isMutable() const +bool AlterCommand::isModifyingData() const { - if (type == COMMENT_COLUMN || type == MODIFY_SETTING) - return false; + /// Change binary representation on disk if (type == MODIFY_COLUMN) return data_type.get() || default_expression; - return true; + + return type == ADD_COLUMN /// We need to change columns.txt in each part + || type == DROP_COLUMN /// We need to change columns.txt in each part + || type == DROP_INDEX; /// We need to remove file from filesystem } bool AlterCommand::isSettingsAlter() const @@ -666,11 +665,11 @@ void AlterCommands::applyForSettingsOnly(SettingsChanges & changes) const changes = std::move(out_changes); } -bool AlterCommands::isMutable() const +bool AlterCommands::isModifyingData() const { for (const auto & param : *this) { - if (param.isMutable()) + if (param.isModifyingData()) return true; } diff --git a/dbms/src/Storages/AlterCommands.h b/dbms/src/Storages/AlterCommands.h index 67fc166067b..1217d96dc29 100644 --- a/dbms/src/Storages/AlterCommands.h +++ b/dbms/src/Storages/AlterCommands.h @@ -47,7 +47,7 @@ struct AlterCommand ColumnDefaultKind default_kind{}; ASTPtr default_expression{}; - String comment; + std::optional comment; /// For ADD - after which column to add a new one. If an empty string, add to the end. To add to the beginning now it is impossible. String after_column; @@ -102,8 +102,11 @@ struct AlterCommand ConstraintsDescription & constraints_description, ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast, SettingsChanges & changes) const; - /// Checks that not only metadata touched by that command - bool isMutable() const; + /// Checks that alter query changes data. For MergeTree: + /// * column files (data and marks) + /// * each part meta (columns.txt) + /// in each part on disk (it's not lightweight alter). + bool isModifyingData() const; /// checks that only settings changed by alter bool isSettingsAlter() const; @@ -124,7 +127,7 @@ public: void applyForSettingsOnly(SettingsChanges & changes) const; void validate(const IStorage & table, const Context & context); - bool isMutable() const; + bool isModifyingData() const; bool isSettingsAlter() const; }; diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index f5c34587fb2..169117f7b44 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -402,7 +402,7 @@ void IStorage::alter( const Context & context, TableStructureWriteLockHolder & table_lock_holder) { - if (params.isMutable()) + if (params.isModifyingData()) throw Exception("Method alter supports only change comment of column for storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); const String database_name = getDatabaseName(); diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 2039c71b04b..8d892fd69d6 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1386,7 +1386,7 @@ void MergeTreeData::checkAlter(const AlterCommands & commands, const Context & c for (const AlterCommand & command : commands) { - if (!command.isMutable()) + if (!command.isModifyingData()) { continue; } @@ -1433,9 +1433,9 @@ void MergeTreeData::checkAlter(const AlterCommands & commands, const Context & c getIndices().indices, new_indices.indices, unused_expression, unused_map, unused_bool); } -void MergeTreeData::createConvertExpression(const DataPartPtr & part, const NamesAndTypesList & old_columns, const NamesAndTypesList & new_columns, - const IndicesASTs & old_indices, const IndicesASTs & new_indices, ExpressionActionsPtr & out_expression, - NameToNameMap & out_rename_map, bool & out_force_update_metadata) const +void MergeTreeData::createConvertExpression(const DataPartPtr & part, const NamesAndTypesList & old_columns, + const NamesAndTypesList & new_columns, const IndicesASTs & old_indices, const IndicesASTs & new_indices, + ExpressionActionsPtr & out_expression, NameToNameMap & out_rename_map, bool & out_force_update_metadata) const { const auto settings = getSettings(); out_expression = nullptr; @@ -1457,7 +1457,7 @@ void MergeTreeData::createConvertExpression(const DataPartPtr & part, const Name /// Remove old indices - std::set new_indices_set; + std::unordered_set new_indices_set; for (const auto & index_decl : new_indices) new_indices_set.emplace(index_decl->as().name); for (const auto & index_decl : old_indices) @@ -1465,8 +1465,8 @@ void MergeTreeData::createConvertExpression(const DataPartPtr & part, const Name const auto & index = index_decl->as(); if (!new_indices_set.count(index.name)) { - out_rename_map["skp_idx_" + index.name + ".idx"] = ""; - out_rename_map["skp_idx_" + index.name + part_mrk_file_extension] = ""; + out_rename_map["skp_idx_" + index.name + ".idx"] = ""; /// drop this file + out_rename_map["skp_idx_" + index.name + part_mrk_file_extension] = ""; /// and this one } } @@ -1494,8 +1494,8 @@ void MergeTreeData::createConvertExpression(const DataPartPtr & part, const Name /// Delete files if they are no longer shared with another column. if (--stream_counts[file_name] == 0) { - out_rename_map[file_name + ".bin"] = ""; - out_rename_map[file_name + part_mrk_file_extension] = ""; + out_rename_map[file_name + ".bin"] = ""; /// drop this file + out_rename_map[file_name + part_mrk_file_extension] = ""; /// and this one } }, {}); } @@ -1847,7 +1847,7 @@ void MergeTreeData::AlterDataPartTransaction::commit() mutable_part.checksums = new_checksums; mutable_part.columns = new_columns; - /// 3) Delete the old files. + /// 3) Delete the old files and drop required columns (DROP COLUMN) for (const auto & from_to : rename_map) { String name = from_to.second.empty() ? from_to.first : from_to.second; diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 41c9335de1e..e6a847ba1bd 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -249,34 +249,10 @@ void StorageMergeTree::alter( const String current_database_name = getDatabaseName(); const String current_table_name = getTableName(); - if (!params.isMutable()) - { - lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); - auto new_columns = getColumns(); - auto new_indices = getIndices(); - auto new_constraints = getConstraints(); - ASTPtr new_order_by_ast = order_by_ast; - ASTPtr new_primary_key_ast = primary_key_ast; - ASTPtr new_ttl_table_ast = ttl_table_ast; - SettingsChanges new_changes; - - params.apply(new_columns, new_indices, new_constraints, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast, new_changes); - - changeSettings(new_changes, table_lock_holder); - - IDatabase::ASTModifier settings_modifier = getSettingsModifier(new_changes); - context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, settings_modifier); - setColumns(std::move(new_columns)); - return; - } - - /// NOTE: Here, as in ReplicatedMergeTree, you can do ALTER which does not block the writing of data for a long time. - /// Also block moves, because they can replace part with old state - auto merge_blocker = merger_mutator.merges_blocker.cancel(); - auto moves_blocked = parts_mover.moves_blocker.cancel(); - lockNewDataStructureExclusively(table_lock_holder, context.getCurrentQueryId()); + checkAlter(params, context); + auto new_columns = getColumns(); auto new_indices = getIndices(); auto new_constraints = getConstraints(); @@ -284,13 +260,11 @@ void StorageMergeTree::alter( ASTPtr new_primary_key_ast = primary_key_ast; ASTPtr new_ttl_table_ast = ttl_table_ast; SettingsChanges new_changes; + params.apply(new_columns, new_indices, new_constraints, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast, new_changes); - auto transactions = prepareAlterTransactions(new_columns, new_indices, context); - - lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); - - IDatabase::ASTModifier storage_modifier = [&] (IAST & ast) + /// Modifier for storage AST in /metadata/storage_db/storage.sql + IDatabase::ASTModifier storage_modifier = [&](IAST & ast) { auto & storage_ast = ast.as(); @@ -310,24 +284,54 @@ void StorageMergeTree::alter( } }; - changeSettings(new_changes, table_lock_holder); - - context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, storage_modifier); - - - /// Reinitialize primary key because primary key column types might have changed. - setProperties(new_order_by_ast, new_primary_key_ast, new_columns, new_indices, new_constraints); - - setTTLExpressions(new_columns.getColumnTTLs(), new_ttl_table_ast); - - for (auto & transaction : transactions) + /// Update metdata in memory + auto update_metadata = [&]() { - transaction->commit(); - transaction.reset(); - } + changeSettings(new_changes, table_lock_holder); + /// Reinitialize primary key because primary key column types might have changed. + setProperties(new_order_by_ast, new_primary_key_ast, new_columns, new_indices, new_constraints); - /// Columns sizes could be changed - recalculateColumnSizes(); + setTTLExpressions(new_columns.getColumnTTLs(), new_ttl_table_ast); + }; + + /// This alter can be performed at metadata level only + if (!params.isModifyingData()) + { + lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); + + params.apply(new_columns, new_indices, new_constraints, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast, new_changes); + + IDatabase::ASTModifier settings_modifier = getSettingsModifier(new_changes); + context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, storage_modifier); + + update_metadata(); + } + else + { + + /// NOTE: Here, as in ReplicatedMergeTree, you can do ALTER which does not block the writing of data for a long time. + /// Also block moves, because they can replace part with old state + auto merge_blocker = merger_mutator.merges_blocker.cancel(); + auto moves_blocked = parts_mover.moves_blocker.cancel(); + + + auto transactions = prepareAlterTransactions(new_columns, new_indices, context); + + lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); + + context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, storage_modifier); + + update_metadata(); + + for (auto & transaction : transactions) + { + transaction->commit(); + transaction.reset(); + } + + /// Columns sizes could be changed + recalculateColumnSizes(); + } } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index b3f69d463f2..ba4f14bd1fc 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3158,6 +3158,10 @@ void StorageReplicatedMergeTree::alter( const String current_database_name = getDatabaseName(); const String current_table_name = getTableName(); + /// We cannot check this alter commands with method isModifyingData() + /// because ReplicatedMergeTree stores both columns and metadata for + /// each replica. So we have to wait AlterThread even with lightweight + /// metadata alter. if (params.isSettingsAlter()) { /// We don't replicate storage_settings_ptr ALTER. It's local operation. From 56870c6b3b03bd5b926f06f07862f8a17c13ff3d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 23 Dec 2019 19:49:06 +0300 Subject: [PATCH 23/53] Fixed -Wshadow --- dbms/src/Storages/System/StorageSystemStackTrace.cpp | 4 ++-- dbms/src/Storages/System/StorageSystemStackTrace.h | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Storages/System/StorageSystemStackTrace.cpp b/dbms/src/Storages/System/StorageSystemStackTrace.cpp index 284c4241f48..b9c299cffaa 100644 --- a/dbms/src/Storages/System/StorageSystemStackTrace.cpp +++ b/dbms/src/Storages/System/StorageSystemStackTrace.cpp @@ -106,8 +106,8 @@ namespace } -StorageSystemStackTrace::StorageSystemStackTrace(const String & name) - : IStorageSystemOneBlock(name) +StorageSystemStackTrace::StorageSystemStackTrace(const String & name_) + : IStorageSystemOneBlock(name_) { notification_pipe.open(); diff --git a/dbms/src/Storages/System/StorageSystemStackTrace.h b/dbms/src/Storages/System/StorageSystemStackTrace.h index 161e7f2e2fb..249ceebfd8c 100644 --- a/dbms/src/Storages/System/StorageSystemStackTrace.h +++ b/dbms/src/Storages/System/StorageSystemStackTrace.h @@ -20,7 +20,7 @@ public: String getName() const override { return "SystemStackTrace"; } static NamesAndTypesList getNamesAndTypes(); - StorageSystemStackTrace(const String & name); + StorageSystemStackTrace(const String & name_); protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; From f04a2a5f7bb17f16dd8a634a5ffa042395945765 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 23 Dec 2019 19:54:51 +0300 Subject: [PATCH 24/53] Fixed Darwin build --- dbms/src/Common/PipeFDs.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/src/Common/PipeFDs.cpp b/dbms/src/Common/PipeFDs.cpp index 463897f2c08..17eeb9aaef7 100644 --- a/dbms/src/Common/PipeFDs.cpp +++ b/dbms/src/Common/PipeFDs.cpp @@ -104,6 +104,8 @@ void LazyPipeFDs::tryIncreaseSize(int desired_size) LOG_TRACE(log, "Pipe capacity is " << formatReadableSizeWithBinarySuffix(std::min(pipe_size, desired_size))); } +#else + (void)desired_size; #endif } From be963a7a6f183d7027771c1b58bd5a6918c1c216 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 23 Dec 2019 21:24:33 +0300 Subject: [PATCH 25/53] Add a build package type for performance tests. --- docker/packager/binary/build.sh | 8 ++++---- docker/packager/packager | 28 ++++++++++++++++++---------- 2 files changed, 22 insertions(+), 14 deletions(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 42d7ebcbdf1..7d6b7a518d4 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -21,14 +21,14 @@ mv ./dbms/unit_tests_dbms /output find . -name '*.so' -print -exec mv '{}' /output \; find . -name '*.so.*' -print -exec mv '{}' /output \; -count=`ls -1 /output/*.so 2>/dev/null | wc -l` -if [ $count != 0 ] +# May be set for split build or for performance test. +if [ "" != "$COMBINED_OUTPUT" ] then mkdir -p /output/config cp ../dbms/programs/server/config.xml /output/config cp ../dbms/programs/server/users.xml /output/config cp -r ../dbms/programs/server/config.d /output/config - tar -czvf shared_build.tgz /output + tar -czvf "$COMBINED_OUTPUT.tgz" /output rm -r /output/* - mv shared_build.tgz /output + mv "$COMBINED_OUTPUT.tgz" /output fi diff --git a/docker/packager/packager b/docker/packager/packager index 9cdba69b3ea..8e385786c5f 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -103,7 +103,7 @@ def run_vagrant_box_with_env(image_path, output_dir, ch_root): logging.info("Copying binary back") vagrant.copy_from_image("~/ClickHouse/dbms/programs/clickhouse", output_dir) -def parse_env_variables(build_type, compiler, sanitizer, package_type, cache, distcc_hosts, unbundled, split_binary, version, author, official, alien_pkgs, with_coverage): +def parse_env_variables(build_type, compiler, sanitizer, package_type, image_type, cache, distcc_hosts, unbundled, split_binary, version, author, official, alien_pkgs, with_coverage): CLANG_PREFIX = "clang" DARWIN_SUFFIX = "-darwin" ARM_SUFFIX = "-aarch64" @@ -135,15 +135,21 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, cache, di cxx = cc.replace('gcc', 'g++').replace('clang', 'clang++') - if package_type == "deb": + if image_type == "deb": result.append("DEB_CC={}".format(cc)) result.append("DEB_CXX={}".format(cxx)) - elif package_type == "binary": + elif image_type == "binary": result.append("CC={}".format(cc)) result.append("CXX={}".format(cxx)) cmake_flags.append('-DCMAKE_C_COMPILER=`which {}`'.format(cc)) cmake_flags.append('-DCMAKE_CXX_COMPILER=`which {}`'.format(cxx)) + # Create combined output archive for split build and for performance tests. + if package_type == "performance": + result.append("COMBINED_OUTPUT=performance") + elif split_binary: + result.append("COMBINED_OUTPUT=shared_build") + if sanitizer: result.append("SANITIZER={}".format(sanitizer)) if build_type: @@ -193,7 +199,8 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, cache, di if __name__ == "__main__": logging.basicConfig(level=logging.INFO, format='%(asctime)s %(message)s') parser = argparse.ArgumentParser(description="ClickHouse building script using prebuilt Docker image") - parser.add_argument("--package-type", choices=IMAGE_MAP.keys(), required=True) + # 'performance' creates a combined .tgz with server and configs to be used for performance test. + parser.add_argument("--package-type", choices=['deb', 'binary', 'performance', 'freebsd'], required=True) parser.add_argument("--clickhouse-repo-path", default="../../") parser.add_argument("--output-dir", required=True) parser.add_argument("--build-type", choices=("debug", ""), default="") @@ -215,25 +222,26 @@ if __name__ == "__main__": if not os.path.isabs(args.output_dir): args.output_dir = os.path.abspath(os.path.join(os.getcwd(), args.output_dir)) - image_name = IMAGE_MAP[args.package_type] + image_type = 'binary' if args.package_type == 'performance' else args.package_type + image_name = IMAGE_MAP[image_type] if not os.path.isabs(args.clickhouse_repo_path): ch_root = os.path.abspath(os.path.join(os.getcwd(), args.clickhouse_repo_path)) else: ch_root = args.clickhouse_repo_path - if args.alien_pkgs and not args.package_type == "deb": + if args.alien_pkgs and not image_type == "deb": raise Exception("Can add alien packages only in deb build") - dockerfile = os.path.join(ch_root, "docker/packager", args.package_type, "Dockerfile") - if args.package_type != "freebsd" and not check_image_exists_locally(image_name) or args.force_build_image: + dockerfile = os.path.join(ch_root, "docker/packager", image_type, "Dockerfile") + if image_type != "freebsd" and not check_image_exists_locally(image_name) or args.force_build_image: if not pull_image(image_name) or args.force_build_image: build_image(image_name, dockerfile) env_prepared = parse_env_variables( - args.build_type, args.compiler, args.sanitizer, args.package_type, + args.build_type, args.compiler, args.sanitizer, args.package_type, image_type, args.cache, args.distcc_hosts, args.unbundled, args.split_binary, args.version, args.author, args.official, args.alien_pkgs, args.with_coverage) - if args.package_type != "freebsd": + if image_type != "freebsd": run_docker_image_with_env(image_name, args.output_dir, env_prepared, ch_root, args.ccache_dir) else: logging.info("Running freebsd build, arguments will be ignored") From 9da4b63fe953392fb004b73b93e54b58e2f6fe03 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 23 Dec 2019 21:56:57 +0300 Subject: [PATCH 26/53] Fixed Darwin build --- dbms/src/Storages/System/StorageSystemStackTrace.cpp | 3 +++ dbms/src/Storages/System/StorageSystemStackTrace.h | 3 +++ dbms/src/Storages/System/attachSystemTables.cpp | 5 +++++ 3 files changed, 11 insertions(+) diff --git a/dbms/src/Storages/System/StorageSystemStackTrace.cpp b/dbms/src/Storages/System/StorageSystemStackTrace.cpp index b9c299cffaa..24426eabca7 100644 --- a/dbms/src/Storages/System/StorageSystemStackTrace.cpp +++ b/dbms/src/Storages/System/StorageSystemStackTrace.cpp @@ -1,3 +1,5 @@ +#ifdef OS_LINUX /// Because of 'sigqueue' functions and RT signals. + #include #include @@ -198,3 +200,4 @@ void StorageSystemStackTrace::fillData(MutableColumns & res_columns, const Conte } +#endif diff --git a/dbms/src/Storages/System/StorageSystemStackTrace.h b/dbms/src/Storages/System/StorageSystemStackTrace.h index 249ceebfd8c..79185ca805a 100644 --- a/dbms/src/Storages/System/StorageSystemStackTrace.h +++ b/dbms/src/Storages/System/StorageSystemStackTrace.h @@ -1,5 +1,7 @@ #pragma once +#ifdef OS_LINUX /// Because of 'sigqueue' functions and RT signals. + #include #include #include @@ -31,3 +33,4 @@ protected: } +#endif diff --git a/dbms/src/Storages/System/attachSystemTables.cpp b/dbms/src/Storages/System/attachSystemTables.cpp index cd224353acb..fc3a36d7c2c 100644 --- a/dbms/src/Storages/System/attachSystemTables.cpp +++ b/dbms/src/Storages/System/attachSystemTables.cpp @@ -38,7 +38,10 @@ #include #include #include + +#ifdef OS_LINUX #include +#endif namespace DB @@ -66,7 +69,9 @@ void attachSystemTablesLocal(IDatabase & system_database) system_database.attachTable("collations", StorageSystemCollations::create("collations")); system_database.attachTable("table_engines", StorageSystemTableEngines::create("table_engines")); system_database.attachTable("contributors", StorageSystemContributors::create("contributors")); +#ifdef OS_LINUX system_database.attachTable("stack_trace", StorageSystemStackTrace::create("stack_trace")); +#endif } void attachSystemTablesServer(IDatabase & system_database, bool has_zookeeper) From e907ce103eb5aa720afd2bcf39391641b5ef98c7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 23 Dec 2019 21:58:42 +0300 Subject: [PATCH 27/53] Fixed warning --- dbms/src/Storages/System/StorageSystemStackTrace.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/System/StorageSystemStackTrace.cpp b/dbms/src/Storages/System/StorageSystemStackTrace.cpp index 24426eabca7..b964ab0f51d 100644 --- a/dbms/src/Storages/System/StorageSystemStackTrace.cpp +++ b/dbms/src/Storages/System/StorageSystemStackTrace.cpp @@ -61,8 +61,10 @@ namespace memcpy(query_id_data, query_id.data, query_id_size); char buf = 0; + ssize_t res = ::write(notification_pipe.fds_rw[1], &buf, 1); + /// We cannot do anything if write failed. - (void)::write(notification_pipe.fds_rw[1], &buf, 1); + (void)res; } /// Wait for data in pipe and read it. From 3b7f3b07cd2f59346857d28937b96fa19a0f934a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 23 Dec 2019 22:23:54 +0300 Subject: [PATCH 28/53] Better handling of signals --- .../System/StorageSystemStackTrace.cpp | 32 +++++++++++++------ .../Storages/System/StorageSystemStackTrace.h | 1 + 2 files changed, 24 insertions(+), 9 deletions(-) diff --git a/dbms/src/Storages/System/StorageSystemStackTrace.cpp b/dbms/src/Storages/System/StorageSystemStackTrace.cpp index b964ab0f51d..20767464038 100644 --- a/dbms/src/Storages/System/StorageSystemStackTrace.cpp +++ b/dbms/src/Storages/System/StorageSystemStackTrace.cpp @@ -35,6 +35,8 @@ namespace const pid_t expected_pid = getpid(); const int sig = SIGRTMIN; + int sequence_num = 0; /// For messages sent via pipe. + UInt32 thread_number{0}; std::optional stack_trace; @@ -51,6 +53,10 @@ namespace if (info->si_pid != expected_pid) return; + /// Signal received too late. + if (info->si_value.sival_int != sequence_num) + return; + /// All these methods are signal-safe. const ucontext_t signal_context = *reinterpret_cast(context); stack_trace.emplace(signal_context); @@ -60,8 +66,8 @@ namespace query_id_size = std::min(query_id.size, max_query_id_size); memcpy(query_id_data, query_id.data, query_id_size); - char buf = 0; - ssize_t res = ::write(notification_pipe.fds_rw[1], &buf, 1); + int notification_num = info->si_value.sival_int; + ssize_t res = ::write(notification_pipe.fds_rw[1], ¬ification_num, sizeof(notification_num)); /// We cannot do anything if write failed. (void)res; @@ -91,10 +97,8 @@ namespace if (poll_res == 0) return false; - char buf = 0; - ssize_t read_res = ::read(fd, &buf, 1); - if (read_res == 1) - return true; + int notification_num = 0; + ssize_t read_res = ::read(fd, ¬ification_num, sizeof(notification_num)); if (read_res < 0) { @@ -104,7 +108,15 @@ namespace throwFromErrno("Cannot read from pipe", ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR); } - throw Exception("Logical error: read for one byte returned more than one byte", ErrorCodes::LOGICAL_ERROR); + if (read_res == sizeof(notification_num)) + { + if (notification_num == sequence_num) + return true; + else + continue; /// Drain delayed notifications. + } + + throw Exception("Logical error: read wrong number of bytes from pipe", ErrorCodes::LOGICAL_ERROR); } } } @@ -160,9 +172,10 @@ void StorageSystemStackTrace::fillData(MutableColumns & res_columns, const Conte std::filesystem::directory_iterator end; for (std::filesystem::directory_iterator it("/proc/self/task"); it != end; ++it) { - sigval sig_value{}; pid_t tid = parse(it->path().filename()); + sigval sig_value{}; + sig_value.sival_int = sequence_num; if (0 != ::sigqueue(tid, sig, sig_value)) { /// The thread may has been already finished. @@ -173,7 +186,6 @@ void StorageSystemStackTrace::fillData(MutableColumns & res_columns, const Conte } /// Just in case we will wait for pipe with timeout. In case signal didn't get processed. - /// TODO How to deal with stale values in a pipe? TSan will also argue. if (wait(100)) { @@ -197,6 +209,8 @@ void StorageSystemStackTrace::fillData(MutableColumns & res_columns, const Conte res_columns[1]->insertDefault(); res_columns[2]->insertDefault(); } + + sequence_num = static_cast(static_cast(sequence_num) + 1); } } diff --git a/dbms/src/Storages/System/StorageSystemStackTrace.h b/dbms/src/Storages/System/StorageSystemStackTrace.h index 79185ca805a..4961d786f59 100644 --- a/dbms/src/Storages/System/StorageSystemStackTrace.h +++ b/dbms/src/Storages/System/StorageSystemStackTrace.h @@ -15,6 +15,7 @@ class Context; /// Allows to introspect stack trace of all server threads. /// It acts like an embedded debugger. +/// More than one instance of this table cannot be used. class StorageSystemStackTrace : public ext::shared_ptr_helper, public IStorageSystemOneBlock { friend struct ext::shared_ptr_helper; From 9d631c2c779b06c552864750ab1209b2a4e4cc54 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 23 Dec 2019 23:19:49 +0300 Subject: [PATCH 29/53] Add query_id to crash messages --- libs/libdaemon/src/BaseDaemon.cpp | 41 +++++++++++++++++++++++++++---- 1 file changed, 36 insertions(+), 5 deletions(-) diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index ce3eb2cd787..1c0f86db600 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -72,7 +72,15 @@ static void call_default_signal_handler(int sig) } -static const size_t buf_size = sizeof(int) + sizeof(siginfo_t) + sizeof(ucontext_t) + sizeof(StackTrace) + sizeof(UInt32); +static constexpr size_t max_query_id_size = 127; + +static const size_t buf_size = + sizeof(int) + + sizeof(siginfo_t) + + sizeof(ucontext_t) + + sizeof(StackTrace) + + sizeof(UInt32) + + max_query_id_size + 1; /// query_id + varint encoded length using signal_function = void(int, siginfo_t*, void*); @@ -107,11 +115,15 @@ static void signalHandler(int sig, siginfo_t * info, void * context) const ucontext_t signal_context = *reinterpret_cast(context); const StackTrace stack_trace(signal_context); + StringRef query_id = CurrentThread::getQueryId(); /// This is signal safe. + query_id.size = std::min(query_id.size, max_query_id_size); + DB::writeBinary(sig, out); DB::writePODBinary(*info, out); DB::writePODBinary(signal_context, out); DB::writePODBinary(stack_trace, out); DB::writeBinary(UInt32(getThreadNumber()), out); + DB::writeStringBinary(query_id, out); out.next(); @@ -187,15 +199,17 @@ public: ucontext_t context; StackTrace stack_trace(NoCapture{}); UInt32 thread_num; + std::string query_id; DB::readPODBinary(info, in); DB::readPODBinary(context, in); DB::readPODBinary(stack_trace, in); DB::readBinary(thread_num, in); + DB::readBinary(query_id, in); /// This allows to receive more signals if failure happens inside onFault function. /// Example: segfault while symbolizing stack trace. - std::thread([=] { onFault(sig, info, context, stack_trace, thread_num); }).detach(); + std::thread([=] { onFault(sig, info, context, stack_trace, thread_num, query_id); }).detach(); } } } @@ -210,11 +224,28 @@ private: LOG_FATAL(log, "(version " << VERSION_STRING << VERSION_OFFICIAL << ") (from thread " << thread_num << ") " << message); } - void onFault(int sig, const siginfo_t & info, const ucontext_t & context, const StackTrace & stack_trace, UInt32 thread_num) const + void onFault( + int sig, + const siginfo_t & info, + const ucontext_t & context, + const StackTrace & stack_trace, + UInt32 thread_num, + const std::string & query_id) const { LOG_FATAL(log, "########################################"); - LOG_FATAL(log, "(version " << VERSION_STRING << VERSION_OFFICIAL << ") (from thread " << thread_num << ") " - << "Received signal " << strsignal(sig) << " (" << sig << ")" << "."); + + { + std::stringstream message; + message << "(version " << VERSION_STRING << VERSION_OFFICIAL << ")"; + message << " (from thread " << thread_num << ")"; + if (query_id.empty()) + message << " (no query)"; + else + message << " (query_id: " << query_id << ")"; + message << "Received signal " << strsignal(sig) << " (" << sig << ")" << "."; + + LOG_FATAL(log, message.rdbuf()); + } LOG_FATAL(log, signalToErrorMessage(sig, info, context)); From f7d9ada51cabadd7a9efee214807184e0911cb25 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 23 Dec 2019 23:26:11 +0300 Subject: [PATCH 30/53] Addition to prev. revision --- libs/libdaemon/src/BaseDaemon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index 1c0f86db600..70cc7157344 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -242,7 +242,7 @@ private: message << " (no query)"; else message << " (query_id: " << query_id << ")"; - message << "Received signal " << strsignal(sig) << " (" << sig << ")" << "."; + message << " Received signal " << strsignal(sig) << " (" << sig << ")" << "."; LOG_FATAL(log, message.rdbuf()); } From badbee23856651c21a632dac0a58989db1308e5d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 23 Dec 2019 23:26:46 +0300 Subject: [PATCH 31/53] Added another mode of diagnostic trap --- dbms/src/Functions/trap.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dbms/src/Functions/trap.cpp b/dbms/src/Functions/trap.cpp index 217b7091dc1..9176a8656af 100644 --- a/dbms/src/Functions/trap.cpp +++ b/dbms/src/Functions/trap.cpp @@ -83,6 +83,10 @@ public: { abort(); } + else if (mode == "std::terminate") + { + std::terminate(); + } else if (mode == "use after free") { int * x_ptr; From bdefa9248c2beeb4527cc2e1bcc0948824bf53cb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 24 Dec 2019 03:54:32 +0300 Subject: [PATCH 32/53] Removed test because it cannot run in Sandbox (CI) --- .../queries/0_stateless/01051_system_stack_trace.reference | 1 - dbms/tests/queries/0_stateless/01051_system_stack_trace.sql | 2 -- 2 files changed, 3 deletions(-) delete mode 100644 dbms/tests/queries/0_stateless/01051_system_stack_trace.reference delete mode 100644 dbms/tests/queries/0_stateless/01051_system_stack_trace.sql diff --git a/dbms/tests/queries/0_stateless/01051_system_stack_trace.reference b/dbms/tests/queries/0_stateless/01051_system_stack_trace.reference deleted file mode 100644 index d00491fd7e5..00000000000 --- a/dbms/tests/queries/0_stateless/01051_system_stack_trace.reference +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/dbms/tests/queries/0_stateless/01051_system_stack_trace.sql b/dbms/tests/queries/0_stateless/01051_system_stack_trace.sql deleted file mode 100644 index 32d344fce7e..00000000000 --- a/dbms/tests/queries/0_stateless/01051_system_stack_trace.sql +++ /dev/null @@ -1,2 +0,0 @@ --- at least this query should be present -SELECT count() > 0 FROM system.stack_trace WHERE query_id != ''; From 69186423895cff6ffb4fa85d284e93f698bce889 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 24 Dec 2019 11:59:30 +0300 Subject: [PATCH 33/53] consistent Ok. in http docs --- docs/en/interfaces/http.md | 2 +- docs/fa/interfaces/http.md | 4 ++-- docs/zh/interfaces/http.md | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index 25a146f78b3..cecb4e79a15 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -3,7 +3,7 @@ The HTTP interface lets you use ClickHouse on any platform from any programming language. We use it for working from Java and Perl, as well as shell scripts. In other departments, the HTTP interface is used from Perl, Python, and Go. The HTTP interface is more limited than the native interface, but it has better compatibility. By default, clickhouse-server listens for HTTP on port 8123 (this can be changed in the config). -If you make a GET / request without parameters, it returns the string "Ok" (with a line feed at the end). You can use this in health-check scripts. +If you make a GET / request without parameters, it returns the string "Ok." (with a line feed at the end). You can use this in health-check scripts. ```bash $ curl 'http://localhost:8123/' diff --git a/docs/fa/interfaces/http.md b/docs/fa/interfaces/http.md index 6462c68b224..ed11cf53eac 100644 --- a/docs/fa/interfaces/http.md +++ b/docs/fa/interfaces/http.md @@ -2,9 +2,9 @@ # HTTP interface -HTTP interface به شما امکان استفاده از ClickHpuse در هر پلتفرم با هر زمان برنامه نویسی را می دهد. ما از این Interface برای زبان های Java و Perl به مانند shell استفاده می کنیم. در دیگر دپارتمان ها، HTTP interface در Perl، Python، و Go استفاده می شود. HTTP Interface محدود تر از native interface می باشد، اما سازگاری بهتری دارد. +HTTP interface به شما امکان استفاده از ClickHouse در هر پلتفرم با هر زمان برنامه نویسی را می دهد. ما از این Interface برای زبان های Java و Perl به مانند shell استفاده می کنیم. در دیگر دپارتمان ها، HTTP interface در Perl، Python، و Go استفاده می شود. HTTP Interface محدود تر از native interface می باشد، اما سازگاری بهتری دارد. -به صورت پیش فرض، clickhouse-server به پرت 8123 در HTTP گوش می دهد. (میتونه در کانفیگ فایل تغییر پیدا کنه). اگر شما یک درخواست GET / بدون پارامتر بسازید، رشته ی "OK" رو دریافت می کنید (به همراه line feed در انتها). شما می توانید از این درخواست برای اسکریپت های health-check استفاده کنید. +به صورت پیش فرض، clickhouse-server به پرت 8123 در HTTP گوش می دهد. (میتونه در کانفیگ فایل تغییر پیدا کنه). اگر شما یک درخواست GET / بدون پارامتر بسازید، رشته ی "Ok." رو دریافت می کنید (به همراه line feed در انتها). شما می توانید از این درخواست برای اسکریپت های health-check استفاده کنید. diff --git a/docs/zh/interfaces/http.md b/docs/zh/interfaces/http.md index 8700f0be3c9..5923359e763 100644 --- a/docs/zh/interfaces/http.md +++ b/docs/zh/interfaces/http.md @@ -3,7 +3,7 @@ HTTP 接口可以让你通过任何平台和编程语言来使用 ClickHouse。我们用 Java 和 Perl 以及 shell 脚本来访问它。在其他的部门中,HTTP 接口会用在 Perl,Python 以及 Go 中。HTTP 接口比 TCP 原生接口更为局限,但是却有更好的兼容性。 默认情况下,clickhouse-server 会在端口 8123 上监控 HTTP 请求(这可以在配置中修改)。 -如果你发送了一个不带参数的 GET 请求,它会返回一个字符串 "Ok"(结尾有换行)。可以将它用在健康检查脚本中。 +如果你发送了一个不带参数的 GET 请求,它会返回一个字符串 "Ok."(结尾有换行)。可以将它用在健康检查脚本中。 ```bash $ curl 'http://localhost:8123/' From a8ab28b61243e7da3fd3e103f1215747203160c9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 24 Dec 2019 12:05:06 +0300 Subject: [PATCH 34/53] Remove flaps from kill mutation test --- .../0_stateless/00834_kill_mutation.sh | 24 +++++++++++++++++-- 1 file changed, 22 insertions(+), 2 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00834_kill_mutation.sh b/dbms/tests/queries/0_stateless/00834_kill_mutation.sh index 726764c654b..ed04c362401 100755 --- a/dbms/tests/queries/0_stateless/00834_kill_mutation.sh +++ b/dbms/tests/queries/0_stateless/00834_kill_mutation.sh @@ -16,7 +16,17 @@ ${CLICKHOUSE_CLIENT} --query="SELECT '*** Create and kill a single invalid mutat ${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.kill_mutation DELETE WHERE toUInt32(s) = 1 SETTINGS mutations_sync = 1" & -sleep 0.1 + +check_query1="SELECT substr(latest_fail_reason, 1, 8) as ErrorCode FROM system.mutations WHERE database = 'test' AND table = 'kill_mutation' AND ErrorCode != ''" + +query_result=`$CLICKHOUSE_CLIENT --query="$check_query1" 2>&1` + +while [ -z "$query_result" ] +do + query_result=`$CLICKHOUSE_CLIENT --query="$check_query1" 2>&1` + sleep 0.1 +done + ${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, latest_failed_part IN ('20000101_1_1_0', '20010101_2_2_0'), latest_fail_time != 0, substr(latest_fail_reason, 1, 8) FROM system.mutations WHERE database = 'test' AND table = 'kill_mutation'" ${CLICKHOUSE_CLIENT} --query="KILL MUTATION WHERE database = 'test' AND table = 'kill_mutation'" @@ -31,9 +41,19 @@ ${CLICKHOUSE_CLIENT} --query="SELECT '*** Create and kill invalid mutation that ${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.kill_mutation DELETE WHERE toUInt32(s) = 1" ${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.kill_mutation DELETE WHERE x = 1 SETTINGS mutations_sync = 1" & +check_query2="SELECT substr(latest_fail_reason, 1, 8) as ErrorCode FROM system.mutations WHERE database = 'test' AND table = 'kill_mutation' AND mutation_id = 'mutation_4.txt' AND ErrorCode != ''" + +query_result=`$CLICKHOUSE_CLIENT --query="$check_query1" 2>&1` + +while [ -z "$query_result" ] +do + query_result=`$CLICKHOUSE_CLIENT --query="$check_query1" 2>&1` + sleep 0.1 +done + ${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, latest_failed_part IN ('20000101_1_1_0', '20010101_2_2_0'), latest_fail_time != 0, substr(latest_fail_reason, 1, 8) FROM system.mutations WHERE database = 'test' AND table = 'kill_mutation' AND mutation_id = 'mutation_4.txt'" -sleep 0.1 + ${CLICKHOUSE_CLIENT} --query="KILL MUTATION WHERE database = 'test' AND table = 'kill_mutation' AND mutation_id = 'mutation_4.txt'" wait From 8bd22bf1903c92e9ca0e4b650977d265816fc9e2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 24 Dec 2019 12:06:44 +0300 Subject: [PATCH 35/53] Removed unused method that is unneeded after removing compiled aggregator --- .../AggregateFunctions/AggregateFunctionArgMinMax.h | 2 -- dbms/src/AggregateFunctions/AggregateFunctionArray.h | 2 -- dbms/src/AggregateFunctions/AggregateFunctionAvg.h | 2 -- .../AggregateFunctions/AggregateFunctionBitwise.h | 2 -- .../AggregateFunctionBoundingRatio.h | 5 ----- .../AggregateFunctionCategoricalInformationValue.h | 5 ----- dbms/src/AggregateFunctions/AggregateFunctionCount.h | 4 ---- .../AggregateFunctions/AggregateFunctionEntropy.h | 2 -- .../AggregateFunctions/AggregateFunctionForEach.h | 2 -- .../AggregateFunctions/AggregateFunctionGroupArray.h | 4 ---- .../AggregateFunctionGroupArrayInsertAt.h | 2 -- .../AggregateFunctionGroupArrayMoving.h | 2 -- .../AggregateFunctionGroupBitmap.h | 4 ---- .../AggregateFunctionGroupUniqArray.h | 4 ---- .../AggregateFunctions/AggregateFunctionHistogram.h | 2 -- dbms/src/AggregateFunctions/AggregateFunctionIf.h | 2 -- .../AggregateFunctions/AggregateFunctionMLMethod.h | 2 -- .../AggregateFunctionMaxIntersections.h | 5 ----- dbms/src/AggregateFunctions/AggregateFunctionMerge.h | 2 -- .../AggregateFunctions/AggregateFunctionMinMaxAny.h | 2 -- .../AggregateFunctions/AggregateFunctionNothing.h | 2 -- dbms/src/AggregateFunctions/AggregateFunctionNull.h | 2 -- .../src/AggregateFunctions/AggregateFunctionOrFill.h | 5 ----- .../AggregateFunctions/AggregateFunctionQuantile.h | 2 -- .../AggregateFunctions/AggregateFunctionResample.h | 5 ----- .../AggregateFunctions/AggregateFunctionRetention.h | 5 ----- .../AggregateFunctionSequenceMatch.h | 2 -- .../AggregateFunctionSimpleLinearRegression.h | 5 ----- dbms/src/AggregateFunctions/AggregateFunctionState.h | 2 -- .../AggregateFunctions/AggregateFunctionStatistics.h | 4 ---- .../AggregateFunctionStatisticsSimple.h | 2 -- dbms/src/AggregateFunctions/AggregateFunctionSum.h | 2 -- .../src/AggregateFunctions/AggregateFunctionSumMap.h | 2 -- .../AggregateFunctionTimeSeriesGroupSum.h | 2 -- dbms/src/AggregateFunctions/AggregateFunctionTopK.h | 4 ---- dbms/src/AggregateFunctions/AggregateFunctionUniq.h | 4 ---- .../AggregateFunctionUniqCombined.h | 12 +----------- .../AggregateFunctions/AggregateFunctionUniqUpTo.h | 4 ---- .../AggregateFunctionWindowFunnel.h | 5 ----- dbms/src/AggregateFunctions/IAggregateFunction.h | 6 ------ 40 files changed, 1 insertion(+), 133 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionArgMinMax.h b/dbms/src/AggregateFunctions/AggregateFunctionArgMinMax.h index ec151baa305..96d97280f03 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionArgMinMax.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionArgMinMax.h @@ -84,8 +84,6 @@ public: { this->data(place).result.insertResultInto(to); } - - const char * getHeaderFilePath() const override { return __FILE__; } }; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionArray.h b/dbms/src/AggregateFunctions/AggregateFunctionArray.h index 66dbcd865d5..cc4d5ffebb2 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionArray.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionArray.h @@ -130,8 +130,6 @@ public: } AggregateFunctionPtr getNestedFunction() const { return nested_func; } - - const char * getHeaderFilePath() const override { return __FILE__; } }; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionAvg.h b/dbms/src/AggregateFunctions/AggregateFunctionAvg.h index 3a79c2a06f6..35bb16550a7 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionAvg.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionAvg.h @@ -87,8 +87,6 @@ public: column.getData().push_back(this->data(place).template result()); } - const char * getHeaderFilePath() const override { return __FILE__; } - protected: UInt32 scale; }; diff --git a/dbms/src/AggregateFunctions/AggregateFunctionBitwise.h b/dbms/src/AggregateFunctions/AggregateFunctionBitwise.h index 23bd5e30d3f..29afa7db8d5 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionBitwise.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionBitwise.h @@ -78,8 +78,6 @@ public: { assert_cast &>(to).getData().push_back(this->data(place).value); } - - const char * getHeaderFilePath() const override { return __FILE__; } }; diff --git a/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.h b/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.h index def1c2eafe4..ecb4d686e59 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.h @@ -154,11 +154,6 @@ public: { assert_cast(to).getData().push_back(getBoundingRatio(data(place))); } - - const char * getHeaderFilePath() const override - { - return __FILE__; - } }; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionCategoricalInformationValue.h b/dbms/src/AggregateFunctions/AggregateFunctionCategoricalInformationValue.h index e00543039fc..25e5c2d1f1a 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionCategoricalInformationValue.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionCategoricalInformationValue.h @@ -33,11 +33,6 @@ public: return "categoricalInformationValue"; } - const char * getHeaderFilePath() const override - { - return __FILE__; - } - void create(AggregateDataPtr place) const override { memset(place, 0, sizeOfData()); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionCount.h b/dbms/src/AggregateFunctions/AggregateFunctionCount.h index edf6277cef5..c63d02931cf 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionCount.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionCount.h @@ -63,8 +63,6 @@ public: assert_cast(to).getData().push_back(data(place).count); } - const char * getHeaderFilePath() const override { return __FILE__; } - /// Reset the state to specified value. This function is not the part of common interface. void set(AggregateDataPtr place, UInt64 new_count) { @@ -115,8 +113,6 @@ public: { assert_cast(to).getData().push_back(data(place).count); } - - const char * getHeaderFilePath() const override { return __FILE__; } }; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionEntropy.h b/dbms/src/AggregateFunctions/AggregateFunctionEntropy.h index 49c7ff704f7..942de8ffe98 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionEntropy.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionEntropy.h @@ -145,8 +145,6 @@ public: auto & column = assert_cast &>(to); column.getData().push_back(this->data(place).get()); } - - const char * getHeaderFilePath() const override { return __FILE__; } }; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionForEach.h b/dbms/src/AggregateFunctions/AggregateFunctionForEach.h index dc89976dbd9..8f47a2de018 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionForEach.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionForEach.h @@ -247,8 +247,6 @@ public: { return true; } - - const char * getHeaderFilePath() const override { return __FILE__; } }; diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h index f5d4410b186..d58739e1dd8 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h @@ -136,8 +136,6 @@ public: { return true; } - - const char * getHeaderFilePath() const override { return __FILE__; } }; @@ -400,8 +398,6 @@ public: { return true; } - - const char * getHeaderFilePath() const override { return __FILE__; } }; #undef AGGREGATE_FUNCTION_GROUP_ARRAY_MAX_ARRAY_SIZE diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h index de551beab97..395d13f7d34 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h @@ -203,8 +203,6 @@ public: to_offsets.push_back(to_offsets.back() + result_array_size); } - - const char * getHeaderFilePath() const override { return __FILE__; } }; diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h index 85c9b9afcda..0e6d1af6a1b 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h @@ -192,8 +192,6 @@ public: { return true; } - - const char * getHeaderFilePath() const override { return __FILE__; } }; #undef AGGREGATE_FUNCTION_MOVING_MAX_ARRAY_SIZE diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupBitmap.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupBitmap.h index 6479eaf3c1f..56901e28e01 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupBitmap.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupBitmap.h @@ -52,8 +52,6 @@ public: { assert_cast &>(to).getData().push_back(this->data(place).rbs.size()); } - - const char * getHeaderFilePath() const override { return __FILE__; } }; @@ -119,8 +117,6 @@ public: { assert_cast &>(to).getData().push_back(this->data(place).rbs.size()); } - - const char * getHeaderFilePath() const override { return __FILE__; } }; template diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h index f4f9f0913d9..09585ecd38c 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h @@ -118,8 +118,6 @@ public: for (auto it = set.begin(); it != set.end(); ++it, ++i) data_to[old_size + i] = it->getValue(); } - - const char * getHeaderFilePath() const override { return __FILE__; } }; @@ -255,8 +253,6 @@ public: deserializeAndInsert(elem.getValue(), data_to); } } - - const char * getHeaderFilePath() const override { return __FILE__; } }; template <> diff --git a/dbms/src/AggregateFunctions/AggregateFunctionHistogram.h b/dbms/src/AggregateFunctions/AggregateFunctionHistogram.h index 9d2e0ea1331..673d78d807a 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionHistogram.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionHistogram.h @@ -369,8 +369,6 @@ public: offsets_to.push_back(to_tuple.size()); } - const char * getHeaderFilePath() const override { return __FILE__; } - String getName() const override { return "histogram"; } }; diff --git a/dbms/src/AggregateFunctions/AggregateFunctionIf.h b/dbms/src/AggregateFunctions/AggregateFunctionIf.h index c6d2e218ccc..e33fb1df53d 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionIf.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionIf.h @@ -109,8 +109,6 @@ public: { return nested_func->isState(); } - - const char * getHeaderFilePath() const override { return __FILE__; } }; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionMLMethod.h b/dbms/src/AggregateFunctions/AggregateFunctionMLMethod.h index a5d558364a5..d23a3dbcf62 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionMLMethod.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionMLMethod.h @@ -394,8 +394,6 @@ public: this->data(place).returnWeights(to); } - const char * getHeaderFilePath() const override { return __FILE__; } - private: UInt64 param_num; Float64 learning_rate; diff --git a/dbms/src/AggregateFunctions/AggregateFunctionMaxIntersections.h b/dbms/src/AggregateFunctions/AggregateFunctionMaxIntersections.h index 69cd6c0f3ec..13ed6ae42fe 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionMaxIntersections.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionMaxIntersections.h @@ -162,11 +162,6 @@ public: result_column.push_back(position_of_max_intersections); } } - - const char * getHeaderFilePath() const override - { - return __FILE__; - } }; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionMerge.h b/dbms/src/AggregateFunctions/AggregateFunctionMerge.h index c34f2fc1869..e2e0f65adda 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionMerge.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionMerge.h @@ -98,8 +98,6 @@ public: { return nested_func->allocatesMemoryInArena(); } - - const char * getHeaderFilePath() const override { return __FILE__; } }; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionMinMaxAny.h b/dbms/src/AggregateFunctions/AggregateFunctionMinMaxAny.h index ca5b1abae8f..a6779eb5d9e 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionMinMaxAny.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionMinMaxAny.h @@ -731,8 +731,6 @@ public: { this->data(place).insertResultInto(to); } - - const char * getHeaderFilePath() const override { return __FILE__; } }; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionNothing.h b/dbms/src/AggregateFunctions/AggregateFunctionNothing.h index aa54d95f158..d9c8f9cea19 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionNothing.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionNothing.h @@ -71,8 +71,6 @@ public: { to.insertDefault(); } - - const char * getHeaderFilePath() const override { return __FILE__; } }; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionNull.h b/dbms/src/AggregateFunctions/AggregateFunctionNull.h index e3128043ba6..a5000f30cd5 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionNull.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionNull.h @@ -176,8 +176,6 @@ public: { return nested_function->isState(); } - - const char * getHeaderFilePath() const override { return __FILE__; } }; diff --git a/dbms/src/AggregateFunctions/AggregateFunctionOrFill.h b/dbms/src/AggregateFunctions/AggregateFunctionOrFill.h index 39cf3f96488..42f6210e7d1 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionOrFill.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionOrFill.h @@ -49,11 +49,6 @@ public: return nested_function->getName() + "OrDefault"; } - const char * getHeaderFilePath() const override - { - return __FILE__; - } - bool isState() const override { return nested_function->isState(); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h b/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h index 78758a93298..50101bb2509 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h @@ -179,8 +179,6 @@ public: } } - const char * getHeaderFilePath() const override { return __FILE__; } - static void assertSecondArg(const DataTypes & types) { if constexpr (has_second_arg) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionResample.h b/dbms/src/AggregateFunctions/AggregateFunctionResample.h index 3864de7db4f..33b03fcdee0 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionResample.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionResample.h @@ -72,11 +72,6 @@ public: return nested_function->getName() + "Resample"; } - const char * getHeaderFilePath() const override - { - return __FILE__; - } - bool isState() const override { return nested_function->isState(); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionRetention.h b/dbms/src/AggregateFunctions/AggregateFunctionRetention.h index 806b34dd543..35df3b68fb7 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionRetention.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionRetention.h @@ -144,11 +144,6 @@ public: offsets_to.push_back(current_offset); } - - const char * getHeaderFilePath() const override - { - return __FILE__; - } }; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSequenceMatch.h b/dbms/src/AggregateFunctions/AggregateFunctionSequenceMatch.h index 903d5ce3493..61fd28f2a70 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSequenceMatch.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionSequenceMatch.h @@ -180,8 +180,6 @@ public: this->data(place).deserialize(buf); } - const char * getHeaderFilePath() const override { return __FILE__; } - private: enum class PatternActionType { diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSimpleLinearRegression.h b/dbms/src/AggregateFunctions/AggregateFunctionSimpleLinearRegression.h index cc5346c6f01..fcd4d113f1f 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSimpleLinearRegression.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionSimpleLinearRegression.h @@ -109,11 +109,6 @@ public: return "simpleLinearRegression"; } - const char * getHeaderFilePath() const override - { - return __FILE__; - } - void add( AggregateDataPtr place, const IColumn ** columns, diff --git a/dbms/src/AggregateFunctions/AggregateFunctionState.h b/dbms/src/AggregateFunctions/AggregateFunctionState.h index d558d93571a..8879a324827 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionState.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionState.h @@ -94,8 +94,6 @@ public: } AggregateFunctionPtr getNestedFunction() const { return nested_func; } - - const char * getHeaderFilePath() const override { return __FILE__; } }; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionStatistics.h b/dbms/src/AggregateFunctions/AggregateFunctionStatistics.h index 284e6dfb8bd..1614e265ccb 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionStatistics.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionStatistics.h @@ -147,8 +147,6 @@ public: { this->data(place).publish(to); } - - const char * getHeaderFilePath() const override { return __FILE__; } }; /** Implementing the varSamp function. @@ -401,8 +399,6 @@ public: { this->data(place).publish(to); } - - const char * getHeaderFilePath() const override { return __FILE__; } }; /** Implementing the covarSamp function. diff --git a/dbms/src/AggregateFunctions/AggregateFunctionStatisticsSimple.h b/dbms/src/AggregateFunctions/AggregateFunctionStatisticsSimple.h index 69b8a25e5c3..185ef3594dd 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionStatisticsSimple.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionStatisticsSimple.h @@ -552,8 +552,6 @@ public: } } - const char * getHeaderFilePath() const override { return __FILE__; } - private: UInt32 src_scale; }; diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSum.h b/dbms/src/AggregateFunctions/AggregateFunctionSum.h index 4aead37e146..5170b4ddd9a 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSum.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionSum.h @@ -147,8 +147,6 @@ public: column.getData().push_back(this->data(place).get()); } - const char * getHeaderFilePath() const override { return __FILE__; } - private: UInt32 scale; }; diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h index f17bfd8b6b0..c201e8e3370 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -261,8 +261,6 @@ public: } } - const char * getHeaderFilePath() const override { return __FILE__; } - bool keepKey(const T & key) const { return static_cast(*this).keepKey(key); } }; diff --git a/dbms/src/AggregateFunctions/AggregateFunctionTimeSeriesGroupSum.h b/dbms/src/AggregateFunctions/AggregateFunctionTimeSeriesGroupSum.h index 94a64ed6331..06748bf7385 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionTimeSeriesGroupSum.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionTimeSeriesGroupSum.h @@ -281,7 +281,5 @@ public: } bool allocatesMemoryInArena() const override { return true; } - - const char * getHeaderFilePath() const override { return __FILE__; } }; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionTopK.h b/dbms/src/AggregateFunctions/AggregateFunctionTopK.h index 84eeb50d189..dec6baf6ed3 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionTopK.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionTopK.h @@ -103,8 +103,6 @@ public: for (auto it = result_vec.begin(); it != result_vec.end(); ++it, ++i) data_to[old_size + i] = it->key; } - - const char * getHeaderFilePath() const override { return __FILE__; } }; @@ -230,8 +228,6 @@ public: data_to.deserializeAndInsertFromArena(elem.key.data); } } - - const char * getHeaderFilePath() const override { return __FILE__; } }; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniq.h b/dbms/src/AggregateFunctions/AggregateFunctionUniq.h index 7c97cfa2853..9e869435ce0 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniq.h @@ -244,8 +244,6 @@ public: { assert_cast(to).getData().push_back(this->data(place).set.size()); } - - const char * getHeaderFilePath() const override { return __FILE__; } }; @@ -300,8 +298,6 @@ public: { assert_cast(to).getData().push_back(this->data(place).set.size()); } - - const char * getHeaderFilePath() const override { return __FILE__; } }; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h index dac5805c374..44d92b72365 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h @@ -109,7 +109,7 @@ struct AggregateFunctionUniqCombinedData : public AggregateFunctionUniqCombinedD }; -/// For String keys, 64 bit hash is always used (both for uniqCombined and uniqCombined64), +/// For String keys, 64 bit hash is always used (both for uniqCombined and uniqCombined64), /// because of backwards compatibility (64 bit hash was already used for uniqCombined). template struct AggregateFunctionUniqCombinedData : public AggregateFunctionUniqCombinedDataWithKey @@ -171,11 +171,6 @@ public: { assert_cast(to).getData().push_back(this->data(place).set.size()); } - - const char * getHeaderFilePath() const override - { - return __FILE__; - } }; /** For multiple arguments. To compute, hashes them. @@ -238,11 +233,6 @@ public: { assert_cast(to).getData().push_back(this->data(place).set.size()); } - - const char * getHeaderFilePath() const override - { - return __FILE__; - } }; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniqUpTo.h b/dbms/src/AggregateFunctions/AggregateFunctionUniqUpTo.h index 47fb02b7ad0..f16a7cc475e 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniqUpTo.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniqUpTo.h @@ -184,8 +184,6 @@ public: { assert_cast(to).getData().push_back(this->data(place).size()); } - - const char * getHeaderFilePath() const override { return __FILE__; } }; @@ -248,8 +246,6 @@ public: { assert_cast(to).getData().push_back(this->data(place).size()); } - - const char * getHeaderFilePath() const override { return __FILE__; } }; diff --git a/dbms/src/AggregateFunctions/AggregateFunctionWindowFunnel.h b/dbms/src/AggregateFunctions/AggregateFunctionWindowFunnel.h index 16d9ac548ad..8aa3e452113 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionWindowFunnel.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionWindowFunnel.h @@ -245,11 +245,6 @@ public: { assert_cast(to).getData().push_back(getEventLevel(this->data(place))); } - - const char * getHeaderFilePath() const override - { - return __FILE__; - } }; } diff --git a/dbms/src/AggregateFunctions/IAggregateFunction.h b/dbms/src/AggregateFunctions/IAggregateFunction.h index 5693f998b3e..94dcf4cbcab 100644 --- a/dbms/src/AggregateFunctions/IAggregateFunction.h +++ b/dbms/src/AggregateFunctions/IAggregateFunction.h @@ -148,12 +148,6 @@ public: addBatchArray(size_t batch_size, AggregateDataPtr * places, size_t place_offset, const IColumn ** columns, const UInt64 * offsets, Arena * arena) const = 0; - /** This is used for runtime code generation to determine, which header files to include in generated source. - * Always implement it as - * const char * getHeaderFilePath() const override { return __FILE__; } - */ - virtual const char * getHeaderFilePath() const = 0; - const DataTypes & getArgumentTypes() const { return argument_types; } const Array & getParameters() const { return parameters; } From 94318c50a13c7655653b96e7df5e34da61e5d3f9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 24 Dec 2019 15:13:20 +0300 Subject: [PATCH 36/53] Renamed a test because it's inconvenient to do Ctrl+F and type 'fail' in browser --- ...nce => 01040_dictionary_invalidate_query_switchover.reference} | 0 ...ailover.sh => 01040_dictionary_invalidate_query_switchover.sh} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename dbms/tests/queries/0_stateless/{01040_dictionary_invalidate_query_failover.reference => 01040_dictionary_invalidate_query_switchover.reference} (100%) rename dbms/tests/queries/0_stateless/{01040_dictionary_invalidate_query_failover.sh => 01040_dictionary_invalidate_query_switchover.sh} (100%) diff --git a/dbms/tests/queries/0_stateless/01040_dictionary_invalidate_query_failover.reference b/dbms/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01040_dictionary_invalidate_query_failover.reference rename to dbms/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover.reference diff --git a/dbms/tests/queries/0_stateless/01040_dictionary_invalidate_query_failover.sh b/dbms/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover.sh similarity index 100% rename from dbms/tests/queries/0_stateless/01040_dictionary_invalidate_query_failover.sh rename to dbms/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover.sh From 6fbc202bb33e2233eaf1914ac9654894c0edff31 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 24 Dec 2019 16:30:28 +0300 Subject: [PATCH 37/53] Try fix tests with processors. --- .../src/Processors/Transforms/AggregatingTransform.cpp | 5 +++++ .../Processors/Transforms/MergingSortedTransform.cpp | 10 +++++++--- .../0_stateless/00909_kill_not_initialized_query.sh | 2 +- 3 files changed, 13 insertions(+), 4 deletions(-) diff --git a/dbms/src/Processors/Transforms/AggregatingTransform.cpp b/dbms/src/Processors/Transforms/AggregatingTransform.cpp index f763d4ff5c0..72a5ff3bb7c 100644 --- a/dbms/src/Processors/Transforms/AggregatingTransform.cpp +++ b/dbms/src/Processors/Transforms/AggregatingTransform.cpp @@ -176,7 +176,12 @@ public: /// Check can output. if (output.isFinished()) + { + for (auto & input : inputs) + input.close(); + return Status::Finished; + } if (!output.canPush()) return Status::PortFull; diff --git a/dbms/src/Processors/Transforms/MergingSortedTransform.cpp b/dbms/src/Processors/Transforms/MergingSortedTransform.cpp index d8f06a7fe4a..705116ca081 100644 --- a/dbms/src/Processors/Transforms/MergingSortedTransform.cpp +++ b/dbms/src/Processors/Transforms/MergingSortedTransform.cpp @@ -134,7 +134,10 @@ IProcessor::Status MergingSortedTransform::prepare() auto chunk = input.pull(); if (!chunk.hasRows()) { - all_inputs_has_data = false; + + if (!input.isFinished()) + all_inputs_has_data = false; + continue; } @@ -176,13 +179,14 @@ IProcessor::Status MergingSortedTransform::prepare() return Status::NeedData; auto chunk = input.pull(); - if (!chunk.hasRows()) + if (!chunk.hasRows() && !input.isFinished()) return Status::NeedData; updateCursor(std::move(chunk), next_input_to_read); pushToQueue(next_input_to_read); - need_data = false; } + + need_data = false; } return Status::Ready; diff --git a/dbms/tests/queries/0_stateless/00909_kill_not_initialized_query.sh b/dbms/tests/queries/0_stateless/00909_kill_not_initialized_query.sh index 67454f676b3..677709dd2c0 100755 --- a/dbms/tests/queries/0_stateless/00909_kill_not_initialized_query.sh +++ b/dbms/tests/queries/0_stateless/00909_kill_not_initialized_query.sh @@ -34,7 +34,7 @@ $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE query='$query_to_kill' ASYNC" &>/dev/nul sleep 1 # Kill $query_for_pending SYNC. This query is not blocker, so it should be killed fast. -timeout 5 $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE query='$query_for_pending' SYNC" &>/dev/null +timeout 10 $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE query='$query_for_pending' SYNC" &>/dev/null # Both queries have to be killed, doesn't matter with SYNC or ASYNC kill for run in {1..15} From 2befa5aec5411c64bbbca1a794c8eae5b6d3f1e9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 24 Dec 2019 19:51:37 +0300 Subject: [PATCH 38/53] Fixed build on old systems --- contrib/libcxxabi | 2 +- libs/libglibc-compatibility/CMakeLists.txt | 4 - .../libcxxabi/LICENSE.TXT | 76 ----------- .../libcxxabi/README.txt | 8 -- .../libcxxabi/cxa_thread_atexit.cpp | 121 ------------------ 5 files changed, 1 insertion(+), 210 deletions(-) delete mode 100644 libs/libglibc-compatibility/libcxxabi/LICENSE.TXT delete mode 100644 libs/libglibc-compatibility/libcxxabi/README.txt delete mode 100644 libs/libglibc-compatibility/libcxxabi/cxa_thread_atexit.cpp diff --git a/contrib/libcxxabi b/contrib/libcxxabi index c26cf36f838..7aacd45028e 160000 --- a/contrib/libcxxabi +++ b/contrib/libcxxabi @@ -1 +1 @@ -Subproject commit c26cf36f8387c5edf2cabb4a630f0975c35aa9fb +Subproject commit 7aacd45028ecf5f1c39985ecbd4f67eed9b11ce5 diff --git a/libs/libglibc-compatibility/CMakeLists.txt b/libs/libglibc-compatibility/CMakeLists.txt index 8405c9450a5..42fc8693dd0 100644 --- a/libs/libglibc-compatibility/CMakeLists.txt +++ b/libs/libglibc-compatibility/CMakeLists.txt @@ -27,10 +27,6 @@ if (GLIBC_COMPATIBILITY) list(APPEND glibc_compatibility_sources musl/getentropy.c) endif() - if(MAKE_STATIC_LIBRARIES) - list(APPEND glibc_compatibility_sources libcxxabi/cxa_thread_atexit.cpp) - endif() - # Need to omit frame pointers to match the performance of glibc set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fomit-frame-pointer") diff --git a/libs/libglibc-compatibility/libcxxabi/LICENSE.TXT b/libs/libglibc-compatibility/libcxxabi/LICENSE.TXT deleted file mode 100644 index f1f163f38fd..00000000000 --- a/libs/libglibc-compatibility/libcxxabi/LICENSE.TXT +++ /dev/null @@ -1,76 +0,0 @@ -============================================================================== -libc++abi License -============================================================================== - -The libc++abi library is dual licensed under both the University of Illinois -"BSD-Like" license and the MIT license. As a user of this code you may choose -to use it under either license. As a contributor, you agree to allow your code -to be used under both. - -Full text of the relevant licenses is included below. - -============================================================================== - -University of Illinois/NCSA -Open Source License - -Copyright (c) 2009-2018 by the contributors listed in CREDITS.TXT - -All rights reserved. - -Developed by: - - LLVM Team - - University of Illinois at Urbana-Champaign - - http://llvm.org - -Permission is hereby granted, free of charge, to any person obtaining a copy of -this software and associated documentation files (the "Software"), to deal with -the Software without restriction, including without limitation the rights to -use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies -of the Software, and to permit persons to whom the Software is furnished to do -so, subject to the following conditions: - - * Redistributions of source code must retain the above copyright notice, - this list of conditions and the following disclaimers. - - * Redistributions in binary form must reproduce the above copyright notice, - this list of conditions and the following disclaimers in the - documentation and/or other materials provided with the distribution. - - * Neither the names of the LLVM Team, University of Illinois at - Urbana-Champaign, nor the names of its contributors may be used to - endorse or promote products derived from this Software without specific - prior written permission. - -THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS -FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -CONTRIBUTORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS WITH THE -SOFTWARE. - -============================================================================== - -Copyright (c) 2009-2014 by the contributors listed in CREDITS.TXT - -Permission is hereby granted, free of charge, to any person obtaining a copy -of this software and associated documentation files (the "Software"), to deal -in the Software without restriction, including without limitation the rights -to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -copies of the Software, and to permit persons to whom the Software is -furnished to do so, subject to the following conditions: - -The above copyright notice and this permission notice shall be included in -all copies or substantial portions of the Software. - -THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -THE SOFTWARE. diff --git a/libs/libglibc-compatibility/libcxxabi/README.txt b/libs/libglibc-compatibility/libcxxabi/README.txt deleted file mode 100644 index 2b5e98ad3df..00000000000 --- a/libs/libglibc-compatibility/libcxxabi/README.txt +++ /dev/null @@ -1,8 +0,0 @@ -Imported from https://github.com/llvm-project/llvm-project-20170507 -revision: ad82e63b9719923cb393bd805730eaca0e3632a8 - -This is needed to avoid linking with "__cxa_thread_atexit_impl" function, that require too new (2.18) glibc library. - -Note: "__cxa_thread_atexit_impl" may provide sophisticated implementation to correct destruction of thread-local objects, -that was created in different DSO. Read https://sourceware.org/glibc/wiki/Destructor%20support%20for%20thread_local%20variables -We simply don't need this implementation, because we don't use thread-local objects from different DSO. diff --git a/libs/libglibc-compatibility/libcxxabi/cxa_thread_atexit.cpp b/libs/libglibc-compatibility/libcxxabi/cxa_thread_atexit.cpp deleted file mode 100644 index 016184e8ae9..00000000000 --- a/libs/libglibc-compatibility/libcxxabi/cxa_thread_atexit.cpp +++ /dev/null @@ -1,121 +0,0 @@ -//===----------------------- cxa_thread_atexit.cpp ------------------------===// -// -// The LLVM Compiler Infrastructure -// -// This file is dual licensed under the MIT and the University of Illinois Open -// Source Licenses. See LICENSE.TXT for details. -// -//===----------------------------------------------------------------------===// - -#include -#include - -using Dtor = void(*)(void*); - -namespace { - // This implementation is used if the C library does not provide - // __cxa_thread_atexit_impl() for us. It has a number of limitations that are - // difficult to impossible to address without ..._impl(): - // - // - dso_symbol is ignored. This means that a shared library may be unloaded - // (via dlclose()) before its thread_local destructors have run. - // - // - thread_local destructors for the main thread are run by the destructor of - // a static object. This is later than expected; they should run before the - // destructors of any objects with static storage duration. - // - // - thread_local destructors on non-main threads run on the first iteration - // through the __libccpp_tls_key destructors. - // std::notify_all_at_thread_exit() and similar functions must be careful to - // wait until the second iteration to provide their intended ordering - // guarantees. - // - // Another limitation, though one shared with ..._impl(), is that any - // thread_locals that are first initialized after non-thread_local global - // destructors begin to run will not be destroyed. [basic.start.term] states - // that all thread_local destructors are sequenced before the destruction of - // objects with static storage duration, resulting in a contradiction if a - // thread_local is constructed after that point. Thus we consider such - // programs ill-formed, and don't bother to run those destructors. (If the - // program terminates abnormally after such a thread_local is constructed, - // the destructor is not expected to run and thus there is no contradiction. - // So construction still has to work.) - - struct DtorList { - Dtor dtor; - void* obj; - DtorList* next; - }; - - // The linked list of thread-local destructors to run - __thread DtorList* dtors = nullptr; - // True if the destructors are currently scheduled to run on this thread - __thread bool dtors_alive = false; - // Used to trigger destructors on thread exit; value is ignored - pthread_key_t dtors_key; - - void run_dtors(void*) { - while (auto head = dtors) { - dtors = head->next; - head->dtor(head->obj); - std::free(head); - } - - dtors_alive = false; - } - - struct DtorsManager { - DtorsManager() { - // There is intentionally no matching std::__libcpp_tls_delete call, as - // __cxa_thread_atexit() may be called arbitrarily late (for example, from - // global destructors or atexit() handlers). - if (pthread_key_create(&dtors_key, run_dtors) != 0) { - abort(); - } - } - - ~DtorsManager() { - // std::__libcpp_tls_key destructors do not run on threads that call exit() - // (including when the main thread returns from main()), so we explicitly - // call the destructor here. This runs at exit time (potentially earlier - // if libc++abi is dlclose()'d). Any thread_locals initialized after this - // point will not be destroyed. - run_dtors(nullptr); - } - }; -} // namespace - - -extern "C" -{ - int __cxa_thread_atexit_impl(Dtor dtor, void* obj, void* dso_symbol) throw() - { - // Initialize the dtors std::__libcpp_tls_key (uses __cxa_guard_*() for - // one-time initialization and __cxa_atexit() for destruction) - static DtorsManager manager; - - if (!dtors_alive) { - if (pthread_setspecific(dtors_key, &dtors_key) != 0) { - return -1; - } - dtors_alive = true; - } - - auto head = static_cast(std::malloc(sizeof(DtorList))); - if (!head) { - return -1; - } - - head->dtor = dtor; - head->obj = obj; - head->next = dtors; - dtors = head; - - return 0; - } - - int __attribute__((__weak__)) __cxa_thread_atexit(Dtor dtor, void* obj, void* dso_symbol) throw() - { - return __cxa_thread_atexit_impl(dtor, obj, dso_symbol); - } -} // extern "C" From bc36c3f1a9aa809904bbe3efb5f41b9266d4a65e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 24 Dec 2019 20:12:08 +0300 Subject: [PATCH 39/53] Avoid std::terminate in destructors of some WriteBuffers --- dbms/src/IO/BrotliWriteBuffer.cpp | 9 ++++++++- dbms/src/IO/WriteBufferFromFileBase.cpp | 4 ---- dbms/src/IO/WriteBufferFromFileBase.h | 2 +- dbms/src/IO/WriteBufferFromVector.h | 8 +++++++- dbms/src/IO/WriteBufferValidUTF8.cpp | 13 +++++++++++++ dbms/src/IO/WriteBufferValidUTF8.h | 5 +---- 6 files changed, 30 insertions(+), 11 deletions(-) diff --git a/dbms/src/IO/BrotliWriteBuffer.cpp b/dbms/src/IO/BrotliWriteBuffer.cpp index 9e8b2a81a63..0a0eeb52956 100644 --- a/dbms/src/IO/BrotliWriteBuffer.cpp +++ b/dbms/src/IO/BrotliWriteBuffer.cpp @@ -46,7 +46,14 @@ BrotliWriteBuffer::BrotliWriteBuffer(WriteBuffer & out_, int compression_level, BrotliWriteBuffer::~BrotliWriteBuffer() { - finish(); + try + { + finish(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } } void BrotliWriteBuffer::nextImpl() diff --git a/dbms/src/IO/WriteBufferFromFileBase.cpp b/dbms/src/IO/WriteBufferFromFileBase.cpp index 7a3bf1e7378..4b989d7ac72 100644 --- a/dbms/src/IO/WriteBufferFromFileBase.cpp +++ b/dbms/src/IO/WriteBufferFromFileBase.cpp @@ -8,10 +8,6 @@ WriteBufferFromFileBase::WriteBufferFromFileBase(size_t buf_size, char * existin { } -WriteBufferFromFileBase::~WriteBufferFromFileBase() -{ -} - off_t WriteBufferFromFileBase::seek(off_t off, int whence) { return doSeek(off, whence); diff --git a/dbms/src/IO/WriteBufferFromFileBase.h b/dbms/src/IO/WriteBufferFromFileBase.h index 18bc6b36085..8d3409c3c58 100644 --- a/dbms/src/IO/WriteBufferFromFileBase.h +++ b/dbms/src/IO/WriteBufferFromFileBase.h @@ -13,7 +13,7 @@ class WriteBufferFromFileBase : public BufferWithOwnMemory { public: WriteBufferFromFileBase(size_t buf_size, char * existing_memory, size_t alignment); - virtual ~WriteBufferFromFileBase(); + ~WriteBufferFromFileBase() override = default; off_t seek(off_t off, int whence = SEEK_SET); void truncate(off_t length = 0); diff --git a/dbms/src/IO/WriteBufferFromVector.h b/dbms/src/IO/WriteBufferFromVector.h index b6a6a226669..02f07b9f228 100644 --- a/dbms/src/IO/WriteBufferFromVector.h +++ b/dbms/src/IO/WriteBufferFromVector.h @@ -89,8 +89,14 @@ public: ~WriteBufferFromVector() override { - if (!is_finished) + try + { finish(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } } }; diff --git a/dbms/src/IO/WriteBufferValidUTF8.cpp b/dbms/src/IO/WriteBufferValidUTF8.cpp index 220b6cd44fc..b40424fc463 100644 --- a/dbms/src/IO/WriteBufferValidUTF8.cpp +++ b/dbms/src/IO/WriteBufferValidUTF8.cpp @@ -133,4 +133,17 @@ void WriteBufferValidUTF8::finish() putReplacement(); } + +WriteBufferValidUTF8::~WriteBufferValidUTF8() +{ + try + { + finish(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} + } diff --git a/dbms/src/IO/WriteBufferValidUTF8.h b/dbms/src/IO/WriteBufferValidUTF8.h index 31151eefbfb..1d934be3779 100644 --- a/dbms/src/IO/WriteBufferValidUTF8.h +++ b/dbms/src/IO/WriteBufferValidUTF8.h @@ -35,10 +35,7 @@ public: const char * replacement_ = "\xEF\xBF\xBD", size_t size = DEFAULT_SIZE); - virtual ~WriteBufferValidUTF8() override - { - finish(); - } + ~WriteBufferValidUTF8() override; }; } From 8d190e68077189387366d46c018b04250e10af92 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 24 Dec 2019 20:16:38 +0300 Subject: [PATCH 40/53] Fixed build --- dbms/src/IO/WriteBufferFromFileBase.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/IO/WriteBufferFromFileBase.h b/dbms/src/IO/WriteBufferFromFileBase.h index 8d3409c3c58..e846045016c 100644 --- a/dbms/src/IO/WriteBufferFromFileBase.h +++ b/dbms/src/IO/WriteBufferFromFileBase.h @@ -18,7 +18,7 @@ public: off_t seek(off_t off, int whence = SEEK_SET); void truncate(off_t length = 0); virtual off_t getPositionInFile() = 0; - virtual void sync() = 0; + void sync() override = 0; virtual std::string getFileName() const = 0; virtual int getFD() const = 0; From 05b933b1d3fa841acab9157537b913264ebab400 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 24 Dec 2019 21:07:51 +0300 Subject: [PATCH 41/53] Better --- dbms/src/Storages/AlterCommands.cpp | 22 +++--- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 70 +++++++++---------- dbms/src/Storages/StorageMergeTree.cpp | 3 - 3 files changed, 46 insertions(+), 49 deletions(-) diff --git a/dbms/src/Storages/AlterCommands.cpp b/dbms/src/Storages/AlterCommands.cpp index b24863a2afb..535105e5264 100644 --- a/dbms/src/Storages/AlterCommands.cpp +++ b/dbms/src/Storages/AlterCommands.cpp @@ -257,10 +257,16 @@ void AlterCommand::apply(ColumnsDescription & columns_description, IndicesDescri if (ttl) column.ttl = ttl; - column.type = data_type; + if (data_type) + column.type = data_type; - column.default_desc.kind = default_kind; - column.default_desc.expression = default_expression; + /// User specified default expression or changed + /// datatype. We have to replace default. + if (default_expression || data_type) + { + column.default_desc.kind = default_kind; + column.default_desc.expression = default_expression; + } }); } else if (type == MODIFY_ORDER_BY) @@ -389,13 +395,13 @@ void AlterCommand::apply(ColumnsDescription & columns_description, IndicesDescri bool AlterCommand::isModifyingData() const { - /// Change binary representation on disk + /// Possible change data representation on disk if (type == MODIFY_COLUMN) - return data_type.get() || default_expression; + return data_type != nullptr; - return type == ADD_COLUMN /// We need to change columns.txt in each part - || type == DROP_COLUMN /// We need to change columns.txt in each part - || type == DROP_INDEX; /// We need to remove file from filesystem + return type == ADD_COLUMN /// We need to change columns.txt in each part for MergeTree + || type == DROP_COLUMN /// We need to change columns.txt in each part for MergeTree + || type == DROP_INDEX; /// We need to remove file from filesystem for MergeTree } bool AlterCommand::isSettingsAlter() const diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 8d892fd69d6..c89af45f2fa 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1342,11 +1342,11 @@ void MergeTreeData::checkAlter(const AlterCommands & commands, const Context & c "before using data skipping indices.", ErrorCodes::BAD_ARGUMENTS); /// Set of columns that shouldn't be altered. - NameSet columns_alter_forbidden; + NameSet columns_alter_type_forbidden; /// Primary key columns can be ALTERed only if they are used in the key as-is /// (and not as a part of some expression) and if the ALTER only affects column metadata. - NameSet columns_alter_metadata_only; + NameSet columns_alter_type_metadata_only; if (partition_key_expr) { @@ -1354,13 +1354,13 @@ void MergeTreeData::checkAlter(const AlterCommands & commands, const Context & c /// TODO: in some cases (e.g. adding an Enum value) a partition key column can still be ALTERed. /// We should allow it. for (const String & col : partition_key_expr->getRequiredColumns()) - columns_alter_forbidden.insert(col); + columns_alter_type_forbidden.insert(col); } for (const auto & index : skip_indices) { for (const String & col : index->expr->getRequiredColumns()) - columns_alter_forbidden.insert(col); + columns_alter_type_forbidden.insert(col); } if (sorting_key_expr) @@ -1368,17 +1368,16 @@ void MergeTreeData::checkAlter(const AlterCommands & commands, const Context & c for (const ExpressionAction & action : sorting_key_expr->getActions()) { auto action_columns = action.getNeededColumns(); - columns_alter_forbidden.insert(action_columns.begin(), action_columns.end()); + columns_alter_type_forbidden.insert(action_columns.begin(), action_columns.end()); } for (const String & col : sorting_key_expr->getRequiredColumns()) - columns_alter_metadata_only.insert(col); + columns_alter_type_metadata_only.insert(col); /// We don't process sample_by_ast separately because it must be among the primary key columns /// and we don't process primary_key_expr separately because it is a prefix of sorting_key_expr. } - if (!merging_params.sign_column.empty()) - columns_alter_forbidden.insert(merging_params.sign_column); + columns_alter_type_forbidden.insert(merging_params.sign_column); std::map old_types; for (const auto & column : getColumns().getAllPhysical()) @@ -1386,34 +1385,26 @@ void MergeTreeData::checkAlter(const AlterCommands & commands, const Context & c for (const AlterCommand & command : commands) { - if (!command.isModifyingData()) + if (command.type == AlterCommand::MODIFY_ORDER_BY && !is_custom_partitioned) { - continue; - } - - if (columns_alter_forbidden.count(command.column_name)) - throw Exception("Trying to ALTER key column " + command.column_name, ErrorCodes::ILLEGAL_COLUMN); - - if (columns_alter_metadata_only.count(command.column_name)) - { - if (command.type == AlterCommand::MODIFY_COLUMN) - { - auto it = old_types.find(command.column_name); - if (it != old_types.end() && isMetadataOnlyConversion(it->second, command.data_type.get())) - continue; - } - throw Exception( - "ALTER of key column " + command.column_name + " must be metadata-only", - ErrorCodes::ILLEGAL_COLUMN); + "ALTER MODIFY ORDER BY is not supported for default-partitioned tables created with the old syntax", + ErrorCodes::BAD_ARGUMENTS); } - - if (command.type == AlterCommand::MODIFY_ORDER_BY) + else if (command.isModifyingData()) { - if (!is_custom_partitioned) - throw Exception( - "ALTER MODIFY ORDER BY is not supported for default-partitioned tables created with the old syntax", - ErrorCodes::BAD_ARGUMENTS); + if (columns_alter_type_forbidden.count(command.column_name)) + throw Exception("Trying to ALTER key column " + command.column_name, ErrorCodes::ILLEGAL_COLUMN); + + if (columns_alter_type_metadata_only.count(command.column_name)) + { + if (command.type == AlterCommand::MODIFY_COLUMN) + { + auto it = old_types.find(command.column_name); + if (it == old_types.end() || !isMetadataOnlyConversion(it->second, command.data_type.get())) + throw Exception("ALTER of key column " + command.column_name + " must be metadata-only", ErrorCodes::ILLEGAL_COLUMN); + } + } } } @@ -1425,12 +1416,15 @@ void MergeTreeData::checkAlter(const AlterCommands & commands, const Context & c for (const auto & setting : new_changes) checkSettingCanBeChanged(setting.name); - /// Check that type conversions are possible. - ExpressionActionsPtr unused_expression; - NameToNameMap unused_map; - bool unused_bool; - createConvertExpression(nullptr, getColumns().getAllPhysical(), new_columns.getAllPhysical(), - getIndices().indices, new_indices.indices, unused_expression, unused_map, unused_bool); + if (commands.isModifyingData()) + { + /// Check that type conversions are possible. + ExpressionActionsPtr unused_expression; + NameToNameMap unused_map; + bool unused_bool; + createConvertExpression(nullptr, getColumns().getAllPhysical(), new_columns.getAllPhysical(), + getIndices().indices, new_indices.indices, unused_expression, unused_map, unused_bool); + } } void MergeTreeData::createConvertExpression(const DataPartPtr & part, const NamesAndTypesList & old_columns, diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index e6a847ba1bd..66d22da5c3b 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -299,9 +299,6 @@ void StorageMergeTree::alter( { lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); - params.apply(new_columns, new_indices, new_constraints, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast, new_changes); - - IDatabase::ASTModifier settings_modifier = getSettingsModifier(new_changes); context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, storage_modifier); update_metadata(); From 6a1dccd27ccb138eee77b85ced071ba1e4687595 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 24 Dec 2019 21:25:00 +0300 Subject: [PATCH 42/53] Change initialization order for StorageDistributedDirectoryMonitor members. --- .../Storages/Distributed/DirectoryMonitor.cpp | 14 +++++++------- dbms/src/Storages/Distributed/DirectoryMonitor.h | 16 ++++++++-------- 2 files changed, 15 insertions(+), 15 deletions(-) diff --git a/dbms/src/Storages/Distributed/DirectoryMonitor.cpp b/dbms/src/Storages/Distributed/DirectoryMonitor.cpp index 111d1ff7aab..94327d129dd 100644 --- a/dbms/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/dbms/src/Storages/Distributed/DirectoryMonitor.cpp @@ -80,11 +80,15 @@ namespace StorageDistributedDirectoryMonitor::StorageDistributedDirectoryMonitor( - StorageDistributed & storage_, const std::string & name_, const ConnectionPoolPtr & pool_, ActionBlocker & monitor_blocker_) + StorageDistributed & storage_, std::string name_, ConnectionPoolPtr pool_, ActionBlocker & monitor_blocker_) + /// It's important to initialize members before `thread` to avoid race. : storage(storage_) - , pool{pool_} - , name{name_} + , pool(std::move(pool_)) + , name(std::move(name_)) , path{storage.path + name + '/'} + , should_batch_inserts(storage.global_context.getSettingsRef().distributed_directory_monitor_batch_inserts) + , min_batched_block_size_rows(storage.global_context.getSettingsRef().min_insert_block_size_rows) + , min_batched_block_size_bytes(storage.global_context.getSettingsRef().min_insert_block_size_bytes) , current_batch_file_path{path + "current_batch.txt"} , default_sleep_time{storage.global_context.getSettingsRef().distributed_directory_monitor_sleep_time_ms.totalMilliseconds()} , sleep_time{default_sleep_time} @@ -92,10 +96,6 @@ StorageDistributedDirectoryMonitor::StorageDistributedDirectoryMonitor( , log{&Logger::get(getLoggerName())} , monitor_blocker(monitor_blocker_) { - const Settings & settings = storage.global_context.getSettingsRef(); - should_batch_inserts = settings.distributed_directory_monitor_batch_inserts; - min_batched_block_size_rows = settings.min_insert_block_size_rows; - min_batched_block_size_bytes = settings.min_insert_block_size_bytes; } diff --git a/dbms/src/Storages/Distributed/DirectoryMonitor.h b/dbms/src/Storages/Distributed/DirectoryMonitor.h index 7e8f6a298f7..ec642d93819 100644 --- a/dbms/src/Storages/Distributed/DirectoryMonitor.h +++ b/dbms/src/Storages/Distributed/DirectoryMonitor.h @@ -20,7 +20,7 @@ class StorageDistributedDirectoryMonitor { public: StorageDistributedDirectoryMonitor( - StorageDistributed & storage_, const std::string & name_, const ConnectionPoolPtr & pool_, ActionBlocker & monitor_blocker_); + StorageDistributed & storage_, std::string name_, ConnectionPoolPtr pool_, ActionBlocker & monitor_blocker_); ~StorageDistributedDirectoryMonitor(); @@ -44,22 +44,22 @@ private: std::string getLoggerName() const; StorageDistributed & storage; - ConnectionPoolPtr pool; - std::string name; + const ConnectionPoolPtr pool; + const std::string name; std::string path; - bool should_batch_inserts = false; - size_t min_batched_block_size_rows = 0; - size_t min_batched_block_size_bytes = 0; + const bool should_batch_inserts = false; + const size_t min_batched_block_size_rows = 0; + const size_t min_batched_block_size_bytes = 0; String current_batch_file_path; struct BatchHeader; struct Batch; size_t error_count{}; - std::chrono::milliseconds default_sleep_time; + const std::chrono::milliseconds default_sleep_time; std::chrono::milliseconds sleep_time; - std::chrono::milliseconds max_sleep_time; + const std::chrono::milliseconds max_sleep_time; std::chrono::time_point last_decrease_time {std::chrono::system_clock::now()}; std::atomic quit {false}; std::mutex mutex; From afaf860810ba388f931cce03f8d0c8d053d95cac Mon Sep 17 00:00:00 2001 From: chertus Date: Tue, 24 Dec 2019 21:51:37 +0300 Subject: [PATCH 43/53] relax ambiguous column check for multiple JOIN ON section --- dbms/src/Interpreters/AnalyzedJoin.cpp | 8 ----- dbms/src/Interpreters/AnalyzedJoin.h | 1 - .../Interpreters/CollectJoinOnKeysVisitor.cpp | 17 +++++++++-- .../Interpreters/CollectJoinOnKeysVisitor.h | 5 ++-- .../Interpreters/DatabaseAndTableWithAlias.h | 14 +++++++++ dbms/src/Interpreters/IdentifierSemantic.cpp | 16 ++++++++++ dbms/src/Interpreters/IdentifierSemantic.h | 3 ++ .../JoinToSubqueryTransformVisitor.cpp | 6 ++-- dbms/src/Interpreters/SyntaxAnalyzer.cpp | 11 +++---- ...01051_same_name_alias_with_joins.reference | 0 .../01051_same_name_alias_with_joins.sql | 29 +++++++++++++++++++ 11 files changed, 88 insertions(+), 22 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/01051_same_name_alias_with_joins.reference create mode 100644 dbms/tests/queries/0_stateless/01051_same_name_alias_with_joins.sql diff --git a/dbms/src/Interpreters/AnalyzedJoin.cpp b/dbms/src/Interpreters/AnalyzedJoin.cpp index 772fcad3eaf..5e4bf1fe53b 100644 --- a/dbms/src/Interpreters/AnalyzedJoin.cpp +++ b/dbms/src/Interpreters/AnalyzedJoin.cpp @@ -98,14 +98,6 @@ NameSet AnalyzedJoin::getQualifiedColumnsSet() const return out; } -NameSet AnalyzedJoin::getOriginalColumnsSet() const -{ - NameSet out; - for (const auto & names : original_names) - out.insert(names.second); - return out; -} - NamesWithAliases AnalyzedJoin::getNamesWithAliases(const NameSet & required_columns) const { NamesWithAliases out; diff --git a/dbms/src/Interpreters/AnalyzedJoin.h b/dbms/src/Interpreters/AnalyzedJoin.h index 9503e2c28e9..677662d949c 100644 --- a/dbms/src/Interpreters/AnalyzedJoin.h +++ b/dbms/src/Interpreters/AnalyzedJoin.h @@ -96,7 +96,6 @@ public: bool hasOn() const { return table_join.on_expression != nullptr; } NameSet getQualifiedColumnsSet() const; - NameSet getOriginalColumnsSet() const; NamesWithAliases getNamesWithAliases(const NameSet & required_columns) const; NamesWithAliases getRequiredColumns(const Block & sample, const Names & action_columns) const; diff --git a/dbms/src/Interpreters/CollectJoinOnKeysVisitor.cpp b/dbms/src/Interpreters/CollectJoinOnKeysVisitor.cpp index 894e1ea3a5a..04484aebe0b 100644 --- a/dbms/src/Interpreters/CollectJoinOnKeysVisitor.cpp +++ b/dbms/src/Interpreters/CollectJoinOnKeysVisitor.cpp @@ -170,11 +170,22 @@ size_t CollectJoinOnKeysMatcher::getTableForIdentifiers(std::vectorname; - bool in_left_table = data.source_columns.count(name); - bool in_right_table = data.joined_columns.count(name); + bool in_left_table = data.left_table.hasColumn(name); + bool in_right_table = data.right_table.hasColumn(name); if (in_left_table && in_right_table) - throw Exception("Column '" + name + "' is ambiguous", ErrorCodes::AMBIGUOUS_COLUMN_NAME); + { + /// Relax ambiguous check for multiple JOINs + if (auto original_name = IdentifierSemantic::uncover(*identifier)) + { + auto match = IdentifierSemantic::canReferColumnToTable(*original_name, data.right_table.table); + if (match == IdentifierSemantic::ColumnMatch::NoMatch) + in_right_table = false; + in_left_table = !in_right_table; + } + else + throw Exception("Column '" + name + "' is ambiguous", ErrorCodes::AMBIGUOUS_COLUMN_NAME); + } if (in_left_table) membership = 1; diff --git a/dbms/src/Interpreters/CollectJoinOnKeysVisitor.h b/dbms/src/Interpreters/CollectJoinOnKeysVisitor.h index 0b4cb1fe857..68109e460e5 100644 --- a/dbms/src/Interpreters/CollectJoinOnKeysVisitor.h +++ b/dbms/src/Interpreters/CollectJoinOnKeysVisitor.h @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -25,8 +26,8 @@ public: struct Data { AnalyzedJoin & analyzed_join; - const NameSet & source_columns; - const NameSet & joined_columns; + const TableWithColumnNames & left_table; + const TableWithColumnNames & right_table; const Aliases & aliases; const bool is_asof{false}; ASTPtr asof_left_key{}; diff --git a/dbms/src/Interpreters/DatabaseAndTableWithAlias.h b/dbms/src/Interpreters/DatabaseAndTableWithAlias.h index 82f0f1fa966..3567a351b14 100644 --- a/dbms/src/Interpreters/DatabaseAndTableWithAlias.h +++ b/dbms/src/Interpreters/DatabaseAndTableWithAlias.h @@ -53,6 +53,20 @@ struct TableWithColumnNames for (auto & column : addition) hidden_columns.push_back(column.name); } + + bool hasColumn(const String & name) const + { + if (columns_set.empty()) + { + columns_set.insert(columns.begin(), columns.end()); + columns_set.insert(hidden_columns.begin(), hidden_columns.end()); + } + + return columns_set.count(name); + } + +private: + mutable NameSet columns_set; }; std::vector getDatabaseAndTables(const ASTSelectQuery & select_query, const String & current_database); diff --git a/dbms/src/Interpreters/IdentifierSemantic.cpp b/dbms/src/Interpreters/IdentifierSemantic.cpp index 959a750c637..17349ab81cd 100644 --- a/dbms/src/Interpreters/IdentifierSemantic.cpp +++ b/dbms/src/Interpreters/IdentifierSemantic.cpp @@ -92,6 +92,22 @@ std::optional IdentifierSemantic::getTableName(const ASTPtr & ast) return {}; } +std::optional IdentifierSemantic::uncover(const ASTIdentifier & identifier) +{ + if (identifier.semantic->covered) + { + std::vector name_parts = identifier.name_parts; + return ASTIdentifier(std::move(name_parts)); + } + return {}; +} + +void IdentifierSemantic::coverName(ASTIdentifier & identifier, const String & alias) +{ + identifier.setShortName(alias); + identifier.semantic->covered = true; +} + bool IdentifierSemantic::canBeAlias(const ASTIdentifier & identifier) { return identifier.semantic->can_be_alias; diff --git a/dbms/src/Interpreters/IdentifierSemantic.h b/dbms/src/Interpreters/IdentifierSemantic.h index dc5859dc05e..e3b69abc61e 100644 --- a/dbms/src/Interpreters/IdentifierSemantic.h +++ b/dbms/src/Interpreters/IdentifierSemantic.h @@ -12,6 +12,7 @@ struct IdentifierSemanticImpl { bool special = false; /// for now it's 'not a column': tables, subselects and some special stuff like FORMAT bool can_be_alias = true; /// if it's a cropped name it could not be an alias + bool covered = false; /// real (compound) name is hidden by an alias (short name) std::optional membership; /// table position in join }; @@ -43,6 +44,8 @@ struct IdentifierSemantic static void setColumnLongName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table); static bool canBeAlias(const ASTIdentifier & identifier); static void setMembership(ASTIdentifier &, size_t table_no); + static void coverName(ASTIdentifier &, const String & alias); + static std::optional uncover(const ASTIdentifier & identifier); static std::optional getMembership(const ASTIdentifier & identifier); static bool chooseTable(const ASTIdentifier &, const std::vector & tables, size_t & best_table_pos, bool ambiguous = false); diff --git a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp index 7e8174b7955..60d8651f453 100644 --- a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp +++ b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp @@ -159,7 +159,7 @@ struct ColumnAliasesMatcher aliases[alias] = long_name; rev_aliases[long_name].push_back(alias); - identifier->setShortName(alias); + IdentifierSemantic::coverName(*identifier, alias); if (is_public) { identifier->setAlias(long_name); @@ -177,7 +177,7 @@ struct ColumnAliasesMatcher if (is_public && allowed_long_names.count(long_name)) ; /// leave original name unchanged for correct output else - identifier->setShortName(it->second[0]); + IdentifierSemantic::coverName(*identifier, it->second[0]); } } } @@ -229,7 +229,7 @@ struct ColumnAliasesMatcher if (!last_table) { - node.setShortName(alias); + IdentifierSemantic::coverName(node, alias); node.setAlias(""); } } diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index aa244909109..a26d8b8253a 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -532,8 +532,8 @@ void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_defaul } /// Find the columns that are obtained by JOIN. -void collectJoinedColumns(AnalyzedJoin & analyzed_join, const ASTSelectQuery & select_query, const NameSet & source_columns, - const Aliases & aliases) +void collectJoinedColumns(AnalyzedJoin & analyzed_join, const ASTSelectQuery & select_query, + const std::vector & tables, const Aliases & aliases) { const ASTTablesInSelectQueryElement * node = select_query.join(); if (!node) @@ -551,7 +551,7 @@ void collectJoinedColumns(AnalyzedJoin & analyzed_join, const ASTSelectQuery & s { bool is_asof = (table_join.strictness == ASTTableJoin::Strictness::Asof); - CollectJoinOnKeysVisitor::Data data{analyzed_join, source_columns, analyzed_join.getOriginalColumnsSet(), aliases, is_asof}; + CollectJoinOnKeysVisitor::Data data{analyzed_join, tables[0], tables[1], aliases, is_asof}; CollectJoinOnKeysVisitor(data).visit(table_join.on_expression); if (!data.has_some) throw Exception("Cannot get JOIN keys from JOIN ON section: " + queryToString(table_join.on_expression), @@ -820,6 +820,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( if (storage) collectSourceColumns(storage->getColumns(), result.source_columns, (select_query != nullptr)); NameSet source_columns_set = removeDuplicateColumns(result.source_columns); + std::vector tables_with_columns; if (select_query) { @@ -837,7 +838,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( } std::vector table_expressions = getTableExpressions(*select_query); - auto tables_with_columns = getTablesWithColumns(table_expressions, context); + tables_with_columns = getTablesWithColumns(table_expressions, context); if (tables_with_columns.empty()) { @@ -935,7 +936,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( setJoinStrictness(*select_query, settings.join_default_strictness, settings.any_join_distinct_right_table_keys, result.analyzed_join->table_join); - collectJoinedColumns(*result.analyzed_join, *select_query, source_columns_set, result.aliases); + collectJoinedColumns(*result.analyzed_join, *select_query, tables_with_columns, result.aliases); } result.aggregates = getAggregates(query); diff --git a/dbms/tests/queries/0_stateless/01051_same_name_alias_with_joins.reference b/dbms/tests/queries/0_stateless/01051_same_name_alias_with_joins.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/01051_same_name_alias_with_joins.sql b/dbms/tests/queries/0_stateless/01051_same_name_alias_with_joins.sql new file mode 100644 index 00000000000..f42eea468fa --- /dev/null +++ b/dbms/tests/queries/0_stateless/01051_same_name_alias_with_joins.sql @@ -0,0 +1,29 @@ +DROP TABLE IF EXISTS a; +DROP TABLE IF EXISTS b; +DROP TABLE IF EXISTS c; + +CREATE TABLE a (x UInt64) ENGINE = Memory; +CREATE TABLE b (x UInt64) ENGINE = Memory; +CREATE TABLE c (x UInt64) ENGINE = Memory; + +SET enable_optimize_predicate_expression = 0; + +SELECT a.x AS x FROM a +LEFT JOIN b ON a.x = b.x +LEFT JOIN c ON a.x = c.x; + +SELECT a.x AS x FROM a +LEFT JOIN b ON a.x = b.x +LEFT JOIN c ON b.x = c.x; + +SELECT b.x AS x FROM a +LEFT JOIN b ON a.x = b.x +LEFT JOIN c ON b.x = c.x; + +SELECT c.x AS x FROM a +LEFT JOIN b ON a.x = b.x +LEFT JOIN c ON b.x = c.x; + +DROP TABLE a; +DROP TABLE b; +DROP TABLE c; From 34d2afa3549f74d672d72303ec8b3017e1e816a8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 24 Dec 2019 23:03:33 +0300 Subject: [PATCH 44/53] Fix obvious bug --- dbms/src/Storages/AlterCommands.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/AlterCommands.cpp b/dbms/src/Storages/AlterCommands.cpp index 535105e5264..217f7787d75 100644 --- a/dbms/src/Storages/AlterCommands.cpp +++ b/dbms/src/Storages/AlterCommands.cpp @@ -225,7 +225,9 @@ void AlterCommand::apply(ColumnsDescription & columns_description, IndicesDescri column.default_desc.kind = default_kind; column.default_desc.expression = default_expression; } - column.comment = *comment; + if (comment) + column.comment = *comment; + column.codec = codec; column.ttl = ttl; From 80f42a1f3b4ee20dc8d573c7fc671df8003015a1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 24 Dec 2019 23:07:44 +0300 Subject: [PATCH 45/53] Show physical addresses in StackTrace --- dbms/src/Common/Dwarf.h | 4 ++-- dbms/src/Common/StackTrace.cpp | 17 ++++++++++------- dbms/src/Common/SymbolIndex.h | 1 + dbms/src/Common/tests/symbol_index.cpp | 2 +- 4 files changed, 14 insertions(+), 10 deletions(-) diff --git a/dbms/src/Common/Dwarf.h b/dbms/src/Common/Dwarf.h index 2f97212d4d7..9abb526a210 100644 --- a/dbms/src/Common/Dwarf.h +++ b/dbms/src/Common/Dwarf.h @@ -127,8 +127,8 @@ public: uint64_t line = 0; }; - /** - * Find the file and line number information corresponding to address. + /** Find the file and line number information corresponding to address. + * The address must be physical - offset in object file without offset in virtual memory where the object is loaded. */ bool findAddress(uintptr_t address, LocationInfo & info, LocationInfoMode mode) const; diff --git a/dbms/src/Common/StackTrace.cpp b/dbms/src/Common/StackTrace.cpp index 2f3c4e9c2fa..597ed2028fa 100644 --- a/dbms/src/Common/StackTrace.cpp +++ b/dbms/src/Common/StackTrace.cpp @@ -258,10 +258,14 @@ static void toStringEveryLineImpl(const StackTrace::Frames & frames, size_t offs for (size_t i = offset; i < size; ++i) { - const void * addr = frames[i]; + const void * virtual_addr = frames[i]; + auto object = symbol_index.findObject(virtual_addr); + uintptr_t virtual_offset = object ? uintptr_t(object->address_begin) : 0; + const void * physical_addr = reinterpret_cast(uintptr_t(virtual_addr) - virtual_offset); - out << i << ". " << addr << " "; - auto symbol = symbol_index.findSymbol(addr); + out << i << ". " << physical_addr << " "; + + auto symbol = symbol_index.findSymbol(virtual_addr); if (symbol) { int status = 0; @@ -272,18 +276,17 @@ static void toStringEveryLineImpl(const StackTrace::Frames & frames, size_t offs out << " "; - if (auto object = symbol_index.findObject(addr)) + if (object) { if (std::filesystem::exists(object->name)) { auto dwarf_it = dwarfs.try_emplace(object->name, *object->elf).first; DB::Dwarf::LocationInfo location; - if (dwarf_it->second.findAddress(uintptr_t(addr) - uintptr_t(object->address_begin), location, DB::Dwarf::LocationInfoMode::FAST)) + if (dwarf_it->second.findAddress(uintptr_t(physical_addr), location, DB::Dwarf::LocationInfoMode::FAST)) out << location.file.toString() << ":" << location.line; - else - out << object->name; } + out << " in " << object->name; } else out << "?"; diff --git a/dbms/src/Common/SymbolIndex.h b/dbms/src/Common/SymbolIndex.h index 0e249c59bb2..1e762780dad 100644 --- a/dbms/src/Common/SymbolIndex.h +++ b/dbms/src/Common/SymbolIndex.h @@ -38,6 +38,7 @@ public: std::unique_ptr elf; }; + /// Address in virtual memory should be passed. These addresses include offset where the object is loaded in memory. const Symbol * findSymbol(const void * address) const; const Object * findObject(const void * address) const; diff --git a/dbms/src/Common/tests/symbol_index.cpp b/dbms/src/Common/tests/symbol_index.cpp index d1867cb524e..1c7e0ffc27d 100644 --- a/dbms/src/Common/tests/symbol_index.cpp +++ b/dbms/src/Common/tests/symbol_index.cpp @@ -49,7 +49,7 @@ int main(int argc, char ** argv) Dwarf dwarf(*object->elf); Dwarf::LocationInfo location; - if (dwarf.findAddress(uintptr_t(address), location, Dwarf::LocationInfoMode::FAST)) + if (dwarf.findAddress(uintptr_t(address) - uintptr_t(info.dli_fbase), location, Dwarf::LocationInfoMode::FAST)) std::cerr << location.file.toString() << ":" << location.line << "\n"; else std::cerr << "Dwarf: Not found\n"; From daae6ba32011c6b6c4df4aeb9d96fa5b7e1805ad Mon Sep 17 00:00:00 2001 From: tai Date: Wed, 25 Dec 2019 13:32:43 +0800 Subject: [PATCH 46/53] fix docs --- docs/en/interfaces/http.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index cecb4e79a15..4383318f98f 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -189,7 +189,7 @@ $ echo 'SELECT 1' | curl 'http://localhost:8123/?user=user&password=password' -d If the user name is not specified, the `default` name is used. If the password is not specified, the empty password is used. You can also use the URL parameters to specify any settings for processing a single query, or entire profiles of settings. Example:http://localhost:8123/?profile=web&max_rows_to_read=1000000000&query=SELECT+1 -For more information, see the [Settings][../operations/settings/index.md] section. +For more information, see the [Settings](../operations/settings/index.md) section. ```bash $ echo 'SELECT number FROM system.numbers LIMIT 10' | curl 'http://localhost:8123/?' --data-binary @- From c72715128057438542cd7e07c5089c83bb9f8f7d Mon Sep 17 00:00:00 2001 From: BayoNet Date: Wed, 25 Dec 2019 09:48:43 +0300 Subject: [PATCH 47/53] DOCS-265: max_http_get_redirects. EN review. RU translation. (#8389) * Update settings.md (#73) * Update url.md (#74) * CLICKHOUSEDOCS-265: EN review, RU translation Co-authored-by: FeehanG <51821376+FeehanG@users.noreply.github.com> --- docs/en/operations/settings/settings.md | 4 ++-- docs/en/operations/table_engines/url.md | 2 +- docs/ru/operations/settings/settings.md | 11 +++++++++++ docs/ru/operations/table_engines/url.md | 2 ++ 4 files changed, 16 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 9117fcb2543..ba7370fef03 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -132,11 +132,11 @@ Default value: 0. ## max_http_get_redirects {#setting-max_http_get_redirects} -Limits the maximum number of HTTP GET redirect hops for [URL](../table_engines/url.md)-engine tables. The setting applies to the both types of tables: created by [CREATE TABLE](../../query_language/create/#create-table-query) query and by [url](../../query_language/table_functions/url.md) table function. +Limits the maximum number of HTTP GET redirect hops for [URL](../table_engines/url.md)-engine tables. The setting applies to both types of tables: those created by the [CREATE TABLE](../../query_language/create/#create-table-query) query and by the [url](../../query_language/table_functions/url.md) table function. Possible values: -- Positive integer number of hops. +- Any positive integer number of hops. - 0 — No hops allowed. Default value: 0. diff --git a/docs/en/operations/table_engines/url.md b/docs/en/operations/table_engines/url.md index cb7b57b35c3..4e46c58b843 100644 --- a/docs/en/operations/table_engines/url.md +++ b/docs/en/operations/table_engines/url.md @@ -17,7 +17,7 @@ additional headers for getting a response from the server. respectively. For processing `POST` requests, the remote server must support [Chunked transfer encoding](https://en.wikipedia.org/wiki/Chunked_transfer_encoding). -You can limit the maximum number of HTTP GET redirect hops by the [max_http_get_redirects](../settings/settings.md#setting-max_http_get_redirects) setting. +You can limit the maximum number of HTTP GET redirect hops using the [max_http_get_redirects](../settings/settings.md#setting-max_http_get_redirects) setting. **Example:** diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index e0045bd42ef..2d5a11bec86 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -127,6 +127,17 @@ ClickHouse применяет настройку в тех случаях, ко Значение по умолчанию — 0. +## max_http_get_redirects {#setting-max_http_get_redirects} + +Ограничивает максимальное количество переходов по редиректам в таблицах с движком [URL](../table_engines/url.md) при выполнении HTTP запросов методом GET. Настройка применяется для обоих типов таблиц: созданных запросом [CREATE TABLE](../../query_language/create/#create-table-query) и с помощью табличной функции [url](../../query_language/table_functions/url.md). + +Возможные значения: + +- Положительное целое число переходов. +- 0 — переходы запрещены. + +Значение по умолчанию: 0. + ## input_format_allow_errors_num Устанавливает максимальное количество допустимых ошибок при чтении из текстовых форматов (CSV, TSV и т.п.). diff --git a/docs/ru/operations/table_engines/url.md b/docs/ru/operations/table_engines/url.md index 8e8313e814e..8a6a6790a15 100644 --- a/docs/ru/operations/table_engines/url.md +++ b/docs/ru/operations/table_engines/url.md @@ -17,6 +17,8 @@ соответственно. Для обработки `POST`-запросов удаленный сервер должен поддерживать [Chunked transfer encoding](https://ru.wikipedia.org/wiki/Chunked_transfer_encoding). +Максимальное количество переходов по редиректам при выполнении HTTP-запроса методом GET можно ограничить с помощью настройки [max_http_get_redirects](../settings/settings.md#setting-max_http_get_redirects). + **Пример:** **1.** Создадим на сервере таблицу `url_engine_table`: From 6e785ea4bd0fbb4cf0ddc471bb49662a01bddbbc Mon Sep 17 00:00:00 2001 From: BayoNet Date: Wed, 25 Dec 2019 13:42:00 +0300 Subject: [PATCH 48/53] DOCS-173: clickhouse benchmark (#7947) * CLICKHOUSEDOCS-173: Draft of a description. * CLICKHOUSEDOCS-173: Table experiment. * CLICKHOUSEDOCS-173: Further writing. * CLICKHOUSEDOCS-173: Final. * CLICKHOUSEDOCS-173: Updated by comments. * CLICKHOUSEDOCS-173: Updated by comments. * CLICKHOUSEDOCS-173: Supported other languages. --- .../operations/utils/clickhouse-benchmark.md | 146 ++++++++++++++++++ docs/en/operations/utils/index.md | 6 +- .../operations/utils/clickhouse-benchmark.md | 1 + .../operations/utils/clickhouse-benchmark.md | 1 + docs/toc_en.yml | 1 + docs/toc_fa.yml | 3 +- docs/toc_ja.yml | 3 +- docs/toc_zh.yml | 3 +- .../operations/utils/clickhouse-benchmark.md | 1 + docs/zh/operations/utils/index.md | 6 +- 10 files changed, 162 insertions(+), 9 deletions(-) create mode 100644 docs/en/operations/utils/clickhouse-benchmark.md create mode 120000 docs/fa/operations/utils/clickhouse-benchmark.md create mode 120000 docs/ja/operations/utils/clickhouse-benchmark.md create mode 120000 docs/zh/operations/utils/clickhouse-benchmark.md diff --git a/docs/en/operations/utils/clickhouse-benchmark.md b/docs/en/operations/utils/clickhouse-benchmark.md new file mode 100644 index 00000000000..5707158e671 --- /dev/null +++ b/docs/en/operations/utils/clickhouse-benchmark.md @@ -0,0 +1,146 @@ +# clickhouse-benchmark + +Connects to a ClickHouse server and repeatedly sends specified queries. + +Syntax: + +```bash +$ echo "single query" | clickhouse-benchmark [keys] +``` +or +```bash +$ clickhouse-benchmark [keys] <<< "single query" +``` + +If you want to send a set of queries, create a text file and place each query on the individual string in this file. For example: + +```sql +SELECT * FROM system.numbers LIMIT 10000000 +SELECT 1 +``` + +Then pass this file to a standard input of `clickhouse-benchmark`. + +```bash +clickhouse-benchmark [keys] < queries_file +``` + +## Keys {#clickhouse-benchmark-keys} + +- `-c N`, `--concurrency=N` — Number of queries that `clickhouse-benchmark` sends simultaneously. Default value: 1. +- `-d N`, `--delay=N` — Interval in seconds between intermediate reports (set 0 to disable reports). Default value: 1. +- `-h WORD`, `--host=WORD` — Server host. Default value: `localhost`. For the [comparison mode](#clickhouse-benchmark-comparison-mode) you can use multiple `-h` keys. +- `-p N`, `--port=N` — Server port. Default value: 9000. For the [comparison mode](#clickhouse-benchmark-comparison-mode) you can use multiple `-p` keys. +- `-i N`, `--iterations=N` — Total number of queries. Default value: 0. +- `-r`, `--randomize` — Random order of queries execution if there is more then one input query. +- `-s`, `--secure` — Using TLS connection. +- `-t N`, `--timelimit=N` — Time limit in seconds. `clickhouse-benchmark` stops sending queries when the specified time limit is reached. Default value: 0 (time limit disabled). +- `--confidence=N` — Level of confidence for T-test. Possible values: 0 (80%), 1 (90%), 2 (95%), 3 (98%), 4 (99%), 5 (99.5%). Default value: 5. In the [comparison mode](#clickhouse-benchmark-comparison-mode) `clickhouse-benchmark` performs the [Independent two-sample Student's t-test](https://en.wikipedia.org/wiki/Student%27s_t-test#Independent_two-sample_t-test) test to determine whether the two distributions aren't different with the selected level of confidence. +- `--cumulative` — Printing cumulative data instead of data per interval. +- `--database=DATABASE_NAME` — ClickHouse database name. Default value: `default`. +- `--json=FILEPATH` — JSON output. When the key is set, `clickhouse-benchmark` outputs a report to the specified JSON-file. +- `--user=USERNAME` — ClickHouse user name. Default value: `default`. +- `--password=PSWD` — ClickHouse user password. Default value: empty string. +- `--stacktrace` — Stack traces output. When the key is set, `clickhouse-bencmark` outputs stack traces of exceptions. +- `--stage=WORD` — Query processing stage at server. ClickHouse stops query processing and returns answer to `clickhouse-benchmark` at the specified stage. Possible values: `complete`, `fetch_columns`, `with_mergeable_state`. Default value: `complete`. +- `--help` — Shows the help message. + +If you want to apply some [settings](../../operations/settings/index.md) for queries, pass them as a key `--= SETTING_VALUE`. For example, `--max_memory_usage=1048576`. + +## Output {#clickhouse-benchmark-output} + +By default, `clickhouse-benchmark` reports for each `--delay` interval. + +Example of the report: + +```text +Queries executed: 10. + +localhost:9000, queries 10, QPS: 6.772, RPS: 67904487.440, MiB/s: 518.070, result RPS: 67721584.984, result MiB/s: 516.675. + +0.000% 0.145 sec. +10.000% 0.146 sec. +20.000% 0.146 sec. +30.000% 0.146 sec. +40.000% 0.147 sec. +50.000% 0.148 sec. +60.000% 0.148 sec. +70.000% 0.148 sec. +80.000% 0.149 sec. +90.000% 0.150 sec. +95.000% 0.150 sec. +99.000% 0.150 sec. +99.900% 0.150 sec. +99.990% 0.150 sec. +``` + +In the report you can find: + +- Number of queries in the `Queries executed: ` field. +- Status string containing (in order): + + - Endpoint of ClickHouse server. + - Number of processed queries. + - QPS: QPS: How many queries server performed per second during a period specified in the `--delay` argument. + - RPS: How many rows server read per second during a period specified in the `--delay` argument. + - MiB/s: How many mebibytes server read per second during a period specified in the `--delay` argument. + - result RPS: How many rows placed by server to the result of a query per second during a period specified in the `--delay` argument. + - result MiB/s. How many mebibytes placed by server to the result of a query per second during a period specified in the `--delay` argument. + +- Percentiles of queries execution time. + + +## Comparison mode {#clickhouse-benchmark-comparison-mode} + +`clickhouse-benchmark` can compare performances for two running ClickHouse servers. + +To use the comparison mode, specify endpoints of both servers by two pairs of `--host`, `--port` keys. Keys matched together by position in arguments list, the first `--host` is matched with the first `--port` and so on. `clickhouse-benchmark` establishes connections to both servers, then sends queries. Each query addressed to a randomly selected server. The results are shown for each server separately. + +## Example {#clickhouse-benchmark-example} + +```bash +$ echo "SELECT * FROM system.numbers LIMIT 10000000 OFFSET 10000000" | clickhouse-benchmark -i 10 +``` +```text +Loaded 1 queries. + +Queries executed: 6. + +localhost:9000, queries 6, QPS: 6.153, RPS: 123398340.957, MiB/s: 941.455, result RPS: 61532982.200, result MiB/s: 469.459. + +0.000% 0.159 sec. +10.000% 0.159 sec. +20.000% 0.159 sec. +30.000% 0.160 sec. +40.000% 0.160 sec. +50.000% 0.162 sec. +60.000% 0.164 sec. +70.000% 0.165 sec. +80.000% 0.166 sec. +90.000% 0.166 sec. +95.000% 0.167 sec. +99.000% 0.167 sec. +99.900% 0.167 sec. +99.990% 0.167 sec. + + + +Queries executed: 10. + +localhost:9000, queries 10, QPS: 6.082, RPS: 121959604.568, MiB/s: 930.478, result RPS: 60815551.642, result MiB/s: 463.986. + +0.000% 0.159 sec. +10.000% 0.159 sec. +20.000% 0.160 sec. +30.000% 0.163 sec. +40.000% 0.164 sec. +50.000% 0.165 sec. +60.000% 0.166 sec. +70.000% 0.166 sec. +80.000% 0.167 sec. +90.000% 0.167 sec. +95.000% 0.170 sec. +99.000% 0.172 sec. +99.900% 0.172 sec. +99.990% 0.172 sec. +``` diff --git a/docs/en/operations/utils/index.md b/docs/en/operations/utils/index.md index ca0f0954150..3fcd66da8d8 100644 --- a/docs/en/operations/utils/index.md +++ b/docs/en/operations/utils/index.md @@ -1,7 +1,7 @@ # ClickHouse Utility -* [clickhouse-local](clickhouse-local.md) — Allows running SQL queries on data without stopping the ClickHouse server, similar to how `awk` does this. -* [clickhouse-copier](clickhouse-copier.md) — Copies (and reshards) data from one cluster to another cluster. - +- [clickhouse-local](clickhouse-local.md) — Allows running SQL queries on data without stopping the ClickHouse server, similar to how `awk` does this. +- [clickhouse-copier](clickhouse-copier.md) — Copies (and reshards) data from one cluster to another cluster. +- [clickhouse-benchmark](clickhouse-benchmark.md) — Loads server with the custom queries and settings. [Original article](https://clickhouse.yandex/docs/en/operations/utils/) diff --git a/docs/fa/operations/utils/clickhouse-benchmark.md b/docs/fa/operations/utils/clickhouse-benchmark.md new file mode 120000 index 00000000000..133b4d2e511 --- /dev/null +++ b/docs/fa/operations/utils/clickhouse-benchmark.md @@ -0,0 +1 @@ +../../../en/operations/utils/clickhouse-benchmark.md \ No newline at end of file diff --git a/docs/ja/operations/utils/clickhouse-benchmark.md b/docs/ja/operations/utils/clickhouse-benchmark.md new file mode 120000 index 00000000000..133b4d2e511 --- /dev/null +++ b/docs/ja/operations/utils/clickhouse-benchmark.md @@ -0,0 +1 @@ +../../../en/operations/utils/clickhouse-benchmark.md \ No newline at end of file diff --git a/docs/toc_en.yml b/docs/toc_en.yml index 8a2b32b240a..a11c40e4907 100644 --- a/docs/toc_en.yml +++ b/docs/toc_en.yml @@ -212,6 +212,7 @@ nav: - 'Overview': 'operations/utils/index.md' - 'clickhouse-copier': 'operations/utils/clickhouse-copier.md' - 'clickhouse-local': 'operations/utils/clickhouse-local.md' + - 'clickhouse-benchmark': 'operations/utils/clickhouse-benchmark.md' - 'Development': - 'hidden': 'development/index.md' diff --git a/docs/toc_fa.yml b/docs/toc_fa.yml index c5a2a7fd80b..710a2ee20f8 100644 --- a/docs/toc_fa.yml +++ b/docs/toc_fa.yml @@ -208,7 +208,8 @@ nav: - 'Overview': 'operations/utils/index.md' - 'clickhouse-copier': 'operations/utils/clickhouse-copier.md' - 'clickhouse-local': 'operations/utils/clickhouse-local.md' - + - 'clickhouse-benchmark': 'operations/utils/clickhouse-benchmark.md' + - 'F.A.Q.': - 'General Questions': 'faq/general.md' diff --git a/docs/toc_ja.yml b/docs/toc_ja.yml index 8a2b32b240a..945042f0fef 100644 --- a/docs/toc_ja.yml +++ b/docs/toc_ja.yml @@ -212,7 +212,8 @@ nav: - 'Overview': 'operations/utils/index.md' - 'clickhouse-copier': 'operations/utils/clickhouse-copier.md' - 'clickhouse-local': 'operations/utils/clickhouse-local.md' - + - 'clickhouse-benchmark': 'operations/utils/clickhouse-benchmark.md' + - 'Development': - 'hidden': 'development/index.md' - 'Overview of ClickHouse Architecture': 'development/architecture.md' diff --git a/docs/toc_zh.yml b/docs/toc_zh.yml index 7395dcfe145..09f9875069b 100644 --- a/docs/toc_zh.yml +++ b/docs/toc_zh.yml @@ -207,7 +207,8 @@ nav: - '介绍': 'operations/utils/index.md' - 'clickhouse-copier': 'operations/utils/clickhouse-copier.md' - 'clickhouse-local': 'operations/utils/clickhouse-local.md' - + - 'clickhouse-benchmark': 'operations/utils/clickhouse-benchmark.md' + - '常见问题': - '一般的问题': 'faq/general.md' diff --git a/docs/zh/operations/utils/clickhouse-benchmark.md b/docs/zh/operations/utils/clickhouse-benchmark.md new file mode 120000 index 00000000000..133b4d2e511 --- /dev/null +++ b/docs/zh/operations/utils/clickhouse-benchmark.md @@ -0,0 +1 @@ +../../../en/operations/utils/clickhouse-benchmark.md \ No newline at end of file diff --git a/docs/zh/operations/utils/index.md b/docs/zh/operations/utils/index.md index ca0f0954150..3fcd66da8d8 100644 --- a/docs/zh/operations/utils/index.md +++ b/docs/zh/operations/utils/index.md @@ -1,7 +1,7 @@ # ClickHouse Utility -* [clickhouse-local](clickhouse-local.md) — Allows running SQL queries on data without stopping the ClickHouse server, similar to how `awk` does this. -* [clickhouse-copier](clickhouse-copier.md) — Copies (and reshards) data from one cluster to another cluster. - +- [clickhouse-local](clickhouse-local.md) — Allows running SQL queries on data without stopping the ClickHouse server, similar to how `awk` does this. +- [clickhouse-copier](clickhouse-copier.md) — Copies (and reshards) data from one cluster to another cluster. +- [clickhouse-benchmark](clickhouse-benchmark.md) — Loads server with the custom queries and settings. [Original article](https://clickhouse.yandex/docs/en/operations/utils/) From e3cb9043ab9623a0d7d8705b4e4be18d6d035320 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 25 Dec 2019 14:01:58 +0300 Subject: [PATCH 49/53] log_to_console.xml should log to console. Confusingly, it doesn't specify 'logger.console' now. --- dbms/programs/server/config.d/log_to_console.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/programs/server/config.d/log_to_console.xml b/dbms/programs/server/config.d/log_to_console.xml index 86160407465..227c53647f3 100644 --- a/dbms/programs/server/config.d/log_to_console.xml +++ b/dbms/programs/server/config.d/log_to_console.xml @@ -1,5 +1,6 @@ + true From 3b6e3db0c285e6d89adac213b6a207723886a112 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 25 Dec 2019 15:34:04 +0300 Subject: [PATCH 50/53] Remove redundant alter_lock, because parts cannot be altered concurrently. --- dbms/src/Interpreters/MutationsInterpreter.cpp | 2 ++ dbms/src/Interpreters/MutationsInterpreter.h | 7 ++++--- dbms/src/Storages/MergeTree/MergeTreeData.h | 4 +--- dbms/src/Storages/MergeTree/MergeTreeDataPart.h | 10 ---------- dbms/src/Storages/StorageMergeTree.cpp | 1 - 5 files changed, 7 insertions(+), 17 deletions(-) diff --git a/dbms/src/Interpreters/MutationsInterpreter.cpp b/dbms/src/Interpreters/MutationsInterpreter.cpp index 31470b08468..8ff10e92dee 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.cpp +++ b/dbms/src/Interpreters/MutationsInterpreter.cpp @@ -339,6 +339,8 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) affected_materialized.emplace(mat_column); } + /// Just to be sure, that we don't change type + /// after update expression execution. const auto & update_expr = kv.second; auto updated_column = makeASTFunction("CAST", makeASTFunction("if", diff --git a/dbms/src/Interpreters/MutationsInterpreter.h b/dbms/src/Interpreters/MutationsInterpreter.h index a5c1df778cb..4aac3182205 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.h +++ b/dbms/src/Interpreters/MutationsInterpreter.h @@ -65,8 +65,9 @@ private: /// Each stage has output_columns that contain columns that are changed at the end of that stage /// plus columns needed for the next mutations. /// - /// First stage is special: it can contain only DELETEs and is executed using InterpreterSelectQuery - /// to take advantage of table indexes (if there are any). + /// First stage is special: it can contain only filters and is executed using InterpreterSelectQuery + /// to take advantage of table indexes (if there are any). It's necessary because all mutations have + /// `WHERE clause` part. struct Stage { @@ -83,7 +84,7 @@ private: /// A chain of actions needed to execute this stage. /// First steps calculate filter columns for DELETEs (in the same order as in `filter_column_names`), - /// then there is (possibly) an UPDATE stage, and finally a projection stage. + /// then there is (possibly) an UPDATE step, and finally a projection step. ExpressionActionsChain expressions_chain; Names filter_column_names; }; diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 248be299fa3..affdade1c9b 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -234,7 +234,7 @@ public: const NamesAndTypesList & getNewColumns() const { return new_columns; } const DataPart::Checksums & getNewChecksums() const { return new_checksums; } - AlterDataPartTransaction(DataPartPtr data_part_) : data_part(data_part_), alter_lock(data_part->alter_mutex) {} + AlterDataPartTransaction(DataPartPtr data_part_) : data_part(data_part_) {} const DataPartPtr & getDataPart() const { return data_part; } bool isValid() const; @@ -244,9 +244,7 @@ public: bool valid = true; - //don't interchange order of data_part & alter_lock DataPartPtr data_part; - DataPartsLock alter_lock; DataPart::Checksums new_checksums; NamesAndTypesList new_columns; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataPart.h b/dbms/src/Storages/MergeTree/MergeTreeDataPart.h index d47411c9068..cf62b84f7ba 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataPart.h +++ b/dbms/src/Storages/MergeTree/MergeTreeDataPart.h @@ -227,16 +227,6 @@ struct MergeTreeDataPart */ mutable std::shared_mutex columns_lock; - /** It is taken for the whole time ALTER a part: from the beginning of the recording of the temporary files to their renaming to permanent. - * It is taken with unlocked `columns_lock`. - * - * NOTE: "You can" do without this mutex if you could turn ReadRWLock into WriteRWLock without removing the lock. - * This transformation is impossible, because it would create a deadlock, if you do it from two threads at once. - * Taking this mutex means that we want to lock columns_lock on read with intention then, not - * unblocking, block it for writing. - */ - mutable std::mutex alter_mutex; - MergeTreeIndexGranularityInfo index_granularity_info; ~MergeTreeDataPart(); diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 9d284800712..f1df178e810 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -454,7 +454,6 @@ void StorageMergeTree::mutate(const MutationCommands & commands, const Context & auto check = [version, this]() { return isMutationDone(version); }; std::unique_lock lock(mutation_wait_mutex); mutation_wait_event.wait(lock, check); - } } From 6701adceb702fc8b776730c2a60a47ae9f9ab6ef Mon Sep 17 00:00:00 2001 From: BayoNet Date: Wed, 25 Dec 2019 16:42:40 +0300 Subject: [PATCH 51/53] DOCS-139: FINAL. EN review, RU translation. (#8391) * Update select.md (#78) * CLICKHOUSEDOCS-85: RU translation. * Update docs/en/query_language/select.md Co-Authored-By: Ivan Blinkov * Update docs/en/query_language/select.md Co-Authored-By: Ivan Blinkov * Update docs/ru/query_language/select.md Co-Authored-By: Ivan Blinkov Co-authored-by: FeehanG <51821376+FeehanG@users.noreply.github.com> Co-authored-by: Ivan Blinkov --- docs/en/query_language/select.md | 12 ++++++------ docs/ru/query_language/select.md | 17 +++++++++++++++++ 2 files changed, 23 insertions(+), 6 deletions(-) diff --git a/docs/en/query_language/select.md b/docs/en/query_language/select.md index cfa3a1e4dc7..e6e7676c643 100644 --- a/docs/en/query_language/select.md +++ b/docs/en/query_language/select.md @@ -114,18 +114,18 @@ If a query does not list any columns (for example, `SELECT count() FROM t`), som #### FINAL Modifier {#select-from-final} -Appliable when selecting data from tables of the [MergeTree](../operations/table_engines/mergetree.md)-engine family, except `GraphiteMergeTree`. When `FINAL` is specified, ClickHouse fully merges data before returning the result and thus performs all data transformations that are supposed to happen during merges for given table engine. +Applicable when selecting data from tables from the [MergeTree](../operations/table_engines/mergetree.md)-engine family other than `GraphiteMergeTree`. When `FINAL` is specified, ClickHouse fully merges the data before returning the result and thus performs all data transformations that happen during merges for the given table engine. Also supported for: - [Replicated](../operations/table_engines/replication.md) versions of `MergeTree` engines. -- [View](../operations/table_engines/view.md), [Buffer](../operations/table_engines/buffer.md), [Distributed](../operations/table_engines/distributed.md), [MaterializedView](../operations/table_engines/materializedview.md) engines that operate over other engines, if they created over `MergeTree`-engine tables. +- [View](../operations/table_engines/view.md), [Buffer](../operations/table_engines/buffer.md), [Distributed](../operations/table_engines/distributed.md), and [MaterializedView](../operations/table_engines/materializedview.md) engines that operate over other engines, provided they were created over `MergeTree`-engine tables. -The queries that use `FINAL` are executed slower than similar queries that don't, because: +Queries that use `FINAL` are executed not as fast as similar queries that don't, because: -- Query is executed in a single thread, and data is merged during query execution. -- Queries with `FINAL` read primary key columns additionally to the columns specified in the query. +- Query is executed in a single thread and data is merged during query execution. +- Queries with `FINAL` read primary key columns in addition to the columns specified in the query. -In the most cases, avoid using `FINAL`. +In most cases, avoid using `FINAL`. ### SAMPLE Clause {#select-sample-clause} diff --git a/docs/ru/query_language/select.md b/docs/ru/query_language/select.md index 56549f21e53..6c652557858 100644 --- a/docs/ru/query_language/select.md +++ b/docs/ru/query_language/select.md @@ -114,6 +114,23 @@ Cекция `FROM` определяет источник данных: Модификатор `FINAL` может быть использован в запросе `SELECT` из таблиц семейства [MergeTree](../operations/table_engines/mergetree.md). При указании `FINAL`, данные будут выбираться полностью "домерженными". Стоит учитывать, что использование `FINAL` приводит к чтению также столбцов, относящихся к первичному ключу. Также, запрос будет выполняться в один поток, и при выполнении запроса будет выполняться слияние данных. Это приводит к тому, что при использовании `FINAL`, запрос выполняется медленнее. В большинстве случаев, следует избегать использования `FINAL`. Модификатор `FINAL` может быть использован для всех таблиц семейства `MergeTree`, которые производят преобразования данных в процессе фоновых слияний (кроме GraphiteMergeTree). +#### FINAL Modifier {#select-from-final} + +Применим при выборке данных из таблиц с движками таблиц семейства [MergeTree](../operations/table_engines/mergetree.md), кроме `GraphiteMergeTree`. Если в запросе используется `FINAL`, то ClickHouse полностью мёржит данные перед выдачей результата, таким образом выполняя все преобразования данных, которые производятся движком таблиц при мёржах. + +Также поддержан для движков: + +- [Replicated](../operations/table_engines/replication.md)-версий `MergeTree`. +- [View](../operations/table_engines/view.md), [Buffer](../operations/table_engines/buffer.md), [Distributed](../operations/table_engines/distributed.md), и [MaterializedView](../operations/table_engines/materializedview.md), которые работают поверх других движков, если они созданы для таблиц с движками семейства `MergeTree`. + +Запросы, использующие `FINAL` исполняются медленнее аналогичных запросов без `FINAL`, поскольку: + +- Запрос исполняется в один поток и данные мёржатся в процессе выполнения. +- Запросы с модификатором `FINAL` дополнительно к столбцам, указанным в запросе, читают столбцы первичного ключа. + +По возможности не используйте модификатор `FINAL`. + + ### Секция SAMPLE {#select-sample-clause} Секция `SAMPLE` позволяет выполнять запросы приближённо. Например, чтобы посчитать статистику по всем визитам, можно обработать 1/10 всех визитов и результат домножить на 10. From 17f175469542cdba68b9725c8234a23b9dd9986d Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 25 Dec 2019 19:08:44 +0300 Subject: [PATCH 52/53] Publish different files for perftest. --- docker/packager/binary/build.sh | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 7d6b7a518d4..51ce99a9af1 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -21,6 +21,14 @@ mv ./dbms/unit_tests_dbms /output find . -name '*.so' -print -exec mv '{}' /output \; find . -name '*.so.*' -print -exec mv '{}' /output \; +# Different files for performance test. +if [ "performance" == "$COMBINED_OUTPUT" ] +then + cp -r ../dbms/tests/performance /output + rm /output/unit_tests_dbms ||: + rm /output/clickhouse-odbc-bridge ||: +fi + # May be set for split build or for performance test. if [ "" != "$COMBINED_OUTPUT" ] then From 438ea5bc22efda6e5218fc5ad35af0360c1e7dd1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 25 Dec 2019 22:17:41 +0300 Subject: [PATCH 53/53] Remove context from formats (#8388) * Remove Context from formats. --- dbms/CMakeLists.txt | 4 +- .../ParallelParsingBlockInputStream.cpp | 2 +- .../ParallelParsingBlockInputStream.h | 12 ++-- dbms/src/Formats/FormatFactory.cpp | 62 ++++++++++++++----- dbms/src/Formats/FormatFactory.h | 4 -- dbms/src/Formats/FormatSchemaInfo.cpp | 14 ++--- dbms/src/Formats/FormatSchemaInfo.h | 2 +- dbms/src/Formats/FormatSettings.h | 21 +++++++ dbms/src/Formats/NativeFormat.cpp | 2 - dbms/src/Formats/NullFormat.cpp | 1 - .../Formats/ParsedTemplateFormatString.cpp | 18 +++--- dbms/src/Formats/ParsedTemplateFormatString.h | 5 +- dbms/src/Functions/FunctionsConversion.cpp | 23 ------- dbms/src/Functions/FunctionsConversion.h | 48 +++++++++----- dbms/src/Interpreters/castColumn.cpp | 12 +++- dbms/src/Interpreters/castColumn.h | 2 +- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 4 +- .../Formats/Impl/ArrowColumnToCHColumn.h | 2 +- .../Formats/Impl/BinaryRowInputFormat.cpp | 2 - .../Formats/Impl/BinaryRowOutputFormat.cpp | 2 - .../Formats/Impl/CSVRowInputFormat.cpp | 1 - .../Formats/Impl/CSVRowOutputFormat.cpp | 1 - .../Formats/Impl/CapnProtoRowInputFormat.cpp | 5 +- .../Impl/JSONCompactEachRowRowInputFormat.cpp | 2 - .../JSONCompactEachRowRowOutputFormat.cpp | 2 - .../Impl/JSONCompactRowOutputFormat.cpp | 1 - .../Impl/JSONEachRowRowInputFormat.cpp | 1 - .../Impl/JSONEachRowRowOutputFormat.cpp | 1 - ...JSONEachRowWithProgressRowOutputFormat.cpp | 1 - .../Formats/Impl/JSONRowOutputFormat.cpp | 1 - .../Formats/Impl/MySQLOutputFormat.cpp | 30 ++++----- .../Formats/Impl/MySQLOutputFormat.h | 15 +++-- .../Processors/Formats/Impl/NativeFormat.cpp | 2 - .../Processors/Formats/Impl/NullFormat.cpp | 1 - .../Impl/ODBCDriver2BlockOutputFormat.cpp | 2 +- .../Impl/ODBCDriverBlockOutputFormat.cpp | 1 - .../Formats/Impl/ORCBlockInputFormat.cpp | 9 ++- .../Formats/Impl/ORCBlockInputFormat.h | 4 +- .../Formats/Impl/ParquetBlockInputFormat.cpp | 10 +-- .../Formats/Impl/ParquetBlockInputFormat.h | 4 +- .../Formats/Impl/ParquetBlockOutputFormat.cpp | 1 - .../Formats/Impl/PrettyBlockOutputFormat.cpp | 2 - .../Impl/PrettyCompactBlockOutputFormat.cpp | 3 - .../Impl/PrettySpaceBlockOutputFormat.cpp | 2 - .../Formats/Impl/ProtobufRowInputFormat.cpp | 6 +- .../Formats/Impl/ProtobufRowOutputFormat.cpp | 8 +-- .../Formats/Impl/TSKVRowInputFormat.cpp | 1 - .../Formats/Impl/TSKVRowOutputFormat.cpp | 1 - .../Impl/TabSeparatedRowInputFormat.cpp | 3 - .../Impl/TabSeparatedRowOutputFormat.cpp | 4 -- .../Impl/TemplateBlockOutputFormat.cpp | 26 ++++---- .../Formats/Impl/TemplateBlockOutputFormat.h | 5 +- .../Formats/Impl/TemplateRowInputFormat.cpp | 39 ++++++------ .../Formats/Impl/TemplateRowInputFormat.h | 7 ++- .../Formats/Impl/ValuesBlockInputFormat.cpp | 7 +-- .../Formats/Impl/ValuesBlockInputFormat.h | 7 ++- .../Formats/Impl/ValuesRowOutputFormat.cpp | 1 - .../Formats/Impl/VerticalRowOutputFormat.cpp | 1 - .../Formats/Impl/XMLRowOutputFormat.cpp | 1 - 59 files changed, 234 insertions(+), 227 deletions(-) diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 45d12f8ed93..466b3daf94f 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -186,8 +186,8 @@ endif() list (APPEND clickhouse_common_io_sources ${CONFIG_BUILD}) list (APPEND clickhouse_common_io_headers ${CONFIG_VERSION} ${CONFIG_COMMON}) -list (APPEND dbms_sources src/Functions/IFunction.cpp src/Functions/FunctionFactory.cpp src/Functions/FunctionHelpers.cpp) -list (APPEND dbms_headers src/Functions/IFunctionImpl.h src/Functions/FunctionFactory.h src/Functions/FunctionHelpers.h) +list (APPEND dbms_sources src/Functions/IFunction.cpp src/Functions/FunctionFactory.cpp src/Functions/FunctionHelpers.cpp src/Functions/extractTimeZoneFromFunctionArguments.cpp) +list (APPEND dbms_headers src/Functions/IFunctionImpl.h src/Functions/FunctionFactory.h src/Functions/FunctionHelpers.h src/Functions/extractTimeZoneFromFunctionArguments.h) list (APPEND dbms_sources src/AggregateFunctions/AggregateFunctionFactory.cpp diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp index c894af82580..3f6ddbd7a15 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp @@ -65,7 +65,7 @@ void ParallelParsingBlockInputStream::parserThreadFunction(size_t current_unit_n */ ReadBuffer read_buffer(unit.segment.data(), unit.segment.size(), 0); auto parser = std::make_unique( - input_processor_creator(read_buffer, header, context, + input_processor_creator(read_buffer, header, row_input_format_params, format_settings)); unit.block_ext.block.clear(); diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h index 8c276f2f7dd..89a9d7c8926 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -55,23 +55,21 @@ private: using InputProcessorCreator = std::function; public: struct InputCreatorParams { - const Block &sample; - const Context &context; - const RowInputFormatParams& row_input_format_params; + const Block & sample; + const RowInputFormatParams & row_input_format_params; const FormatSettings &settings; }; struct Params { ReadBuffer & read_buffer; - const InputProcessorCreator &input_processor_creator; - const InputCreatorParams &input_creator_params; + const InputProcessorCreator & input_processor_creator; + const InputCreatorParams & input_creator_params; FormatFactory::FileSegmentationEngine file_segmentation_engine; int max_threads; size_t min_chunk_bytes; @@ -79,7 +77,6 @@ public: explicit ParallelParsingBlockInputStream(const Params & params) : header(params.input_creator_params.sample), - context(params.input_creator_params.context), row_input_format_params(params.input_creator_params.row_input_format_params), format_settings(params.input_creator_params.settings), input_processor_creator(params.input_processor_creator), @@ -149,7 +146,6 @@ protected: private: const Block header; - const Context context; const RowInputFormatParams row_input_format_params; const FormatSettings format_settings; const InputProcessorCreator input_processor_creator; diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index aa65fe5765f..240e591123f 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -12,6 +12,8 @@ #include #include #include +#include +#include namespace DB @@ -34,7 +36,7 @@ const FormatFactory::Creators & FormatFactory::getCreators(const String & name) } -static FormatSettings getInputFormatSetting(const Settings & settings) +static FormatSettings getInputFormatSetting(const Settings & settings, const Context & context) { FormatSettings format_settings; format_settings.csv.delimiter = settings.format_csv_delimiter; @@ -56,11 +58,21 @@ static FormatSettings getInputFormatSetting(const Settings & settings) format_settings.template_settings.row_format = settings.format_template_row; format_settings.template_settings.row_between_delimiter = settings.format_template_rows_between_delimiter; format_settings.tsv.empty_as_default = settings.input_format_tsv_empty_as_default; + format_settings.schema.format_schema = settings.format_schema; + format_settings.schema.format_schema_path = context.getFormatSchemaPath(); + format_settings.schema.is_server = context.hasGlobalContext() && (context.getGlobalContext().getApplicationType() == Context::ApplicationType::SERVER); + format_settings.custom.result_before_delimiter = settings.format_custom_result_before_delimiter; + format_settings.custom.result_after_delimiter = settings.format_custom_result_after_delimiter; + format_settings.custom.escaping_rule = settings.format_custom_escaping_rule; + format_settings.custom.field_delimiter = settings.format_custom_field_delimiter; + format_settings.custom.row_before_delimiter = settings.format_custom_row_before_delimiter; + format_settings.custom.row_after_delimiter = settings.format_custom_row_after_delimiter; + format_settings.custom.row_between_delimiter = settings.format_custom_row_between_delimiter; return format_settings; } -static FormatSettings getOutputFormatSetting(const Settings & settings) +static FormatSettings getOutputFormatSetting(const Settings & settings, const Context & context) { FormatSettings format_settings; format_settings.json.quote_64bit_integers = settings.output_format_json_quote_64bit_integers; @@ -77,6 +89,16 @@ static FormatSettings getOutputFormatSetting(const Settings & settings) format_settings.template_settings.row_between_delimiter = settings.format_template_rows_between_delimiter; format_settings.write_statistics = settings.output_format_write_statistics; format_settings.parquet.row_group_size = settings.output_format_parquet_row_group_size; + format_settings.schema.format_schema = settings.format_schema; + format_settings.schema.format_schema_path = context.getFormatSchemaPath(); + format_settings.schema.is_server = context.hasGlobalContext() && (context.getGlobalContext().getApplicationType() == Context::ApplicationType::SERVER); + format_settings.custom.result_before_delimiter = settings.format_custom_result_before_delimiter; + format_settings.custom.result_after_delimiter = settings.format_custom_result_after_delimiter; + format_settings.custom.escaping_rule = settings.format_custom_escaping_rule; + format_settings.custom.field_delimiter = settings.format_custom_field_delimiter; + format_settings.custom.row_before_delimiter = settings.format_custom_row_before_delimiter; + format_settings.custom.row_after_delimiter = settings.format_custom_row_after_delimiter; + format_settings.custom.row_between_delimiter = settings.format_custom_row_between_delimiter; return format_settings; } @@ -100,9 +122,9 @@ BlockInputStreamPtr FormatFactory::getInput( throw Exception("Format " + name + " is not suitable for input", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT); const Settings & settings = context.getSettingsRef(); - FormatSettings format_settings = getInputFormatSetting(settings); + FormatSettings format_settings = getInputFormatSetting(settings, context); - return input_getter(buf, sample, context, max_block_size, callback ? callback : ReadCallback(), format_settings); + return input_getter(buf, sample, max_block_size, callback ? callback : ReadCallback(), format_settings); } const Settings & settings = context.getSettingsRef(); @@ -118,7 +140,7 @@ BlockInputStreamPtr FormatFactory::getInput( if (!input_getter) throw Exception("Format " + name + " is not suitable for input", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT); - FormatSettings format_settings = getInputFormatSetting(settings); + FormatSettings format_settings = getInputFormatSetting(settings, context); RowInputFormatParams row_input_format_params; row_input_format_params.max_block_size = max_block_size; @@ -128,7 +150,7 @@ BlockInputStreamPtr FormatFactory::getInput( row_input_format_params.max_execution_time = settings.max_execution_time; row_input_format_params.timeout_overflow_mode = settings.timeout_overflow_mode; - auto input_creator_params = ParallelParsingBlockInputStream::InputCreatorParams{sample, context, row_input_format_params, format_settings}; + auto input_creator_params = ParallelParsingBlockInputStream::InputCreatorParams{sample, row_input_format_params, format_settings}; ParallelParsingBlockInputStream::Params params{buf, input_getter, input_creator_params, file_segmentation_engine, static_cast(settings.max_threads), @@ -164,16 +186,16 @@ BlockOutputStreamPtr FormatFactory::getOutput( throw Exception("Format " + name + " is not suitable for output", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_OUTPUT); const Settings & settings = context.getSettingsRef(); - FormatSettings format_settings = getOutputFormatSetting(settings); + FormatSettings format_settings = getOutputFormatSetting(settings, context); /** Materialization is needed, because formats can use the functions `IDataType`, * which only work with full columns. */ return std::make_shared( - output_getter(buf, sample, context, callback, format_settings), sample); + output_getter(buf, sample, std::move(callback), format_settings), sample); } - auto format = getOutputFormat(name, buf, sample, context, callback); + auto format = getOutputFormat(name, buf, sample, context, std::move(callback)); return std::make_shared(std::make_shared(format), sample); } @@ -191,7 +213,7 @@ InputFormatPtr FormatFactory::getInputFormat( throw Exception("Format " + name + " is not suitable for input", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT); const Settings & settings = context.getSettingsRef(); - FormatSettings format_settings = getInputFormatSetting(settings); + FormatSettings format_settings = getInputFormatSetting(settings, context); RowInputFormatParams params; params.max_block_size = max_block_size; @@ -201,7 +223,13 @@ InputFormatPtr FormatFactory::getInputFormat( params.max_execution_time = settings.max_execution_time; params.timeout_overflow_mode = settings.timeout_overflow_mode; - return input_getter(buf, sample, context, params, format_settings); + auto format = input_getter(buf, sample, params, format_settings); + + /// It's a kludge. Because I cannot remove context from values format. + if (auto * values = typeid_cast(format.get())) + values->setContext(context); + + return format; } @@ -213,12 +241,18 @@ OutputFormatPtr FormatFactory::getOutputFormat( throw Exception("Format " + name + " is not suitable for output", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_OUTPUT); const Settings & settings = context.getSettingsRef(); - FormatSettings format_settings = getOutputFormatSetting(settings); + FormatSettings format_settings = getOutputFormatSetting(settings, context); /** TODO: Materialization is needed, because formats can use the functions `IDataType`, * which only work with full columns. */ - return output_getter(buf, sample, context, callback, format_settings); + auto format = output_getter(buf, sample, std::move(callback), format_settings); + + /// It's a kludge. Because I cannot remove context from MySQL format. + if (auto * mysql = typeid_cast(format.get())) + mysql->setContext(context); + + return format; } @@ -259,7 +293,7 @@ void FormatFactory::registerFileSegmentationEngine(const String & name, FileSegm auto & target = dict[name].file_segmentation_engine; if (target) throw Exception("FormatFactory: File segmentation engine " + name + " is already registered", ErrorCodes::LOGICAL_ERROR); - target = file_segmentation_engine; + target = std::move(file_segmentation_engine); } FormatFactory::FormatFactory() diff --git a/dbms/src/Formats/FormatFactory.h b/dbms/src/Formats/FormatFactory.h index ee2cf3ee444..cbf64afeaec 100644 --- a/dbms/src/Formats/FormatFactory.h +++ b/dbms/src/Formats/FormatFactory.h @@ -59,7 +59,6 @@ private: using InputCreator = std::function; @@ -67,21 +66,18 @@ private: using OutputCreator = std::function; using InputProcessorCreator = std::function; using OutputProcessorCreator = std::function; diff --git a/dbms/src/Formats/FormatSchemaInfo.cpp b/dbms/src/Formats/FormatSchemaInfo.cpp index fab8fc7fa63..707f9babe8d 100644 --- a/dbms/src/Formats/FormatSchemaInfo.cpp +++ b/dbms/src/Formats/FormatSchemaInfo.cpp @@ -26,7 +26,7 @@ namespace } -FormatSchemaInfo::FormatSchemaInfo(const Context & context, const String & format_schema, const String & format, bool require_message) +FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String & format, bool require_message, bool is_server, const std::string & format_schema_path) { if (format_schema.empty()) throw Exception( @@ -54,29 +54,25 @@ FormatSchemaInfo::FormatSchemaInfo(const Context & context, const String & forma else path.assign(format_schema).makeFile().getFileName(); - auto default_schema_directory = [&context]() + auto default_schema_directory = [&format_schema_path]() { - static const String str = Poco::Path(context.getFormatSchemaPath()).makeAbsolute().makeDirectory().toString(); + static const String str = Poco::Path(format_schema_path).makeAbsolute().makeDirectory().toString(); return str; }; - auto is_server = [&context]() - { - return context.hasGlobalContext() && (context.getGlobalContext().getApplicationType() == Context::ApplicationType::SERVER); - }; if (path.getExtension().empty() && !default_file_extension.empty()) path.setExtension(default_file_extension); if (path.isAbsolute()) { - if (is_server()) + if (is_server) throw Exception("Absolute path in the 'format_schema' setting is prohibited: " + path.toString(), ErrorCodes::BAD_ARGUMENTS); schema_path = path.getFileName(); schema_directory = path.makeParent().toString(); } else if (path.depth() >= 1 && path.directory(0) == "..") { - if (is_server()) + if (is_server) throw Exception( "Path in the 'format_schema' setting shouldn't go outside the 'format_schema_path' directory: " + path.toString(), ErrorCodes::BAD_ARGUMENTS); diff --git a/dbms/src/Formats/FormatSchemaInfo.h b/dbms/src/Formats/FormatSchemaInfo.h index 3360698c81f..7af0d56a0cf 100644 --- a/dbms/src/Formats/FormatSchemaInfo.h +++ b/dbms/src/Formats/FormatSchemaInfo.h @@ -10,7 +10,7 @@ class Context; class FormatSchemaInfo { public: - FormatSchemaInfo(const Context & context, const String & format_schema, const String & format, bool require_message); + FormatSchemaInfo(const String & format_schema, const String & format, bool require_message, bool is_server, const std::string & format_schema_path); /// Returns path to the schema file. const String & schemaPath() const { return schema_path; } diff --git a/dbms/src/Formats/FormatSettings.h b/dbms/src/Formats/FormatSettings.h index dfd5d5b86f6..6219edf6e6d 100644 --- a/dbms/src/Formats/FormatSettings.h +++ b/dbms/src/Formats/FormatSettings.h @@ -89,6 +89,27 @@ struct FormatSettings UInt64 row_group_size = 1000000; } parquet; + struct Schema + { + std::string format_schema; + std::string format_schema_path; + bool is_server = false; + }; + + Schema schema; + + struct Custom + { + std::string result_before_delimiter; + std::string result_after_delimiter; + std::string row_before_delimiter; + std::string row_after_delimiter; + std::string row_between_delimiter; + std::string field_delimiter; + std::string escaping_rule; + }; + + Custom custom; }; } diff --git a/dbms/src/Formats/NativeFormat.cpp b/dbms/src/Formats/NativeFormat.cpp index 11835c01123..f9cafbe5459 100644 --- a/dbms/src/Formats/NativeFormat.cpp +++ b/dbms/src/Formats/NativeFormat.cpp @@ -11,7 +11,6 @@ void registerInputFormatNative(FormatFactory & factory) factory.registerInputFormat("Native", []( ReadBuffer & buf, const Block & sample, - const Context &, UInt64 /* max_block_size */, FormatFactory::ReadCallback /* callback */, const FormatSettings &) @@ -25,7 +24,6 @@ void registerOutputFormatNative(FormatFactory & factory) factory.registerOutputFormat("Native", []( WriteBuffer & buf, const Block & sample, - const Context &, FormatFactory::WriteCallback, const FormatSettings &) { diff --git a/dbms/src/Formats/NullFormat.cpp b/dbms/src/Formats/NullFormat.cpp index d5376b70fe1..c07723211eb 100644 --- a/dbms/src/Formats/NullFormat.cpp +++ b/dbms/src/Formats/NullFormat.cpp @@ -10,7 +10,6 @@ void registerOutputFormatNull(FormatFactory & factory) factory.registerOutputFormat("Null", []( WriteBuffer &, const Block & sample, - const Context &, FormatFactory::WriteCallback, const FormatSettings &) { diff --git a/dbms/src/Formats/ParsedTemplateFormatString.cpp b/dbms/src/Formats/ParsedTemplateFormatString.cpp index d6773970c0c..981d43089a2 100644 --- a/dbms/src/Formats/ParsedTemplateFormatString.cpp +++ b/dbms/src/Formats/ParsedTemplateFormatString.cpp @@ -234,36 +234,32 @@ void ParsedTemplateFormatString::throwInvalidFormat(const String & message, size ErrorCodes::INVALID_TEMPLATE_FORMAT); } -ParsedTemplateFormatString ParsedTemplateFormatString::setupCustomSeparatedResultsetFormat(const Context & context) +ParsedTemplateFormatString ParsedTemplateFormatString::setupCustomSeparatedResultsetFormat(const FormatSettings::Custom & settings) { - const Settings & settings = context.getSettingsRef(); - /// Set resultset format to "result_before_delimiter ${data} result_after_delimiter" ParsedTemplateFormatString resultset_format; - resultset_format.delimiters.emplace_back(settings.format_custom_result_before_delimiter); - resultset_format.delimiters.emplace_back(settings.format_custom_result_after_delimiter); + resultset_format.delimiters.emplace_back(settings.result_before_delimiter); + resultset_format.delimiters.emplace_back(settings.result_after_delimiter); resultset_format.formats.emplace_back(ParsedTemplateFormatString::ColumnFormat::None); resultset_format.format_idx_to_column_idx.emplace_back(0); resultset_format.column_names.emplace_back("data"); return resultset_format; } -ParsedTemplateFormatString ParsedTemplateFormatString::setupCustomSeparatedRowFormat(const Context & context, const Block & sample) +ParsedTemplateFormatString ParsedTemplateFormatString::setupCustomSeparatedRowFormat(const FormatSettings::Custom & settings, const Block & sample) { - const Settings & settings = context.getSettingsRef(); - /// Set row format to /// "row_before_delimiter ${Col0:escaping} field_delimiter ${Col1:escaping} field_delimiter ... ${ColN:escaping} row_after_delimiter" - ParsedTemplateFormatString::ColumnFormat escaping = ParsedTemplateFormatString::stringToFormat(settings.format_custom_escaping_rule); + ParsedTemplateFormatString::ColumnFormat escaping = ParsedTemplateFormatString::stringToFormat(settings.escaping_rule); ParsedTemplateFormatString row_format; - row_format.delimiters.emplace_back(settings.format_custom_row_before_delimiter); + row_format.delimiters.emplace_back(settings.row_before_delimiter); for (size_t i = 0; i < sample.columns(); ++i) { row_format.formats.emplace_back(escaping); row_format.format_idx_to_column_idx.emplace_back(i); row_format.column_names.emplace_back(sample.getByPosition(i).name); bool last_column = i == sample.columns() - 1; - row_format.delimiters.emplace_back(last_column ? settings.format_custom_row_after_delimiter : settings.format_custom_field_delimiter); + row_format.delimiters.emplace_back(last_column ? settings.row_after_delimiter : settings.field_delimiter); } return row_format; } diff --git a/dbms/src/Formats/ParsedTemplateFormatString.h b/dbms/src/Formats/ParsedTemplateFormatString.h index cb751d1412a..2da8a074679 100644 --- a/dbms/src/Formats/ParsedTemplateFormatString.h +++ b/dbms/src/Formats/ParsedTemplateFormatString.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -49,8 +50,8 @@ struct ParsedTemplateFormatString String dump() const; [[noreturn]] void throwInvalidFormat(const String & message, size_t column) const; - static ParsedTemplateFormatString setupCustomSeparatedResultsetFormat(const Context & context); - static ParsedTemplateFormatString setupCustomSeparatedRowFormat(const Context & context, const Block & sample); + static ParsedTemplateFormatString setupCustomSeparatedResultsetFormat(const FormatSettings::Custom & settings); + static ParsedTemplateFormatString setupCustomSeparatedRowFormat(const FormatSettings::Custom & settings, const Block & sample); }; } diff --git a/dbms/src/Functions/FunctionsConversion.cpp b/dbms/src/Functions/FunctionsConversion.cpp index 4e6e0fe6e29..1d6a24d99b4 100644 --- a/dbms/src/Functions/FunctionsConversion.cpp +++ b/dbms/src/Functions/FunctionsConversion.cpp @@ -5,29 +5,6 @@ namespace DB { -void throwExceptionForIncompletelyParsedValue( - ReadBuffer & read_buffer, Block & block, size_t result) -{ - const IDataType & to_type = *block.getByPosition(result).type; - - WriteBufferFromOwnString message_buf; - message_buf << "Cannot parse string " << quote << String(read_buffer.buffer().begin(), read_buffer.buffer().size()) - << " as " << to_type.getName() - << ": syntax error"; - - if (read_buffer.offset()) - message_buf << " at position " << read_buffer.offset() - << " (parsed just " << quote << String(read_buffer.buffer().begin(), read_buffer.offset()) << ")"; - else - message_buf << " at begin of string"; - - if (isNativeNumber(to_type)) - message_buf << ". Note: there are to" << to_type.getName() << "OrZero and to" << to_type.getName() << "OrNull functions, which returns zero/NULL instead of throwing exception."; - - throw Exception(message_buf.str(), ErrorCodes::CANNOT_PARSE_TEXT); -} - - void registerFunctionsConversion(FunctionFactory & factory) { factory.registerFunction(); diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index f6d6f615f0c..1f0360b3f1b 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -501,7 +501,26 @@ inline bool tryParseImpl(DataTypeDateTime::FieldType & x, Read /** Throw exception with verbose message when string value is not parsed completely. */ -[[noreturn]] void throwExceptionForIncompletelyParsedValue(ReadBuffer & read_buffer, Block & block, size_t result); +[[noreturn]] inline void throwExceptionForIncompletelyParsedValue(ReadBuffer & read_buffer, Block & block, size_t result) +{ + const IDataType & to_type = *block.getByPosition(result).type; + + WriteBufferFromOwnString message_buf; + message_buf << "Cannot parse string " << quote << String(read_buffer.buffer().begin(), read_buffer.buffer().size()) + << " as " << to_type.getName() + << ": syntax error"; + + if (read_buffer.offset()) + message_buf << " at position " << read_buffer.offset() + << " (parsed just " << quote << String(read_buffer.buffer().begin(), read_buffer.offset()) << ")"; + else + message_buf << " at begin of string"; + + if (isNativeNumber(to_type)) + message_buf << ". Note: there are to" << to_type.getName() << "OrZero and to" << to_type.getName() << "OrNull functions, which returns zero/NULL instead of throwing exception."; + + throw Exception(message_buf.str(), ErrorCodes::CANNOT_PARSE_TEXT); +} enum class ConvertFromStringExceptionMode @@ -886,6 +905,7 @@ public: static constexpr bool to_datetime64 = std::is_same_v; static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create() { return std::make_shared(); } String getName() const override { @@ -1083,6 +1103,7 @@ public: std::is_same_v>; static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create() { return std::make_shared(); } String getName() const override { @@ -1231,6 +1252,7 @@ class FunctionToFixedString : public IFunction public: static constexpr auto name = "toFixedString"; static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create() { return std::make_shared(); } String getName() const override { @@ -1686,9 +1708,9 @@ public: using WrapperType = std::function; using MonotonicityForRange = std::function; - FunctionCast(const Context & context_, const char * name_, MonotonicityForRange && monotonicity_for_range_ + FunctionCast(const char * name_, MonotonicityForRange && monotonicity_for_range_ , const DataTypes & argument_types_, const DataTypePtr & return_type_) - : context(context_), name(name_), monotonicity_for_range(monotonicity_for_range_) + : name(name_), monotonicity_for_range(monotonicity_for_range_) , argument_types(argument_types_), return_type(return_type_) { } @@ -1719,7 +1741,6 @@ public: private: - const Context & context; const char * name; MonotonicityForRange monotonicity_for_range; @@ -1735,10 +1756,10 @@ private: { /// In case when converting to Nullable type, we apply different parsing rule, /// that will not throw an exception but return NULL in case of malformed input. - function = FunctionConvertFromString::create(context); + function = FunctionConvertFromString::create(); } else - function = FunctionTo::Type::create(context); + function = FunctionTo::Type::create(); auto function_adaptor = FunctionOverloadResolverAdaptor(std::make_unique(function)) @@ -1752,7 +1773,7 @@ private: WrapperType createStringWrapper(const DataTypePtr & from_type) const { - FunctionPtr function = FunctionToString::create(context); + FunctionPtr function = FunctionToString::create(); auto function_adaptor = FunctionOverloadResolverAdaptor(std::make_unique(function)) @@ -1780,7 +1801,7 @@ private: if (requested_result_is_nullable) throw Exception{"CAST AS Nullable(UUID) is not implemented", ErrorCodes::NOT_IMPLEMENTED}; - FunctionPtr function = FunctionTo::Type::create(context); + FunctionPtr function = FunctionTo::Type::create(); auto function_adaptor = FunctionOverloadResolverAdaptor(std::make_unique(function)) @@ -1985,7 +2006,7 @@ private: return createStringToEnumWrapper(); else if (isNativeNumber(from_type) || isEnum(from_type)) { - auto function = Function::create(context); + auto function = Function::create(); auto func_or_adaptor = FunctionOverloadResolverAdaptor(std::make_unique(function)) .build(ColumnsWithTypeAndName{{nullptr, from_type, "" }}); @@ -2337,9 +2358,10 @@ public: using MonotonicityForRange = FunctionCast::MonotonicityForRange; static constexpr auto name = "CAST"; - static FunctionOverloadResolverImplPtr create(const Context & context) { return std::make_unique(context); } + static FunctionOverloadResolverImplPtr create(const Context &) { return createImpl(); } + static FunctionOverloadResolverImplPtr createImpl() { return std::make_unique(); } - CastOverloadResolver(const Context & context_) : context(context_) {} + CastOverloadResolver() {} String getName() const override { return name; } @@ -2357,7 +2379,7 @@ protected: data_types[i] = arguments[i].type; auto monotonicity = getMonotonicityInformation(arguments.front().type, return_type.get()); - return std::make_unique(context, name, std::move(monotonicity), data_types, return_type); + return std::make_unique(name, std::move(monotonicity), data_types, return_type); } DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments) const override @@ -2418,8 +2440,6 @@ private: /// other types like Null, FixedString, Array and Tuple have no monotonicity defined return {}; } - - const Context & context; }; } diff --git a/dbms/src/Interpreters/castColumn.cpp b/dbms/src/Interpreters/castColumn.cpp index bbbf82a681a..dd281540b51 100644 --- a/dbms/src/Interpreters/castColumn.cpp +++ b/dbms/src/Interpreters/castColumn.cpp @@ -2,13 +2,14 @@ #include #include #include -#include +#include +#include namespace DB { -ColumnPtr castColumn(const ColumnWithTypeAndName & arg, const DataTypePtr & type, const Context & context) +ColumnPtr castColumn(const ColumnWithTypeAndName & arg, const DataTypePtr & type) { if (arg.type->equals(*type)) return arg.column; @@ -28,7 +29,7 @@ ColumnPtr castColumn(const ColumnWithTypeAndName & arg, const DataTypePtr & type } }; - FunctionOverloadResolverPtr func_builder_cast = FunctionFactory::instance().get("CAST", context); + FunctionOverloadResolverPtr func_builder_cast = std::make_shared(CastOverloadResolver::createImpl()); ColumnsWithTypeAndName arguments{ temporary_block.getByPosition(0), temporary_block.getByPosition(1) }; auto func_cast = func_builder_cast->build(arguments); @@ -37,4 +38,9 @@ ColumnPtr castColumn(const ColumnWithTypeAndName & arg, const DataTypePtr & type return temporary_block.getByPosition(2).column; } +ColumnPtr castColumn(const ColumnWithTypeAndName & arg, const DataTypePtr & type, const Context &) +{ + return castColumn(arg, type); +} + } diff --git a/dbms/src/Interpreters/castColumn.h b/dbms/src/Interpreters/castColumn.h index f8efd7bb1da..28914f34977 100644 --- a/dbms/src/Interpreters/castColumn.h +++ b/dbms/src/Interpreters/castColumn.h @@ -6,7 +6,7 @@ namespace DB { - +ColumnPtr castColumn(const ColumnWithTypeAndName & arg, const DataTypePtr & type); ColumnPtr castColumn(const ColumnWithTypeAndName & arg, const DataTypePtr & type, const Context & context); } diff --git a/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index edb8d5c15f4..b7ff829922b 100644 --- a/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -248,7 +248,7 @@ namespace DB void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk &res, std::shared_ptr &table, arrow::Status &read_status, const Block &header, - int &row_group_current, const Context &context, std::string format_name) + int &row_group_current, std::string format_name) { Columns columns_list; UInt64 num_rows = 0; @@ -389,7 +389,7 @@ namespace DB else column.column = std::move(read_column); - column.column = castColumn(column, column_type, context); + column.column = castColumn(column, column_type); column.type = column_type; num_rows = column.column->size(); columns_list.push_back(std::move(column.column)); diff --git a/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h b/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h index 34b58a80091..720b4df47cc 100644 --- a/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h +++ b/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h @@ -39,7 +39,7 @@ namespace DB static void arrowTableToCHChunk(Chunk &res, std::shared_ptr &table, arrow::Status &read_status, const Block &header, - int &row_group_current, const Context &context, std::string format_name); + int &row_group_current, std::string format_name); }; } #endif diff --git a/dbms/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp index 53e00d295f1..f49f521d474 100644 --- a/dbms/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp @@ -61,7 +61,6 @@ void registerInputFormatProcessorRowBinary(FormatFactory & factory) factory.registerInputFormatProcessor("RowBinary", []( ReadBuffer & buf, const Block & sample, - const Context &, const IRowInputFormat::Params & params, const FormatSettings &) { @@ -71,7 +70,6 @@ void registerInputFormatProcessorRowBinary(FormatFactory & factory) factory.registerInputFormatProcessor("RowBinaryWithNamesAndTypes", []( ReadBuffer & buf, const Block & sample, - const Context &, const IRowInputFormat::Params & params, const FormatSettings &) { diff --git a/dbms/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp index 1bbdfbf93fc..726a9ca45e8 100644 --- a/dbms/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp @@ -52,7 +52,6 @@ void registerOutputFormatProcessorRowBinary(FormatFactory & factory) factory.registerOutputFormatProcessor("RowBinary", []( WriteBuffer & buf, const Block & sample, - const Context &, FormatFactory::WriteCallback callback, const FormatSettings &) { @@ -62,7 +61,6 @@ void registerOutputFormatProcessorRowBinary(FormatFactory & factory) factory.registerOutputFormatProcessor("RowBinaryWithNamesAndTypes", []( WriteBuffer & buf, const Block & sample, - const Context &, FormatFactory::WriteCallback callback, const FormatSettings &) { diff --git a/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 3dc373109be..e5920f33dc6 100644 --- a/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -421,7 +421,6 @@ void registerInputFormatProcessorCSV(FormatFactory & factory) factory.registerInputFormatProcessor(with_names ? "CSVWithNames" : "CSV", [=]( ReadBuffer & buf, const Block & sample, - const Context &, IRowInputFormat::Params params, const FormatSettings & settings) { diff --git a/dbms/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp index 081dcf890a0..45962e9779c 100644 --- a/dbms/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp @@ -76,7 +76,6 @@ void registerOutputFormatProcessorCSV(FormatFactory & factory) factory.registerOutputFormatProcessor(with_names ? "CSVWithNames" : "CSV", [=]( WriteBuffer & buf, const Block & sample, - const Context &, FormatFactory::WriteCallback callback, const FormatSettings & format_settings) { diff --git a/dbms/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp index be4e6eaaf3f..afa9f4d4fa0 100644 --- a/dbms/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp @@ -301,10 +301,11 @@ void registerInputFormatProcessorCapnProto(FormatFactory & factory) { factory.registerInputFormatProcessor( "CapnProto", - [](ReadBuffer & buf, const Block & sample, const Context & context, IRowInputFormat::Params params, const FormatSettings &) + [](ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, const FormatSettings & settings) { return std::make_shared(buf, sample, std::move(params), - FormatSchemaInfo(context, context.getSettingsRef().format_schema, "CapnProto", true)); + FormatSchemaInfo(settings.schema.format_schema, "CapnProto", true, + settings.schema.is_server, settings.schema.format_schema_path)); }); } diff --git a/dbms/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp index d4530e7b09d..61a5f649ed1 100644 --- a/dbms/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp @@ -217,7 +217,6 @@ void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory) factory.registerInputFormatProcessor("JSONCompactEachRow", []( ReadBuffer & buf, const Block & sample, - const Context &, IRowInputFormat::Params params, const FormatSettings & settings) { @@ -227,7 +226,6 @@ void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory) factory.registerInputFormatProcessor("JSONCompactEachRowWithNamesAndTypes", []( ReadBuffer & buf, const Block & sample, - const Context &, IRowInputFormat::Params params, const FormatSettings & settings) { diff --git a/dbms/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp index 433cc4515ae..02b7cc220a0 100644 --- a/dbms/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp @@ -94,7 +94,6 @@ void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory) factory.registerOutputFormatProcessor("JSONCompactEachRow", []( WriteBuffer & buf, const Block & sample, - const Context &, FormatFactory::WriteCallback callback, const FormatSettings & format_settings) { @@ -104,7 +103,6 @@ void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory) factory.registerOutputFormatProcessor("JSONCompactEachRowWithNamesAndTypes", []( WriteBuffer &buf, const Block &sample, - const Context &, FormatFactory::WriteCallback callback, const FormatSettings &format_settings) { diff --git a/dbms/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp index 011054dfce7..7e56a4643da 100644 --- a/dbms/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp @@ -80,7 +80,6 @@ void registerOutputFormatProcessorJSONCompact(FormatFactory & factory) factory.registerOutputFormatProcessor("JSONCompact", []( WriteBuffer & buf, const Block & sample, - const Context &, FormatFactory::WriteCallback callback, const FormatSettings & format_settings) { diff --git a/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index 1ffe50d87e6..a597af956b6 100644 --- a/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -271,7 +271,6 @@ void registerInputFormatProcessorJSONEachRow(FormatFactory & factory) factory.registerInputFormatProcessor("JSONEachRow", []( ReadBuffer & buf, const Block & sample, - const Context &, IRowInputFormat::Params params, const FormatSettings & settings) { diff --git a/dbms/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp index 513420d29b4..f862a07173b 100644 --- a/dbms/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp @@ -56,7 +56,6 @@ void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory) factory.registerOutputFormatProcessor("JSONEachRow", []( WriteBuffer & buf, const Block & sample, - const Context &, FormatFactory::WriteCallback callback, const FormatSettings & format_settings) { diff --git a/dbms/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp index 33fc1c9920b..a611b5a129b 100644 --- a/dbms/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp @@ -33,7 +33,6 @@ void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factor factory.registerOutputFormatProcessor("JSONEachRowWithProgress", []( WriteBuffer & buf, const Block & sample, - const Context &, FormatFactory::WriteCallback callback, const FormatSettings & format_settings) { diff --git a/dbms/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp index 90cff3f1498..c23f3812c0b 100644 --- a/dbms/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp @@ -246,7 +246,6 @@ void registerOutputFormatProcessorJSON(FormatFactory & factory) factory.registerOutputFormatProcessor("JSON", []( WriteBuffer & buf, const Block & sample, - const Context &, FormatFactory::WriteCallback callback, const FormatSettings & format_settings) { diff --git a/dbms/src/Processors/Formats/Impl/MySQLOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/MySQLOutputFormat.cpp index f913087da9b..b04de32ca5a 100644 --- a/dbms/src/Processors/Formats/Impl/MySQLOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/MySQLOutputFormat.cpp @@ -12,13 +12,10 @@ namespace DB using namespace MySQLProtocol; -MySQLOutputFormat::MySQLOutputFormat(WriteBuffer & out_, const Block & header_, const Context & context_, const FormatSettings & settings_) +MySQLOutputFormat::MySQLOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & settings_) : IOutputFormat(header_, out_) - , context(context_) - , packet_sender(out, const_cast(context_.mysql.sequence_id)) /// TODO: fix it , format_settings(settings_) { - packet_sender.max_packet_size = context_.mysql.max_packet_size; } void MySQLOutputFormat::initialize() @@ -32,17 +29,17 @@ void MySQLOutputFormat::initialize() if (header.columns()) { - packet_sender.sendPacket(LengthEncodedNumber(header.columns())); + packet_sender->sendPacket(LengthEncodedNumber(header.columns())); for (size_t i = 0; i < header.columns(); i++) { const auto & column_name = header.getColumnsWithTypeAndName()[i].name; - packet_sender.sendPacket(getColumnDefinition(column_name, data_types[i]->getTypeId())); + packet_sender->sendPacket(getColumnDefinition(column_name, data_types[i]->getTypeId())); } - if (!(context.mysql.client_capabilities & Capability::CLIENT_DEPRECATE_EOF)) + if (!(context->mysql.client_capabilities & Capability::CLIENT_DEPRECATE_EOF)) { - packet_sender.sendPacket(EOF_Packet(0, 0)); + packet_sender->sendPacket(EOF_Packet(0, 0)); } } } @@ -53,7 +50,7 @@ void MySQLOutputFormat::consume(Chunk chunk) for (size_t i = 0; i < chunk.getNumRows(); i++) { ProtocolText::ResultsetRow row_packet(data_types, chunk.getColumns(), i); - packet_sender.sendPacket(row_packet); + packet_sender->sendPacket(row_packet); } } @@ -61,7 +58,7 @@ void MySQLOutputFormat::finalize() { size_t affected_rows = 0; std::stringstream human_readable_info; - if (QueryStatus * process_list_elem = context.getProcessListElement()) + if (QueryStatus * process_list_elem = context->getProcessListElement()) { CurrentThread::finalizePerformanceCounters(); QueryStatusInfo info = process_list_elem->getInfo(); @@ -74,17 +71,17 @@ void MySQLOutputFormat::finalize() const auto & header = getPort(PortKind::Main).getHeader(); if (header.columns() == 0) - packet_sender.sendPacket(OK_Packet(0x0, context.mysql.client_capabilities, affected_rows, 0, 0, "", human_readable_info.str()), true); + packet_sender->sendPacket(OK_Packet(0x0, context->mysql.client_capabilities, affected_rows, 0, 0, "", human_readable_info.str()), true); else - if (context.mysql.client_capabilities & CLIENT_DEPRECATE_EOF) - packet_sender.sendPacket(OK_Packet(0xfe, context.mysql.client_capabilities, affected_rows, 0, 0, "", human_readable_info.str()), true); + if (context->mysql.client_capabilities & CLIENT_DEPRECATE_EOF) + packet_sender->sendPacket(OK_Packet(0xfe, context->mysql.client_capabilities, affected_rows, 0, 0, "", human_readable_info.str()), true); else - packet_sender.sendPacket(EOF_Packet(0, 0), true); + packet_sender->sendPacket(EOF_Packet(0, 0), true); } void MySQLOutputFormat::flush() { - packet_sender.out->next(); + packet_sender->out->next(); } void registerOutputFormatProcessorMySQLWrite(FormatFactory & factory) @@ -93,9 +90,8 @@ void registerOutputFormatProcessorMySQLWrite(FormatFactory & factory) "MySQLWire", [](WriteBuffer & buf, const Block & sample, - const Context & context, FormatFactory::WriteCallback, - const FormatSettings & settings) { return std::make_shared(buf, sample, context, settings); }); + const FormatSettings & settings) { return std::make_shared(buf, sample, settings); }); } } diff --git a/dbms/src/Processors/Formats/Impl/MySQLOutputFormat.h b/dbms/src/Processors/Formats/Impl/MySQLOutputFormat.h index 780a0c4ea05..2604e7fc42e 100644 --- a/dbms/src/Processors/Formats/Impl/MySQLOutputFormat.h +++ b/dbms/src/Processors/Formats/Impl/MySQLOutputFormat.h @@ -16,13 +16,20 @@ class Context; /** A stream for outputting data in a binary line-by-line format. */ -class MySQLOutputFormat: public IOutputFormat +class MySQLOutputFormat final : public IOutputFormat { public: - MySQLOutputFormat(WriteBuffer & out_, const Block & header_, const Context & context_, const FormatSettings & settings_); + MySQLOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & settings_); String getName() const override { return "MySQLOutputFormat"; } + void setContext(const Context & context_) + { + context = &context_; + packet_sender = std::make_unique(out, const_cast(context_.mysql.sequence_id)); /// TODO: fix it + packet_sender->max_packet_size = context_.mysql.max_packet_size; + } + void consume(Chunk) override; void finalize() override; void flush() override; @@ -34,8 +41,8 @@ private: bool initialized = false; - const Context & context; - MySQLProtocol::PacketSender packet_sender; + const Context * context = nullptr; + std::unique_ptr packet_sender; FormatSettings format_settings; DataTypes data_types; }; diff --git a/dbms/src/Processors/Formats/Impl/NativeFormat.cpp b/dbms/src/Processors/Formats/Impl/NativeFormat.cpp index 846d1c3dbc6..bbc0d05272b 100644 --- a/dbms/src/Processors/Formats/Impl/NativeFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/NativeFormat.cpp @@ -156,7 +156,6 @@ void registerInputFormatProcessorNative(FormatFactory & factory) factory.registerInputFormatProcessor("Native", []( ReadBuffer & buf, const Block & sample, - const Context &, const RowInputFormatParams &, const FormatSettings &) { @@ -169,7 +168,6 @@ void registerOutputFormatProcessorNative(FormatFactory & factory) factory.registerOutputFormatProcessor("Native", []( WriteBuffer & buf, const Block & sample, - const Context &, FormatFactory::WriteCallback, const FormatSettings &) { diff --git a/dbms/src/Processors/Formats/Impl/NullFormat.cpp b/dbms/src/Processors/Formats/Impl/NullFormat.cpp index cf1dc7186ab..c8c773e7acf 100644 --- a/dbms/src/Processors/Formats/Impl/NullFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/NullFormat.cpp @@ -21,7 +21,6 @@ void registerOutputFormatProcessorNull(FormatFactory & factory) factory.registerOutputFormatProcessor("Null", []( WriteBuffer & buf, const Block & sample, - const Context &, FormatFactory::WriteCallback, const FormatSettings &) { diff --git a/dbms/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp index 9ada3d16516..5cad02cf2cf 100644 --- a/dbms/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp @@ -107,7 +107,7 @@ void ODBCDriver2BlockOutputFormat::writePrefix() void registerOutputFormatProcessorODBCDriver2(FormatFactory & factory) { factory.registerOutputFormatProcessor( - "ODBCDriver2", [](WriteBuffer & buf, const Block & sample, const Context &, FormatFactory::WriteCallback, const FormatSettings & format_settings) + "ODBCDriver2", [](WriteBuffer & buf, const Block & sample, FormatFactory::WriteCallback, const FormatSettings & format_settings) { return std::make_shared(buf, sample, format_settings); }); diff --git a/dbms/src/Processors/Formats/Impl/ODBCDriverBlockOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/ODBCDriverBlockOutputFormat.cpp index 467bfb614e3..eaccc47346d 100644 --- a/dbms/src/Processors/Formats/Impl/ODBCDriverBlockOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ODBCDriverBlockOutputFormat.cpp @@ -69,7 +69,6 @@ void registerOutputFormatProcessorODBCDriver(FormatFactory & factory) factory.registerOutputFormatProcessor("ODBCDriver", []( WriteBuffer & buf, const Block & sample, - const Context &, FormatFactory::WriteCallback, const FormatSettings & format_settings) { diff --git a/dbms/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/dbms/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index c368e979495..6049420cac0 100644 --- a/dbms/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -12,8 +12,8 @@ namespace DB { - ORCBlockInputFormat::ORCBlockInputFormat(ReadBuffer &in_, Block header_, const Context &context_) - : IInputFormat(std::move(header_), in_), context{context_} { + ORCBlockInputFormat::ORCBlockInputFormat(ReadBuffer &in_, Block header_) : IInputFormat(std::move(header_), in_) + { } Chunk ORCBlockInputFormat::generate() @@ -57,7 +57,7 @@ namespace DB arrow::Status read_status = file_reader->Read(&table); - ArrowColumnToCHColumn::arrowTableToCHChunk(res, table, read_status, header, row_group_current, context, "ORC"); + ArrowColumnToCHColumn::arrowTableToCHChunk(res, table, read_status, header, row_group_current, "ORC"); return res; } @@ -78,11 +78,10 @@ namespace DB "ORC", [](ReadBuffer &buf, const Block &sample, - const Context &context, const RowInputFormatParams &, const FormatSettings & /* settings */) { - return std::make_shared(buf, sample, context); + return std::make_shared(buf, sample); }); } diff --git a/dbms/src/Processors/Formats/Impl/ORCBlockInputFormat.h b/dbms/src/Processors/Formats/Impl/ORCBlockInputFormat.h index 331d60af9dd..c0c688f0158 100644 --- a/dbms/src/Processors/Formats/Impl/ORCBlockInputFormat.h +++ b/dbms/src/Processors/Formats/Impl/ORCBlockInputFormat.h @@ -17,7 +17,7 @@ class Context; class ORCBlockInputFormat: public IInputFormat { public: - ORCBlockInputFormat(ReadBuffer & in_, Block header_, const Context & context_); + ORCBlockInputFormat(ReadBuffer & in_, Block header_); String getName() const override { return "ORCBlockInputFormat"; } @@ -30,8 +30,6 @@ private: // TODO: check that this class implements every part of its parent - const Context & context; - std::unique_ptr file_reader; std::string file_data; int row_group_total = 0; diff --git a/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 55c2ebccf41..3c9b50c2c75 100644 --- a/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -15,8 +15,9 @@ namespace DB { - ParquetBlockInputFormat::ParquetBlockInputFormat(ReadBuffer &in_, Block header_, const Context &context_) - : IInputFormat(std::move(header_), in_), context{context_} { + ParquetBlockInputFormat::ParquetBlockInputFormat(ReadBuffer & in_, Block header_) + : IInputFormat(std::move(header_), in_) + { } Chunk ParquetBlockInputFormat::generate() @@ -59,7 +60,7 @@ namespace DB std::shared_ptr table; arrow::Status read_status = file_reader->ReadRowGroup(row_group_current, &table); - ArrowColumnToCHColumn::arrowTableToCHChunk(res, table, read_status, header, row_group_current, context, "Parquet"); + ArrowColumnToCHColumn::arrowTableToCHChunk(res, table, read_status, header, row_group_current, "Parquet"); return res; } @@ -80,11 +81,10 @@ namespace DB "Parquet", [](ReadBuffer &buf, const Block &sample, - const Context &context, const RowInputFormatParams &, const FormatSettings & /* settings */) { - return std::make_shared(buf, sample, context); + return std::make_shared(buf, sample); }); } diff --git a/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.h b/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.h index 94489f8fbaa..665665557a5 100644 --- a/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.h +++ b/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.h @@ -16,7 +16,7 @@ class Context; class ParquetBlockInputFormat: public IInputFormat { public: - ParquetBlockInputFormat(ReadBuffer & in_, Block header_, const Context & context_); + ParquetBlockInputFormat(ReadBuffer & in_, Block header_); void resetParser() override; @@ -30,8 +30,6 @@ private: // TODO: check that this class implements every part of its parent - const Context & context; - std::unique_ptr file_reader; std::string file_data; std::unique_ptr buffer; diff --git a/dbms/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index f7092bc9a5b..855036e05ff 100644 --- a/dbms/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -426,7 +426,6 @@ void registerOutputFormatProcessorParquet(FormatFactory & factory) "Parquet", [](WriteBuffer & buf, const Block & sample, - const Context & /*context*/, FormatFactory::WriteCallback, const FormatSettings & format_settings) { diff --git a/dbms/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index 5a0a43f487a..c0b1b2e4ffc 100644 --- a/dbms/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -260,7 +260,6 @@ void registerOutputFormatProcessorPretty(FormatFactory & factory) factory.registerOutputFormatProcessor("Pretty", []( WriteBuffer & buf, const Block & sample, - const Context &, FormatFactory::WriteCallback, const FormatSettings & format_settings) { @@ -270,7 +269,6 @@ void registerOutputFormatProcessorPretty(FormatFactory & factory) factory.registerOutputFormatProcessor("PrettyNoEscapes", []( WriteBuffer & buf, const Block & sample, - const Context &, FormatFactory::WriteCallback, const FormatSettings & format_settings) { diff --git a/dbms/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp index 10d475f599a..84e222d073c 100644 --- a/dbms/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp @@ -134,7 +134,6 @@ void registerOutputFormatProcessorPrettyCompact(FormatFactory & factory) factory.registerOutputFormatProcessor("PrettyCompact", []( WriteBuffer & buf, const Block & sample, - const Context &, FormatFactory::WriteCallback, const FormatSettings & format_settings) { @@ -144,7 +143,6 @@ void registerOutputFormatProcessorPrettyCompact(FormatFactory & factory) factory.registerOutputFormatProcessor("PrettyCompactNoEscapes", []( WriteBuffer & buf, const Block & sample, - const Context &, FormatFactory::WriteCallback, const FormatSettings & format_settings) { @@ -157,7 +155,6 @@ void registerOutputFormatProcessorPrettyCompact(FormatFactory & factory) // factory.registerOutputFormat("PrettyCompactMonoBlock", []( // WriteBuffer & buf, // const Block & sample, -// const Context &, // const FormatSettings & format_settings) // { // BlockOutputStreamPtr impl = std::make_shared(buf, sample, format_settings); diff --git a/dbms/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp index 85655a33004..8b852b3d194 100644 --- a/dbms/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp @@ -97,7 +97,6 @@ void registerOutputFormatProcessorPrettySpace(FormatFactory & factory) factory.registerOutputFormatProcessor("PrettySpace", []( WriteBuffer & buf, const Block & sample, - const Context &, FormatFactory::WriteCallback, const FormatSettings & format_settings) { @@ -107,7 +106,6 @@ void registerOutputFormatProcessorPrettySpace(FormatFactory & factory) factory.registerOutputFormatProcessor("PrettySpaceNoEscapes", []( WriteBuffer & buf, const Block & sample, - const Context &, FormatFactory::WriteCallback, const FormatSettings & format_settings) { diff --git a/dbms/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp index 6fce4437219..50d9cef9bfc 100644 --- a/dbms/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp @@ -70,12 +70,12 @@ void registerInputFormatProcessorProtobuf(FormatFactory & factory) factory.registerInputFormatProcessor("Protobuf", []( ReadBuffer & buf, const Block & sample, - const Context & context, IRowInputFormat::Params params, - const FormatSettings &) + const FormatSettings & settings) { return std::make_shared(buf, sample, std::move(params), - FormatSchemaInfo(context, context.getSettingsRef().format_schema, "Protobuf", true)); + FormatSchemaInfo(settings.schema.format_schema, "Protobuf", true, + settings.schema.is_server, settings.schema.format_schema_path)); }); } diff --git a/dbms/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp index d1007492203..da567e60475 100644 --- a/dbms/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp @@ -50,12 +50,12 @@ void registerOutputFormatProcessorProtobuf(FormatFactory & factory) "Protobuf", [](WriteBuffer & buf, const Block & header, - const Context & context, FormatFactory::WriteCallback callback, - const FormatSettings &) + const FormatSettings & settings) { - return std::make_shared(buf, header, callback, - FormatSchemaInfo(context, context.getSettingsRef().format_schema, "Protobuf", true)); + return std::make_shared(buf, header, std::move(callback), + FormatSchemaInfo(settings.schema.format_schema, "Protobuf", true, + settings.schema.is_server, settings.schema.format_schema_path)); }); } diff --git a/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp index dcc7d64fcd2..1e4fa511983 100644 --- a/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp @@ -210,7 +210,6 @@ void registerInputFormatProcessorTSKV(FormatFactory & factory) factory.registerInputFormatProcessor("TSKV", []( ReadBuffer & buf, const Block & sample, - const Context &, IRowInputFormat::Params params, const FormatSettings & settings) { diff --git a/dbms/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp index f1fbc72b9ee..a8f1acb59bb 100644 --- a/dbms/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp @@ -45,7 +45,6 @@ void registerOutputFormatProcessorTSKV(FormatFactory & factory) factory.registerOutputFormatProcessor("TSKV", []( WriteBuffer & buf, const Block & sample, - const Context &, FormatFactory::WriteCallback callback, const FormatSettings & settings) { diff --git a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index b578a2c07ce..d20d7802d5c 100644 --- a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -356,7 +356,6 @@ void registerInputFormatProcessorTabSeparated(FormatFactory & factory) factory.registerInputFormatProcessor(name, []( ReadBuffer & buf, const Block & sample, - const Context &, IRowInputFormat::Params params, const FormatSettings & settings) { @@ -369,7 +368,6 @@ void registerInputFormatProcessorTabSeparated(FormatFactory & factory) factory.registerInputFormatProcessor(name, []( ReadBuffer & buf, const Block & sample, - const Context &, IRowInputFormat::Params params, const FormatSettings & settings) { @@ -382,7 +380,6 @@ void registerInputFormatProcessorTabSeparated(FormatFactory & factory) factory.registerInputFormatProcessor(name, []( ReadBuffer & buf, const Block & sample, - const Context &, IRowInputFormat::Params params, const FormatSettings & settings) { diff --git a/dbms/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp index ae3a1fb9af0..0acbbdcdfbe 100644 --- a/dbms/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp @@ -78,7 +78,6 @@ void registerOutputFormatProcessorTabSeparated(FormatFactory & factory) factory.registerOutputFormatProcessor(name, []( WriteBuffer & buf, const Block & sample, - const Context &, FormatFactory::WriteCallback callback, const FormatSettings & settings) { @@ -91,7 +90,6 @@ void registerOutputFormatProcessorTabSeparated(FormatFactory & factory) factory.registerOutputFormatProcessor(name, []( WriteBuffer & buf, const Block & sample, - const Context &, FormatFactory::WriteCallback callback, const FormatSettings & settings) { @@ -104,7 +102,6 @@ void registerOutputFormatProcessorTabSeparated(FormatFactory & factory) factory.registerOutputFormatProcessor(name, []( WriteBuffer & buf, const Block & sample, - const Context &, FormatFactory::WriteCallback callback, const FormatSettings & settings) { @@ -117,7 +114,6 @@ void registerOutputFormatProcessorTabSeparated(FormatFactory & factory) factory.registerOutputFormatProcessor(name, []( WriteBuffer & buf, const Block & sample, - const Context &, FormatFactory::WriteCallback callback, const FormatSettings & settings) { diff --git a/dbms/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp index aa7bfbeaff1..bd49b25934e 100644 --- a/dbms/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp @@ -14,8 +14,10 @@ namespace ErrorCodes } TemplateBlockOutputFormat::TemplateBlockOutputFormat(const Block & header_, WriteBuffer & out_, const FormatSettings & settings_, - ParsedTemplateFormatString format_, ParsedTemplateFormatString row_format_) - : IOutputFormat(header_, out_), settings(settings_), format(std::move(format_)), row_format(std::move(row_format_)) + ParsedTemplateFormatString format_, ParsedTemplateFormatString row_format_, + std::string row_between_delimiter_) + : IOutputFormat(header_, out_), settings(settings_), format(std::move(format_)) + , row_format(std::move(row_format_)), row_between_delimiter(std::move(row_between_delimiter_)) { auto & sample = getPort(PortKind::Main).getHeader(); size_t columns = sample.columns(); @@ -152,7 +154,7 @@ void TemplateBlockOutputFormat::consume(Chunk chunk) for (size_t i = 0; i < rows; ++i) { if (row_count) - writeString(settings.template_settings.row_between_delimiter, out); + writeString(row_between_delimiter, out); writeRow(chunk, i); ++row_count; @@ -230,7 +232,6 @@ void registerOutputFormatProcessorTemplate(FormatFactory & factory) factory.registerOutputFormatProcessor("Template", []( WriteBuffer & buf, const Block & sample, - const Context & context, FormatFactory::WriteCallback, const FormatSettings & settings) { @@ -247,7 +248,8 @@ void registerOutputFormatProcessorTemplate(FormatFactory & factory) { /// Read format string from file resultset_format = ParsedTemplateFormatString( - FormatSchemaInfo(context, settings.template_settings.resultset_format, "Template", false), + FormatSchemaInfo(settings.template_settings.resultset_format, "Template", false, + settings.schema.is_server, settings.schema.format_schema_path), [&](const String & partName) { return static_cast(TemplateBlockOutputFormat::stringToResultsetPart(partName)); @@ -255,28 +257,26 @@ void registerOutputFormatProcessorTemplate(FormatFactory & factory) } ParsedTemplateFormatString row_format = ParsedTemplateFormatString( - FormatSchemaInfo(context, settings.template_settings.row_format, "Template", false), + FormatSchemaInfo(settings.template_settings.row_format, "Template", false, + settings.schema.is_server, settings.schema.format_schema_path), [&](const String & colName) { return sample.getPositionByName(colName); }); - return std::make_shared(sample, buf, settings, resultset_format, row_format); + return std::make_shared(sample, buf, settings, resultset_format, row_format, settings.template_settings.row_between_delimiter); }); factory.registerOutputFormatProcessor("CustomSeparated", []( WriteBuffer & buf, const Block & sample, - const Context & context, FormatFactory::WriteCallback, const FormatSettings & settings) { - ParsedTemplateFormatString resultset_format = ParsedTemplateFormatString::setupCustomSeparatedResultsetFormat(context); - ParsedTemplateFormatString row_format = ParsedTemplateFormatString::setupCustomSeparatedRowFormat(context, sample); - FormatSettings format_settings = settings; - format_settings.template_settings.row_between_delimiter = context.getSettingsRef().format_custom_row_between_delimiter; + ParsedTemplateFormatString resultset_format = ParsedTemplateFormatString::setupCustomSeparatedResultsetFormat(settings.custom); + ParsedTemplateFormatString row_format = ParsedTemplateFormatString::setupCustomSeparatedRowFormat(settings.custom, sample); - return std::make_shared(sample, buf, format_settings, resultset_format, row_format); + return std::make_shared(sample, buf, settings, resultset_format, row_format, settings.custom.row_between_delimiter); }); } } diff --git a/dbms/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h b/dbms/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h index 25a6a832bc8..f29d31eb3f1 100644 --- a/dbms/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h +++ b/dbms/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h @@ -15,7 +15,8 @@ class TemplateBlockOutputFormat : public IOutputFormat using ColumnFormat = ParsedTemplateFormatString::ColumnFormat; public: TemplateBlockOutputFormat(const Block & header_, WriteBuffer & out_, const FormatSettings & settings_, - ParsedTemplateFormatString format_, ParsedTemplateFormatString row_format_); + ParsedTemplateFormatString format_, ParsedTemplateFormatString row_format_, + std::string row_between_delimiter_); String getName() const override { return "TemplateBlockOutputFormat"; } @@ -65,6 +66,8 @@ protected: size_t row_count = 0; bool need_write_prefix = true; + + std::string row_between_delimiter; }; } diff --git a/dbms/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp index d4de40eddec..9b007d9f29b 100644 --- a/dbms/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp @@ -20,12 +20,13 @@ extern const int SYNTAX_ERROR; TemplateRowInputFormat::TemplateRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_, - const FormatSettings & settings_, bool ignore_spaces_, - ParsedTemplateFormatString format_, ParsedTemplateFormatString row_format_) + FormatSettings settings_, bool ignore_spaces_, + ParsedTemplateFormatString format_, ParsedTemplateFormatString row_format_, + std::string row_between_delimiter_) : RowInputFormatWithDiagnosticInfo(header_, buf, params_), buf(in_), data_types(header_.getDataTypes()), - settings(settings_), ignore_spaces(ignore_spaces_), + settings(std::move(settings_)), ignore_spaces(ignore_spaces_), format(std::move(format_)), row_format(std::move(row_format_)), - default_csv_delimiter(settings.csv.delimiter) + default_csv_delimiter(settings.csv.delimiter), row_between_delimiter(std::move(row_between_delimiter_)) { /// Validate format string for result set bool has_data = false; @@ -160,7 +161,7 @@ bool TemplateRowInputFormat::readRow(MutableColumns & columns, RowReadExtension updateDiagnosticInfo(); if (likely(row_num != 1)) - assertString(settings.template_settings.row_between_delimiter, buf); + assertString(row_between_delimiter, buf); extra.read_columns.assign(columns.size(), false); @@ -339,11 +340,11 @@ bool TemplateRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & col try { if (likely(row_num != 1)) - assertString(settings.template_settings.row_between_delimiter, buf); + assertString(row_between_delimiter, buf); } catch (const DB::Exception &) { - writeErrorStringForWrongDelimiter(out, "delimiter between rows", settings.template_settings.row_between_delimiter); + writeErrorStringForWrongDelimiter(out, "delimiter between rows", row_between_delimiter); return false; } @@ -428,7 +429,7 @@ bool TemplateRowInputFormat::isGarbageAfterField(size_t, ReadBuffer::Position) bool TemplateRowInputFormat::allowSyncAfterError() const { - return !row_format.delimiters.back().empty() || !settings.template_settings.row_between_delimiter.empty(); + return !row_format.delimiters.back().empty() || !row_between_delimiter.empty(); } void TemplateRowInputFormat::syncAfterError() @@ -450,10 +451,10 @@ void TemplateRowInputFormat::syncAfterError() bool last_delimiter_in_row_found = !row_format.delimiters.back().empty(); - if (last_delimiter_in_row_found && checkString(settings.template_settings.row_between_delimiter, buf)) + if (last_delimiter_in_row_found && checkString(row_between_delimiter, buf)) at_beginning_of_row_or_eof = true; else - skipToNextDelimiterOrEof(settings.template_settings.row_between_delimiter); + skipToNextDelimiterOrEof(row_between_delimiter); if (buf.eof()) at_beginning_of_row_or_eof = end_of_stream = true; @@ -509,7 +510,6 @@ void registerInputFormatProcessorTemplate(FormatFactory & factory) factory.registerInputFormatProcessor(ignore_spaces ? "TemplateIgnoreSpaces" : "Template", [=]( ReadBuffer & buf, const Block & sample, - const Context & context, IRowInputFormat::Params params, const FormatSettings & settings) { @@ -526,7 +526,8 @@ void registerInputFormatProcessorTemplate(FormatFactory & factory) { /// Read format string from file resultset_format = ParsedTemplateFormatString( - FormatSchemaInfo(context, settings.template_settings.resultset_format, "Template", false), + FormatSchemaInfo(settings.template_settings.resultset_format, "Template", false, + settings.schema.is_server, settings.schema.format_schema_path), [&](const String & partName) -> std::optional { if (partName == "data") @@ -537,13 +538,14 @@ void registerInputFormatProcessorTemplate(FormatFactory & factory) } ParsedTemplateFormatString row_format = ParsedTemplateFormatString( - FormatSchemaInfo(context, settings.template_settings.row_format, "Template", false), + FormatSchemaInfo(settings.template_settings.row_format, "Template", false, + settings.schema.is_server, settings.schema.format_schema_path), [&](const String & colName) -> std::optional { return sample.getPositionByName(colName); }); - return std::make_shared(sample, buf, params, settings, ignore_spaces, resultset_format, row_format); + return std::make_shared(sample, buf, params, settings, ignore_spaces, resultset_format, row_format, settings.template_settings.row_between_delimiter); }); } @@ -552,16 +554,13 @@ void registerInputFormatProcessorTemplate(FormatFactory & factory) factory.registerInputFormatProcessor(ignore_spaces ? "CustomSeparatedIgnoreSpaces" : "CustomSeparated", [=]( ReadBuffer & buf, const Block & sample, - const Context & context, IRowInputFormat::Params params, const FormatSettings & settings) { - ParsedTemplateFormatString resultset_format = ParsedTemplateFormatString::setupCustomSeparatedResultsetFormat(context); - ParsedTemplateFormatString row_format = ParsedTemplateFormatString::setupCustomSeparatedRowFormat(context, sample); - FormatSettings format_settings = settings; - format_settings.template_settings.row_between_delimiter = context.getSettingsRef().format_custom_row_between_delimiter; + ParsedTemplateFormatString resultset_format = ParsedTemplateFormatString::setupCustomSeparatedResultsetFormat(settings.custom); + ParsedTemplateFormatString row_format = ParsedTemplateFormatString::setupCustomSeparatedRowFormat(settings.custom, sample); - return std::make_shared(sample, buf, params, format_settings, ignore_spaces, resultset_format, row_format); + return std::make_shared(sample, buf, params, settings, ignore_spaces, resultset_format, row_format, settings.custom.row_between_delimiter); }); } } diff --git a/dbms/src/Processors/Formats/Impl/TemplateRowInputFormat.h b/dbms/src/Processors/Formats/Impl/TemplateRowInputFormat.h index a9180c52606..4a39396399d 100644 --- a/dbms/src/Processors/Formats/Impl/TemplateRowInputFormat.h +++ b/dbms/src/Processors/Formats/Impl/TemplateRowInputFormat.h @@ -16,8 +16,9 @@ class TemplateRowInputFormat : public RowInputFormatWithDiagnosticInfo using ColumnFormat = ParsedTemplateFormatString::ColumnFormat; public: TemplateRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_, - const FormatSettings & settings_, bool ignore_spaces_, - ParsedTemplateFormatString format_, ParsedTemplateFormatString row_format_); + FormatSettings settings_, bool ignore_spaces_, + ParsedTemplateFormatString format_, ParsedTemplateFormatString row_format_, + std::string row_between_delimiter); String getName() const override { return "TemplateRowInputFormat"; } @@ -61,6 +62,8 @@ private: bool end_of_stream = false; std::vector always_default_columns; char default_csv_delimiter; + + std::string row_between_delimiter; }; } diff --git a/dbms/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/dbms/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index c42b638fb48..b324719527a 100644 --- a/dbms/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -33,8 +33,8 @@ namespace ErrorCodes ValuesBlockInputFormat::ValuesBlockInputFormat(ReadBuffer & in_, const Block & header_, const RowInputFormatParams & params_, - const Context & context_, const FormatSettings & format_settings_) - : IInputFormat(header_, buf), buf(in_), params(params_), context(std::make_unique(context_)), + const FormatSettings & format_settings_) + : IInputFormat(header_, buf), buf(in_), params(params_), format_settings(format_settings_), num_columns(header_.columns()), parser_type_for_column(num_columns, ParserType::Streaming), attempts_to_deduce_template(num_columns), attempts_to_deduce_template_cached(num_columns), @@ -424,11 +424,10 @@ void registerInputFormatProcessorValues(FormatFactory & factory) factory.registerInputFormatProcessor("Values", []( ReadBuffer & buf, const Block & header, - const Context & context, const RowInputFormatParams & params, const FormatSettings & settings) { - return std::make_shared(buf, header, params, context, settings); + return std::make_shared(buf, header, params, settings); }); } diff --git a/dbms/src/Processors/Formats/Impl/ValuesBlockInputFormat.h b/dbms/src/Processors/Formats/Impl/ValuesBlockInputFormat.h index 9c2473a0a04..059a15e1e86 100644 --- a/dbms/src/Processors/Formats/Impl/ValuesBlockInputFormat.h +++ b/dbms/src/Processors/Formats/Impl/ValuesBlockInputFormat.h @@ -18,7 +18,7 @@ class ReadBuffer; /** Stream to read data in VALUES format (as in INSERT query). */ -class ValuesBlockInputFormat : public IInputFormat +class ValuesBlockInputFormat final : public IInputFormat { public: /** Data is parsed using fast, streaming parser. @@ -29,12 +29,15 @@ public: * than interpreting expressions in each row separately, but it's still slower than streaming parsing) */ ValuesBlockInputFormat(ReadBuffer & in_, const Block & header_, const RowInputFormatParams & params_, - const Context & context_, const FormatSettings & format_settings_); + const FormatSettings & format_settings_); String getName() const override { return "ValuesBlockInputFormat"; } void resetParser() override; + /// TODO: remove context somehow. + void setContext(const Context & context_) { context = std::make_unique(context_); } + const BlockMissingValues & getMissingValues() const override { return block_missing_values; } private: diff --git a/dbms/src/Processors/Formats/Impl/ValuesRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/ValuesRowOutputFormat.cpp index e773cbf167c..6294a829456 100644 --- a/dbms/src/Processors/Formats/Impl/ValuesRowOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ValuesRowOutputFormat.cpp @@ -46,7 +46,6 @@ void registerOutputFormatProcessorValues(FormatFactory & factory) factory.registerOutputFormatProcessor("Values", []( WriteBuffer & buf, const Block & sample, - const Context &, FormatFactory::WriteCallback callback, const FormatSettings & settings) { diff --git a/dbms/src/Processors/Formats/Impl/VerticalRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/VerticalRowOutputFormat.cpp index 6e794c9a79b..ccef2a0898a 100644 --- a/dbms/src/Processors/Formats/Impl/VerticalRowOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/VerticalRowOutputFormat.cpp @@ -168,7 +168,6 @@ void registerOutputFormatProcessorVertical(FormatFactory & factory) factory.registerOutputFormatProcessor("Vertical", []( WriteBuffer & buf, const Block & sample, - const Context &, FormatFactory::WriteCallback callback, const FormatSettings & settings) { diff --git a/dbms/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp index 07335fc6c49..9c2384691f6 100644 --- a/dbms/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp @@ -245,7 +245,6 @@ void registerOutputFormatProcessorXML(FormatFactory & factory) factory.registerOutputFormatProcessor("XML", []( WriteBuffer & buf, const Block & sample, - const Context &, FormatFactory::WriteCallback callback, const FormatSettings & settings) {