From 0fa4c37e855036acfad7c0a541ca32911690bc76 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Tue, 22 Sep 2020 20:47:07 +0300 Subject: [PATCH 001/256] fix not equi join on issue --- src/Interpreters/CollectJoinOnKeysVisitor.cpp | 6 ++++++ tests/queries/0_stateless/01478_not_equi-join_on.reference | 0 tests/queries/0_stateless/01478_not_equi-join_on.sql | 7 +++++++ 3 files changed, 13 insertions(+) create mode 100644 tests/queries/0_stateless/01478_not_equi-join_on.reference create mode 100644 tests/queries/0_stateless/01478_not_equi-join_on.sql diff --git a/src/Interpreters/CollectJoinOnKeysVisitor.cpp b/src/Interpreters/CollectJoinOnKeysVisitor.cpp index e0fce4854d2..943db234c7e 100644 --- a/src/Interpreters/CollectJoinOnKeysVisitor.cpp +++ b/src/Interpreters/CollectJoinOnKeysVisitor.cpp @@ -127,6 +127,12 @@ std::pair CollectJoinOnKeysMatcher::getTableNumbers(const ASTPtr getIdentifiers(left_ast, left_identifiers); getIdentifiers(right_ast, right_identifiers); + if (left_identifiers.empty() || right_identifiers.empty()) + { + throw Exception("Not equi-join ON expression: " + queryToString(expr) + ". No columns in one of equality side.", + ErrorCodes::INVALID_JOIN_ON_EXPRESSION); + } + size_t left_idents_table = getTableForIdentifiers(left_identifiers, data); size_t right_idents_table = getTableForIdentifiers(right_identifiers, data); diff --git a/tests/queries/0_stateless/01478_not_equi-join_on.reference b/tests/queries/0_stateless/01478_not_equi-join_on.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01478_not_equi-join_on.sql b/tests/queries/0_stateless/01478_not_equi-join_on.sql new file mode 100644 index 00000000000..b52af5fcb46 --- /dev/null +++ b/tests/queries/0_stateless/01478_not_equi-join_on.sql @@ -0,0 +1,7 @@ +SELECT * FROM (SELECT NULL AS a, 1 AS b) AS foo +RIGHT JOIN (SELECT 1024 AS b) AS bar +ON 1 = foo.b; -- { serverError 403 } + +SELECT * FROM (SELECT NULL AS a, 1 AS b) AS foo +RIGHT JOIN (SELECT 1024 AS b) AS bar +ON 1 = bar.b; -- { serverError 403 } From e349769524ed3cdab6883782f1922700ddaf6146 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Tue, 22 Sep 2020 22:22:47 +0300 Subject: [PATCH 002/256] update tests --- .../0_stateless/00878_join_unexpected_results.reference | 4 ---- .../queries/0_stateless/00878_join_unexpected_results.sql | 8 ++++---- .../01081_PartialSortingTransform_full_column.sql | 4 ++-- .../0_stateless/01101_literal_column_clash.reference | 4 ---- tests/queries/0_stateless/01101_literal_column_clash.sql | 5 ++--- 5 files changed, 8 insertions(+), 17 deletions(-) diff --git a/tests/queries/0_stateless/00878_join_unexpected_results.reference b/tests/queries/0_stateless/00878_join_unexpected_results.reference index 65fcbc257ca..a389cb47a96 100644 --- a/tests/queries/0_stateless/00878_join_unexpected_results.reference +++ b/tests/queries/0_stateless/00878_join_unexpected_results.reference @@ -23,8 +23,6 @@ join_use_nulls = 1 - \N \N - -1 1 \N \N -2 2 \N \N - 1 1 1 1 2 2 \N \N @@ -51,8 +49,6 @@ join_use_nulls = 0 - - - -1 1 0 0 -2 2 0 0 - 1 1 1 1 2 2 0 0 diff --git a/tests/queries/0_stateless/00878_join_unexpected_results.sql b/tests/queries/0_stateless/00878_join_unexpected_results.sql index 6f6cd6e6479..0aef5208b26 100644 --- a/tests/queries/0_stateless/00878_join_unexpected_results.sql +++ b/tests/queries/0_stateless/00878_join_unexpected_results.sql @@ -30,11 +30,11 @@ select * from t left outer join s on (t.a=s.a and t.b=s.b) where s.a is null; select '-'; select s.* from t left outer join s on (t.a=s.a and t.b=s.b) where s.a is null; select '-'; -select t.*, s.* from t left join s on (s.a=t.a and t.b=s.b and t.a=toInt64(2)) order by t.a; +select t.*, s.* from t left join s on (s.a=t.a and t.b=s.b and t.a=toInt64(2)) order by t.a; -- {serverError 403 } select '-'; select t.*, s.* from t left join s on (s.a=t.a) order by t.a; select '-'; -select t.*, s.* from t left join s on (t.b=toInt64(2) and s.a=t.a) where s.b=2; +select t.*, s.* from t left join s on (t.b=toInt64(2) and s.a=t.a) where s.b=2; -- {serverError 403 } select 'join_use_nulls = 0'; set join_use_nulls = 0; @@ -58,11 +58,11 @@ select '-'; select '-'; -- select s.* from t left outer join s on (t.a=s.a and t.b=s.b) where s.a is null; -- TODO select '-'; -select t.*, s.* from t left join s on (s.a=t.a and t.b=s.b and t.a=toInt64(2)) order by t.a; +select t.*, s.* from t left join s on (s.a=t.a and t.b=s.b and t.a=toInt64(2)) order by t.a; -- {serverError 403 } select '-'; select t.*, s.* from t left join s on (s.a=t.a) order by t.a; select '-'; -select t.*, s.* from t left join s on (t.b=toInt64(2) and s.a=t.a) where s.b=2; +select t.*, s.* from t left join s on (t.b=toInt64(2) and s.a=t.a) where s.b=2; -- {serverError 403 } drop table t; drop table s; diff --git a/tests/queries/0_stateless/01081_PartialSortingTransform_full_column.sql b/tests/queries/0_stateless/01081_PartialSortingTransform_full_column.sql index 146107d6f3d..cfa37b06477 100644 --- a/tests/queries/0_stateless/01081_PartialSortingTransform_full_column.sql +++ b/tests/queries/0_stateless/01081_PartialSortingTransform_full_column.sql @@ -2,7 +2,7 @@ drop table if exists test_01081; create table test_01081 (key Int) engine=MergeTree() order by key; insert into test_01081 select * from system.numbers limit 10; -select 1 from remote('127.{1,2}', currentDatabase(), test_01081) lhs join system.one as rhs on rhs.dummy = 1 order by 1; +select 1 from remote('127.{1,2}', currentDatabase(), test_01081) lhs join system.one as rhs on rhs.dummy = 1 order by 1; -- { serverError 403 } -- With multiple blocks triggers: -- @@ -13,6 +13,6 @@ select 1 from remote('127.{1,2}', currentDatabase(), test_01081) lhs join system -- With experimental_use_processors=1 (default at the time of writing). insert into test_01081 select * from system.numbers limit 10; -select 1 from remote('127.{1,2}', currentDatabase(), test_01081) lhs join system.one as rhs on rhs.dummy = 1 order by 1; +select 1 from remote('127.{1,2}', currentDatabase(), test_01081) lhs join system.one as rhs on rhs.dummy = 1 order by 1; -- { serverError 403 } drop table if exists test_01081; diff --git a/tests/queries/0_stateless/01101_literal_column_clash.reference b/tests/queries/0_stateless/01101_literal_column_clash.reference index 47458541f79..e63f96416af 100644 --- a/tests/queries/0_stateless/01101_literal_column_clash.reference +++ b/tests/queries/0_stateless/01101_literal_column_clash.reference @@ -8,7 +8,3 @@ xyzabc 2 1 0 0 3 \N 1 2 \N 0 \N 1 0 \N 3 -1 -1 -1 hello\0\0\0 -1 hello\0\0\0 diff --git a/tests/queries/0_stateless/01101_literal_column_clash.sql b/tests/queries/0_stateless/01101_literal_column_clash.sql index 2c665d3833b..f96aad4bacf 100644 --- a/tests/queries/0_stateless/01101_literal_column_clash.sql +++ b/tests/queries/0_stateless/01101_literal_column_clash.sql @@ -17,7 +17,6 @@ select null, isConstant(null), * from (select 2 x, null) a right join (select 3 -- other cases with joins and constants -select cast(1, 'UInt8') from (select arrayJoin([1, 2]) as a) t1 left join (select 1 as b) t2 on b = ignore('UInt8'); - -select isConstant('UInt8'), toFixedString('hello', toUInt8(substring('UInt8', 5, 1))) from (select arrayJoin([1, 2]) as a) t1 left join (select 1 as b) t2 on b = ignore('UInt8'); +select cast(1, 'UInt8') from (select arrayJoin([1, 2]) as a) t1 left join (select 1 as b) t2 on b = ignore('UInt8'); -- { serverError 403 } +select isConstant('UInt8'), toFixedString('hello', toUInt8(substring('UInt8', 5, 1))) from (select arrayJoin([1, 2]) as a) t1 left join (select 1 as b) t2 on b = ignore('UInt8'); -- { serverError 403 } From 1127232446a1133a1137a6fcd62661eb9eaa6673 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Wed, 23 Sep 2020 00:03:20 +0300 Subject: [PATCH 003/256] one more test fix --- .../queries/0_stateless/01018_Distributed__shard_num.reference | 2 -- tests/queries/0_stateless/01018_Distributed__shard_num.sql | 3 ++- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01018_Distributed__shard_num.reference b/tests/queries/0_stateless/01018_Distributed__shard_num.reference index 679383f969e..f4b92011759 100644 --- a/tests/queries/0_stateless/01018_Distributed__shard_num.reference +++ b/tests/queries/0_stateless/01018_Distributed__shard_num.reference @@ -29,8 +29,6 @@ remote(Distributed) JOIN system.clusters 1 10 localhost ::1 9000 1 20 localhost ::1 9000 -1 10 localhost ::1 9000 -1 20 localhost ::1 9000 dist_3 100 foo foo 100 foo diff --git a/tests/queries/0_stateless/01018_Distributed__shard_num.sql b/tests/queries/0_stateless/01018_Distributed__shard_num.sql index ddb8505a415..f6d5f23eca8 100644 --- a/tests/queries/0_stateless/01018_Distributed__shard_num.sql +++ b/tests/queries/0_stateless/01018_Distributed__shard_num.sql @@ -48,11 +48,12 @@ FROM (SELECT *, _shard_num FROM dist_1) a JOIN system.clusters b ON a._shard_num = b.shard_num WHERE b.cluster = 'test_cluster_two_shards_localhost'; + SELECT _shard_num, key, b.host_name, b.host_address, b.port FROM dist_1 a JOIN system.clusters b ON _shard_num = b.shard_num -WHERE b.cluster = 'test_cluster_two_shards_localhost'; +WHERE b.cluster = 'test_cluster_two_shards_localhost'; -- { serverError 403 } -- rewrite does not work with aliases, hence Missing columns (47) SELECT a._shard_num, key FROM dist_1 a; -- { serverError 47; } From 47d150b11639ec2fbfb5c691b38c172da344d6da Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 25 Sep 2020 02:29:16 +0300 Subject: [PATCH 004/256] Revert "Revert "Avoid deadlocks in Log/TinyLog"" --- src/Common/FileChecker.cpp | 5 ++ src/Common/FileChecker.h | 4 +- src/Storages/StorageFile.cpp | 40 ++++++--- src/Storages/StorageFile.h | 2 +- src/Storages/StorageLog.cpp | 60 +++++++++---- src/Storages/StorageLog.h | 4 +- src/Storages/StorageStripeLog.cpp | 45 +++++++--- src/Storages/StorageStripeLog.h | 2 +- src/Storages/StorageTinyLog.cpp | 87 +++++++++++++------ src/Storages/StorageTinyLog.h | 4 +- .../0_stateless/01499_log_deadlock.reference | 3 + .../0_stateless/01499_log_deadlock.sql | 26 ++++++ ...2_long_log_tinylog_deadlock_race.reference | 6 ++ .../01502_long_log_tinylog_deadlock_race.sh | 85 ++++++++++++++++++ .../01505_log_distributed_deadlock.reference | 0 .../01505_log_distributed_deadlock.sql | 12 +++ .../queries/0_stateless/arcadia_skip_list.txt | 1 + 17 files changed, 307 insertions(+), 79 deletions(-) create mode 100644 tests/queries/0_stateless/01499_log_deadlock.reference create mode 100644 tests/queries/0_stateless/01499_log_deadlock.sql create mode 100644 tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.reference create mode 100755 tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.sh create mode 100644 tests/queries/0_stateless/01505_log_distributed_deadlock.reference create mode 100644 tests/queries/0_stateless/01505_log_distributed_deadlock.sql diff --git a/src/Common/FileChecker.cpp b/src/Common/FileChecker.cpp index 6cbec3bda77..b306c3af990 100644 --- a/src/Common/FileChecker.cpp +++ b/src/Common/FileChecker.cpp @@ -41,6 +41,11 @@ void FileChecker::setEmpty(const String & full_file_path) map[fileName(full_file_path)] = 0; } +FileChecker::Map FileChecker::getFileSizes() const +{ + return map; +} + CheckResults FileChecker::check() const { // Read the files again every time you call `check` - so as not to violate the constancy. diff --git a/src/Common/FileChecker.h b/src/Common/FileChecker.h index 015d4cadb07..59e7331952e 100644 --- a/src/Common/FileChecker.h +++ b/src/Common/FileChecker.h @@ -27,10 +27,12 @@ public: /// The purpose of this function is to rollback a group of unfinished writes. void repair(); -private: /// File name -> size. using Map = std::map; + Map getFileSizes() const; + +private: void initialize(); void updateImpl(const String & file_path); void load(Map & local_map, const String & path) const; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index cc47047dc78..7b094f9bc06 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -52,6 +52,7 @@ namespace ErrorCodes extern const int UNKNOWN_IDENTIFIER; extern const int INCORRECT_FILE_NAME; extern const int FILE_DOESNT_EXIST; + extern const int TIMEOUT_EXCEEDED; } namespace @@ -199,6 +200,17 @@ StorageFile::StorageFile(CommonArguments args) setInMemoryMetadata(storage_metadata); } + +static std::chrono::seconds getLockTimeout(const Context & context) +{ + const Settings & settings = context.getSettingsRef(); + Int64 lock_timeout = settings.lock_acquire_timeout.totalSeconds(); + if (settings.max_execution_time.totalSeconds() != 0 && settings.max_execution_time.totalSeconds() < lock_timeout) + lock_timeout = settings.max_execution_time.totalSeconds(); + return std::chrono::seconds{lock_timeout}; +} + + class StorageFileSource : public SourceWithProgress { public: @@ -245,7 +257,9 @@ public: { if (storage->use_table_fd) { - unique_lock = std::unique_lock(storage->rwlock); + unique_lock = std::unique_lock(storage->rwlock, getLockTimeout(context)); + if (!unique_lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); /// We could use common ReadBuffer and WriteBuffer in storage to leverage cache /// and add ability to seek unseekable files, but cache sync isn't supported. @@ -264,7 +278,9 @@ public: } else { - shared_lock = std::shared_lock(storage->rwlock); + shared_lock = std::shared_lock(storage->rwlock, getLockTimeout(context)); + if (!shared_lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); } } @@ -373,8 +389,8 @@ private: bool finished_generate = false; - std::shared_lock shared_lock; - std::unique_lock unique_lock; + std::shared_lock shared_lock; + std::unique_lock unique_lock; }; @@ -417,7 +433,7 @@ Pipe StorageFile::read( for (size_t i = 0; i < num_streams; ++i) pipes.emplace_back(std::make_shared( - this_ptr, metadata_snapshot, context, max_block_size, files_info, metadata_snapshot->getColumns().getDefaults())); + this_ptr, metadata_snapshot, context, max_block_size, files_info, metadata_snapshot->getColumns().getDefaults())); return Pipe::unitePipes(std::move(pipes)); } @@ -429,12 +445,16 @@ public: explicit StorageFileBlockOutputStream( StorageFile & storage_, const StorageMetadataPtr & metadata_snapshot_, + std::unique_lock && lock_, const CompressionMethod compression_method, const Context & context) : storage(storage_) , metadata_snapshot(metadata_snapshot_) - , lock(storage.rwlock) + , lock(std::move(lock_)) { + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + std::unique_ptr naked_buffer = nullptr; if (storage.use_table_fd) { @@ -488,7 +508,7 @@ public: private: StorageFile & storage; StorageMetadataPtr metadata_snapshot; - std::unique_lock lock; + std::unique_lock lock; std::unique_ptr write_buf; BlockOutputStreamPtr writer; bool prefix_written{false}; @@ -506,7 +526,7 @@ BlockOutputStreamPtr StorageFile::write( if (!paths.empty()) path = paths[0]; - return std::make_shared(*this, metadata_snapshot, + return std::make_shared(*this, metadata_snapshot, std::unique_lock{rwlock, getLockTimeout(context)}, chooseCompressionMethod(path, compression_method), context); } @@ -529,8 +549,6 @@ void StorageFile::rename(const String & new_path_to_table_data, const StorageID if (path_new == paths[0]) return; - std::unique_lock lock(rwlock); - Poco::File(Poco::Path(path_new).parent()).createDirectories(); Poco::File(paths[0]).renameTo(path_new); @@ -547,8 +565,6 @@ void StorageFile::truncate( if (paths.size() != 1) throw Exception("Can't truncate table '" + getStorageID().getNameForLogs() + "' in readonly mode", ErrorCodes::DATABASE_ACCESS_DENIED); - std::unique_lock lock(rwlock); - if (use_table_fd) { if (0 != ::ftruncate(table_fd, 0)) diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index ea70dcd5311..babc56e3a11 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -89,7 +89,7 @@ private: std::atomic table_fd_was_used{false}; /// To detect repeating reads from stdin off_t table_fd_init_offset = -1; /// Initial position of fd, used for repeating reads - mutable std::shared_mutex rwlock; + mutable std::shared_timed_mutex rwlock; Poco::Logger * log = &Poco::Logger::get("StorageFile"); }; diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index e437bfb05f1..2fbce21655c 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -39,6 +39,7 @@ namespace DB namespace ErrorCodes { + extern const int TIMEOUT_EXCEEDED; extern const int LOGICAL_ERROR; extern const int DUPLICATE_COLUMN; extern const int SIZES_OF_MARKS_FILES_ARE_INCONSISTENT; @@ -50,7 +51,6 @@ namespace ErrorCodes class LogSource final : public SourceWithProgress { public: - static Block getHeader(const NamesAndTypesList & columns) { Block res; @@ -116,13 +116,16 @@ private: class LogBlockOutputStream final : public IBlockOutputStream { public: - explicit LogBlockOutputStream(StorageLog & storage_, const StorageMetadataPtr & metadata_snapshot_) + explicit LogBlockOutputStream( + StorageLog & storage_, const StorageMetadataPtr & metadata_snapshot_, std::unique_lock && lock_) : storage(storage_) , metadata_snapshot(metadata_snapshot_) - , lock(storage.rwlock) + , lock(std::move(lock_)) , marks_stream( storage.disk->writeFile(storage.marks_file_path, 4096, WriteMode::Rewrite)) { + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); } ~LogBlockOutputStream() override @@ -149,7 +152,7 @@ public: private: StorageLog & storage; StorageMetadataPtr metadata_snapshot; - std::unique_lock lock; + std::unique_lock lock; bool done = false; struct Stream @@ -507,9 +510,11 @@ void StorageLog::addFiles(const String & column_name, const IDataType & type) } -void StorageLog::loadMarks() +void StorageLog::loadMarks(std::chrono::seconds lock_timeout) { - std::unique_lock lock(rwlock); + std::unique_lock lock(rwlock, lock_timeout); + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); if (loaded_marks) return; @@ -552,8 +557,6 @@ void StorageLog::rename(const String & new_path_to_table_data, const StorageID & { assert(table_path != new_path_to_table_data); { - std::unique_lock lock(rwlock); - disk->moveDirectory(table_path, new_path_to_table_data); table_path = new_path_to_table_data; @@ -569,8 +572,6 @@ void StorageLog::rename(const String & new_path_to_table_data, const StorageID & void StorageLog::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &) { - std::shared_lock lock(rwlock); - files.clear(); file_count = 0; loaded_marks = false; @@ -610,6 +611,17 @@ const StorageLog::Marks & StorageLog::getMarksWithRealRowCount(const StorageMeta return it->second.marks; } + +static std::chrono::seconds getLockTimeout(const Context & context) +{ + const Settings & settings = context.getSettingsRef(); + Int64 lock_timeout = settings.lock_acquire_timeout.totalSeconds(); + if (settings.max_execution_time.totalSeconds() != 0 && settings.max_execution_time.totalSeconds() < lock_timeout) + lock_timeout = settings.max_execution_time.totalSeconds(); + return std::chrono::seconds{lock_timeout}; +} + + Pipe StorageLog::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, @@ -620,11 +632,15 @@ Pipe StorageLog::read( unsigned num_streams) { metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); - loadMarks(); + + auto lock_timeout = getLockTimeout(context); + loadMarks(lock_timeout); NamesAndTypesList all_columns = Nested::collect(metadata_snapshot->getColumns().getAllPhysical().addTypes(column_names)); - std::shared_lock lock(rwlock); + std::shared_lock lock(rwlock, lock_timeout); + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); Pipes pipes; @@ -653,18 +669,28 @@ Pipe StorageLog::read( max_read_buffer_size)); } + /// No need to hold lock while reading because we read fixed range of data that does not change while appending more data. return Pipe::unitePipes(std::move(pipes)); } -BlockOutputStreamPtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) +BlockOutputStreamPtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & context) { - loadMarks(); - return std::make_shared(*this, metadata_snapshot); + auto lock_timeout = getLockTimeout(context); + loadMarks(lock_timeout); + + std::unique_lock lock(rwlock, lock_timeout); + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + + return std::make_shared(*this, metadata_snapshot, std::move(lock)); } -CheckResults StorageLog::checkData(const ASTPtr & /* query */, const Context & /* context */) +CheckResults StorageLog::checkData(const ASTPtr & /* query */, const Context & context) { - std::shared_lock lock(rwlock); + std::shared_lock lock(rwlock, getLockTimeout(context)); + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + return file_checker.check(); } diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index 49fc9a576c5..3553426b9e6 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -83,7 +83,7 @@ private: DiskPtr disk; String table_path; - mutable std::shared_mutex rwlock; + mutable std::shared_timed_mutex rwlock; Files files; @@ -104,7 +104,7 @@ private: /// Read marks files if they are not already read. /// It is done lazily, so that with a large number of tables, the server starts quickly. /// You can not call with a write locked `rwlock`. - void loadMarks(); + void loadMarks(std::chrono::seconds lock_timeout); /** For normal columns, the number of rows in the block is specified in the marks. * For array columns and nested structures, there are more than one group of marks that correspond to different files diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index c4344cf6f1f..8ff8035c128 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -47,13 +47,13 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int INCORRECT_FILE_NAME; + extern const int TIMEOUT_EXCEEDED; } class StripeLogSource final : public SourceWithProgress { public: - static Block getHeader( StorageStripeLog & storage, const StorageMetadataPtr & metadata_snapshot, @@ -157,10 +157,11 @@ private: class StripeLogBlockOutputStream final : public IBlockOutputStream { public: - explicit StripeLogBlockOutputStream(StorageStripeLog & storage_, const StorageMetadataPtr & metadata_snapshot_) + explicit StripeLogBlockOutputStream( + StorageStripeLog & storage_, const StorageMetadataPtr & metadata_snapshot_, std::unique_lock && lock_) : storage(storage_) , metadata_snapshot(metadata_snapshot_) - , lock(storage.rwlock) + , lock(std::move(lock_)) , data_out_file(storage.table_path + "data.bin") , data_out_compressed(storage.disk->writeFile(data_out_file, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append)) , data_out(std::make_unique( @@ -170,6 +171,8 @@ public: , index_out(std::make_unique(*index_out_compressed)) , block_out(*data_out, 0, metadata_snapshot->getSampleBlock(), false, index_out.get(), storage.disk->getFileSize(data_out_file)) { + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); } ~StripeLogBlockOutputStream() override @@ -223,7 +226,7 @@ public: private: StorageStripeLog & storage; StorageMetadataPtr metadata_snapshot; - std::unique_lock lock; + std::unique_lock lock; String data_out_file; std::unique_ptr data_out_compressed; @@ -286,8 +289,6 @@ void StorageStripeLog::rename(const String & new_path_to_table_data, const Stora { assert(table_path != new_path_to_table_data); { - std::unique_lock lock(rwlock); - disk->moveDirectory(table_path, new_path_to_table_data); table_path = new_path_to_table_data; @@ -297,6 +298,16 @@ void StorageStripeLog::rename(const String & new_path_to_table_data, const Stora } +static std::chrono::seconds getLockTimeout(const Context & context) +{ + const Settings & settings = context.getSettingsRef(); + Int64 lock_timeout = settings.lock_acquire_timeout.totalSeconds(); + if (settings.max_execution_time.totalSeconds() != 0 && settings.max_execution_time.totalSeconds() < lock_timeout) + lock_timeout = settings.max_execution_time.totalSeconds(); + return std::chrono::seconds{lock_timeout}; +} + + Pipe StorageStripeLog::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, @@ -306,7 +317,9 @@ Pipe StorageStripeLog::read( const size_t /*max_block_size*/, unsigned num_streams) { - std::shared_lock lock(rwlock); + std::shared_lock lock(rwlock, getLockTimeout(context)); + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); @@ -345,24 +358,28 @@ Pipe StorageStripeLog::read( } -BlockOutputStreamPtr StorageStripeLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) +BlockOutputStreamPtr StorageStripeLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & context) { - return std::make_shared(*this, metadata_snapshot); + std::unique_lock lock(rwlock, getLockTimeout(context)); + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + + return std::make_shared(*this, metadata_snapshot, std::move(lock)); } -CheckResults StorageStripeLog::checkData(const ASTPtr & /* query */, const Context & /* context */) +CheckResults StorageStripeLog::checkData(const ASTPtr & /* query */, const Context & context) { - std::shared_lock lock(rwlock); + std::shared_lock lock(rwlock, getLockTimeout(context)); + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + return file_checker.check(); } void StorageStripeLog::truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) { - std::shared_lock lock(rwlock); - disk->clearDirectory(table_path); - file_checker = FileChecker{disk, table_path + "sizes.json"}; } diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index f88120a932e..ca3bfe4ff75 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -67,7 +67,7 @@ private: size_t max_compress_block_size; FileChecker file_checker; - mutable std::shared_mutex rwlock; + mutable std::shared_timed_mutex rwlock; Poco::Logger * log; }; diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 0bdcab8abf4..4d646c7451e 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -13,6 +13,7 @@ #include #include +#include #include #include #include @@ -46,6 +47,7 @@ namespace DB namespace ErrorCodes { + extern const int TIMEOUT_EXCEEDED; extern const int DUPLICATE_COLUMN; extern const int INCORRECT_FILE_NAME; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; @@ -55,7 +57,6 @@ namespace ErrorCodes class TinyLogSource final : public SourceWithProgress { public: - static Block getHeader(const NamesAndTypesList & columns) { Block res; @@ -66,10 +67,17 @@ public: return Nested::flatten(res); } - TinyLogSource(size_t block_size_, const NamesAndTypesList & columns_, StorageTinyLog & storage_, size_t max_read_buffer_size_) + TinyLogSource( + size_t block_size_, + const NamesAndTypesList & columns_, + StorageTinyLog & storage_, + size_t max_read_buffer_size_, + FileChecker::Map file_sizes_) : SourceWithProgress(getHeader(columns_)) - , block_size(block_size_), columns(columns_), storage(storage_), lock(storage_.rwlock) - , max_read_buffer_size(max_read_buffer_size_) {} + , block_size(block_size_), columns(columns_), storage(storage_) + , max_read_buffer_size(max_read_buffer_size_), file_sizes(std::move(file_sizes_)) + { + } String getName() const override { return "TinyLog"; } @@ -80,19 +88,21 @@ private: size_t block_size; NamesAndTypesList columns; StorageTinyLog & storage; - std::shared_lock lock; bool is_finished = false; size_t max_read_buffer_size; + FileChecker::Map file_sizes; struct Stream { - Stream(const DiskPtr & disk, const String & data_path, size_t max_read_buffer_size_) + Stream(const DiskPtr & disk, const String & data_path, size_t max_read_buffer_size_, size_t file_size) : plain(disk->readFile(data_path, std::min(max_read_buffer_size_, disk->getFileSize(data_path)))), + limited(std::make_unique(*plain, file_size, false)), compressed(*plain) { } std::unique_ptr plain; + std::unique_ptr limited; CompressedReadBuffer compressed; }; @@ -110,9 +120,14 @@ private: class TinyLogBlockOutputStream final : public IBlockOutputStream { public: - explicit TinyLogBlockOutputStream(StorageTinyLog & storage_, const StorageMetadataPtr & metadata_snapshot_) - : storage(storage_), metadata_snapshot(metadata_snapshot_), lock(storage_.rwlock) + explicit TinyLogBlockOutputStream( + StorageTinyLog & storage_, + const StorageMetadataPtr & metadata_snapshot_, + std::unique_lock && lock_) + : storage(storage_), metadata_snapshot(metadata_snapshot_), lock(std::move(lock_)) { + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); } ~TinyLogBlockOutputStream() override @@ -140,7 +155,7 @@ public: private: StorageTinyLog & storage; StorageMetadataPtr metadata_snapshot; - std::unique_lock lock; + std::unique_lock lock; bool done = false; struct Stream @@ -231,13 +246,17 @@ void TinyLogSource::readData(const String & name, const IDataType & type, IColum String stream_name = IDataType::getFileNameForStream(name, path); if (!streams.count(stream_name)) - streams[stream_name] = std::make_unique(storage.disk, storage.files[stream_name].data_file_path, max_read_buffer_size); + { + String file_path = storage.files[stream_name].data_file_path; + streams[stream_name] = std::make_unique( + storage.disk, file_path, max_read_buffer_size, file_sizes[fileName(file_path)]); + } return &streams[stream_name]->compressed; }; if (deserialize_states.count(name) == 0) - type.deserializeBinaryBulkStatePrefix(settings, deserialize_states[name]); + type.deserializeBinaryBulkStatePrefix(settings, deserialize_states[name]); type.deserializeBinaryBulkWithMultipleStreams(column, limit, settings, deserialize_states[name]); } @@ -410,8 +429,6 @@ void StorageTinyLog::rename(const String & new_path_to_table_data, const Storage { assert(table_path != new_path_to_table_data); { - std::unique_lock lock(rwlock); - disk->moveDirectory(table_path, new_path_to_table_data); table_path = new_path_to_table_data; @@ -424,6 +441,16 @@ void StorageTinyLog::rename(const String & new_path_to_table_data, const Storage } +static std::chrono::seconds getLockTimeout(const Context & context) +{ + const Settings & settings = context.getSettingsRef(); + Int64 lock_timeout = settings.lock_acquire_timeout.totalSeconds(); + if (settings.max_execution_time.totalSeconds() != 0 && settings.max_execution_time.totalSeconds() < lock_timeout) + lock_timeout = settings.max_execution_time.totalSeconds(); + return std::chrono::seconds{lock_timeout}; +} + + Pipe StorageTinyLog::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, @@ -437,28 +464,40 @@ Pipe StorageTinyLog::read( // When reading, we lock the entire storage, because we only have one file // per column and can't modify it concurrently. + const Settings & settings = context.getSettingsRef(); + + std::shared_lock lock{rwlock, getLockTimeout(context)}; + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + + /// No need to hold lock while reading because we read fixed range of data that does not change while appending more data. return Pipe(std::make_shared( - max_block_size, Nested::collect(metadata_snapshot->getColumns().getAllPhysical().addTypes(column_names)), *this, context.getSettingsRef().max_read_buffer_size)); + max_block_size, + Nested::collect(metadata_snapshot->getColumns().getAllPhysical().addTypes(column_names)), + *this, + settings.max_read_buffer_size, + file_checker.getFileSizes())); } -BlockOutputStreamPtr StorageTinyLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) +BlockOutputStreamPtr StorageTinyLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & context) { - return std::make_shared(*this, metadata_snapshot); + return std::make_shared(*this, metadata_snapshot, std::unique_lock{rwlock, getLockTimeout(context)}); } -CheckResults StorageTinyLog::checkData(const ASTPtr & /* query */, const Context & /* context */) +CheckResults StorageTinyLog::checkData(const ASTPtr & /* query */, const Context & context) { - std::shared_lock lock(rwlock); + std::shared_lock lock(rwlock, getLockTimeout(context)); + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + return file_checker.check(); } void StorageTinyLog::truncate( const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &) { - std::unique_lock lock(rwlock); - disk->clearDirectory(table_path); files.clear(); @@ -468,14 +507,6 @@ void StorageTinyLog::truncate( addFiles(column.name, *column.type); } -void StorageTinyLog::drop() -{ - std::unique_lock lock(rwlock); - if (disk->exists(table_path)) - disk->removeRecursive(table_path); - files.clear(); -} - void registerStorageTinyLog(StorageFactory & factory) { diff --git a/src/Storages/StorageTinyLog.h b/src/Storages/StorageTinyLog.h index dc6ff101503..95b7d9f2941 100644 --- a/src/Storages/StorageTinyLog.h +++ b/src/Storages/StorageTinyLog.h @@ -43,8 +43,6 @@ public: void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &) override; - void drop() override; - protected: StorageTinyLog( DiskPtr disk_, @@ -70,7 +68,7 @@ private: Files files; FileChecker file_checker; - mutable std::shared_mutex rwlock; + mutable std::shared_timed_mutex rwlock; Poco::Logger * log; diff --git a/tests/queries/0_stateless/01499_log_deadlock.reference b/tests/queries/0_stateless/01499_log_deadlock.reference new file mode 100644 index 00000000000..166be640db5 --- /dev/null +++ b/tests/queries/0_stateless/01499_log_deadlock.reference @@ -0,0 +1,3 @@ +6 +6 +6 diff --git a/tests/queries/0_stateless/01499_log_deadlock.sql b/tests/queries/0_stateless/01499_log_deadlock.sql new file mode 100644 index 00000000000..e98b37f2455 --- /dev/null +++ b/tests/queries/0_stateless/01499_log_deadlock.sql @@ -0,0 +1,26 @@ +DROP TABLE IF EXISTS t; +CREATE TABLE t (x UInt8) ENGINE = TinyLog; + +INSERT INTO t VALUES (1), (2), (3); +INSERT INTO t SELECT * FROM t; +SELECT count() FROM t; + +DROP TABLE t; + + +CREATE TABLE t (x UInt8) ENGINE = Log; + +INSERT INTO t VALUES (1), (2), (3); +INSERT INTO t SELECT * FROM t; +SELECT count() FROM t; + +DROP TABLE t; + + +CREATE TABLE t (x UInt8) ENGINE = StripeLog; + +INSERT INTO t VALUES (1), (2), (3); +INSERT INTO t SELECT * FROM t; +SELECT count() FROM t; + +DROP TABLE t; diff --git a/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.reference b/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.reference new file mode 100644 index 00000000000..4bf85ae79f3 --- /dev/null +++ b/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.reference @@ -0,0 +1,6 @@ +Testing TinyLog +Done TinyLog +Testing StripeLog +Done StripeLog +Testing Log +Done Log diff --git a/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.sh b/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.sh new file mode 100755 index 00000000000..29c5f868617 --- /dev/null +++ b/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.sh @@ -0,0 +1,85 @@ +#!/usr/bin/env bash + +set -e + +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + + +function thread_create { + while true; do + $CLICKHOUSE_CLIENT --query "CREATE TABLE IF NOT EXISTS $1 (x UInt64, s Array(Nullable(String))) ENGINE = $2" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (60|57)' + sleep 0.0$RANDOM + done +} + +function thread_drop { + while true; do + $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS $1" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (60|57)' + sleep 0.0$RANDOM + done +} + +function thread_rename { + while true; do + $CLICKHOUSE_CLIENT --query "RENAME TABLE $1 TO $2" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (60|57)' + sleep 0.0$RANDOM + done +} + +function thread_select { + while true; do + $CLICKHOUSE_CLIENT --query "SELECT * FROM $1 FORMAT Null" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (60|218)' + sleep 0.0$RANDOM + done +} + +function thread_insert { + while true; do + $CLICKHOUSE_CLIENT --query "INSERT INTO $1 SELECT rand64(1), [toString(rand64(2))] FROM numbers($2)" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (60|218)' + sleep 0.0$RANDOM + done +} + +function thread_insert_select { + while true; do + $CLICKHOUSE_CLIENT --query "INSERT INTO $1 SELECT * FROM $2" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (60|218)' + sleep 0.0$RANDOM + done +} + +export -f thread_create +export -f thread_drop +export -f thread_rename +export -f thread_select +export -f thread_insert +export -f thread_insert_select + + +# Do randomized queries and expect nothing extraordinary happens. + +function test_with_engine { + echo "Testing $1" + + timeout 10 bash -c "thread_create t1 $1" & + timeout 10 bash -c "thread_create t2 $1" & + timeout 10 bash -c 'thread_drop t1' & + timeout 10 bash -c 'thread_drop t2' & + timeout 10 bash -c 'thread_rename t1 t2' & + timeout 10 bash -c 'thread_rename t2 t1' & + timeout 10 bash -c 'thread_select t1' & + timeout 10 bash -c 'thread_select t2' & + timeout 10 bash -c 'thread_insert t1 5' & + timeout 10 bash -c 'thread_insert t2 10' & + timeout 10 bash -c 'thread_insert_select t1 t2' & + timeout 10 bash -c 'thread_insert_select t2 t1' & + + wait + echo "Done $1" +} + +test_with_engine TinyLog +test_with_engine StripeLog +test_with_engine Log diff --git a/tests/queries/0_stateless/01505_log_distributed_deadlock.reference b/tests/queries/0_stateless/01505_log_distributed_deadlock.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01505_log_distributed_deadlock.sql b/tests/queries/0_stateless/01505_log_distributed_deadlock.sql new file mode 100644 index 00000000000..2b0b2b97188 --- /dev/null +++ b/tests/queries/0_stateless/01505_log_distributed_deadlock.sql @@ -0,0 +1,12 @@ +DROP TABLE IF EXISTS t_local; +DROP TABLE IF EXISTS t_dist; + +create table t_local(a int) engine Log; +create table t_dist (a int) engine Distributed(test_shard_localhost, currentDatabase(), 't_local', cityHash64(a)); + +set insert_distributed_sync = 1; + +insert into t_dist values (1); + +DROP TABLE t_local; +DROP TABLE t_dist; diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index 69391ca9fd4..6d1c6444d1b 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -145,3 +145,4 @@ 01461_query_start_time_microseconds 01455_shard_leaf_max_rows_bytes_to_read 01505_distributed_local_type_conversion_enum +01505_log_distributed_deadlock From 91e80598cd1ff50cb7af1c32ae4fb47b53a0816d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Oct 2020 04:54:35 +0300 Subject: [PATCH 005/256] Add "harmful" library --- base/glibc-compatibility/musl/lgamma.c | 8 --- base/glibc-compatibility/musl/lgammal.c | 9 --- base/harmful/CMakeLists.txt | 3 + base/harmful/README.md | 1 + base/harmful/harmful.c | 89 +++++++++++++++++++++++++ cmake/linux/default_libs.cmake | 1 + 6 files changed, 94 insertions(+), 17 deletions(-) create mode 100644 base/harmful/CMakeLists.txt create mode 100644 base/harmful/README.md create mode 100644 base/harmful/harmful.c diff --git a/base/glibc-compatibility/musl/lgamma.c b/base/glibc-compatibility/musl/lgamma.c index fbbe1fedc92..fb9d105d0fa 100644 --- a/base/glibc-compatibility/musl/lgamma.c +++ b/base/glibc-compatibility/musl/lgamma.c @@ -258,11 +258,3 @@ double lgamma_r(double x, int *signgamp) r = nadj - r; return r; } - - -int signgam; - -double lgamma(double x) -{ - return lgamma_r(x, &signgam); -} diff --git a/base/glibc-compatibility/musl/lgammal.c b/base/glibc-compatibility/musl/lgammal.c index 3b5d94c5051..a8208a6d9c0 100644 --- a/base/glibc-compatibility/musl/lgammal.c +++ b/base/glibc-compatibility/musl/lgammal.c @@ -328,12 +328,3 @@ long double lgammal_r(long double x, int *sg) return lgamma_r(x, sg); } #endif - - -int signgam_lgammal; - -long double lgammal(long double x) -{ - return lgammal_r(x, &signgam_lgammal); -} - diff --git a/base/harmful/CMakeLists.txt b/base/harmful/CMakeLists.txt new file mode 100644 index 00000000000..f4ae3ba82b9 --- /dev/null +++ b/base/harmful/CMakeLists.txt @@ -0,0 +1,3 @@ +add_library(harmful harmful.c) +target_link_libraries(global-libs INTERFACE harmful) +install(TARGETS harmful EXPORT global ARCHIVE DESTINATION lib) diff --git a/base/harmful/README.md b/base/harmful/README.md new file mode 100644 index 00000000000..7e56c1f62e4 --- /dev/null +++ b/base/harmful/README.md @@ -0,0 +1 @@ +A library that traps whenever harmful functions from libc are called. diff --git a/base/harmful/harmful.c b/base/harmful/harmful.c new file mode 100644 index 00000000000..893d451e0c6 --- /dev/null +++ b/base/harmful/harmful.c @@ -0,0 +1,89 @@ +#pragma GCC diagnostic ignored "-Wincompatible-library-redeclaration" + +#define TRAP(func) void func() { __builtin_trap(); } + +/// Trap all non thread-safe functions: +/// nm -D /lib/x86_64-linux-gnu/{libc.so.6,libdl.so.2,libm.so.6,libpthread.so.0,librt.so.1,libnss_dns.so.2,libresolv.so.2} | grep -P '_r@?$' | awk '{ print $3 }' | sed -r -e 's/_r//' | grep -vP '^_' + +TRAP(asctime) +TRAP(ctime) +TRAP(drand48) +TRAP(ecvt) +TRAP(erand48) +TRAP(ether_aton) +TRAP(ether_ntoa) +TRAP(fcvt) +TRAP(fgetgrent) +TRAP(fgetpwent) +TRAP(fgetsgent) +TRAP(fgetspent) +TRAP(getaliasbyname) +TRAP(getaliasent) +TRAP(getdate) +TRAP(getgrent) +TRAP(getgrgid) +TRAP(getgrnam) +TRAP(gethostbyaddr) +TRAP(gethostbyname2) +TRAP(gethostbyname) +TRAP(gethostent) +TRAP(getlogin) +TRAP(getmntent) +TRAP(getnetbyaddr) +TRAP(getnetbyname) +TRAP(getnetent) +TRAP(getnetgrent) +TRAP(getprotobyname) +TRAP(getprotobynumber) +TRAP(getprotoent) +TRAP(getpwent) +TRAP(getpwnam) +TRAP(getpwuid) +TRAP(getrpcbyname) +TRAP(getrpcbynumber) +TRAP(getrpcent) +TRAP(getservbyname) +TRAP(getservbyport) +TRAP(getservent) +TRAP(getsgent) +TRAP(getsgnam) +TRAP(getspent) +TRAP(getspnam) +TRAP(getutent) +TRAP(getutid) +TRAP(getutline) +TRAP(gmtime) +TRAP(hcreate) +TRAP(hdestroy) +TRAP(hsearch) +TRAP(initstate) +TRAP(jrand48) +TRAP(lcong48) +TRAP(localtime) +TRAP(lrand48) +TRAP(mrand48) +TRAP(nrand48) +TRAP(ptsname) +TRAP(qecvt) +TRAP(qfcvt) +TRAP(random) +TRAP(rand) +TRAP(seed48) +TRAP(setstate) +TRAP(sgetsgent) +TRAP(sgetspent) +TRAP(srand48) +TRAP(srandom) +TRAP(strerror) +TRAP(strtok) +TRAP(tmpnam) +TRAP(ttyname) +TRAP(twalk) +TRAP(lgammaf128) +TRAP(lgammaf32) +TRAP(lgammaf32x) +TRAP(lgammaf64) +TRAP(lgammaf64x) +TRAP(lgammaf) +TRAP(lgammal) +TRAP(lgamma) diff --git a/cmake/linux/default_libs.cmake b/cmake/linux/default_libs.cmake index 2a325b4d095..c4c441634f4 100644 --- a/cmake/linux/default_libs.cmake +++ b/cmake/linux/default_libs.cmake @@ -43,6 +43,7 @@ find_package(Threads REQUIRED) if (NOT OS_ANDROID) # Our compatibility layer doesn't build under Android, many errors in musl. add_subdirectory(base/glibc-compatibility) + add_subdirectory(base/harmful) endif () include (cmake/find/unwind.cmake) From 53ef792fe507cec3caa575424a79ee75770ef933 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Oct 2020 06:12:04 +0300 Subject: [PATCH 006/256] Fix gcc build --- base/harmful/harmful.c | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/base/harmful/harmful.c b/base/harmful/harmful.c index 893d451e0c6..cd53da9e9ac 100644 --- a/base/harmful/harmful.c +++ b/base/harmful/harmful.c @@ -1,4 +1,8 @@ -#pragma GCC diagnostic ignored "-Wincompatible-library-redeclaration" +#if defined(__clang__) + #pragma clang diagnostic ignored "-Wincompatible-library-redeclaration" +#else + #pragma GCC diagnostic ignored "-Wbuiltin-declaration-mismatch" +#endif #define TRAP(func) void func() { __builtin_trap(); } From 349571a4582d3358fc8dcb623dbc818e28db1dac Mon Sep 17 00:00:00 2001 From: Ken Chen Date: Fri, 27 Nov 2020 03:25:19 +0000 Subject: [PATCH 007/256] Add asterisk_include_materialized_columns and asterisk_include_materialized_columns switches to include materialized columns and alias columns respectively for wildcard query --- src/Core/Settings.h | 2 ++ src/Interpreters/DatabaseAndTableWithAlias.h | 23 +++++++++++++++++-- .../TranslateQualifiedNamesVisitor.cpp | 10 +++++--- src/Interpreters/getTableExpressions.cpp | 12 ++++++++++ 4 files changed, 42 insertions(+), 5 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6836a597047..9f7e00701ed 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -397,6 +397,8 @@ class IColumn; M(Bool, allow_non_metadata_alters, true, "Allow to execute alters which affects not only tables metadata, but also data on disk", 0) \ M(Bool, enable_global_with_statement, false, "Propagate WITH statements to UNION queries and all subqueries", 0) \ M(Bool, aggregate_functions_null_for_empty, false, "Rewrite all aggregate functions in a query, adding -OrNull suffix to them", 0) \ + M(Bool, asterisk_include_materialized_columns, false, "Include MATERIALIZED columns for wildcard query", 0) \ + M(Bool, asterisk_include_alias_columns, false, "Include ALIAS columns for wildcard query", 0) \ \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ diff --git a/src/Interpreters/DatabaseAndTableWithAlias.h b/src/Interpreters/DatabaseAndTableWithAlias.h index 07a41c12983..b889509c264 100644 --- a/src/Interpreters/DatabaseAndTableWithAlias.h +++ b/src/Interpreters/DatabaseAndTableWithAlias.h @@ -49,7 +49,9 @@ struct TableWithColumnNamesAndTypes { DatabaseAndTableWithAlias table; NamesAndTypesList columns; - NamesAndTypesList hidden_columns; /// Not general columns like MATERIALIZED and ALIAS. They are omitted in * and t.* results. + NamesAndTypesList hidden_columns; /// Not general columns like MATERIALIZED, ALIAS, VIRTUAL. They are omitted in * and t.* results by default. + NamesAndTypesList alias_columns; + NamesAndTypesList materialized_columns; TableWithColumnNamesAndTypes(const DatabaseAndTableWithAlias & table_, const NamesAndTypesList & columns_) : table(table_) @@ -63,11 +65,28 @@ struct TableWithColumnNamesAndTypes void addHiddenColumns(const NamesAndTypesList & addition) { - hidden_columns.insert(hidden_columns.end(), addition.begin(), addition.end()); + addAdditionalColumns(hidden_columns, addition); + } + + void addAliasColumns(const NamesAndTypesList & addition) + { + addAdditionalColumns(alias_columns, addition); + } + + void addMaterializedColumns(const NamesAndTypesList & addition) + { + addAdditionalColumns(alias_columns, addition); + } + +private: + void addAdditionalColumns(NamesAndTypesList & target, const NamesAndTypesList & addition) + { + target.insert(target.end(), addition.begin(), addition.end()); for (auto & col : addition) names.insert(col.name); } + private: NameSet names; }; diff --git a/src/Interpreters/TranslateQualifiedNamesVisitor.cpp b/src/Interpreters/TranslateQualifiedNamesVisitor.cpp index 98ed2166c40..53a5df29ce0 100644 --- a/src/Interpreters/TranslateQualifiedNamesVisitor.cpp +++ b/src/Interpreters/TranslateQualifiedNamesVisitor.cpp @@ -222,11 +222,15 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt bool first_table = true; for (const auto & table : tables_with_columns) { - for (const auto & column : table.columns) + auto all_columns = {&table.columns, &table.alias_columns, &table.materialized_columns}; + for (const auto cols: all_columns) { - if (first_table || !data.join_using_columns.count(column.name)) + for (const auto & column : *cols) { - addIdentifier(columns, table.table, column.name); + if (first_table || !data.join_using_columns.count(column.name)) + { + addIdentifier(columns, table.table, column.name); + } } } diff --git a/src/Interpreters/getTableExpressions.cpp b/src/Interpreters/getTableExpressions.cpp index 56ca614dc2d..9234aaa831a 100644 --- a/src/Interpreters/getTableExpressions.cpp +++ b/src/Interpreters/getTableExpressions.cpp @@ -124,6 +124,8 @@ TablesWithColumns getDatabaseAndTablesWithColumns(const std::vector Date: Fri, 4 Dec 2020 11:42:58 +0800 Subject: [PATCH 008/256] Add metrics for part number in MergeTree --- src/Common/CurrentMetrics.cpp | 3 ++ src/Storages/MergeTree/MergeTreeData.cpp | 50 ++++++++++++++++++++++++ 2 files changed, 53 insertions(+) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index c48e76e1d98..14fdf74b5e0 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -5,6 +5,9 @@ #define APPLY_FOR_METRICS(M) \ M(Query, "Number of executing queries") \ M(Merge, "Number of executing background merges") \ + M(Parts, "Total number of data parts") \ + M(PartsActive, "Number of active data parts") \ + M(PartsInactive, "Number of inactive data parts") \ M(PartMutation, "Number of mutations (ALTER DELETE/UPDATE)") \ M(ReplicatedFetch, "Number of data parts being fetched from replica") \ M(ReplicatedSend, "Number of data parts being sent to replicas") \ diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b213bb7b6f9..af3e323763e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -72,6 +72,9 @@ namespace CurrentMetrics { extern const Metric DelayedInserts; extern const Metric BackgroundMovePoolTask; + extern const Metric Parts; + extern const Metric PartsActive; + extern const Metric PartsInactive; } @@ -877,6 +880,9 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) std::lock_guard loading_lock(mutex); if (!data_parts_indexes.insert(part).second) throw Exception("Part " + part->name + " already exists", ErrorCodes::DUPLICATE_DATA_PART); + + CurrentMetrics::add(CurrentMetrics::Parts); + CurrentMetrics::add(CurrentMetrics::PartsActive); }); } @@ -893,6 +899,9 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) if (!data_parts_indexes.insert(part).second) throw Exception("Part " + part->name + " already exists", ErrorCodes::DUPLICATE_DATA_PART); + + CurrentMetrics::add(CurrentMetrics::Parts); + CurrentMetrics::add(CurrentMetrics::PartsActive); } if (has_non_adaptive_parts && has_adaptive_parts && !settings->enable_mixed_granularity_parts) @@ -924,6 +933,9 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) { (*it)->remove_time.store((*it)->modification_time, std::memory_order_relaxed); modifyPartState(it, DataPartState::Outdated); + + CurrentMetrics::sub(CurrentMetrics::PartsActive); + CurrentMetrics::add(CurrentMetrics::PartsInactive); }; (*prev_jt)->assertState({DataPartState::Committed}); @@ -1091,6 +1103,9 @@ void MergeTreeData::removePartsFinally(const MergeTreeData::DataPartsVector & pa (*it)->assertState({DataPartState::Deleting}); data_parts_indexes.erase(it); + + CurrentMetrics::sub(CurrentMetrics::Parts); + CurrentMetrics::sub(CurrentMetrics::PartsInactive); } } @@ -1265,10 +1280,15 @@ void MergeTreeData::dropAllData() LOG_TRACE(log, "dropAllData: removing data from memory."); DataPartsVector all_parts(data_parts_by_info.begin(), data_parts_by_info.end()); + DataPartsVector committed_parts = getDataPartsVector({DataPartState::Committed}); data_parts_indexes.clear(); column_sizes.clear(); + CurrentMetrics::sub(CurrentMetrics::Parts, all_parts.size()); + CurrentMetrics::sub(CurrentMetrics::PartsActive, committed_parts.size()); + CurrentMetrics::sub(CurrentMetrics::PartsInactive, all_parts.size() - committed_parts.size()); + /// Tables in atomic databases have UUID and stored in persistent locations. /// No need to drop caches (that are keyed by filesystem path) because collision is not possible. if (!getStorageID().hasUUID()) @@ -1981,6 +2001,9 @@ bool MergeTreeData::renameTempPartAndReplace( auto part_it = data_parts_indexes.insert(part).first; + CurrentMetrics::add(CurrentMetrics::Parts); + CurrentMetrics::add(CurrentMetrics::PartsInactive); + if (out_transaction) { out_transaction->precommitted_parts.insert(part); @@ -1993,10 +2016,16 @@ bool MergeTreeData::renameTempPartAndReplace( covered_part->remove_time.store(current_time, std::memory_order_relaxed); modifyPartState(covered_part, DataPartState::Outdated); removePartContributionToColumnSizes(covered_part); + + CurrentMetrics::sub(CurrentMetrics::PartsActive); + CurrentMetrics::add(CurrentMetrics::PartsInactive); } modifyPartState(part_it, DataPartState::Committed); addPartContributionToColumnSizes(part); + + CurrentMetrics::add(CurrentMetrics::PartsActive); + CurrentMetrics::sub(CurrentMetrics::PartsInactive); } auto part_in_memory = asInMemoryPart(part); @@ -2037,7 +2066,11 @@ void MergeTreeData::removePartsFromWorkingSet(const MergeTreeData::DataPartsVect for (const DataPartPtr & part : remove) { if (part->state == IMergeTreeDataPart::State::Committed) + { removePartContributionToColumnSizes(part); + CurrentMetrics::sub(CurrentMetrics::PartsActive); + CurrentMetrics::add(CurrentMetrics::PartsInactive); + } if (part->state == IMergeTreeDataPart::State::Committed || clear_without_timeout) part->remove_time.store(remove_time, std::memory_order_relaxed); @@ -2063,6 +2096,9 @@ void MergeTreeData::removePartsFromWorkingSetImmediatelyAndSetTemporaryState(con modifyPartState(part, IMergeTreeDataPart::State::Temporary); /// Erase immediately data_parts_indexes.erase(it_part); + + CurrentMetrics::sub(CurrentMetrics::Parts); + CurrentMetrics::sub(CurrentMetrics::PartsInactive); } } @@ -2150,13 +2186,21 @@ restore_covered) DataPartPtr part = *it_part; if (part->state == DataPartState::Committed) + { removePartContributionToColumnSizes(part); + CurrentMetrics::sub(CurrentMetrics::PartsActive); + CurrentMetrics::add(CurrentMetrics::PartsInactive); + } + modifyPartState(it_part, DataPartState::Deleting); part->renameToDetached(prefix); data_parts_indexes.erase(it_part); + CurrentMetrics::sub(CurrentMetrics::Parts); + CurrentMetrics::sub(CurrentMetrics::PartsInactive); + if (restore_covered && part->info.level == 0) { LOG_WARNING(log, "Will not recover parts covered by zero-level part {}", part->name); @@ -3385,10 +3429,16 @@ MergeTreeData::DataPartsVector MergeTreeData::Transaction::commit(MergeTreeData: covered_part->remove_time.store(current_time, std::memory_order_relaxed); data.modifyPartState(covered_part, DataPartState::Outdated); data.removePartContributionToColumnSizes(covered_part); + + CurrentMetrics::sub(CurrentMetrics::PartsActive); + CurrentMetrics::add(CurrentMetrics::PartsInactive); } data.modifyPartState(part, DataPartState::Committed); data.addPartContributionToColumnSizes(part); + + CurrentMetrics::add(CurrentMetrics::PartsActive); + CurrentMetrics::sub(CurrentMetrics::PartsInactive); } } } From 5914e6e451700d5e9b0d17ae57406f920cf2ea87 Mon Sep 17 00:00:00 2001 From: spongedc Date: Mon, 7 Dec 2020 16:04:29 +0800 Subject: [PATCH 009/256] Refine documents for system log table, and fix a link error in config.xml --- docs/en/operations/system-tables/index.md | 28 ++++++++++++++++++++++- docs/zh/operations/system-tables/index.md | 28 ++++++++++++++++++++++- programs/server/config.xml | 2 +- 3 files changed, 55 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/system-tables/index.md b/docs/en/operations/system-tables/index.md index 7a9e386d419..e660b418a76 100644 --- a/docs/en/operations/system-tables/index.md +++ b/docs/en/operations/system-tables/index.md @@ -20,7 +20,33 @@ System tables: Most of system tables store their data in RAM. A ClickHouse server creates such system tables at the start. -Unlike other system tables, the system tables [metric_log](../../operations/system-tables/metric_log.md#system_tables-metric_log), [query_log](../../operations/system-tables/query_log.md#system_tables-query_log), [query_thread_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log), [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) are served by [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table engine and store their data in a storage filesystem. If you remove a table from a filesystem, the ClickHouse server creates the empty one again at the time of the next data writing. If system table schema changed in a new release, then ClickHouse renames the current table and creates a new one. +Unlike other system tables, the system log tables [metric_log](../../operations/system-tables/metric_log.md#system_tables-metric_log), [query_log](../../operations/system-tables/query_log.md#system_tables-query_log), [query_thread_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log), [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log), [part_log](../../operations/system-tables/part_log.md#system.part_log), [crash_log](../../operations/system-tables/crash_log.md#system.crash_log) and [text_log](../../operations/system-tables/crash_log.md#system.text_log) are served by [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table engine and store their data in a storage filesystem by default. If you remove a table from a filesystem, the ClickHouse server creates the empty one again at the time of the next data writing. If system table schema changed in a new release, then ClickHouse renames the current table and creates a new one. + +System log tables can be customized by creating a config file with the same name as the table under `/etc/clickhouse-server/config.d/`, or setting corresponding elements in `/etc/clickhouse-server/config.xml`. Elements can be customized are: + +- `database`: database the system log table belongs to. This option is deprecated now. All system log tables are under database `system`. +- `table`: table to insert data. +- `partition_by`: specify [PARTITION BY](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) expression. +- `ttl`: specify table [TTL](../../sql-reference/statements/alter/ttl.md#manipulations-with-table-ttl) expression. +- `flush_interval_milliseconds`: interval of flushing data to disk. +- `engine`: provide full engine expression (starting with `ENGINE =` ) with parameters. This option is contradict with `partition_by` and `ttl`. If set together, the server would raise an exception and exit. + +An example: + +``` + + + system + query_log
+ toYYYYMM(event_date) + event_date + INTERVAL 30 DAY DELETE + + 7500 +
+
+``` By default, table growth is unlimited. To control a size of a table, you can use [TTL](../../sql-reference/statements/alter/ttl.md#manipulations-with-table-ttl) settings for removing outdated log records. Also you can use the partitioning feature of `MergeTree`-engine tables. diff --git a/docs/zh/operations/system-tables/index.md b/docs/zh/operations/system-tables/index.md index fcf6741761b..2dc1e2712e5 100644 --- a/docs/zh/operations/system-tables/index.md +++ b/docs/zh/operations/system-tables/index.md @@ -22,7 +22,33 @@ toc_title: "\u7CFB\u7EDF\u8868" 大多数系统表将数据存储在RAM中。 ClickHouse服务器在开始时创建此类系统表。 -与其他系统表不同,系统表 [metric_log](../../operations/system-tables/metric_log.md#system_tables-metric_log), [query_log](../../operations/system-tables/query_log.md#system_tables-query_log), [query_thread_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log), [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) 由 [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) 表引擎并将其数据存储在存储文件系统中。 如果从文件系统中删除表,ClickHouse服务器会在下一次写入数据时再次创建空表。 如果系统表架构在新版本中发生更改,则ClickHouse会重命名当前表并创建一个新表。 +与其他系统表不同,系统日志表 [metric_log](../../operations/system-tables/metric_log.md#system_tables-metric_log), [query_log](../../operations/system-tables/query_log.md#system_tables-query_log), [query_thread_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log), [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log), [part_log](../../operations/system-tables/part_log.md#system.part_log), [crash_log](../../operations/system-tables/crash_log.md#system.crash_log) and [text_log](../../operations/system-tables/crash_log.md#system.text_log) 默认采用[MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) 引擎并将其数据存储在存储文件系统中。 如果从文件系统中删除表,ClickHouse服务器会在下一次写入数据时再次创建空表。 如果系统表架构在新版本中发生更改,则ClickHouse会重命名当前表并创建一个新表。 + +用户可以通过在`/etc/clickhouse-server/config.d/`下创建与系统表同名的配置文件, 或者在`/etc/clickhouse-server/config.xml`中设置相应配置项,来自定义系统日志表的结构。可以自定义的配置项如下: + +- `database`: 系统日志表所在的数据库。这个选项目前已经废弃。所有的系统日表都位于`system`库中。 +- `table`: 系统日志表名。 +- `partition_by`: 指定[PARTITION BY](../../engines/table-engines/mergetree-family/custom-partitioning-key.md)表达式。 +- `ttl`: 指定系统日志表[TTL](../../sql-reference/statements/alter/ttl.md#manipulations-with-table-ttl)选项。 +- `flush_interval_milliseconds`: 指定系统日志表数据落盘时间。 +- `engine`: 指定完整的表引擎定义。(以`ENGINE = `开始)。 这个选项与`partition_by`以及`ttl`冲突。如果两者一起设置,服务启动时会抛出异常并且退出。 + +一个配置定义的例子如下: + +``` + + + system + query_log
+ toYYYYMM(event_date) + event_date + INTERVAL 30 DAY DELETE + + 7500 +
+
+``` 默认情况下,表增长是无限的。 要控制表的大小,可以使用 [TTL](../../sql-reference/statements/alter.md#manipulations-with-table-ttl) 删除过期日志记录的设置。 你也可以使用分区功能 `MergeTree`-发动机表。 diff --git a/programs/server/config.xml b/programs/server/config.xml index dde3702a44b..e062c8d7b80 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -581,7 +581,7 @@ system query_log
From e476dcdba78a1f24dd4522b6ef042bbe4756b1b8 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 4 Jan 2021 20:58:26 +0300 Subject: [PATCH 084/256] Update replication.md --- .../table-engines/mergetree-family/replication.md | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/replication.md b/docs/en/engines/table-engines/mergetree-family/replication.md index 04a239e0b06..ef34c8d3804 100644 --- a/docs/en/engines/table-engines/mergetree-family/replication.md +++ b/docs/en/engines/table-engines/mergetree-family/replication.md @@ -38,15 +38,15 @@ Example of setting the addresses of the ZooKeeper cluster: ``` xml - + example1 2181 - + example2 2181 - + example3 2181 @@ -61,21 +61,21 @@ Example of setting the addresses of the auxiliary ZooKeeper cluster: ``` xml - + example_2_1 2181 - + example_2_2 2181 - + example_2_3 2181 - + example_3_1 2181 From 063360511ac09e3eb40474daa59026f940f90f73 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 4 Jan 2021 21:01:17 +0300 Subject: [PATCH 085/256] Remove obsolete "incl" from /etc/metrika.xml; correct links in config --- programs/server/config.xml | 36 ++++++++++++++++++++++++++++-------- programs/server/embedded.xml | 2 +- programs/server/users.xml | 2 +- 3 files changed, 30 insertions(+), 10 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index 2bdcbd47ccf..a4c13be493e 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -204,7 +204,7 @@ /etc/clickhouse-server/server.key /etc/clickhouse-server/dhparam.pem @@ -432,7 +432,7 @@ - + - + - + @@ -810,8 +830,8 @@ - + --> diff --git a/programs/server/embedded.xml b/programs/server/embedded.xml index 1ac568682f9..a66f57d1eb7 100644 --- a/programs/server/embedded.xml +++ b/programs/server/embedded.xml @@ -20,7 +20,7 @@ - + ::/0 diff --git a/programs/server/users.xml b/programs/server/users.xml index 838b46e6938..3223d855651 100644 --- a/programs/server/users.xml +++ b/programs/server/users.xml @@ -77,7 +77,7 @@ Strongly recommended that regexp is ends with $ All results of DNS requests are cached till server restart. --> - + ::/0 From 12a0ef907ece6cbda587209ace70c58fee9b0b1a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 4 Jan 2021 21:07:52 +0300 Subject: [PATCH 086/256] Slightly better --- src/IO/WriteHelpers.h | 4 ++-- src/IO/WriteIntText.h | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 58fdaeab6f0..624a6c3496a 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -730,7 +730,7 @@ static const char digits100[201] = template inline void writeDateText(const LocalDate & date, WriteBuffer & buf) { - if (buf.position() && buf.position() + 10 <= buf.buffer().end()) + if (reinterpret_cast(buf.position()) + 10 <= reinterpret_cast(buf.buffer().end())) { memcpy(buf.position(), &digits100[date.year() / 100 * 2], 2); buf.position() += 2; @@ -767,7 +767,7 @@ inline void writeDateText(DayNum date, WriteBuffer & buf) template inline void writeDateTimeText(const LocalDateTime & datetime, WriteBuffer & buf) { - if (buf.position() && buf.position() + 19 <= buf.buffer().end()) + if (reinterpret_cast(buf.position()) + 19 <= reinterpret_cast(buf.buffer().end())) { memcpy(buf.position(), &digits100[datetime.year() / 100 * 2], 2); buf.position() += 2; diff --git a/src/IO/WriteIntText.h b/src/IO/WriteIntText.h index 2dca2af8c40..93444e7df73 100644 --- a/src/IO/WriteIntText.h +++ b/src/IO/WriteIntText.h @@ -24,7 +24,7 @@ namespace detail template void writeIntText(T x, WriteBuffer & buf) { - if (likely(buf.position() && buf.position() + WRITE_HELPERS_MAX_INT_WIDTH < buf.buffer().end())) + if (likely(reinterpret_cast(buf.position()) + WRITE_HELPERS_MAX_INT_WIDTH < reinterpret_cast(buf.buffer().end()))) buf.position() = itoa(x, buf.position()); else detail::writeUIntTextFallback(x, buf); From fc9c440d3ecfe3cb4489b7dcf6269995ff7f3374 Mon Sep 17 00:00:00 2001 From: fastio Date: Tue, 5 Jan 2021 04:42:57 +0800 Subject: [PATCH 087/256] Fix the unexpected behaviour of show tables when antlr parser enabled (#18431) * Fix the unexpected behaviour of show tables when antlr parser enabled * fix compile error --- src/Interpreters/executeQuery.cpp | 2 +- src/Parsers/New/ParseTreeVisitor.cpp | 15 ++++++++------- src/Parsers/New/ParseTreeVisitor.h | 2 ++ src/Parsers/New/parseQuery.cpp | 9 ++++----- src/Parsers/New/parseQuery.h | 7 +++---- utils/syntax-analyzer/main.cpp | 2 +- 6 files changed, 19 insertions(+), 18 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index ae35e009bce..5928da156f3 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -351,7 +351,7 @@ static std::tuple executeQueryImpl( #if !defined(ARCADIA_BUILD) if (settings.use_antlr_parser) { - ast = parseQuery(begin, end, max_query_size, settings.max_parser_depth); + ast = parseQuery(begin, end, max_query_size, settings.max_parser_depth, context.getCurrentDatabase()); } else { diff --git a/src/Parsers/New/ParseTreeVisitor.cpp b/src/Parsers/New/ParseTreeVisitor.cpp index 2179e44d78b..a7c7a2758eb 100644 --- a/src/Parsers/New/ParseTreeVisitor.cpp +++ b/src/Parsers/New/ParseTreeVisitor.cpp @@ -34,7 +34,6 @@ // Include last, because antlr-runtime undefines EOF macros, which is required in boost multiprecision numbers. #include - namespace DB { @@ -114,15 +113,17 @@ antlrcpp::Any ParseTreeVisitor::visitShowTablesStmt(ClickHouseParser::ShowTables auto and_args = PtrTo(new ColumnExprList{ColumnExpr::createLiteral(Literal::createNumber("1"))}); + auto current_database = ColumnExpr::createLiteral(Literal::createString(current_database_name)); if (ctx->databaseIdentifier()) { - auto database = std::make_shared(nullptr, std::make_shared("database")); - auto args = PtrTo(new ColumnExprList{ - ColumnExpr::createIdentifier(database), - ColumnExpr::createLiteral(Literal::createString(visit(ctx->databaseIdentifier()).as>()->getName())) - }); - and_args->push(ColumnExpr::createFunction(std::make_shared("equals"), nullptr, args)); + current_database = ColumnExpr::createLiteral(Literal::createString(visit(ctx->databaseIdentifier()).as>()->getName())); } + auto database = std::make_shared(nullptr, std::make_shared("database")); + auto equals_args = PtrTo(new ColumnExprList{ + ColumnExpr::createIdentifier(database), + current_database + }); + and_args->push(ColumnExpr::createFunction(std::make_shared("equals"), nullptr, equals_args)); if (ctx->LIKE()) { diff --git a/src/Parsers/New/ParseTreeVisitor.h b/src/Parsers/New/ParseTreeVisitor.h index 4f6d93f0327..8301cb5c5b8 100644 --- a/src/Parsers/New/ParseTreeVisitor.h +++ b/src/Parsers/New/ParseTreeVisitor.h @@ -7,7 +7,9 @@ namespace DB { class ParseTreeVisitor : public ClickHouseParserVisitor { + const String & current_database_name; public: + explicit ParseTreeVisitor(const String & database_name) : ClickHouseParserVisitor(), current_database_name(database_name) {} virtual ~ParseTreeVisitor() override = default; // Top-level statements diff --git a/src/Parsers/New/parseQuery.cpp b/src/Parsers/New/parseQuery.cpp index af334717392..c66772385ca 100644 --- a/src/Parsers/New/parseQuery.cpp +++ b/src/Parsers/New/parseQuery.cpp @@ -12,7 +12,6 @@ #include - namespace DB { @@ -20,7 +19,7 @@ using namespace antlr4; using namespace AST; // For testing only -PtrTo parseQuery(const String & query) +PtrTo parseQuery(const String & query, const String & current_database) { ANTLRInputStream input(query); ClickHouseLexer lexer(&input); @@ -34,12 +33,12 @@ PtrTo parseQuery(const String & query) lexer.addErrorListener(&lexer_error_listener); parser.addErrorListener(&parser_error_listener); - ParseTreeVisitor visitor; + ParseTreeVisitor visitor { current_database }; return visitor.visit(parser.queryStmt()); } -ASTPtr parseQuery(const char * begin, const char * end, size_t, size_t) +ASTPtr parseQuery(const char * begin, const char * end, size_t, size_t, const String & current_database) { // TODO: do not ignore |max_parser_depth|. @@ -60,7 +59,7 @@ ASTPtr parseQuery(const char * begin, const char * end, size_t, size_t) lexer.addErrorListener(&lexer_error_listener); parser.addErrorListener(&parser_error_listener); - ParseTreeVisitor visitor; + ParseTreeVisitor visitor { current_database }; PtrTo new_ast = visitor.visit(parser.queryStmt()); auto old_ast = new_ast->convertToOld(); diff --git a/src/Parsers/New/parseQuery.h b/src/Parsers/New/parseQuery.h index ae86dc32eda..8d9c8efd337 100644 --- a/src/Parsers/New/parseQuery.h +++ b/src/Parsers/New/parseQuery.h @@ -2,14 +2,13 @@ #include #include - +#include namespace DB { // Compatibility interface - -AST::PtrTo parseQuery(const std::string & query); -ASTPtr parseQuery(const char * begin, const char * end, size_t max_query_size, size_t max_parser_depth); +AST::PtrTo parseQuery(const std::string & query, const String & current_database); +ASTPtr parseQuery(const char * begin, const char * end, size_t max_query_size, size_t max_parser_depth, const String & current_database); } diff --git a/utils/syntax-analyzer/main.cpp b/utils/syntax-analyzer/main.cpp index 7f1d2c979ec..cf264160407 100644 --- a/utils/syntax-analyzer/main.cpp +++ b/utils/syntax-analyzer/main.cpp @@ -36,7 +36,7 @@ int main(int argc, const char **) ASTPtr orig_ast = parseQuery(parser, q, 10000000, 10000); std::cout << std::endl << "New AST:" << std::endl; - auto new_ast = parseQuery(q); + auto new_ast = parseQuery(q, ""); new_ast->dump(); auto old_ast = new_ast->convertToOld(); From 858c2a474aacb16bd7513b14df651563477b420d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 5 Jan 2021 00:28:57 +0300 Subject: [PATCH 088/256] Do not throw from Parser --- src/Parsers/ParserCreateQuery.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 8e6040169c9..552ea9df320 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -228,7 +228,8 @@ bool ParserTablePropertiesDeclarationList::parseImpl(Pos & pos, ASTPtr & node, E { if (primary_key) { - throw Exception("Multiple primary keys are not allowed.", ErrorCodes::BAD_ARGUMENTS); + /// Multiple primary keys are not allowed. + return false; } primary_key = elem; } From a15eb69f049fe0ab78f21236468fc64248a1afa1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 5 Jan 2021 00:56:00 +0300 Subject: [PATCH 089/256] Remove pink screen with confusing questions about Kerberos --- debian/control | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/debian/control b/debian/control index 809b5bb5d2a..9b34e982698 100644 --- a/debian/control +++ b/debian/control @@ -40,7 +40,7 @@ Description: Common files for ClickHouse Package: clickhouse-server Architecture: all Depends: ${shlibs:Depends}, ${misc:Depends}, clickhouse-common-static (= ${binary:Version}), adduser -Recommends: libcap2-bin, krb5-user +Recommends: libcap2-bin Replaces: clickhouse-server-common, clickhouse-server-base Provides: clickhouse-server-common Description: Server binary for ClickHouse From 039edbb72434a1f9ba93936a8337457f8e1699de Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 5 Jan 2021 01:31:34 +0300 Subject: [PATCH 090/256] Don't allow conversion between UUID and numeric types --- src/Functions/FunctionsConversion.h | 4 ++ .../queries/0_stateless/00735_conditional.sql | 54 ------------------- .../0_stateless/01634_uuid_fuzz.reference | 0 tests/queries/0_stateless/01634_uuid_fuzz.sql | 1 + 4 files changed, 5 insertions(+), 54 deletions(-) create mode 100644 tests/queries/0_stateless/01634_uuid_fuzz.reference create mode 100644 tests/queries/0_stateless/01634_uuid_fuzz.sql diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 6669a369a9d..91a67799115 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -192,6 +192,10 @@ struct ConvertImpl else throw Exception("Unexpected UInt128 to big int conversion", ErrorCodes::NOT_IMPLEMENTED); } + else if constexpr (std::is_same_v != std::is_same_v) + { + throw Exception("Conversion between numeric types and UUID is not supported", ErrorCodes::NOT_IMPLEMENTED); + } else { if constexpr (IsDataTypeDecimal || IsDataTypeDecimal) diff --git a/tests/queries/0_stateless/00735_conditional.sql b/tests/queries/0_stateless/00735_conditional.sql index 0c272ac69f8..04439f4062e 100644 --- a/tests/queries/0_stateless/00735_conditional.sql +++ b/tests/queries/0_stateless/00735_conditional.sql @@ -14,7 +14,6 @@ SELECT toInt8(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), SELECT toInt8(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toInt8(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT toInt8(0) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toInt8(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toInt8(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toInt8(0) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toInt8(0) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); @@ -31,7 +30,6 @@ SELECT toInt16(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x) SELECT toInt16(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toInt16(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT toInt16(0) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toInt16(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toInt16(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toInt16(0) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toInt16(0) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); @@ -48,7 +46,6 @@ SELECT toInt32(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x) SELECT toInt32(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toInt32(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT toInt32(0) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toInt32(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toInt32(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toInt32(0) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toInt32(0) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); @@ -65,7 +62,6 @@ SELECT toInt64(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x) SELECT toInt64(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toInt64(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT toInt64(0) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toInt64(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toInt64(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toInt64(0) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toInt64(0) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); @@ -82,7 +78,6 @@ SELECT toUInt8(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x) SELECT toUInt8(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toUInt8(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT toUInt8(0) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toUInt8(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUInt8(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toUInt8(0) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toUInt8(0) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); @@ -99,7 +94,6 @@ SELECT toUInt16(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x SELECT toUInt16(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toUInt16(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT toUInt16(0) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toUInt16(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUInt16(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toUInt16(0) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toUInt16(0) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); @@ -116,7 +110,6 @@ SELECT toUInt32(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x SELECT toUInt32(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toUInt32(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT toUInt32(0) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toUInt32(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUInt32(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toUInt32(0) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toUInt32(0) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); @@ -133,7 +126,6 @@ SELECT toUInt64(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x SELECT toUInt64(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUInt64(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT toUInt64(0) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toUInt64(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUInt64(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toUInt64(0) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toUInt64(0) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); @@ -150,7 +142,6 @@ SELECT toDate(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), SELECT toDate(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT toDate(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toDate('2000-01-01') AS x, toDateTime('2000-01-01 00:00:01', 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -SELECT toDate(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT toDate(0) AS x, toDecimal32(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT toDate(0) AS x, toDecimal64(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT toDate(0) AS x, toDecimal128(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } @@ -167,28 +158,10 @@ SELECT toDateTime(0, 'Europe/Moscow') AS x, toFloat32(1) AS y, ((x > y) ? x : y) SELECT toDateTime(0, 'Europe/Moscow') AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toDateTime('2000-01-01 00:00:00', 'Europe/Moscow') AS x, toDate('2000-01-02') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toDateTime(0, 'Europe/Moscow') AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -SELECT toDateTime(0, 'Europe/Moscow') AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toDateTime(0, 'Europe/Moscow') AS x, toDecimal32(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toDateTime(0, 'Europe/Moscow') AS x, toDecimal64(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toDateTime(0, 'Europe/Moscow') AS x, toDecimal128(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toUUID(0) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toUUID(0) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toUUID(0) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toUUID(0) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toUUID(0) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toUUID(0) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toUUID(0) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toUUID(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toUUID(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toUUID(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toUUID(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } -SELECT toUUID(0) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toUUID(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -SELECT toUUID(0) AS x, toDecimal32(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toUUID(0) AS x, toDecimal64(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toUUID(0) AS x, toDecimal128(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } - SELECT 'column vs value'; SELECT materialize(toInt8(0)) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); @@ -203,7 +176,6 @@ SELECT materialize(toInt8(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, t SELECT materialize(toInt8(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toInt8(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toInt8(0)) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toInt8(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toInt8(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toInt8(0)) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toInt8(0)) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } @@ -220,7 +192,6 @@ SELECT materialize(toInt16(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, SELECT materialize(toInt16(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toInt16(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toInt16(0)) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toInt16(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toInt16(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toInt16(0)) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toInt16(0)) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } @@ -237,7 +208,6 @@ SELECT materialize(toInt32(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, SELECT materialize(toInt32(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toInt32(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toInt32(0)) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toInt32(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toInt32(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toInt32(0)) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toInt32(0)) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } @@ -254,7 +224,6 @@ SELECT materialize(toInt64(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, SELECT materialize(toInt64(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toInt64(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toInt64(0)) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toInt64(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toInt64(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toInt64(0)) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toInt64(0)) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } @@ -271,7 +240,6 @@ SELECT materialize(toUInt8(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, SELECT materialize(toUInt8(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toUInt8(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toUInt8(0)) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toUInt8(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUInt8(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toUInt8(0)) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toUInt8(0)) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } @@ -288,7 +256,6 @@ SELECT materialize(toUInt16(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, SELECT materialize(toUInt16(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toUInt16(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toUInt16(0)) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toUInt16(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUInt16(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toUInt16(0)) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toUInt16(0)) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } @@ -305,7 +272,6 @@ SELECT materialize(toUInt32(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, SELECT materialize(toUInt32(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toUInt32(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toUInt32(0)) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toUInt32(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUInt32(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toUInt32(0)) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toUInt32(0)) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } @@ -322,7 +288,6 @@ SELECT materialize(toUInt64(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, SELECT materialize(toUInt64(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUInt64(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toUInt64(0)) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toUInt64(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUInt64(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toUInt64(0)) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toUInt64(0)) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } @@ -339,7 +304,6 @@ SELECT materialize(toDate(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, t SELECT materialize(toDate(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toDate(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toDate('2000-01-01')) AS x, toDateTime('2000-01-01 00:00:01', 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -SELECT materialize(toDate(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toDate(0)) AS x, toDecimal32(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toDate(0)) AS x, toDecimal64(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toDate(0)) AS x, toDecimal128(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } @@ -356,24 +320,6 @@ SELECT materialize(toDateTime(0, 'Europe/Moscow')) AS x, toFloat32(1) AS y, ((x SELECT materialize(toDateTime(0, 'Europe/Moscow')) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toDateTime('2000-01-01 00:00:00', 'Europe/Moscow')) AS x, toDate('2000-01-02') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toDateTime(0, 'Europe/Moscow')) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -SELECT materialize(toDateTime(0, 'Europe/Moscow')) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toDateTime(0, 'Europe/Moscow')) AS x, toDecimal32(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toDateTime(0, 'Europe/Moscow')) AS x, toDecimal64(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toDateTime(0, 'Europe/Moscow')) AS x, toDecimal128(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } - -SELECT materialize(toUUID(0)) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toUUID(0)) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toUUID(0)) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toUUID(0)) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toUUID(0)) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toUUID(0)) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toUUID(0)) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toUUID(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toUUID(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toUUID(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toUUID(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } -SELECT materialize(toUUID(0)) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toUUID(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -SELECT materialize(toUUID(0)) AS x, toDecimal32(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toUUID(0)) AS x, toDecimal64(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toUUID(0)) AS x, toDecimal128(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } diff --git a/tests/queries/0_stateless/01634_uuid_fuzz.reference b/tests/queries/0_stateless/01634_uuid_fuzz.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01634_uuid_fuzz.sql b/tests/queries/0_stateless/01634_uuid_fuzz.sql new file mode 100644 index 00000000000..62ca209f6f3 --- /dev/null +++ b/tests/queries/0_stateless/01634_uuid_fuzz.sql @@ -0,0 +1 @@ +SELECT toUUID(-1.1); -- { serverError 48 } From 3bbdacc66b7d86669f278a8f036e2edd19879565 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 5 Jan 2021 01:35:54 +0300 Subject: [PATCH 091/256] Update test --- .../0_stateless/01516_create_table_primary_key.sql | 8 -------- 1 file changed, 8 deletions(-) diff --git a/tests/queries/0_stateless/01516_create_table_primary_key.sql b/tests/queries/0_stateless/01516_create_table_primary_key.sql index 0611612f89f..c0e9ce4a34d 100644 --- a/tests/queries/0_stateless/01516_create_table_primary_key.sql +++ b/tests/queries/0_stateless/01516_create_table_primary_key.sql @@ -18,10 +18,6 @@ ATTACH TABLE primary_key_test(v Int32) ENGINE=ReplacingMergeTree ORDER BY v PRIM SELECT * FROM primary_key_test FINAL; DROP TABLE primary_key_test; -CREATE TABLE primary_key_test(v Int32, PRIMARY KEY(v), PRIMARY KEY(v)) ENGINE=ReplacingMergeTree ORDER BY v; -- { clientError 36; } - -CREATE TABLE primary_key_test(v Int32, PRIMARY KEY(v)) ENGINE=ReplacingMergeTree ORDER BY v PRIMARY KEY(v); -- { clientError 36; } - CREATE TABLE primary_key_test(v1 Int32, v2 Int32, PRIMARY KEY(v1, v2)) ENGINE=ReplacingMergeTree ORDER BY (v1, v2); INSERT INTO primary_key_test VALUES (1, 1), (1, 1), (1, 1); DETACH TABLE primary_key_test; @@ -36,10 +32,6 @@ ATTACH TABLE primary_key_test(v1 Int32, v2 Int32) ENGINE=ReplacingMergeTree ORDE SELECT * FROM primary_key_test FINAL; DROP TABLE primary_key_test; -CREATE TABLE primary_key_test(v1 Int32, v2 Int32, PRIMARY KEY(v1, v2), PRIMARY KEY(v1, v2)) ENGINE=ReplacingMergeTree ORDER BY (v1, v2); -- { clientError 36; } - -CREATE TABLE primary_key_test(v1 Int32, v2 Int32, PRIMARY KEY(v1, v2)) ENGINE=ReplacingMergeTree ORDER BY (v1, v2) PRIMARY KEY(v1, v2); -- { clientError 36; } - CREATE TABLE primary_key_test(v1 Int64, v2 Int32, v3 String, PRIMARY KEY(v1, gcd(v1, v2))) ENGINE=ReplacingMergeTree ORDER BY v1; -- { serverError 36; } CREATE TABLE primary_key_test(v1 Int64, v2 Int32, v3 String, PRIMARY KEY(v1, gcd(v1, v2))) ENGINE=ReplacingMergeTree ORDER BY (v1, gcd(v1, v2)); From 535889003987145bc366f874e486540daa24ec53 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 5 Jan 2021 02:49:31 +0300 Subject: [PATCH 092/256] Remove bad code in HashJoin --- src/Interpreters/HashJoin.cpp | 22 +++++++++++++++++++--- src/Interpreters/HashJoin.h | 8 ++++---- 2 files changed, 23 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 8f74a7be493..7c472a27d88 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -357,10 +357,14 @@ void HashJoin::init(Type type_) joinDispatch(kind, strictness, data->maps, [&](auto, auto, auto & map) { map.create(data->type); }); } -size_t HashJoin::getTotalRowCount() const +bool HashJoin::overDictionary() const { - std::shared_lock lock(data->rwlock); - return getTotalRowCountLocked(); + return data->type == Type::DICT; +} + +bool HashJoin::empty() const +{ + return data->type == Type::EMPTY; } size_t HashJoin::getTotalByteCount() const @@ -369,6 +373,18 @@ size_t HashJoin::getTotalByteCount() const return getTotalByteCountLocked(); } +size_t HashJoin::getTotalRowCount() const +{ + std::shared_lock lock(data->rwlock); + return getTotalRowCountLocked(); +} + +bool HashJoin::alwaysReturnsEmptySet() const +{ + std::shared_lock lock(data->rwlock); + return isInnerOrRight(getKind()) && data->empty && !overDictionary(); +} + size_t HashJoin::getTotalRowCountLocked() const { size_t res = 0; diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 37bc9a9d345..c14ad4e5a99 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -150,9 +150,6 @@ class HashJoin : public IJoin public: HashJoin(std::shared_ptr table_join_, const Block & right_sample_block, bool any_take_last_row_ = false); - bool empty() const { return data->type == Type::EMPTY; } - bool overDictionary() const { return data->type == Type::DICT; } - /** Add block of data from right hand of JOIN to the map. * Returns false, if some limit was exceeded and you should not insert more data. */ @@ -188,7 +185,7 @@ public: /// Sum size in bytes of all buffers, used for JOIN maps and for all memory pools. size_t getTotalByteCount() const final; - bool alwaysReturnsEmptySet() const final { return isInnerOrRight(getKind()) && data->empty && !overDictionary(); } + bool alwaysReturnsEmptySet() const final; ASTTableJoin::Kind getKind() const { return kind; } ASTTableJoin::Strictness getStrictness() const { return strictness; } @@ -397,6 +394,9 @@ private: /// Call with already locked rwlock. size_t getTotalRowCountLocked() const; size_t getTotalByteCountLocked() const; + + bool empty() const; + bool overDictionary() const; }; } From 06405531a1b5118ef20112dfd7044599cc720aa2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 5 Jan 2021 02:58:16 +0300 Subject: [PATCH 093/256] Update tests --- .../0_stateless/00306_insert_values_and_expressions.sql | 2 +- tests/queries/0_stateless/00735_conditional.reference | 2 -- .../queries/0_stateless/01322_cast_keep_nullable.reference | 1 - tests/queries/0_stateless/01322_cast_keep_nullable.sql | 1 - .../01554_bloom_filter_index_big_integer_uuid.reference | 6 +++--- .../01554_bloom_filter_index_big_integer_uuid.sql | 6 +++--- 6 files changed, 7 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/00306_insert_values_and_expressions.sql b/tests/queries/0_stateless/00306_insert_values_and_expressions.sql index 10a1415f287..01a66282bb8 100644 --- a/tests/queries/0_stateless/00306_insert_values_and_expressions.sql +++ b/tests/queries/0_stateless/00306_insert_values_and_expressions.sql @@ -1,7 +1,7 @@ DROP TABLE IF EXISTS insert; CREATE TABLE insert (i UInt64, s String, u UUID, d Date, t DateTime, a Array(UInt32)) ENGINE = Memory; -INSERT INTO insert VALUES (1, 'Hello', 'ab41bdd6-5cd4-11e7-907b-a6006ad3dba0', '2016-01-01', '2016-01-02 03:04:05', [1, 2, 3]), (1 + 1, concat('Hello', ', world'), toUUID(0), toDate('2016-01-01') + 1, toStartOfMinute(toDateTime('2016-01-02 03:04:05')), [[0,1],[2]][1]), (round(pi()), concat('hello', ', world!'), toUUID(toString('ab41bdd6-5cd4-11e7-907b-a6006ad3dba0')), toDate(toDateTime('2016-01-03 03:04:05')), toStartOfHour(toDateTime('2016-01-02 03:04:05')), []), (4, 'World', 'ab41bdd6-5cd4-11e7-907b-a6006ad3dba0', '2016-01-04', '2016-12-11 10:09:08', [3,2,1]); +INSERT INTO insert VALUES (1, 'Hello', 'ab41bdd6-5cd4-11e7-907b-a6006ad3dba0', '2016-01-01', '2016-01-02 03:04:05', [1, 2, 3]), (1 + 1, concat('Hello', ', world'), toUUID('00000000-0000-0000-0000-000000000000'), toDate('2016-01-01') + 1, toStartOfMinute(toDateTime('2016-01-02 03:04:05')), [[0,1],[2]][1]), (round(pi()), concat('hello', ', world!'), toUUID(toString('ab41bdd6-5cd4-11e7-907b-a6006ad3dba0')), toDate(toDateTime('2016-01-03 03:04:05')), toStartOfHour(toDateTime('2016-01-02 03:04:05')), []), (4, 'World', 'ab41bdd6-5cd4-11e7-907b-a6006ad3dba0', '2016-01-04', '2016-12-11 10:09:08', [3,2,1]); SELECT * FROM insert ORDER BY i; DROP TABLE insert; diff --git a/tests/queries/0_stateless/00735_conditional.reference b/tests/queries/0_stateless/00735_conditional.reference index b37fce430ec..6bee974769d 100644 --- a/tests/queries/0_stateless/00735_conditional.reference +++ b/tests/queries/0_stateless/00735_conditional.reference @@ -95,7 +95,6 @@ value vs value 2000-01-01 2000-01-01 00:00:01 2000-01-01 00:00:01 Date DateTime(\'Europe/Moscow\') DateTime 2000-01-01 00:00:00 2000-01-02 2000-01-02 00:00:00 DateTime(\'Europe/Moscow\') Date DateTime 1970-01-01 03:00:00 1970-01-01 03:00:01 1970-01-01 03:00:01 DateTime(\'Europe/Moscow\') DateTime(\'Europe/Moscow\') DateTime(\'Europe/Moscow\') -00000000-0000-0000-0000-000000000000 00000000-0000-0001-0000-000000000000 00000000-0000-0001-0000-000000000000 UUID UUID UUID column vs value 0 1 1 Int8 Int8 Int8 0 1 1 Int8 Int16 Int16 @@ -169,4 +168,3 @@ column vs value 2000-01-01 2000-01-01 00:00:01 2000-01-01 00:00:01 Date DateTime(\'Europe/Moscow\') DateTime 2000-01-01 00:00:00 2000-01-02 2000-01-02 00:00:00 DateTime(\'Europe/Moscow\') Date DateTime 1970-01-01 03:00:00 1970-01-01 03:00:01 1970-01-01 03:00:01 DateTime(\'Europe/Moscow\') DateTime(\'Europe/Moscow\') DateTime(\'Europe/Moscow\') -00000000-0000-0000-0000-000000000000 00000000-0000-0001-0000-000000000000 00000000-0000-0001-0000-000000000000 UUID UUID UUID diff --git a/tests/queries/0_stateless/01322_cast_keep_nullable.reference b/tests/queries/0_stateless/01322_cast_keep_nullable.reference index cfe80a7c1b1..8ad99a10170 100644 --- a/tests/queries/0_stateless/01322_cast_keep_nullable.reference +++ b/tests/queries/0_stateless/01322_cast_keep_nullable.reference @@ -4,7 +4,6 @@ 1 Nullable(Int32) 2 Nullable(Float32) 2 Nullable(UInt8) -00000000-0000-0002-0000-000000000000 Nullable(UUID) 3 Nullable(Int32) \N Nullable(Int32) 42 Nullable(Int32) diff --git a/tests/queries/0_stateless/01322_cast_keep_nullable.sql b/tests/queries/0_stateless/01322_cast_keep_nullable.sql index ef5f5747a7d..2f6fd80c35f 100644 --- a/tests/queries/0_stateless/01322_cast_keep_nullable.sql +++ b/tests/queries/0_stateless/01322_cast_keep_nullable.sql @@ -10,7 +10,6 @@ SELECT CAST(toNullable(toInt8(1)) AS Int32) as x, toTypeName(x); SELECT CAST(toNullable(toFloat32(2)), 'Float32') as x, toTypeName(x); SELECT CAST(toNullable(toFloat32(2)), 'UInt8') as x, toTypeName(x); -SELECT CAST(toNullable(toFloat32(2)), 'UUID') as x, toTypeName(x); SELECT CAST(if(1 = 1, toNullable(toInt8(3)), NULL) AS Int32) as x, toTypeName(x); SELECT CAST(if(1 = 0, toNullable(toInt8(3)), NULL) AS Int32) as x, toTypeName(x); diff --git a/tests/queries/0_stateless/01554_bloom_filter_index_big_integer_uuid.reference b/tests/queries/0_stateless/01554_bloom_filter_index_big_integer_uuid.reference index 39beae69d65..f3b92461df7 100644 --- a/tests/queries/0_stateless/01554_bloom_filter_index_big_integer_uuid.reference +++ b/tests/queries/0_stateless/01554_bloom_filter_index_big_integer_uuid.reference @@ -7,6 +7,6 @@ 1 1 2 -00000000-0000-0001-0000-000000000000 -00000000-0000-0001-0000-000000000000 -00000000-0000-0002-0000-000000000000 +00000000-0000-0000-0000-000000000001 +00000000-0000-0000-0000-000000000001 +00000000-0000-0000-0000-000000000002 diff --git a/tests/queries/0_stateless/01554_bloom_filter_index_big_integer_uuid.sql b/tests/queries/0_stateless/01554_bloom_filter_index_big_integer_uuid.sql index 265f931c556..3472f41092d 100644 --- a/tests/queries/0_stateless/01554_bloom_filter_index_big_integer_uuid.sql +++ b/tests/queries/0_stateless/01554_bloom_filter_index_big_integer_uuid.sql @@ -19,7 +19,7 @@ SELECT x FROM 01154_test WHERE x IN (1, 2); DROP TABLE 01154_test; CREATE TABLE 01154_test (x UUID, INDEX ix_x x TYPE bloom_filter(0.01) GRANULARITY 1) ENGINE = MergeTree() ORDER BY x SETTINGS index_granularity=8192; -INSERT INTO 01154_test VALUES (toUUID(1)), (toUUID(2)), (toUUID(3)); -SELECT x FROM 01154_test WHERE x = toUUID(1); -SELECT x FROM 01154_test WHERE x IN (toUUID(1), toUUID(2)); +INSERT INTO 01154_test VALUES (toUUID('00000000-0000-0000-0000-000000000001')), (toUUID('00000000-0000-0000-0000-000000000002')), (toUUID('00000000-0000-0000-0000-000000000003')); +SELECT x FROM 01154_test WHERE x = toUUID('00000000-0000-0000-0000-000000000001'); +SELECT x FROM 01154_test WHERE x IN (toUUID('00000000-0000-0000-0000-000000000001'), toUUID('00000000-0000-0000-0000-000000000002')); DROP TABLE 01154_test; From 6ecf4fe921934aa79fe77f54d25f1295cce2bf1b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 5 Jan 2021 03:02:40 +0300 Subject: [PATCH 094/256] More robust stateful test --- docker/test/stateful/run.sh | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 91c040ce6ad..f2fcefd604f 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -37,7 +37,15 @@ chmod 777 -R /var/lib/clickhouse clickhouse-client --query "SHOW DATABASES" clickhouse-client --query "ATTACH DATABASE datasets ENGINE = Ordinary" clickhouse-client --query "CREATE DATABASE test" -service clickhouse-server restart && sleep 5 + +service clickhouse-server restart + +# Wait for server to start accepting connections +for _ in {1..120}; do + clickhouse-client --query "SELECT 1" && break + sleep 1 +done + clickhouse-client --query "SHOW TABLES FROM datasets" clickhouse-client --query "SHOW TABLES FROM test" clickhouse-client --query "RENAME TABLE datasets.hits_v1 TO test.hits" From 21467cf97a1b40522ca6de1c02307fedd8f243c4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 5 Jan 2021 03:44:14 +0300 Subject: [PATCH 095/256] Added a test from #15641 --- .../0_stateless/01635_nullable_fuzz.reference | 0 .../0_stateless/01635_nullable_fuzz.sql | 21 +++++++++++++++++++ 2 files changed, 21 insertions(+) create mode 100644 tests/queries/0_stateless/01635_nullable_fuzz.reference create mode 100644 tests/queries/0_stateless/01635_nullable_fuzz.sql diff --git a/tests/queries/0_stateless/01635_nullable_fuzz.reference b/tests/queries/0_stateless/01635_nullable_fuzz.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01635_nullable_fuzz.sql b/tests/queries/0_stateless/01635_nullable_fuzz.sql new file mode 100644 index 00000000000..c134578b221 --- /dev/null +++ b/tests/queries/0_stateless/01635_nullable_fuzz.sql @@ -0,0 +1,21 @@ +SELECT + 'Nul\0able\0String)Nul\0\0ble(String)Nul\0able(String)Nul\0able(String)', + NULL AND 2, + '', + number, + NULL AS k +FROM +( + SELECT + materialize(NULL) OR materialize(-9223372036854775808), + number + FROM system.numbers + LIMIT 1000000 +) +ORDER BY + k ASC, + number ASC, + k ASC +LIMIT 1023, 1023 +SETTINGS max_bytes_before_external_sort = 1000000 +FORMAT Null; From f6dff13db8545967475f52416d01946d0d660269 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 5 Jan 2021 03:56:56 +0300 Subject: [PATCH 096/256] Add a test for #14974 --- .../01636_nullable_fuzz2.reference | 6 ++++ .../0_stateless/01636_nullable_fuzz2.sql | 35 +++++++++++++++++++ 2 files changed, 41 insertions(+) create mode 100644 tests/queries/0_stateless/01636_nullable_fuzz2.reference create mode 100644 tests/queries/0_stateless/01636_nullable_fuzz2.sql diff --git a/tests/queries/0_stateless/01636_nullable_fuzz2.reference b/tests/queries/0_stateless/01636_nullable_fuzz2.reference new file mode 100644 index 00000000000..b26b82b1cb9 --- /dev/null +++ b/tests/queries/0_stateless/01636_nullable_fuzz2.reference @@ -0,0 +1,6 @@ +2020-07-09 +2020-07-10 +2020-07-11 +2020-07-12 + +1970-01-01 diff --git a/tests/queries/0_stateless/01636_nullable_fuzz2.sql b/tests/queries/0_stateless/01636_nullable_fuzz2.sql new file mode 100644 index 00000000000..a40da51c38c --- /dev/null +++ b/tests/queries/0_stateless/01636_nullable_fuzz2.sql @@ -0,0 +1,35 @@ +DROP TABLE IF EXISTS open_events_tmp; +DROP TABLE IF EXISTS tracking_events_tmp; + +CREATE TABLE open_events_tmp (`APIKey` UInt32, `EventDate` Date) ENGINE = MergeTree PARTITION BY toMonday(EventDate) ORDER BY (APIKey, EventDate); +CREATE TABLE tracking_events_tmp (`APIKey` UInt32, `EventDate` Date) ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (APIKey, EventDate); + +insert into open_events_tmp select 2, '2020-07-10' from numbers(32); +insert into open_events_tmp select 2, '2020-07-11' from numbers(31); +insert into open_events_tmp select 2, '2020-07-12' from numbers(30); + +insert into tracking_events_tmp select 2, '2020-07-09' from numbers(1555); +insert into tracking_events_tmp select 2, '2020-07-10' from numbers(1881); +insert into tracking_events_tmp select 2, '2020-07-11' from numbers(1623); + +SELECT EventDate +FROM +( + SELECT EventDate + FROM tracking_events_tmp AS t1 + WHERE (EventDate >= toDate('2020-07-09')) AND (EventDate <= toDate('2020-07-11')) AND (APIKey = 2) + GROUP BY EventDate +) +FULL OUTER JOIN +( + SELECT EventDate + FROM remote('127.0.0.{1,3}', currentDatabase(), open_events_tmp) AS t2 + WHERE (EventDate <= toDate('2020-07-12')) AND (APIKey = 2) + GROUP BY EventDate + WITH TOTALS +) USING (EventDate) +ORDER BY EventDate ASC +SETTINGS totals_mode = 'after_having_auto', group_by_overflow_mode = 'any', max_rows_to_group_by = 10000000, joined_subquery_requires_alias = 0; + +DROP TABLE open_events_tmp; +DROP TABLE tracking_events_tmp; From c2c1d97da26412f57b252c077a1879ae26fd9652 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 5 Jan 2021 03:59:09 +0300 Subject: [PATCH 097/256] Arcadia does not support distributed queries --- tests/queries/0_stateless/arcadia_skip_list.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index c1f7c14f58a..ec5acc3e4d2 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -179,3 +179,4 @@ 01558_ttest_scipy 01561_mann_whitney_scipy 01601_custom_tld +01636_nullable_fuzz2 From ab6798addd9de3b26e0eb801f051b87404f4617c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 5 Jan 2021 04:03:38 +0300 Subject: [PATCH 098/256] One more test --- tests/queries/0_stateless/01637_nullable_fuzz3.reference | 2 ++ tests/queries/0_stateless/01637_nullable_fuzz3.sql | 4 ++++ 2 files changed, 6 insertions(+) create mode 100644 tests/queries/0_stateless/01637_nullable_fuzz3.reference create mode 100644 tests/queries/0_stateless/01637_nullable_fuzz3.sql diff --git a/tests/queries/0_stateless/01637_nullable_fuzz3.reference b/tests/queries/0_stateless/01637_nullable_fuzz3.reference new file mode 100644 index 00000000000..d9cf16b9d2a --- /dev/null +++ b/tests/queries/0_stateless/01637_nullable_fuzz3.reference @@ -0,0 +1,2 @@ + +0 diff --git a/tests/queries/0_stateless/01637_nullable_fuzz3.sql b/tests/queries/0_stateless/01637_nullable_fuzz3.sql new file mode 100644 index 00000000000..21bf8999eae --- /dev/null +++ b/tests/queries/0_stateless/01637_nullable_fuzz3.sql @@ -0,0 +1,4 @@ +DROP TABLE IF EXISTS t; +CREATE TABLE t (`item_id` UInt64, `price_sold` Float32, `date` Date) ENGINE = MergeTree ORDER BY item_id; +SELECT item_id FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS) AS l FULL OUTER JOIN (SELECT item_id FROM t GROUP BY item_id WITH TOTALS) AS r USING (item_id); +DROP TABLE t; From a1ce6c41659eb6f6a530b737fb913e0fc054adb7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 5 Jan 2021 04:26:29 +0300 Subject: [PATCH 099/256] Fix errors --- src/AggregateFunctions/AggregateFunctionSumMap.h | 12 +++++++----- .../0_stateless/01422_map_skip_null.reference | 7 ++----- tests/queries/0_stateless/01422_map_skip_null.sql | 6 +++--- .../0_stateless/01621_summap_check_types.reference | 1 + .../queries/0_stateless/01621_summap_check_types.sql | 5 +++++ 5 files changed, 18 insertions(+), 13 deletions(-) create mode 100644 tests/queries/0_stateless/01621_summap_check_types.reference create mode 100644 tests/queries/0_stateless/01621_summap_check_types.sql diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.h b/src/AggregateFunctions/AggregateFunctionSumMap.h index 8be25194dae..6c8155f967c 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -96,6 +96,11 @@ public: if constexpr (overflow) { + if (value_type->onlyNull()) + throw Exception{ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Cannot calculate {} of type {}", + getName(), value_type->getName()}; + // Overflow, meaning that the returned type is the same as // the input type. Nulls are skipped. result_type = removeNullable(value_type); @@ -190,11 +195,8 @@ public: { // Create a value array for this key Array new_values; - new_values.resize(values_types.size()); - for (size_t k = 0; k < new_values.size(); ++k) - { - new_values[k] = (k == col) ? value : values_types[k]->getDefault(); - } + new_values.resize(size); + new_values[col] = value; if constexpr (IsDecimalNumber) { diff --git a/tests/queries/0_stateless/01422_map_skip_null.reference b/tests/queries/0_stateless/01422_map_skip_null.reference index 7211e0ac75d..83456126278 100644 --- a/tests/queries/0_stateless/01422_map_skip_null.reference +++ b/tests/queries/0_stateless/01422_map_skip_null.reference @@ -1,7 +1,4 @@ -([],[]) -([],[]) -([],[]) -([2],[11]) -([2],[22]) +([1,2],[0,11]) +([1,2],[0,22]) ([2],[33]) ([2],[33]) diff --git a/tests/queries/0_stateless/01422_map_skip_null.sql b/tests/queries/0_stateless/01422_map_skip_null.sql index 9af46758289..683757a473b 100644 --- a/tests/queries/0_stateless/01422_map_skip_null.sql +++ b/tests/queries/0_stateless/01422_map_skip_null.sql @@ -1,7 +1,7 @@ -select minMap(arrayJoin([([1], [null]), ([1], [null])])); -select maxMap(arrayJoin([([1], [null]), ([1], [null])])); +select minMap(arrayJoin([([1], [null]), ([1], [null])])); -- { serverError 43 } +select maxMap(arrayJoin([([1], [null]), ([1], [null])])); -- { serverError 43 } select sumMap(arrayJoin([([1], [null]), ([1], [null])])); -- { serverError 43 } -select sumMapWithOverflow(arrayJoin([([1], [null]), ([1], [null])])); +select sumMapWithOverflow(arrayJoin([([1], [null]), ([1], [null])])); -- { serverError 43 } select minMap(arrayJoin([([1, 2], [null, 11]), ([1, 2], [null, 22])])); select maxMap(arrayJoin([([1, 2], [null, 11]), ([1, 2], [null, 22])])); diff --git a/tests/queries/0_stateless/01621_summap_check_types.reference b/tests/queries/0_stateless/01621_summap_check_types.reference new file mode 100644 index 00000000000..1aeb897d4d0 --- /dev/null +++ b/tests/queries/0_stateless/01621_summap_check_types.reference @@ -0,0 +1 @@ +([1,2],[1,2],[1,0]) diff --git a/tests/queries/0_stateless/01621_summap_check_types.sql b/tests/queries/0_stateless/01621_summap_check_types.sql new file mode 100644 index 00000000000..a950f3ea094 --- /dev/null +++ b/tests/queries/0_stateless/01621_summap_check_types.sql @@ -0,0 +1,5 @@ +select initializeAggregation('sumMap', [1, 2], [1, 2], [1, null]); + +CREATE TEMPORARY TABLE sum_map_overflow (events Array(UInt8), counts Array(UInt8)); +INSERT INTO sum_map_overflow VALUES ([1], [255]), ([1], [2]); +SELECT [NULL], sumMapWithOverflow(events, [NULL], [[(NULL)]], counts) FROM sum_map_overflow; -- { serverError 43 } From 5368b20a719740f7572d5df3b5cd58b6523a6c3a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 5 Jan 2021 04:49:15 +0300 Subject: [PATCH 100/256] Fix error --- src/Common/FileChecker.h | 1 + src/Storages/StorageLog.cpp | 188 +++++++++++++++--------------- src/Storages/StorageStripeLog.cpp | 13 ++- src/Storages/StorageTinyLog.cpp | 14 ++- 4 files changed, 116 insertions(+), 100 deletions(-) diff --git a/src/Common/FileChecker.h b/src/Common/FileChecker.h index b540112b5c4..73e4470f231 100644 --- a/src/Common/FileChecker.h +++ b/src/Common/FileChecker.h @@ -18,6 +18,7 @@ public: void update(const String & full_file_path); void setEmpty(const String & full_file_path); void save() const; + bool empty() const { return map.empty(); } /// Check the files whose parameters are specified in sizes.json CheckResults check() const; diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index f9e6685b908..06e9bb8a2d6 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -113,93 +113,6 @@ private: }; -class LogBlockOutputStream final : public IBlockOutputStream -{ -public: - explicit LogBlockOutputStream( - StorageLog & storage_, const StorageMetadataPtr & metadata_snapshot_, std::unique_lock && lock_) - : storage(storage_) - , metadata_snapshot(metadata_snapshot_) - , lock(std::move(lock_)) - , marks_stream( - storage.disk->writeFile(storage.marks_file_path, 4096, WriteMode::Rewrite)) - { - if (!lock) - throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); - } - - ~LogBlockOutputStream() override - { - try - { - if (!done) - { - /// Rollback partial writes. - streams.clear(); - storage.file_checker.repair(); - } - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } - - Block getHeader() const override { return metadata_snapshot->getSampleBlock(); } - void write(const Block & block) override; - void writeSuffix() override; - -private: - StorageLog & storage; - StorageMetadataPtr metadata_snapshot; - std::unique_lock lock; - bool done = false; - - struct Stream - { - Stream(const DiskPtr & disk, const String & data_path, CompressionCodecPtr codec, size_t max_compress_block_size) : - plain(disk->writeFile(data_path, max_compress_block_size, WriteMode::Append)), - compressed(*plain, std::move(codec), max_compress_block_size), - plain_offset(disk->getFileSize(data_path)) - { - } - - std::unique_ptr plain; - CompressedWriteBuffer compressed; - - size_t plain_offset; /// How many bytes were in the file at the time the LogBlockOutputStream was created. - - void finalize() - { - compressed.next(); - plain->next(); - } - }; - - using Mark = StorageLog::Mark; - using MarksForColumns = std::vector>; - - using FileStreams = std::map; - FileStreams streams; - - using WrittenStreams = std::set; - - std::unique_ptr marks_stream; /// Declared below `lock` to make the file open when rwlock is captured. - - using SerializeState = IDataType::SerializeBinaryBulkStatePtr; - using SerializeStates = std::map; - SerializeStates serialize_states; - - IDataType::OutputStreamGetter createStreamGetter(const String & name, WrittenStreams & written_streams); - - void writeData(const String & name, const IDataType & type, const IColumn & column, - MarksForColumns & out_marks, - WrittenStreams & written_streams); - - void writeMarks(MarksForColumns && marks); -}; - - Chunk LogSource::generate() { Block res; @@ -207,7 +120,7 @@ Chunk LogSource::generate() if (rows_read == rows_limit) return {}; - if (storage.disk->isDirectoryEmpty(storage.table_path)) + if (storage.file_checker.empty()) return {}; /// How many rows to read for the next block. @@ -284,6 +197,101 @@ void LogSource::readData(const String & name, const IDataType & type, IColumn & } +class LogBlockOutputStream final : public IBlockOutputStream +{ +public: + explicit LogBlockOutputStream( + StorageLog & storage_, const StorageMetadataPtr & metadata_snapshot_, std::unique_lock && lock_) + : storage(storage_) + , metadata_snapshot(metadata_snapshot_) + , lock(std::move(lock_)) + , marks_stream( + storage.disk->writeFile(storage.marks_file_path, 4096, WriteMode::Rewrite)) + { + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + + /// If there were no files, add info to rollback in case of error. + if (storage.file_checker.empty()) + { + for (const auto & file : storage.files) + storage.file_checker.setEmpty(file.second.data_file_path); + storage.file_checker.save(); + } + } + + ~LogBlockOutputStream() override + { + try + { + if (!done) + { + /// Rollback partial writes. + streams.clear(); + storage.file_checker.repair(); + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + + Block getHeader() const override { return metadata_snapshot->getSampleBlock(); } + void write(const Block & block) override; + void writeSuffix() override; + +private: + StorageLog & storage; + StorageMetadataPtr metadata_snapshot; + std::unique_lock lock; + bool done = false; + + struct Stream + { + Stream(const DiskPtr & disk, const String & data_path, CompressionCodecPtr codec, size_t max_compress_block_size) : + plain(disk->writeFile(data_path, max_compress_block_size, WriteMode::Append)), + compressed(*plain, std::move(codec), max_compress_block_size), + plain_offset(disk->getFileSize(data_path)) + { + } + + std::unique_ptr plain; + CompressedWriteBuffer compressed; + + size_t plain_offset; /// How many bytes were in the file at the time the LogBlockOutputStream was created. + + void finalize() + { + compressed.next(); + plain->next(); + } + }; + + using Mark = StorageLog::Mark; + using MarksForColumns = std::vector>; + + using FileStreams = std::map; + FileStreams streams; + + using WrittenStreams = std::set; + + std::unique_ptr marks_stream; /// Declared below `lock` to make the file open when rwlock is captured. + + using SerializeState = IDataType::SerializeBinaryBulkStatePtr; + using SerializeStates = std::map; + SerializeStates serialize_states; + + IDataType::OutputStreamGetter createStreamGetter(const String & name, WrittenStreams & written_streams); + + void writeData(const String & name, const IDataType & type, const IColumn & column, + MarksForColumns & out_marks, + WrittenStreams & written_streams); + + void writeMarks(MarksForColumns && marks); +}; + + void LogBlockOutputStream::write(const Block & block) { metadata_snapshot->check(block, true); @@ -477,10 +485,6 @@ StorageLog::StorageLog( addFiles(column.name, *column.type); marks_file_path = table_path + DBMS_STORAGE_LOG_MARKS_FILE_NAME; - - if (!attach) - for (const auto & file : files) - file_checker.setEmpty(file.second.data_file_path); } diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 0bf70d4e837..db4fbff78cd 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -98,6 +98,9 @@ public: protected: Chunk generate() override { + if (storage.file_checker.empty()) + return {}; + Block res; start(); @@ -170,6 +173,13 @@ public: { if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + + if (storage.file_checker.empty()) + { + storage.file_checker.setEmpty(storage.table_path + "data.bin"); + storage.file_checker.setEmpty(storage.table_path + "index.mrk"); + storage.file_checker.save(); + } } ~StripeLogBlockOutputStream() override @@ -264,9 +274,6 @@ StorageStripeLog::StorageStripeLog( { /// create directories if they do not exist disk->createDirectories(table_path); - - file_checker.setEmpty(table_path + "data.bin"); - file_checker.setEmpty(table_path + "index.mrk"); } else { diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 4f3dcec4ddc..fe8a25ba13b 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -121,7 +121,7 @@ Chunk TinyLogSource::generate() { Block res; - if (is_finished || (!streams.empty() && streams.begin()->second->compressed.eof())) + if (is_finished || file_sizes.empty() || (!streams.empty() && streams.begin()->second->compressed.eof())) { /** Close the files (before destroying the object). * When many sources are created, but simultaneously reading only a few of them, @@ -196,6 +196,14 @@ public: { if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + + /// If there were no files, add info to rollback in case of error. + if (storage.file_checker.empty()) + { + for (const auto & file : storage.files) + storage.file_checker.setEmpty(file.second.data_file_path); + storage.file_checker.save(); + } } ~TinyLogBlockOutputStream() override @@ -393,10 +401,6 @@ StorageTinyLog::StorageTinyLog( for (const auto & col : storage_metadata.getColumns().getAllPhysical()) addFiles(col.name, *col.type); - - if (!attach) - for (const auto & file : files) - file_checker.setEmpty(file.second.data_file_path); } From 0b65605c34a95ae3a21493ba77bf19fd909b6414 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 5 Jan 2021 05:30:24 +0300 Subject: [PATCH 101/256] Mark some TestFlows as flaky --- tests/testflows/ldap/authentication/regression.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/testflows/ldap/authentication/regression.py b/tests/testflows/ldap/authentication/regression.py index ff004a998ca..b18e5a93b67 100755 --- a/tests/testflows/ldap/authentication/regression.py +++ b/tests/testflows/ldap/authentication/regression.py @@ -23,7 +23,9 @@ xfails = { "connection protocols/starttls with custom port": [(Fail, "it seems that starttls is not enabled by default on custom plain-text ports in LDAP server")], "connection protocols/tls cipher suite": - [(Fail, "can't get it to work")] + [(Fail, "can't get it to work")], + "external user directory/user authentications/valid verification cooldown value ldap unavailable": + [(Fail, "flaky, ask Vitaly Zakaznikov, Telegram @vzakaznikov")] } @TestFeature From dab4719aac3077e41a7759798355c690580941e6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 5 Jan 2021 06:22:06 +0300 Subject: [PATCH 102/256] Remove some headers --- src/DataStreams/PushingToViewsBlockOutputStream.cpp | 3 +++ src/DataStreams/PushingToViewsBlockOutputStream.h | 8 +++++--- src/Interpreters/InterpreterInsertQuery.cpp | 1 + src/Processors/Merges/MergingSortedTransform.cpp | 1 - src/Storages/StorageMerge.cpp | 1 - 5 files changed, 9 insertions(+), 5 deletions(-) diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 1252dd7f4de..35607bcb182 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -2,6 +2,8 @@ #include #include #include +#include +#include #include #include #include @@ -14,6 +16,7 @@ #include #include #include +#include namespace DB diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.h b/src/DataStreams/PushingToViewsBlockOutputStream.h index 18dc3ffd800..b97b3fe2061 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.h +++ b/src/DataStreams/PushingToViewsBlockOutputStream.h @@ -1,14 +1,16 @@ #pragma once -#include #include -#include -#include +#include +#include +#include + namespace DB { class ReplicatedMergeTreeBlockOutputStream; +class Context; /** Writes data to the specified table and to all dependent materialized views. diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 2a193dc7249..742c9f6736f 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -29,6 +29,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Processors/Merges/MergingSortedTransform.cpp b/src/Processors/Merges/MergingSortedTransform.cpp index e680304ccee..ec1bdc59683 100644 --- a/src/Processors/Merges/MergingSortedTransform.cpp +++ b/src/Processors/Merges/MergingSortedTransform.cpp @@ -1,7 +1,6 @@ #include #include #include -#include #include diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 18534d36e71..bc156b22abd 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1,6 +1,5 @@ #include #include -#include #include #include #include From 38b3d2844345ca844b829220f75cf407123450bf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 5 Jan 2021 06:26:53 +0300 Subject: [PATCH 103/256] Mark another flaky test --- tests/testflows/ldap/authentication/regression.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/testflows/ldap/authentication/regression.py b/tests/testflows/ldap/authentication/regression.py index b18e5a93b67..50677d78cb8 100755 --- a/tests/testflows/ldap/authentication/regression.py +++ b/tests/testflows/ldap/authentication/regression.py @@ -25,6 +25,8 @@ xfails = { "connection protocols/tls cipher suite": [(Fail, "can't get it to work")], "external user directory/user authentications/valid verification cooldown value ldap unavailable": + [(Fail, "flaky, ask Vitaly Zakaznikov, Telegram @vzakaznikov")], + "user authentications/rbac=True/verification cooldown/verification cooldown performance": [(Fail, "flaky, ask Vitaly Zakaznikov, Telegram @vzakaznikov")] } From 99335563d122ab3801ce15631f1f3c275d9180f4 Mon Sep 17 00:00:00 2001 From: spongedc Date: Tue, 5 Jan 2021 13:03:19 +0800 Subject: [PATCH 104/256] [For #18707] MySQL compatibility: support DIV and MOD operators --- src/Parsers/ExpressionListParsers.cpp | 2 + .../0_stateless/00977_int_div.reference | 40 +++++++++++++++++++ tests/queries/0_stateless/00977_int_div.sql | 6 +++ .../0_stateless/01412_mod_float.reference | 4 ++ tests/queries/0_stateless/01412_mod_float.sql | 4 ++ 5 files changed, 56 insertions(+) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index f50cf71f54d..afe85f069c7 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -18,6 +18,8 @@ const char * ParserMultiplicativeExpression::operators[] = "*", "multiply", "/", "divide", "%", "modulo", + "MOD", "modulo", + "DIV", "intDiv", nullptr }; diff --git a/tests/queries/0_stateless/00977_int_div.reference b/tests/queries/0_stateless/00977_int_div.reference index 39e135d8eed..e5882f4aaba 100644 --- a/tests/queries/0_stateless/00977_int_div.reference +++ b/tests/queries/0_stateless/00977_int_div.reference @@ -81,3 +81,43 @@ -1 -1 -1 +-1 +0 +0 +0 +0 +0 +0 +0 +0 +0 +-1 +-2 +-3 +-4 +-5 +-6 +-7 +-8 +-9 +-10 +-1 +-2 +-3 +-4 +-5 +-6 +-7 +-8 +-9 +-10 +-1 +-1 +-1 +-1 +-1 +-1 +-1 +-1 +-1 +-1 diff --git a/tests/queries/0_stateless/00977_int_div.sql b/tests/queries/0_stateless/00977_int_div.sql index 08085020ca5..4184475e3a0 100644 --- a/tests/queries/0_stateless/00977_int_div.sql +++ b/tests/queries/0_stateless/00977_int_div.sql @@ -23,3 +23,9 @@ SELECT intDiv(toInt64(number), -1) FROM numbers(1, 10); SELECT intDivOrZero(toInt64(number), -1) FROM numbers(1, 10); SELECT intDiv(number, -number) FROM numbers(1, 10); SELECT intDivOrZero(number, -number) FROM numbers(1, 10); + +SELECT -1 DIV number FROM numbers(1, 10); +SELECT toInt32(number) DIV -1 FROM numbers(1, 10); +SELECT toInt64(number) DIV -1 FROM numbers(1, 10); +SELECT number DIV -number FROM numbers(1, 10); +SELECT -1 DIV 0; -- { serverError 153 } diff --git a/tests/queries/0_stateless/01412_mod_float.reference b/tests/queries/0_stateless/01412_mod_float.reference index 77e3431f7a9..94f7f6d3af7 100644 --- a/tests/queries/0_stateless/01412_mod_float.reference +++ b/tests/queries/0_stateless/01412_mod_float.reference @@ -1,3 +1,7 @@ 1 -1 1 -1 0.125 -0.125 0.125 -0.125 1 -1 1 -1 +1 -1 1 -1 +0.125 -0.125 0.125 -0.125 +nan +nan diff --git a/tests/queries/0_stateless/01412_mod_float.sql b/tests/queries/0_stateless/01412_mod_float.sql index 147cb88ff24..f2a5f2fcee6 100644 --- a/tests/queries/0_stateless/01412_mod_float.sql +++ b/tests/queries/0_stateless/01412_mod_float.sql @@ -1,3 +1,7 @@ WITH 8.5 AS a, 2.5 AS b SELECT a % b, -a % b, a % -b, -a % -b; WITH 10.125 AS a, 2.5 AS b SELECT a % b, -a % b, a % -b, -a % -b; WITH 8.5 AS a, 2.5 AS b SELECT mod(a, b), MOD(-a, b), modulo(a, -b), moduloOrZero(-a, -b); +WITH 8.5 AS a, 2.5 AS b SELECT a MOD b, -a MOD b, a MOD -b, -a MOD -b; +WITH 10.125 AS a, 2.5 AS b SELECT a MOD b, -a MOD b, a MOD -b, -a MOD -b; +SELECT 3.5 % 0; +SELECT 3.5 MOD 0; From 6c50f537a014ce744da22738d54bcd3999e60caa Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 5 Jan 2021 15:13:10 +0300 Subject: [PATCH 105/256] Add a regression test for marks corruption In #17120 (that was reverted in #17918 already) marks adjustemnt was introduced, but it may lead to corruption of marks files, that may cause two things: 1) Incorrect marks files, which lead to reading more rows than there are (can be reproduced with `max_threads`>1 or/and `PREWHERE`, and `optimize_move_to_prewhere`) 2) Can't adjust last granule because it has X rows, but try to subtract Y rows (#9260) And 1) is pretty hard to diagnosis, since it does not throw any error, it just may return wrong result. Fortunately both problems can be fixed with `OPTIMIZE TABLE ... [ FINAL]`. Cc: @alesapin Refs: #17943 Refs: #18223 --- ...1640_marks_corruption_regression.reference | 4 ++ .../01640_marks_corruption_regression.sql | 55 +++++++++++++++++++ 2 files changed, 59 insertions(+) create mode 100644 tests/queries/0_stateless/01640_marks_corruption_regression.reference create mode 100644 tests/queries/0_stateless/01640_marks_corruption_regression.sql diff --git a/tests/queries/0_stateless/01640_marks_corruption_regression.reference b/tests/queries/0_stateless/01640_marks_corruption_regression.reference new file mode 100644 index 00000000000..a71b69fef89 --- /dev/null +++ b/tests/queries/0_stateless/01640_marks_corruption_regression.reference @@ -0,0 +1,4 @@ +"marks",7 +"optimize_trivial_count_query",16384 +"max_threads=1",16384 +"max_threads=100",16384 diff --git a/tests/queries/0_stateless/01640_marks_corruption_regression.sql b/tests/queries/0_stateless/01640_marks_corruption_regression.sql new file mode 100644 index 00000000000..63c180e518f --- /dev/null +++ b/tests/queries/0_stateless/01640_marks_corruption_regression.sql @@ -0,0 +1,55 @@ +DROP TABLE IF EXISTS adaptive_table; + +CREATE TABLE adaptive_table( + key UInt64, + value String +) ENGINE MergeTree() +ORDER BY key +SETTINGS + index_granularity_bytes=1048576, + min_bytes_for_wide_part=0, + old_parts_lifetime=0, + index_granularity=8192 +; + +-- This triggers adjustment of the granules that was introduced in PR#17120 +INSERT INTO adaptive_table SELECT number, randomPrintableASCII(if(number BETWEEN 8192-30 AND 8192, 102400, 1)) FROM system.numbers LIMIT 16384; +-- This creates the following marks: +-- +-- $ check-marks /path/to/db/adaptive_table/all_*/key.{mrk2,bin} +-- Mark 0, points to 0, 0, has rows after 8192, decompressed size 72808. + + + From 4f726784c92ae43389725d0981f96c58498078c3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 6 Jan 2021 04:56:10 +0300 Subject: [PATCH 129/256] Fix shellcheck --- docker/test/fuzzer/run-fuzzer.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 2cf97d683ac..0488cdce155 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -81,9 +81,9 @@ function fuzz echo Server started fuzzer_exit_code=0 - # SC2012: Use find instead of ls to better handle non-alphanumeric filenames. - # They are all alphanumeric. - # shellcheck disable=SC2012 + # SC2012: Use find instead of ls to better handle non-alphanumeric filenames. They are all alphanumeric. + # SC2046: Quote this to prevent word splitting. Actually I need word splitting. + # shellcheck disable=SC2012,SC2046 ./clickhouse-client --query-fuzzer-runs=1000 --queries-file $(ls -1 ch/tests/queries/0_stateless/*.sql | sort -R) \ > >(tail -n 10000 > fuzzer.log) \ 2>&1 \ From 3f5b94fe3bfea38e628d9df16a697bc478dbb53e Mon Sep 17 00:00:00 2001 From: booknouse <34814088+booknouse@users.noreply.github.com> Date: Wed, 6 Jan 2021 15:12:28 +0800 Subject: [PATCH 130/256] Update style.md Update translation by machine, and improve document more human-readable. --- docs/zh/development/style.md | 26 +++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/docs/zh/development/style.md b/docs/zh/development/style.md index 36e4acb6a24..8f104e3a7d8 100644 --- a/docs/zh/development/style.md +++ b/docs/zh/development/style.md @@ -267,7 +267,7 @@ void executeQuery( **9.** 多行注释的开头和结尾不得有空行(关闭多行注释的行除外)。 -**10.** 要注释掉代码,请使用基本注释,而不是«记录»注释。 +**10.** 要注释掉代码,请使用基本注释,而不是“文档”注释。 **11.** 在提交之前删除代码的无效注释部分。 @@ -335,7 +335,7 @@ template struct ExtractDomain ``` -**7.** 对于抽象类型(接口),用 `I` 前缀。 +**7.** 对于抽象类(接口),用 `I` 前缀。 ``` cpp class IBlockInputStream @@ -349,7 +349,7 @@ class IBlockInputStream bool info_successfully_loaded = false; ``` -**9.** `define` 和全局常量的名称使用带下划线的 `ALL_CAPS`。 +**9.** `define` 和全局常量的名称使用全大写带下划线的形式,如 `ALL_CAPS`。 ``` cpp #define MAX_SRC_TABLE_NAMES_TO_STORE 1000 @@ -357,14 +357,14 @@ bool info_successfully_loaded = false; **10.** 文件名应使用与其内容相同的样式。 -如果文件包含单个类,则以与该类名称相同的方式命名该文件。 +如果文件包含单个类,则以与该类名称相同的方式命名该文件(CamelCase)。 -如果文件包含单个函数,则以与函数名称相同的方式命名文件。 +如果文件包含单个函数,则以与函数名称相同的方式命名文件(camelCase)。 **11.** 如果名称包含缩写,则: - 对于变量名,缩写应使用小写字母 `mysql_connection`(不是 `mySQL_connection` )。 -- 对于类和函数的名称,请将大写字母保留在缩写 `MySQLConnection`(不是 `MySqlConnection` 。 +- 对于类和函数的名称,请将大写字母保留在缩写 `MySQLConnection`(不是 `MySqlConnection`)。 **12.** 仅用于初始化类成员的构造方法参数的命名方式应与类成员相同,但最后使用下划线。 @@ -411,7 +411,7 @@ enum class CompressionMethod 如果缩短版本是常用的,则可以接受不完整的单词。 -如果注释中旁边包含全名,您也可以使用缩写。 +如果旁边有注释包含全名,您也可以使用缩写。 **17.** C++ 源码文件名称必须为 `.cpp` 拓展名。 头文件必须为 `.h` 拓展名。 @@ -441,7 +441,7 @@ enum class CompressionMethod 在离线数据处理应用程序中,通常可以接受不捕获异常。 -在处理用户请求的服务器中,通常足以捕获连接处理程序顶层的异常。 +在处理用户请求的服务器中,捕获连接处理程序顶层的异常通常就足够了。 在线程函数中,你应该在 `join` 之后捕获并保留所有异常以在主线程中重新抛出它们。 @@ -548,7 +548,7 @@ Fork不用于并行化。 **10.** 常量。 -使用 const 引用,指向常量的指针,`const_iterator`和 const 指针。 +使用 const 引用、指针,指向常量、`const_iterator`和 const 方法。 将 `const` 视为默认值,仅在必要时使用非 `const`。 @@ -560,7 +560,7 @@ Fork不用于并行化。 **12.** 数值类型。 -使用 `UInt8`, `UInt16`, `UInt32`, `UInt64`, `Int8`, `Int16`, `Int32`, 以及 `Int64`, `size_t`, `ssize_t` 还有 `ptrdiff_t`。 +使用 `UInt8`, `UInt16`, `UInt32`, `UInt64`, `Int8`, `Int16`, `Int32` 和 `Int64`,同样还有 `size_t`, `ssize_t` 和 `ptrdiff_t`。 不要使用这些类型:`signed / unsigned long`,`long long`,`short`,`signed / unsigned char`,`char`。 @@ -732,11 +732,11 @@ CPU指令集是我们服务器中支持的最小集合。 目前,它是SSE 4.2 **8.** 尽可能经常地进行提交,即使代码只是部分准备好了。 -目的明确的功能,使用分支。 +为了这种目的可以创建分支。 -如果 `master` 分支中的代码尚不可构建,请在 `push` 之前将其从构建中排除。您需要在几天内完成或删除它。 +如果您的代码在 `master` 分支中尚不可构建,在 `push` 之前需要将其从构建中排除。您需要在几天内完成或删除它。 -**9.** 对于不重要的更改,请使用分支并在服务器上发布它们。 +**9.** 对于非一般的更改,请使用分支并在服务器上发布它们。 **10.** 未使用的代码将从 repo 中删除。 From 0260953a4785ac1daa6b3c17d6dc555ce59f8caf Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 6 Jan 2021 17:18:48 +0800 Subject: [PATCH 131/256] better --- src/Core/Settings.h | 2 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 5 +++-- .../01632_max_partitions_to_read.reference | 12 ++++++------ .../0_stateless/01632_max_partitions_to_read.sql | 10 +++++----- 4 files changed, 15 insertions(+), 14 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index ba8f6563ec9..e2aa4e2f2d8 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -141,7 +141,6 @@ class IColumn; \ M(Bool, force_index_by_date, 0, "Throw an exception if there is a partition key in a table, and it is not used.", 0) \ M(Bool, force_primary_key, 0, "Throw an exception if there is primary key in a table, and it is not used.", 0) \ - M(Bool, force_max_partition_limit, 0, "Throw an exception if max_partitions_to_read is violated.", 0) \ M(String, force_data_skipping_indices, "", "Comma separated list of strings or literals with the name of the data skipping indices that should be used during query execution, otherwise an exception will be thrown.", 0) \ \ M(Float, max_streams_to_max_threads_ratio, 1, "Allows you to use more sources than the number of threads - to more evenly distribute work across threads. It is assumed that this is a temporary solution, since it will be possible in the future to make the number of sources equal to the number of threads, but for each source to dynamically select available work for itself.", 0) \ @@ -354,6 +353,7 @@ class IColumn; M(Bool, allow_introspection_functions, false, "Allow functions for introspection of ELF and DWARF for query profiling. These functions are slow and may impose security considerations.", 0) \ \ M(UInt64, max_partitions_per_insert_block, 100, "Limit maximum number of partitions in single INSERTed block. Zero means unlimited. Throw exception if the block contains too many partitions. This setting is a safety threshold, because using large number of partitions is a common misconception.", 0) \ + M(UInt64, max_partitions_to_read, 0, "Limit the max number of partitions that can be accessed in one query. 0 means unlimited.", 0) \ M(Bool, check_query_single_value_result, true, "Return check query result as single 1/0 value", 0) \ M(Bool, allow_drop_detached, false, "Allow ALTER TABLE ... DROP DETACHED PART[ITION] ... queries", 0) \ \ diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index ada216b38b9..75760e145d3 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -707,8 +707,9 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( if (parts_with_ranges.empty()) return std::make_unique(); - auto max_partitions_to_read = data.getSettings()->max_partitions_to_read; - if (settings.force_max_partition_limit && max_partitions_to_read) + auto max_partitions_to_read + = settings.max_partitions_to_read.changed ? settings.max_partitions_to_read : data.getSettings()->max_partitions_to_read; + if (max_partitions_to_read) { std::set partitions; for (auto & part_with_ranges : parts_with_ranges) diff --git a/tests/queries/0_stateless/01632_max_partitions_to_read.reference b/tests/queries/0_stateless/01632_max_partitions_to_read.reference index e24dfbd1231..ea2526e1301 100644 --- a/tests/queries/0_stateless/01632_max_partitions_to_read.reference +++ b/tests/queries/0_stateless/01632_max_partitions_to_read.reference @@ -1,6 +1,6 @@ -2021-01-03 1 2 -2021-01-04 3 4 -2021-01-03 1 2 -2021-01-04 3 4 -2021-01-03 1 2 -2021-01-04 3 4 +2021-01-01 1 2 +2021-01-02 4 5 +2021-01-01 1 2 +2021-01-02 4 5 +2021-01-01 1 2 +2021-01-02 4 5 diff --git a/tests/queries/0_stateless/01632_max_partitions_to_read.sql b/tests/queries/0_stateless/01632_max_partitions_to_read.sql index 8e0d2ba0ed6..b91405569bc 100644 --- a/tests/queries/0_stateless/01632_max_partitions_to_read.sql +++ b/tests/queries/0_stateless/01632_max_partitions_to_read.sql @@ -2,16 +2,16 @@ drop table if exists p; create table p(d Date, i int, j int) engine MergeTree partition by d order by i settings max_partitions_to_read = 1; -insert into p values (yesterday(), 1, 2), (today(), 3, 4); +insert into p values ('2021-01-01', 1, 2), ('2021-01-02', 4, 5); -select * from p order by i; -- default no limit +select * from p order by i; -- { serverError 565 } -select * from p order by i settings force_max_partition_limit = 0; +select * from p order by i settings max_partitions_to_read = 2; -select * from p order by i settings force_max_partition_limit = 1; -- { serverError 565 } +select * from p order by i settings max_partitions_to_read = 0; -- unlimited alter table p modify setting max_partitions_to_read = 2; -select * from p order by i settings force_max_partition_limit = 1; +select * from p order by i; drop table if exists p; From b2e4c0ee718b7840cb646c4ca0891f0e5e65799b Mon Sep 17 00:00:00 2001 From: weeds085490 <917923572@qq.com> Date: Wed, 6 Jan 2021 19:29:51 +0800 Subject: [PATCH 132/256] refine test --- .../01600_count_of_parts_metrics.reference | 1 + .../01600_count_of_parts_metrics.sh | 45 ++++++++++++++++ .../01600_count_of_parts_metrics.sql | 54 ------------------- 3 files changed, 46 insertions(+), 54 deletions(-) create mode 100755 tests/queries/0_stateless/01600_count_of_parts_metrics.sh delete mode 100644 tests/queries/0_stateless/01600_count_of_parts_metrics.sql diff --git a/tests/queries/0_stateless/01600_count_of_parts_metrics.reference b/tests/queries/0_stateless/01600_count_of_parts_metrics.reference index e8183f05f5d..98fb6a68656 100644 --- a/tests/queries/0_stateless/01600_count_of_parts_metrics.reference +++ b/tests/queries/0_stateless/01600_count_of_parts_metrics.reference @@ -1,3 +1,4 @@ 1 1 1 +1 diff --git a/tests/queries/0_stateless/01600_count_of_parts_metrics.sh b/tests/queries/0_stateless/01600_count_of_parts_metrics.sh new file mode 100755 index 00000000000..f55ee58ad15 --- /dev/null +++ b/tests/queries/0_stateless/01600_count_of_parts_metrics.sh @@ -0,0 +1,45 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +verify_sql="SELECT COUNT(1) +FROM +(SELECT + SUM(IF(metric = 'Parts', value, 0)) AS Parts, + SUM(IF(metric = 'PartsActive', value, 0)) AS PartsActive, + SUM(IF(metric = 'PartsInactive', value, 0)) AS PartsInactive +FROM system.metrics) as a INNER JOIN +(SELECT + toInt64(SUM(1)) AS Parts, + toInt64(SUM(IF(active = 1, 1, 0))) AS PartsActive, + toInt64(SUM(IF(active = 0, 1, 0))) AS PartsInactive +FROM system.parts +) as b USING (Parts,PartsActive,PartsInactive)" + +verify(){ +for i in $(seq 1 10) +do +result=$( $CLICKHOUSE_CLIENT -m --query="$verify_sql" ) +if [ "$result" = "1" ];then + echo 1 + return +fi +done +echo 0 +} + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test_table" 2>/dev/null +$CLICKHOUSE_CLIENT --query="CREATE TABLE test_table(data Date) ENGINE = MergeTree PARTITION BY toYear(data) ORDER BY data;" 2>/dev/null + +$CLICKHOUSE_CLIENT --query="INSERT INTO test_table VALUES ('1992-01-01')" 2>/dev/null +verify + +$CLICKHOUSE_CLIENT --query="INSERT INTO test_table VALUES ('1992-01-02')" 2>/dev/null +verify + +$CLICKHOUSE_CLIENT --query="OPTIMIZE TABLE test_table FINAL" 2>/dev/null +verify + +$CLICKHOUSE_CLIENT --query="DROP TABLE test_table" 2>/dev/null +verify diff --git a/tests/queries/0_stateless/01600_count_of_parts_metrics.sql b/tests/queries/0_stateless/01600_count_of_parts_metrics.sql deleted file mode 100644 index 31c8f75f3dc..00000000000 --- a/tests/queries/0_stateless/01600_count_of_parts_metrics.sql +++ /dev/null @@ -1,54 +0,0 @@ - -DROP TABLE IF EXISTS test_table; -CREATE TABLE test_table(data Date) ENGINE = MergeTree PARTITION BY toYear(data) ORDER BY data; - -INSERT INTO test_table VALUES ('1992-01-01'); - -SELECT COUNT(1) -FROM -(SELECT - SUM(IF(metric = 'Parts', value, 0)) AS Parts, - SUM(IF(metric = 'PartsActive', value, 0)) AS PartsActive, - SUM(IF(metric = 'PartsInactive', value, 0)) AS PartsInactive -FROM system.metrics) as a INNER JOIN -(SELECT - toInt64(SUM(1)) AS Parts, - toInt64(SUM(IF(active = 1, 1, 0))) AS PartsActive, - toInt64(SUM(IF(active = 0, 1, 0))) AS PartsInactive -FROM system.parts -) as b USING (Parts,PartsActive,PartsInactive); - -INSERT INTO test_table VALUES ('1992-01-02'); - -SELECT COUNT(1) -FROM -(SELECT - SUM(IF(metric = 'Parts', value, 0)) AS Parts, - SUM(IF(metric = 'PartsActive', value, 0)) AS PartsActive, - SUM(IF(metric = 'PartsInactive', value, 0)) AS PartsInactive -FROM system.metrics) as a INNER JOIN -(SELECT - toInt64(SUM(1)) AS Parts, - toInt64(SUM(IF(active = 1, 1, 0))) AS PartsActive, - toInt64(SUM(IF(active = 0, 1, 0))) AS PartsInactive -FROM system.parts -) as b USING (Parts,PartsActive,PartsInactive); - -OPTIMIZE TABLE test_table FINAL; - - -SELECT COUNT(1) -FROM -(SELECT - SUM(IF(metric = 'Parts', value, 0)) AS Parts, - SUM(IF(metric = 'PartsActive', value, 0)) AS PartsActive, - SUM(IF(metric = 'PartsInactive', value, 0)) AS PartsInactive -FROM system.metrics) as a INNER JOIN -(SELECT - toInt64(SUM(1)) AS Parts, - toInt64(SUM(IF(active = 1, 1, 0))) AS PartsActive, - toInt64(SUM(IF(active = 0, 1, 0))) AS PartsInactive -FROM system.parts -) as b USING (Parts,PartsActive,PartsInactive); - -DROP TABLE test_table; From a32f6dcf1b914e4e769902bf3b357595d66b0d10 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 6 Jan 2021 20:05:19 +0800 Subject: [PATCH 133/256] Apply alter settings on cluster to all replicas --- src/Interpreters/DDLWorker.cpp | 7 +++ src/Parsers/ASTAlterQuery.cpp | 16 ++++++ src/Parsers/ASTAlterQuery.h | 2 + .../__init__.py | 0 .../configs/config.d/clusters.xml | 17 ++++++ .../configs/config.d/distributed_ddl.xml | 5 ++ .../test_alter_settings_on_cluster/test.py | 53 +++++++++++++++++++ 7 files changed, 100 insertions(+) create mode 100644 tests/integration/test_alter_settings_on_cluster/__init__.py create mode 100644 tests/integration/test_alter_settings_on_cluster/configs/config.d/clusters.xml create mode 100644 tests/integration/test_alter_settings_on_cluster/configs/config.d/distributed_ddl.xml create mode 100644 tests/integration/test_alter_settings_on_cluster/test.py diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index cce62b1a6c4..e0b5591a708 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -847,6 +847,13 @@ bool DDLWorker::taskShouldBeExecutedOnLeader(const ASTPtr ast_ddl, const Storage if (!ast_ddl->as() && !ast_ddl->as() && !ast_ddl->as()) return false; + if (auto * alter = ast_ddl->as()) + { + // Setting alters should be executed on all replicas + if (alter->isSettingsAlter()) + return false; + } + return storage->supportsReplication(); } diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index b997164684b..8a44dcc7c3b 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -344,6 +344,22 @@ void ASTAlterCommand::formatImpl( throw Exception("Unexpected type of ALTER", ErrorCodes::UNEXPECTED_AST_STRUCTURE); } +bool ASTAlterQuery::isSettingsAlter() const +{ + if (command_list) + { + if (command_list->children.empty()) + return false; + for (const auto & child : command_list->children) + { + const auto & command = child->as(); + if (command.type != ASTAlterCommand::MODIFY_SETTING) + return false; + } + return true; + } + return false; +} /** Get the text that identifies this element. */ String ASTAlterQuery::getID(char delim) const diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index 91c80867738..f53a987905e 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -187,6 +187,8 @@ public: ASTExpressionList * command_list = nullptr; + bool isSettingsAlter() const; + String getID(char) const override; ASTPtr clone() const override; diff --git a/tests/integration/test_alter_settings_on_cluster/__init__.py b/tests/integration/test_alter_settings_on_cluster/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_alter_settings_on_cluster/configs/config.d/clusters.xml b/tests/integration/test_alter_settings_on_cluster/configs/config.d/clusters.xml new file mode 100644 index 00000000000..26c9caa63db --- /dev/null +++ b/tests/integration/test_alter_settings_on_cluster/configs/config.d/clusters.xml @@ -0,0 +1,17 @@ + + + + + true + + ch1 + 9000 + + + ch2 + 9000 + + + + + diff --git a/tests/integration/test_alter_settings_on_cluster/configs/config.d/distributed_ddl.xml b/tests/integration/test_alter_settings_on_cluster/configs/config.d/distributed_ddl.xml new file mode 100644 index 00000000000..6a88929c8ac --- /dev/null +++ b/tests/integration/test_alter_settings_on_cluster/configs/config.d/distributed_ddl.xml @@ -0,0 +1,5 @@ + + + /clickhouse/task_queue/ddl + + diff --git a/tests/integration/test_alter_settings_on_cluster/test.py b/tests/integration/test_alter_settings_on_cluster/test.py new file mode 100644 index 00000000000..d56ef4461aa --- /dev/null +++ b/tests/integration/test_alter_settings_on_cluster/test.py @@ -0,0 +1,53 @@ +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +ch1 = cluster.add_instance( + "ch1", + main_configs=[ + "configs/config.d/clusters.xml", + "configs/config.d/distributed_ddl.xml", + ], + with_zookeeper=True, +) +ch2 = cluster.add_instance( + "ch2", + main_configs=[ + "configs/config.d/clusters.xml", + "configs/config.d/distributed_ddl.xml", + ], + with_zookeeper=True, +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + ch1.query("CREATE DATABASE test_default_database ON CLUSTER 'cluster';") + yield cluster + + finally: + cluster.shutdown() + + +def test_default_database_on_cluster(started_cluster): + ch1.query( + database="test_default_database", + sql="CREATE TABLE test_local_table (x UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_local_table', 'r1') ORDER BY tuple();", + ) + + ch2.query( + database="test_default_database", + sql="CREATE TABLE test_local_table (x UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_local_table', 'r2') ORDER BY tuple();", + ) + + ch1.query( + database="test_default_database", + sql="ALTER TABLE test_local_table ON CLUSTER 'cluster' MODIFY SETTING old_parts_lifetime = 100;", + ) + + for node in [ch1, ch2]: + assert node.query("SHOW CREATE test_local_table FORMAT TSV").endswith( + "old_parts_lifetime = 100\n" + ) From 38bc0f8348e1578c75336912f026e68bf3ff553b Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Wed, 6 Jan 2021 08:16:01 -0500 Subject: [PATCH 134/256] Updating `verification cooldown performance` and `valid verification cooldown value ldap unavailable` tests to fix fails due to timing issues caused by slow test executions. --- .../external_user_directory/tests/authentications.py | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/tests/testflows/ldap/external_user_directory/tests/authentications.py b/tests/testflows/ldap/external_user_directory/tests/authentications.py index 8229947adf7..ced54e95115 100644 --- a/tests/testflows/ldap/external_user_directory/tests/authentications.py +++ b/tests/testflows/ldap/external_user_directory/tests/authentications.py @@ -874,7 +874,7 @@ def valid_verification_cooldown_value_ldap_unavailable(self, server, rbac=False, "enable_tls": "no", "auth_dn_prefix": "cn=", "auth_dn_suffix": ",ou=users,dc=company,dc=com", - "verification_cooldown": "2" + "verification_cooldown": "300" }} self.context.ldap_node = self.context.cluster.node(server) @@ -898,11 +898,6 @@ def valid_verification_cooldown_value_ldap_unavailable(self, server, rbac=False, with Then("when I try to login again with the server offline it should work"): login_and_execute_query(username=user["cn"], password=user["userpassword"]) - with And("when I sleep for 2 seconds and try to log in, it should fail"): - time.sleep(2) - login_and_execute_query(username=user["cn"], password=user["userpassword"], - exitcode=error_exitcode, message=error_message) - finally: with Finally("I start the ldap server back up"): self.context.ldap_node.start() @@ -957,7 +952,7 @@ def repeat_requests(self, server, iterations, vcd_value, rbac=False): RQ_SRS_009_LDAP_ExternalUserDirectory_Authentication_VerificationCooldown_Performance("1.0") ) def verification_cooldown_performance(self, server, rbac=False, iterations=5000): - """Check that login performance is better when the verification cooldown + """Check login performance when the verification cooldown parameter is set to a positive value when comparing to the case when the verification cooldown parameter is turned off. """ @@ -973,9 +968,6 @@ def verification_cooldown_performance(self, server, rbac=False, iterations=5000) no_vcd_time = repeat_requests(server=server, iterations=iterations, vcd_value="0", rbac=rbac) metric("login_with_vcd_value_0", units="seconds", value=no_vcd_time) - with Then("The performance with verification cooldown parameter set is better than the performance with no verification cooldown parameter."): - assert no_vcd_time > vcd_time, error() - with And("Log the performance improvement as a percentage."): metric("percentage_improvement", units="%", value=100*(no_vcd_time - vcd_time)/vcd_time) From 85003c8b5c2a7039999a636b891419dc1c97aeb6 Mon Sep 17 00:00:00 2001 From: Bertrand Junqua Date: Wed, 6 Jan 2021 16:36:51 +0100 Subject: [PATCH 135/256] Fix typo in array functions' documentation --- docs/en/sql-reference/functions/array-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index c4b7156ad95..ee4449d46fa 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -1308,7 +1308,7 @@ Note that the `arraySum` is a [higher-order function](../../sql-reference/functi ## arrayAvg(\[func,\] arr1, …) {#array-avg} -Returns the sum of the `func` values. If the function is omitted, it just returns the average of the array elements. +Returns the average of the `func` values. If the function is omitted, it just returns the average of the array elements. Note that the `arrayAvg` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You can pass a lambda function to it as the first argument. From 22562273612b65e73abb0b027ac4f393d52b24ab Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Wed, 6 Jan 2021 11:21:31 +0800 Subject: [PATCH 136/256] SQL compatibility: provide POSITION(needle IN haystack) syntax. --- src/Parsers/ExpressionElementParsers.cpp | 16 ++++++++++++++++ .../00233_position_function_family.reference | 13 +++++++++++++ .../00233_position_function_family.sql | 15 +++++++++++++++ 3 files changed, 44 insertions(+) diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 7c82c4aca1e..8960803d936 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -343,6 +343,22 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) throw Exception("Argument of function toDate is unquoted: toDate(" + contents_str + "), must be: toDate('" + contents_str + "')" , ErrorCodes::SYNTAX_ERROR); } + else if (Poco::toLower(getIdentifierName(identifier)) == "position") + { + /// POSITION(needle IN haystack) is equivalent to function position(haystack, needle) + const auto & list = expr_list_args->as(); + if (list.children.size() == 1) + { + const auto & in_func = list.children[0]->as(); + if (in_func.name == "in") + { + // switch the two arguments + const auto & arg_list = in_func.arguments->as(); + expr_list_args->children = {arg_list.children[1], arg_list.children[0]}; + } + + } + } /// The parametric aggregate function has two lists (parameters and arguments) in parentheses. Example: quantile(0.9)(x). if (allow_function_parameters && pos->type == TokenType::OpeningRoundBracket) diff --git a/tests/queries/0_stateless/00233_position_function_family.reference b/tests/queries/0_stateless/00233_position_function_family.reference index 1523094261f..97f5276f200 100644 --- a/tests/queries/0_stateless/00233_position_function_family.reference +++ b/tests/queries/0_stateless/00233_position_function_family.reference @@ -23796,3 +23796,16 @@ 1 1 1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/00233_position_function_family.sql b/tests/queries/0_stateless/00233_position_function_family.sql index 8e443f5cd10..21e688c8f35 100644 --- a/tests/queries/0_stateless/00233_position_function_family.sql +++ b/tests/queries/0_stateless/00233_position_function_family.sql @@ -13,6 +13,14 @@ select 1 = position('a', '', 1); select 2 = position('a', '', 2); select 0 = position('a', '', 3); +select 1 = position('' in ''); +select 1 = position('' in 'abc'); +select 0 = position('abc' in ''); +select 1 = position('abc' in 'abc'); +select 2 = position('bc' in 'abc'); +select 3 = position('c' in 'abc'); +select 6 = position('/' IN s) FROM (SELECT 'Hello/World' AS s); + select [1, 1, 2, 3, 4, 5, 0, 0, 0, 0] = groupArray(position('aaaa', '', number)) from numbers(10); select [1, 1, 2, 3, 4, 5, 0, 0, 0, 0] = groupArray(position(materialize('aaaa'), '', number)) from numbers(10); select [1, 1, 2, 3, 4, 5, 0, 0, 0, 0] = groupArray(position('aaaa', materialize(''), number)) from numbers(10); @@ -54,6 +62,13 @@ select 0 = position('abcabc', 'b', 6); select 2 = position('abcabc', 'bca', 0); select 0 = position('abcabc', 'bca', 3); +select 1 = position('' in ''); +select 1 = position('' in 'абв'); +select 0 = position('абв' in ''); +select 1 = position('абв' in 'абв'); +select 3 = position('бв' in 'абв'); +select 5 = position('в' in 'абв'); + select 1 = position(materialize(''), ''); select 1 = position(materialize('абв'), ''); select 0 = position(materialize(''), 'абв'); From 4950377fbf7f3d36a5ebcb9f5ee99de32fc52b30 Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Wed, 6 Jan 2021 15:42:23 +0800 Subject: [PATCH 137/256] code changes according to reviewer's comments --- src/Parsers/ExpressionElementParsers.cpp | 20 +++++++++++-------- .../00233_position_function_family.reference | 13 ------------ .../00233_position_function_family.sql | 15 -------------- ...sition_function_sql_comparibilty.reference | 13 ++++++++++++ ...233_position_function_sql_comparibilty.sql | 16 +++++++++++++++ 5 files changed, 41 insertions(+), 36 deletions(-) create mode 100644 tests/queries/0_stateless/00233_position_function_sql_comparibilty.reference create mode 100644 tests/queries/0_stateless/00233_position_function_sql_comparibilty.sql diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 8960803d936..ff7c4d2d53e 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -346,17 +346,21 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) else if (Poco::toLower(getIdentifierName(identifier)) == "position") { /// POSITION(needle IN haystack) is equivalent to function position(haystack, needle) - const auto & list = expr_list_args->as(); - if (list.children.size() == 1) + if (const auto * list = expr_list_args->as()) { - const auto & in_func = list.children[0]->as(); - if (in_func.name == "in") + if (list->children.size() == 1) { - // switch the two arguments - const auto & arg_list = in_func.arguments->as(); - expr_list_args->children = {arg_list.children[1], arg_list.children[0]}; + if (const auto * in_func = list->children[0]->as()) + { + if (in_func->name == "in") + { + // switch the two arguments + const auto & arg_list = in_func->arguments->as(); + if (arg_list.children.size() == 2) + expr_list_args->children = {arg_list.children[1], arg_list.children[0]}; + } + } } - } } diff --git a/tests/queries/0_stateless/00233_position_function_family.reference b/tests/queries/0_stateless/00233_position_function_family.reference index 97f5276f200..1523094261f 100644 --- a/tests/queries/0_stateless/00233_position_function_family.reference +++ b/tests/queries/0_stateless/00233_position_function_family.reference @@ -23796,16 +23796,3 @@ 1 1 1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 diff --git a/tests/queries/0_stateless/00233_position_function_family.sql b/tests/queries/0_stateless/00233_position_function_family.sql index 21e688c8f35..8e443f5cd10 100644 --- a/tests/queries/0_stateless/00233_position_function_family.sql +++ b/tests/queries/0_stateless/00233_position_function_family.sql @@ -13,14 +13,6 @@ select 1 = position('a', '', 1); select 2 = position('a', '', 2); select 0 = position('a', '', 3); -select 1 = position('' in ''); -select 1 = position('' in 'abc'); -select 0 = position('abc' in ''); -select 1 = position('abc' in 'abc'); -select 2 = position('bc' in 'abc'); -select 3 = position('c' in 'abc'); -select 6 = position('/' IN s) FROM (SELECT 'Hello/World' AS s); - select [1, 1, 2, 3, 4, 5, 0, 0, 0, 0] = groupArray(position('aaaa', '', number)) from numbers(10); select [1, 1, 2, 3, 4, 5, 0, 0, 0, 0] = groupArray(position(materialize('aaaa'), '', number)) from numbers(10); select [1, 1, 2, 3, 4, 5, 0, 0, 0, 0] = groupArray(position('aaaa', materialize(''), number)) from numbers(10); @@ -62,13 +54,6 @@ select 0 = position('abcabc', 'b', 6); select 2 = position('abcabc', 'bca', 0); select 0 = position('abcabc', 'bca', 3); -select 1 = position('' in ''); -select 1 = position('' in 'абв'); -select 0 = position('абв' in ''); -select 1 = position('абв' in 'абв'); -select 3 = position('бв' in 'абв'); -select 5 = position('в' in 'абв'); - select 1 = position(materialize(''), ''); select 1 = position(materialize('абв'), ''); select 0 = position(materialize(''), 'абв'); diff --git a/tests/queries/0_stateless/00233_position_function_sql_comparibilty.reference b/tests/queries/0_stateless/00233_position_function_sql_comparibilty.reference new file mode 100644 index 00000000000..71c9a23879f --- /dev/null +++ b/tests/queries/0_stateless/00233_position_function_sql_comparibilty.reference @@ -0,0 +1,13 @@ +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/00233_position_function_sql_comparibilty.sql b/tests/queries/0_stateless/00233_position_function_sql_comparibilty.sql new file mode 100644 index 00000000000..53b08de296a --- /dev/null +++ b/tests/queries/0_stateless/00233_position_function_sql_comparibilty.sql @@ -0,0 +1,16 @@ +SET send_logs_level = 'fatal'; +select 1 = position('' in ''); +select 1 = position('' in 'abc'); +select 0 = position('abc' in ''); +select 1 = position('abc' in 'abc'); +select 2 = position('bc' in 'abc'); +select 3 = position('c' in 'abc'); + +select 1 = position('' in ''); +select 1 = position('' in 'абв'); +select 0 = position('абв' in ''); +select 1 = position('абв' in 'абв'); +select 3 = position('бв' in 'абв'); +select 5 = position('в' in 'абв'); + +select 6 = position('/' IN s) FROM (SELECT 'Hello/World' AS s); \ No newline at end of file From 8ab40d2747435a42c58f5fae8869a4c4491ecc2e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 6 Jan 2021 21:06:34 +0300 Subject: [PATCH 138/256] Newline --- .../0_stateless/00233_position_function_sql_comparibilty.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00233_position_function_sql_comparibilty.sql b/tests/queries/0_stateless/00233_position_function_sql_comparibilty.sql index 53b08de296a..ae9409cd0c0 100644 --- a/tests/queries/0_stateless/00233_position_function_sql_comparibilty.sql +++ b/tests/queries/0_stateless/00233_position_function_sql_comparibilty.sql @@ -13,4 +13,4 @@ select 1 = position('абв' in 'абв'); select 3 = position('бв' in 'абв'); select 5 = position('в' in 'абв'); -select 6 = position('/' IN s) FROM (SELECT 'Hello/World' AS s); \ No newline at end of file +select 6 = position('/' IN s) FROM (SELECT 'Hello/World' AS s); From 5b9876765229f744ddb5f401861a419bb6baa331 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 6 Jan 2021 21:29:45 +0300 Subject: [PATCH 139/256] Update MergeTreeSettings.h --- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 98bce556fa7..97c6114a0be 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -110,7 +110,7 @@ struct Settings; M(Bool, allow_nullable_key, false, "Allow Nullable types as primary keys.", 0) \ M(Bool, remove_empty_parts, true, "Remove empty parts after they were pruned by TTL, mutation, or collapsing merge algorithm", 0) \ M(Bool, assign_part_uuids, false, "Generate UUIDs for parts. Before enabling check that all replicas support new format.", 0) \ - M(UInt64, max_partitions_to_read, 0, "Limit the max number of partitions that can be accessed in one query. 0 means unlimited.", 0) \ + M(UInt64, max_partitions_to_read, 0, "Limit the max number of partitions that can be accessed in one query. 0 means unlimited. This setting is the default that can be overridden by the query-level setting with the same name.", 0) \ \ /** Obsolete settings. Kept for backward compatibility only. */ \ M(UInt64, min_relative_delay_to_yield_leadership, 120, "Obsolete setting, does nothing.", 0) \ From 6261d4135cab3cb0c368dad1293ba3cd8c1b153e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 6 Jan 2021 22:23:49 +0300 Subject: [PATCH 140/256] Add sanity checks for Sim/Min hash arguments Fixes: #18799 Fixes: #18524 --- src/Functions/FunctionsStringHash.h | 13 ++++++++++--- tests/queries/0_stateless/01016_simhash_minhash.sql | 4 ++++ 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/src/Functions/FunctionsStringHash.h b/src/Functions/FunctionsStringHash.h index c09abc33319..907625ebaf5 100644 --- a/src/Functions/FunctionsStringHash.h +++ b/src/Functions/FunctionsStringHash.h @@ -18,6 +18,7 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; + extern const int ARGUMENT_OUT_OF_BOUND; } // FunctionStringHash @@ -30,6 +31,8 @@ public: static constexpr auto name = Name::name; static constexpr size_t default_shingle_size = 3; static constexpr size_t default_num_hashes = 6; + static constexpr size_t max_shingle_size = 1000; + static constexpr size_t max_num_hashes = 1000; static FunctionPtr create(const Context &) { return std::make_shared(); } @@ -100,10 +103,14 @@ public: } if (shingle_size == 0) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument (shingle size) of function {} cannot be zero", getName()); - + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Second argument (shingle size) of function {} cannot be zero", getName()); if (num_hashes == 0) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Third argument (num hashes) of function {} cannot be zero", getName()); + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Third argument (num hashes) of function {} cannot be zero", getName()); + + if (shingle_size > max_shingle_size) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Second argument (shingle size) of function {} cannot be greater then {}", getName(), max_shingle_size); + if (num_hashes > max_num_hashes) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Third argument (num hashes) of function {} cannot be greater then {}", getName(), max_num_hashes); auto type = std::make_shared(); if constexpr (is_simhash) diff --git a/tests/queries/0_stateless/01016_simhash_minhash.sql b/tests/queries/0_stateless/01016_simhash_minhash.sql index 225bdd2bdc6..01af9451381 100644 --- a/tests/queries/0_stateless/01016_simhash_minhash.sql +++ b/tests/queries/0_stateless/01016_simhash_minhash.sql @@ -108,4 +108,8 @@ SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), wordShingleMinH SELECT 'wordShingleMinHashCaseInsensitiveUTF8'; SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), wordShingleMinHashCaseInsensitiveUTF8(s, 2, 3) as h FROM defaults GROUP BY h; +SELECT wordShingleSimHash('foobar', 9223372036854775807); -- { serverError 69 } +SELECT wordShingleSimHash('foobar', 1001); -- { serverError 69 } +SELECT wordShingleSimHash('foobar', 0); -- { serverError 69 } + DROP TABLE defaults; From 8e0cbea7387249d303d782a371b59e419d429295 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 6 Jan 2021 22:49:48 +0300 Subject: [PATCH 141/256] Update brown-benchmark.md --- docs/en/getting-started/example-datasets/brown-benchmark.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/getting-started/example-datasets/brown-benchmark.md b/docs/en/getting-started/example-datasets/brown-benchmark.md index 80dbd0c1b6a..24b8f9e8eb0 100644 --- a/docs/en/getting-started/example-datasets/brown-benchmark.md +++ b/docs/en/getting-started/example-datasets/brown-benchmark.md @@ -410,3 +410,5 @@ GROUP BY yr, ORDER BY yr, mo; ``` + +The data is also available for interactive queries in the [Playground](https://gh-api.clickhouse.tech/play?user=play), [example](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUIG1hY2hpbmVfbmFtZSwKICAgICAgIE1JTihjcHUpIEFTIGNwdV9taW4sCiAgICAgICBNQVgoY3B1KSBBUyBjcHVfbWF4LAogICAgICAgQVZHKGNwdSkgQVMgY3B1X2F2ZywKICAgICAgIE1JTihuZXRfaW4pIEFTIG5ldF9pbl9taW4sCiAgICAgICBNQVgobmV0X2luKSBBUyBuZXRfaW5fbWF4LAogICAgICAgQVZHKG5ldF9pbikgQVMgbmV0X2luX2F2ZywKICAgICAgIE1JTihuZXRfb3V0KSBBUyBuZXRfb3V0X21pbiwKICAgICAgIE1BWChuZXRfb3V0KSBBUyBuZXRfb3V0X21heCwKICAgICAgIEFWRyhuZXRfb3V0KSBBUyBuZXRfb3V0X2F2ZwpGUk9NICgKICBTRUxFQ1QgbWFjaGluZV9uYW1lLAogICAgICAgICBDT0FMRVNDRShjcHVfdXNlciwgMC4wKSBBUyBjcHUsCiAgICAgICAgIENPQUxFU0NFKGJ5dGVzX2luLCAwLjApIEFTIG5ldF9pbiwKICAgICAgICAgQ09BTEVTQ0UoYnl0ZXNfb3V0LCAwLjApIEFTIG5ldF9vdXQKICBGUk9NIG1nYmVuY2gubG9nczEKICBXSEVSRSBtYWNoaW5lX25hbWUgSU4gKCdhbmFuc2knLCdhcmFnb2cnLCd1cmQnKQogICAgQU5EIGxvZ190aW1lID49IFRJTUVTVEFNUCAnMjAxNy0wMS0xMSAwMDowMDowMCcKKSBBUyByCkdST1VQIEJZIG1hY2hpbmVfbmFtZQ==) From 058cdcc62c2e617a0198310ca54b4cb44e55f0ae Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 6 Jan 2021 22:50:08 +0300 Subject: [PATCH 142/256] Update brown-benchmark.md --- docs/en/getting-started/example-datasets/brown-benchmark.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting-started/example-datasets/brown-benchmark.md b/docs/en/getting-started/example-datasets/brown-benchmark.md index 24b8f9e8eb0..b5ca23eddb9 100644 --- a/docs/en/getting-started/example-datasets/brown-benchmark.md +++ b/docs/en/getting-started/example-datasets/brown-benchmark.md @@ -411,4 +411,4 @@ ORDER BY yr, mo; ``` -The data is also available for interactive queries in the [Playground](https://gh-api.clickhouse.tech/play?user=play), [example](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUIG1hY2hpbmVfbmFtZSwKICAgICAgIE1JTihjcHUpIEFTIGNwdV9taW4sCiAgICAgICBNQVgoY3B1KSBBUyBjcHVfbWF4LAogICAgICAgQVZHKGNwdSkgQVMgY3B1X2F2ZywKICAgICAgIE1JTihuZXRfaW4pIEFTIG5ldF9pbl9taW4sCiAgICAgICBNQVgobmV0X2luKSBBUyBuZXRfaW5fbWF4LAogICAgICAgQVZHKG5ldF9pbikgQVMgbmV0X2luX2F2ZywKICAgICAgIE1JTihuZXRfb3V0KSBBUyBuZXRfb3V0X21pbiwKICAgICAgIE1BWChuZXRfb3V0KSBBUyBuZXRfb3V0X21heCwKICAgICAgIEFWRyhuZXRfb3V0KSBBUyBuZXRfb3V0X2F2ZwpGUk9NICgKICBTRUxFQ1QgbWFjaGluZV9uYW1lLAogICAgICAgICBDT0FMRVNDRShjcHVfdXNlciwgMC4wKSBBUyBjcHUsCiAgICAgICAgIENPQUxFU0NFKGJ5dGVzX2luLCAwLjApIEFTIG5ldF9pbiwKICAgICAgICAgQ09BTEVTQ0UoYnl0ZXNfb3V0LCAwLjApIEFTIG5ldF9vdXQKICBGUk9NIG1nYmVuY2gubG9nczEKICBXSEVSRSBtYWNoaW5lX25hbWUgSU4gKCdhbmFuc2knLCdhcmFnb2cnLCd1cmQnKQogICAgQU5EIGxvZ190aW1lID49IFRJTUVTVEFNUCAnMjAxNy0wMS0xMSAwMDowMDowMCcKKSBBUyByCkdST1VQIEJZIG1hY2hpbmVfbmFtZQ==) +The data is also available for interactive queries in the [Playground](https://gh-api.clickhouse.tech/play?user=play), [example](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUIG1hY2hpbmVfbmFtZSwKICAgICAgIE1JTihjcHUpIEFTIGNwdV9taW4sCiAgICAgICBNQVgoY3B1KSBBUyBjcHVfbWF4LAogICAgICAgQVZHKGNwdSkgQVMgY3B1X2F2ZywKICAgICAgIE1JTihuZXRfaW4pIEFTIG5ldF9pbl9taW4sCiAgICAgICBNQVgobmV0X2luKSBBUyBuZXRfaW5fbWF4LAogICAgICAgQVZHKG5ldF9pbikgQVMgbmV0X2luX2F2ZywKICAgICAgIE1JTihuZXRfb3V0KSBBUyBuZXRfb3V0X21pbiwKICAgICAgIE1BWChuZXRfb3V0KSBBUyBuZXRfb3V0X21heCwKICAgICAgIEFWRyhuZXRfb3V0KSBBUyBuZXRfb3V0X2F2ZwpGUk9NICgKICBTRUxFQ1QgbWFjaGluZV9uYW1lLAogICAgICAgICBDT0FMRVNDRShjcHVfdXNlciwgMC4wKSBBUyBjcHUsCiAgICAgICAgIENPQUxFU0NFKGJ5dGVzX2luLCAwLjApIEFTIG5ldF9pbiwKICAgICAgICAgQ09BTEVTQ0UoYnl0ZXNfb3V0LCAwLjApIEFTIG5ldF9vdXQKICBGUk9NIG1nYmVuY2gubG9nczEKICBXSEVSRSBtYWNoaW5lX25hbWUgSU4gKCdhbmFuc2knLCdhcmFnb2cnLCd1cmQnKQogICAgQU5EIGxvZ190aW1lID49IFRJTUVTVEFNUCAnMjAxNy0wMS0xMSAwMDowMDowMCcKKSBBUyByCkdST1VQIEJZIG1hY2hpbmVfbmFtZQ==). From af0a359ceab7dc48ce8257dfb0a8b087290753b9 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 6 Jan 2021 22:53:23 +0300 Subject: [PATCH 143/256] Update metrica.md --- docs/en/getting-started/example-datasets/metrica.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting-started/example-datasets/metrica.md b/docs/en/getting-started/example-datasets/metrica.md index cdbb9b56eeb..159c99b15a0 100644 --- a/docs/en/getting-started/example-datasets/metrica.md +++ b/docs/en/getting-started/example-datasets/metrica.md @@ -71,4 +71,4 @@ clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1" [ClickHouse tutorial](../../getting-started/tutorial.md) is based on Yandex.Metrica dataset and the recommended way to get started with this dataset is to just go through tutorial. -Additional examples of queries to these tables can be found among [stateful tests](https://github.com/ClickHouse/ClickHouse/tree/master/tests/queries/1_stateful) of ClickHouse (they are named `test.hists` and `test.visits` there). +Additional examples of queries to these tables can be found among [stateful tests](https://github.com/ClickHouse/ClickHouse/tree/master/tests/queries/1_stateful) of ClickHouse (they are named `test.hits` and `test.visits` there). From f35edecebbd1427555dbb41e17606cab255eb0c2 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 6 Jan 2021 23:27:00 +0300 Subject: [PATCH 144/256] Update ontime.md --- docs/en/getting-started/example-datasets/ontime.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/getting-started/example-datasets/ontime.md b/docs/en/getting-started/example-datasets/ontime.md index 5e499cafb2a..6e46cddba52 100644 --- a/docs/en/getting-started/example-datasets/ontime.md +++ b/docs/en/getting-started/example-datasets/ontime.md @@ -398,6 +398,8 @@ ORDER BY c DESC LIMIT 10; ``` +You can also play with the data in Playground, [example](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUIERheU9mV2VlaywgY291bnQoKikgQVMgYwpGUk9NIG9udGltZQpXSEVSRSBZZWFyPj0yMDAwIEFORCBZZWFyPD0yMDA4CkdST1VQIEJZIERheU9mV2VlawpPUkRFUiBCWSBjIERFU0M7Cg==). + This performance test was created by Vadim Tkachenko. See: - https://www.percona.com/blog/2009/10/02/analyzing-air-traffic-performance-with-infobright-and-monetdb/ From 7e1d5b038723f54bb3d29ab5af5fac9da2e180a4 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 7 Jan 2021 00:10:54 +0300 Subject: [PATCH 145/256] Update FunctionsStringHash.h --- src/Functions/FunctionsStringHash.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionsStringHash.h b/src/Functions/FunctionsStringHash.h index 907625ebaf5..72fb6e4c76d 100644 --- a/src/Functions/FunctionsStringHash.h +++ b/src/Functions/FunctionsStringHash.h @@ -31,8 +31,8 @@ public: static constexpr auto name = Name::name; static constexpr size_t default_shingle_size = 3; static constexpr size_t default_num_hashes = 6; - static constexpr size_t max_shingle_size = 1000; - static constexpr size_t max_num_hashes = 1000; + static constexpr size_t max_shingle_size = 25; + static constexpr size_t max_num_hashes = 25; static FunctionPtr create(const Context &) { return std::make_shared(); } From fdcfacda6033d8d1fe5236a7bf4218e4d2e7d3ee Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 7 Jan 2021 01:55:01 +0300 Subject: [PATCH 146/256] Fix *If combinator with Nullable types sumIf(Nullable()) and similar unary functions (unary w/o If combinator) was working incorrectly, since it returns "sum" from the getName() helper, and so distributed query processing fails. The problem is in the optimization in AggregateFunctionIfNullUnary::add() for the unary functions. It pass only one column to write result to, instead of all passed arguments + result columns. While AggregateFunctionIf::add() assumes that it accepts arguments + result columns, and use last column as a result. Introduced-in: #16610 Fixes: #18210 --- .../AggregateFunctionIf.cpp | 30 +++++++++++++++---- .../01642_if_nullable_regression.reference | 5 ++++ .../01642_if_nullable_regression.sql | 7 +++++ 3 files changed, 36 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/01642_if_nullable_regression.reference create mode 100644 tests/queries/0_stateless/01642_if_nullable_regression.sql diff --git a/src/AggregateFunctions/AggregateFunctionIf.cpp b/src/AggregateFunctions/AggregateFunctionIf.cpp index 276abb90920..5e7e3844956 100644 --- a/src/AggregateFunctions/AggregateFunctionIf.cpp +++ b/src/AggregateFunctions/AggregateFunctionIf.cpp @@ -53,17 +53,35 @@ class AggregateFunctionIfNullUnary final private: size_t num_arguments; + /// The name of the nested function, including combinators (i.e. *If) + /// + /// getName() from the nested_function cannot be used because in case of *If combinator + /// with Nullable argument nested_function will point to the function w/o combinator. + /// (I.e. sumIf(Nullable, 1) -> sum()), and distributed query processing will fail. + /// + /// And nested_function cannot point to the function with *If since + /// due to optimization in the add() which pass only one column with the result, + /// and so AggregateFunctionIf::add() cannot be called this way + /// (it write to the last argument -- num_arguments-1). + /// + /// And to avoid extra level of indirection, the name of function is cached: + /// + /// AggregateFunctionIfNullUnary::add -> [ AggregateFunctionIf::add -> ] AggregateFunctionSum::add + String name; + using Base = AggregateFunctionNullBase>; public: String getName() const override { - return Base::getName(); + return name; } - AggregateFunctionIfNullUnary(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) - : Base(std::move(nested_function_), arguments, params), num_arguments(arguments.size()) + AggregateFunctionIfNullUnary(const String & name_, AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) + : Base(std::move(nested_function_), arguments, params) + , num_arguments(arguments.size()) + , name(name_) { if (num_arguments == 0) throw Exception("Aggregate function " + getName() + " require at least one argument", @@ -174,14 +192,14 @@ AggregateFunctionPtr AggregateFunctionIf::getOwnNullAdapter( { if (return_type_is_nullable) { - return std::make_shared>(nested_func, arguments, params); + return std::make_shared>(nested_function->getName(), nested_func, arguments, params); } else { if (serialize_flag) - return std::make_shared>(nested_func, arguments, params); + return std::make_shared>(nested_function->getName(), nested_func, arguments, params); else - return std::make_shared>(nested_func, arguments, params); + return std::make_shared>(nested_function->getName(), nested_func, arguments, params); } } else diff --git a/tests/queries/0_stateless/01642_if_nullable_regression.reference b/tests/queries/0_stateless/01642_if_nullable_regression.reference new file mode 100644 index 00000000000..66430601da2 --- /dev/null +++ b/tests/queries/0_stateless/01642_if_nullable_regression.reference @@ -0,0 +1,5 @@ +\N +\N +\N +0 +90 diff --git a/tests/queries/0_stateless/01642_if_nullable_regression.sql b/tests/queries/0_stateless/01642_if_nullable_regression.sql new file mode 100644 index 00000000000..9b307cf667a --- /dev/null +++ b/tests/queries/0_stateless/01642_if_nullable_regression.sql @@ -0,0 +1,7 @@ +SELECT sumIf(dummy, dummy) FROM remote('127.0.0.{1,2}', view(SELECT cast(Null AS Nullable(UInt8)) AS dummy FROM system.one)); +SELECT sumIf(dummy, 1) FROM remote('127.0.0.{1,2}', view(SELECT cast(Null AS Nullable(UInt8)) AS dummy FROM system.one)); +-- Before #16610 it returns 0 while with this patch it will return NULL +SELECT sumIf(dummy, dummy) FROM remote('127.0.0.{1,2}', view(SELECT cast(dummy AS Nullable(UInt8)) AS dummy FROM system.one)); +SELECT sumIf(dummy, 1) FROM remote('127.0.0.{1,2}', view(SELECT cast(dummy AS Nullable(UInt8)) AS dummy FROM system.one)); + +SELECT sumIf(n, 1) FROM remote('127.0.0.{1,2}', view(SELECT cast(* AS Nullable(UInt8)) AS n FROM system.numbers limit 10)) From d99133118dd5943f53824ab75d8d9c5f8cbef1ab Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Jan 2021 04:03:53 +0300 Subject: [PATCH 147/256] Generate build id when ClickHouse is linked with lld --- CMakeLists.txt | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/CMakeLists.txt b/CMakeLists.txt index 7716fe82677..e77e3b3feb6 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -214,6 +214,12 @@ if (NOT CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE") endif () endif() +# Create BuildID when using lld. For other linkers it is created by default. +if (LINKER_NAME MATCHES "lld$") + # SHA1 is not cryptographically secure but it is the best what lld is offering. + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--build-id=sha1") +endif () + cmake_host_system_information(RESULT AVAILABLE_PHYSICAL_MEMORY QUERY AVAILABLE_PHYSICAL_MEMORY) # Not available under freebsd From 6f75901ffbde003b63051384c528f273b66f286a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Jan 2021 04:09:18 +0300 Subject: [PATCH 148/256] Update test --- tests/queries/0_stateless/01326_build_id.reference | 2 +- tests/queries/0_stateless/01326_build_id.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01326_build_id.reference b/tests/queries/0_stateless/01326_build_id.reference index b6a7d89c68e..d00491fd7e5 100644 --- a/tests/queries/0_stateless/01326_build_id.reference +++ b/tests/queries/0_stateless/01326_build_id.reference @@ -1 +1 @@ -16 +1 diff --git a/tests/queries/0_stateless/01326_build_id.sql b/tests/queries/0_stateless/01326_build_id.sql index f9bf2731374..4451ec360aa 100644 --- a/tests/queries/0_stateless/01326_build_id.sql +++ b/tests/queries/0_stateless/01326_build_id.sql @@ -1 +1 @@ -SELECT length(buildId()); +SELECT length(buildId()) >= 16; From 6f481d75129dfd2c573d865cf04cdad14a35b4dd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Jan 2021 05:56:57 +0300 Subject: [PATCH 149/256] Add integrity checks for ClickHouse binary --- CMakeLists.txt | 6 ++++++ base/daemon/BaseDaemon.cpp | 42 +++++++++++++++++++++++++++++++++++++ base/daemon/BaseDaemon.h | 4 ++++ programs/CMakeLists.txt | 4 ++++ programs/main.cpp | 10 +++++++++ programs/server/Server.cpp | 43 +++++++++++++++++++++++++++++++++++++- src/Common/Elf.cpp | 9 ++++++++ src/Common/Elf.h | 3 +++ 8 files changed, 120 insertions(+), 1 deletion(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index e77e3b3feb6..95f95464857 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -220,6 +220,12 @@ if (LINKER_NAME MATCHES "lld$") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--build-id=sha1") endif () +# Add a section with the hash of the compiled machine code for integrity checks. +# Only for official builds, because adding a section can be time consuming (rewrite of several GB). +if (OBJCOPY_PATH AND YANDEX_OFFICIAL_BUILD) + set (USE_BINARY_HASH 1) +endif () + cmake_host_system_information(RESULT AVAILABLE_PHYSICAL_MEMORY QUERY AVAILABLE_PHYSICAL_MEMORY) # Not available under freebsd diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index 1224d9171ea..2c20eccf914 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -56,6 +56,9 @@ #include #include #include +#include +#include +#include #if !defined(ARCADIA_BUILD) # include @@ -340,6 +343,32 @@ private: /// Write symbolized stack trace line by line for better grep-ability. stack_trace.toStringEveryLine([&](const std::string & s) { LOG_FATAL(log, s); }); +#if defined(__linux__) + /// Write information about binary checksum. It can be difficult to calculate, so do it only after printing stack trace. + String calculated_binary_hash = getHashOfLoadedBinaryHex(); + if (daemon.stored_binary_hash.empty()) + { + LOG_FATAL(log, "Calculated checksum of the binary: {}." + " There is no information about the reference checksum.", calculated_binary_hash); + } + else if (calculated_binary_hash == daemon.stored_binary_hash) + { + LOG_FATAL(log, "Checksum of the binary: {}, integrity check passed.", calculated_binary_hash); + } + else + { + LOG_FATAL(log, "Calculated checksum of the ClickHouse binary ({0}) does not correspond" + " to the reference checksum stored in the binary ({1})." + " It may indicate one of the following:" + " - the file was changed just after startup;" + " - the file is damaged on disk due to faulty hardware;" + " - the loaded executable is damaged in memory due to faulty hardware;" + " - the file was intentionally modified;" + " - logical error in code." + , calculated_binary_hash, daemon.stored_binary_hash); + } +#endif + /// Write crash to system.crash_log table if available. if (collectCrashLog) collectCrashLog(sig, thread_num, query_id, stack_trace); @@ -799,6 +828,13 @@ void BaseDaemon::initializeTerminationAndSignalProcessing() #else build_id_info = "no build id"; #endif + +#if defined(__linux__) + std::string executable_path = getExecutablePath(); + + if (!executable_path.empty()) + stored_binary_hash = DB::Elf(executable_path).getBinaryHash(); +#endif } void BaseDaemon::logRevision() const @@ -1010,3 +1046,9 @@ void BaseDaemon::setupWatchdog() #endif } } + + +String BaseDaemon::getStoredBinaryHash() const +{ + return stored_binary_hash; +} diff --git a/base/daemon/BaseDaemon.h b/base/daemon/BaseDaemon.h index 090d4997606..3c065970504 100644 --- a/base/daemon/BaseDaemon.h +++ b/base/daemon/BaseDaemon.h @@ -121,6 +121,9 @@ public: /// argv0 is needed to change process name (consequently, it is needed for scripts involving "pgrep", "pidof" to work correctly). void shouldSetupWatchdog(char * argv0_); + /// Hash of the binary for integrity checks. + String getStoredBinaryHash() const; + protected: virtual void logRevision() const; @@ -168,6 +171,7 @@ protected: Poco::Util::AbstractConfiguration * last_configuration = nullptr; String build_id_info; + String stored_binary_hash; std::vector handled_signals; diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index a1b5467f234..9adca58b55a 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -318,6 +318,10 @@ else () if (USE_GDB_ADD_INDEX) add_custom_command(TARGET clickhouse POST_BUILD COMMAND ${GDB_ADD_INDEX_EXE} clickhouse COMMENT "Adding .gdb-index to clickhouse" VERBATIM) endif() + + if (USE_BINARY_HASH) + add_custom_command(TARGET clickhouse POST_BUILD COMMAND ./clickhouse hash-binary > hash && ${OBJCOPY_PATH} --add-section .note.ClickHouse.hash=hash clickhouse COMMENT "Adding .note.ClickHouse.hash to clickhouse" VERBATIM) + endif() endif () if (ENABLE_TESTS AND USE_GTEST) diff --git a/programs/main.cpp b/programs/main.cpp index dee02c55832..cbb22b7a87b 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -18,6 +18,7 @@ #endif #include +#include #include #include @@ -62,6 +63,14 @@ int mainEntryClickHouseStatus(int argc, char ** argv); int mainEntryClickHouseRestart(int argc, char ** argv); #endif +int mainEntryClickHouseHashBinary(int, char **) +{ + /// Intentionally without newline. So you can run: + /// objcopy --add-section .note.ClickHouse.hash=<(./clickhouse hash-binary) clickhouse + std::cout << getHashOfLoadedBinaryHex(); + return 0; +} + #define ARRAY_SIZE(a) (sizeof(a)/sizeof((a)[0])) namespace @@ -110,6 +119,7 @@ std::pair clickhouse_applications[] = {"status", mainEntryClickHouseStatus}, {"restart", mainEntryClickHouseRestart}, #endif + {"hash-binary", mainEntryClickHouseHashBinary}, }; diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 76765c0374c..63caec25099 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -65,6 +65,8 @@ #include #include #include +#include +#include #include #include #include @@ -184,6 +186,7 @@ namespace ErrorCodes extern const int FAILED_TO_GETPWUID; extern const int MISMATCHING_USERS_FOR_PROCESS_AND_DATA; extern const int NETWORK_ERROR; + extern const int CORRUPTED_DATA; } @@ -436,7 +439,45 @@ int Server::main(const std::vector & /*args*/) #if defined(OS_LINUX) std::string executable_path = getExecutablePath(); - if (executable_path.empty()) + + if (!executable_path.empty()) + { + /// Integrity check based on checksum of the executable code. + /// Note: it is not intended to protect from malicious party, + /// because the reference checksum can be easily modified as well. + /// And we don't involve asymmetric encryption with PKI yet. + /// It's only intended to protect from faulty hardware. + /// Note: it is only based on machine code. + /// But there are other sections of the binary (e.g. exception handling tables) + /// that are interpreted (not executed) but can alter the behaviour of the program as well. + + String calculated_binary_hash = getHashOfLoadedBinaryHex(); + String stored_binary_hash = getStoredBinaryHash(); + + if (stored_binary_hash.empty()) + { + LOG_WARNING(log, "Calculated checksum of the binary: {}." + " There is no information about the reference checksum.", calculated_binary_hash); + } + else if (calculated_binary_hash == stored_binary_hash) + { + LOG_INFO(log, "Calculated checksum of the binary: {}, integrity check passed.", calculated_binary_hash); + } + else + { + throw Exception(ErrorCodes::CORRUPTED_DATA, + "Calculated checksum of the ClickHouse binary ({0}) does not correspond" + " to the reference checksum stored in the binary ({1})." + " It may indicate one of the following:" + " - the file {2} was changed just after startup;" + " - the file {2} is damaged on disk due to faulty hardware;" + " - the loaded executable is damaged in memory due to faulty hardware;" + " - the file {2} was intentionally modified;" + " - logical error in code." + , calculated_binary_hash, stored_binary_hash, executable_path); + } + } + else executable_path = "/usr/bin/clickhouse"; /// It is used for information messages. /// After full config loaded diff --git a/src/Common/Elf.cpp b/src/Common/Elf.cpp index 0c2359b3418..ee78c988f69 100644 --- a/src/Common/Elf.cpp +++ b/src/Common/Elf.cpp @@ -151,6 +151,15 @@ String Elf::getBuildID(const char * nhdr_pos, size_t size) } +String Elf::getBinaryHash() const +{ + if (auto section = findSectionByName(".note.ClickHouse.hash")) + return {section->begin(), section->end()}; + else + return {}; +} + + const char * Elf::Section::name() const { if (!elf.section_names) diff --git a/src/Common/Elf.h b/src/Common/Elf.h index 632d7e6f0b1..90783ddc18d 100644 --- a/src/Common/Elf.h +++ b/src/Common/Elf.h @@ -59,6 +59,9 @@ public: String getBuildID() const; static String getBuildID(const char * nhdr_pos, size_t size); + /// Hash of the binary for integrity checks. + String getBinaryHash() const; + private: MMapReadBufferFromFile in; size_t elf_size; From 15113a5e5fe6deb8c84c25d2cc1d96f7f9772770 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Jan 2021 06:01:15 +0300 Subject: [PATCH 150/256] Add files --- src/Common/getHashOfLoadedBinary.cpp | 40 ++++++++++++++++++++++++++++ src/Common/getHashOfLoadedBinary.h | 15 +++++++++++ 2 files changed, 55 insertions(+) create mode 100644 src/Common/getHashOfLoadedBinary.cpp create mode 100644 src/Common/getHashOfLoadedBinary.h diff --git a/src/Common/getHashOfLoadedBinary.cpp b/src/Common/getHashOfLoadedBinary.cpp new file mode 100644 index 00000000000..b3f69a0a6bb --- /dev/null +++ b/src/Common/getHashOfLoadedBinary.cpp @@ -0,0 +1,40 @@ +#include +#include +#include +#include + + +static int callback(dl_phdr_info * info, size_t, void * data) +{ + SipHash & hash = *reinterpret_cast(data); + + for (size_t header_index = 0; header_index < info->dlpi_phnum; ++header_index) + { + const auto & phdr = info->dlpi_phdr[header_index]; + + if (phdr.p_type == PT_LOAD && (phdr.p_flags & PF_X)) + { + hash.update(phdr.p_filesz); + hash.update(reinterpret_cast(info->dlpi_addr + phdr.p_vaddr), phdr.p_filesz); + } + } + + return 1; /// Do not continue iterating. +} + + +SipHash getHashOfLoadedBinary() +{ + SipHash hash; + dl_iterate_phdr(callback, &hash); + return hash; +} + + +std::string getHashOfLoadedBinaryHex() +{ + SipHash hash = getHashOfLoadedBinary(); + std::array checksum; + hash.get128(checksum[0], checksum[1]); + return getHexUIntUppercase(checksum); +} diff --git a/src/Common/getHashOfLoadedBinary.h b/src/Common/getHashOfLoadedBinary.h new file mode 100644 index 00000000000..699a0da8b8f --- /dev/null +++ b/src/Common/getHashOfLoadedBinary.h @@ -0,0 +1,15 @@ +#include +#include + +/** Calculate hash of the executable loaded segments of the first loaded object. + * It can be used for integrity checks. + * Does not work when ClickHouse is build as multiple shared libraries. + * Note: we don't hash all loaded readonly segments, because some of them are modified by 'strip' + * and we want something that survives 'strip'. + * Note: program behaviour can be affected not only by machine code but also by the data in these segments, + * so the integrity check is going to be incomplete. + */ +SipHash getHashOfLoadedBinary(); +std::string getHashOfLoadedBinaryHex(); + + From 63f0348291eb899076bf22765fc6080a7840cd04 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 7 Jan 2021 11:31:23 +0800 Subject: [PATCH 151/256] Fix test --- tests/integration/test_alter_settings_on_cluster/test.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_alter_settings_on_cluster/test.py b/tests/integration/test_alter_settings_on_cluster/test.py index d56ef4461aa..6ab3d446b59 100644 --- a/tests/integration/test_alter_settings_on_cluster/test.py +++ b/tests/integration/test_alter_settings_on_cluster/test.py @@ -48,6 +48,7 @@ def test_default_database_on_cluster(started_cluster): ) for node in [ch1, ch2]: - assert node.query("SHOW CREATE test_local_table FORMAT TSV").endswith( - "old_parts_lifetime = 100\n" - ) + assert node.query( + database="test_default_database", + sql="SHOW CREATE test_local_table FORMAT TSV", + ).endswith("old_parts_lifetime = 100\n") From 65b4008d1fa28094f67a0d1c70cb94331d29aa30 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 7 Jan 2021 06:36:00 +0300 Subject: [PATCH 152/256] Update 01600_count_of_parts_metrics.sh --- tests/queries/0_stateless/01600_count_of_parts_metrics.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01600_count_of_parts_metrics.sh b/tests/queries/0_stateless/01600_count_of_parts_metrics.sh index f55ee58ad15..703fe1703a6 100755 --- a/tests/queries/0_stateless/01600_count_of_parts_metrics.sh +++ b/tests/queries/0_stateless/01600_count_of_parts_metrics.sh @@ -1,6 +1,7 @@ #!/usr/bin/env bash CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh verify_sql="SELECT COUNT(1) @@ -18,7 +19,7 @@ FROM system.parts ) as b USING (Parts,PartsActive,PartsInactive)" verify(){ -for i in $(seq 1 10) +for _ in $(seq 1 10) do result=$( $CLICKHOUSE_CLIENT -m --query="$verify_sql" ) if [ "$result" = "1" ];then From 703e16db5afaf1f6220fd111d6b72749e2e08de3 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 7 Jan 2021 06:38:05 +0300 Subject: [PATCH 153/256] Update StorageMergeTree.cpp --- src/Storages/StorageMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 3310cfdf126..c959ee613a8 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -171,7 +171,7 @@ void StorageMergeTree::shutdown() { if (parts_info->state == DataPartState::Committed) { - committed_parts_count++; + ++committed_parts_count; } } From dd42f911291011ab64b62bf41194976540e557ec Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Jan 2021 06:40:24 +0300 Subject: [PATCH 154/256] Fix style --- src/Common/getHashOfLoadedBinary.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/getHashOfLoadedBinary.h b/src/Common/getHashOfLoadedBinary.h index 699a0da8b8f..3a4a22b1bf1 100644 --- a/src/Common/getHashOfLoadedBinary.h +++ b/src/Common/getHashOfLoadedBinary.h @@ -1,3 +1,5 @@ +#pragma once + #include #include @@ -11,5 +13,3 @@ */ SipHash getHashOfLoadedBinary(); std::string getHashOfLoadedBinaryHex(); - - From fe36a5582d53d4088e1bb0c58ca23b18f08bc41e Mon Sep 17 00:00:00 2001 From: long2ice Date: Thu, 7 Jan 2021 14:30:59 +0800 Subject: [PATCH 155/256] Update client-libraries.md --- docs/zh/interfaces/third-party/client-libraries.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/zh/interfaces/third-party/client-libraries.md b/docs/zh/interfaces/third-party/client-libraries.md index e94eb8bcfc0..e2412f2b8de 100644 --- a/docs/zh/interfaces/third-party/client-libraries.md +++ b/docs/zh/interfaces/third-party/client-libraries.md @@ -13,6 +13,7 @@ Yandex**没有**维护下面列出的库,也没有做过任何广泛的测试 - [clickhouse-driver](https://github.com/mymarilyn/clickhouse-driver) - [clickhouse-client](https://github.com/yurial/clickhouse-client) - [aiochclient](https://github.com/maximdanilchenko/aiochclient) + - [asynch](https://github.com/long2ice/asynch) - PHP - [smi2/phpclickhouse](https://packagist.org/packages/smi2/phpClickHouse) - [8bitov/clickhouse-php-client](https://packagist.org/packages/8bitov/clickhouse-php-client) From f96208ac7c3b461aa749666b47847502c8256805 Mon Sep 17 00:00:00 2001 From: long2ice Date: Thu, 7 Jan 2021 14:31:35 +0800 Subject: [PATCH 156/256] Update client-libraries.md --- docs/en/interfaces/third-party/client-libraries.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/interfaces/third-party/client-libraries.md b/docs/en/interfaces/third-party/client-libraries.md index f3a6381aeca..c08eec61b1c 100644 --- a/docs/en/interfaces/third-party/client-libraries.md +++ b/docs/en/interfaces/third-party/client-libraries.md @@ -13,6 +13,7 @@ toc_title: Client Libraries - [clickhouse-driver](https://github.com/mymarilyn/clickhouse-driver) - [clickhouse-client](https://github.com/yurial/clickhouse-client) - [aiochclient](https://github.com/maximdanilchenko/aiochclient) + - [asynch](https://github.com/long2ice/asynch) - PHP - [smi2/phpclickhouse](https://packagist.org/packages/smi2/phpClickHouse) - [8bitov/clickhouse-php-client](https://packagist.org/packages/8bitov/clickhouse-php-client) From c82602abffc0a4229cfcef7c9c4fd2175eaa64c8 Mon Sep 17 00:00:00 2001 From: long2ice Date: Thu, 7 Jan 2021 14:31:56 +0800 Subject: [PATCH 157/256] Update client-libraries.md --- docs/es/interfaces/third-party/client-libraries.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/es/interfaces/third-party/client-libraries.md b/docs/es/interfaces/third-party/client-libraries.md index 818bdbbc6f0..b61ab1a5d9c 100644 --- a/docs/es/interfaces/third-party/client-libraries.md +++ b/docs/es/interfaces/third-party/client-libraries.md @@ -13,6 +13,7 @@ toc_title: Client Libraries - [clickhouse-driver](https://github.com/mymarilyn/clickhouse-driver) - [clickhouse-client](https://github.com/yurial/clickhouse-client) - [aiochclient](https://github.com/maximdanilchenko/aiochclient) + - [asynch](https://github.com/long2ice/asynch) - PHP - [smi2/phpclickhouse](https://packagist.org/packages/smi2/phpClickHouse) - [8bitov/clickhouse-php-client](https://packagist.org/packages/8bitov/clickhouse-php-client) From b5bc5d6798042dcb1be1fdfe83829b8d9550a85c Mon Sep 17 00:00:00 2001 From: long2ice Date: Thu, 7 Jan 2021 14:32:24 +0800 Subject: [PATCH 158/256] Update client-libraries.md --- docs/fr/interfaces/third-party/client-libraries.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/fr/interfaces/third-party/client-libraries.md b/docs/fr/interfaces/third-party/client-libraries.md index 5a86d12a09c..7949aa1d7cf 100644 --- a/docs/fr/interfaces/third-party/client-libraries.md +++ b/docs/fr/interfaces/third-party/client-libraries.md @@ -15,6 +15,7 @@ toc_title: "Biblioth\xE8ques Clientes" - [clickhouse-chauffeur](https://github.com/mymarilyn/clickhouse-driver) - [clickhouse-client](https://github.com/yurial/clickhouse-client) - [aiochclient](https://github.com/maximdanilchenko/aiochclient) + - [asynch](https://github.com/long2ice/asynch) - PHP - [smi2 / phpclickhouse](https://packagist.org/packages/smi2/phpClickHouse) - [8bitov / clickhouse-PHP-client](https://packagist.org/packages/8bitov/clickhouse-php-client) From 7fce85f639fc46cc3b02cd3a2afa27d48e54b576 Mon Sep 17 00:00:00 2001 From: feng lv Date: Thu, 7 Jan 2021 06:32:33 +0000 Subject: [PATCH 159/256] replace PODArray with stack PODArray --- src/AggregateFunctions/AggregateFunctionWindowFunnel.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionWindowFunnel.h b/src/AggregateFunctions/AggregateFunctionWindowFunnel.h index 96f0eb183be..77d99e55eb7 100644 --- a/src/AggregateFunctions/AggregateFunctionWindowFunnel.h +++ b/src/AggregateFunctions/AggregateFunctionWindowFunnel.h @@ -33,7 +33,7 @@ template struct AggregateFunctionWindowFunnelData { using TimestampEvent = std::pair; - using TimestampEvents = PODArray; + using TimestampEvents = PODArrayWithStackMemory; using Comparator = ComparePairFirst; bool sorted = true; From a347bb06006a15b54df984a5fdf890c0b90d339c Mon Sep 17 00:00:00 2001 From: long2ice Date: Thu, 7 Jan 2021 14:32:42 +0800 Subject: [PATCH 160/256] Update client-libraries.md --- docs/ja/interfaces/third-party/client-libraries.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/ja/interfaces/third-party/client-libraries.md b/docs/ja/interfaces/third-party/client-libraries.md index ffe7b641c38..c7bd368bc4c 100644 --- a/docs/ja/interfaces/third-party/client-libraries.md +++ b/docs/ja/interfaces/third-party/client-libraries.md @@ -15,6 +15,7 @@ toc_title: "\u30AF\u30E9\u30A4\u30A2\u30F3\u30C8" - [clickhouse-ドライバ](https://github.com/mymarilyn/clickhouse-driver) - [clickhouse-クライアント](https://github.com/yurial/clickhouse-client) - [aiochclient](https://github.com/maximdanilchenko/aiochclient) + - [asynch](https://github.com/long2ice/asynch) - PHP - [smi2/phpclickhouse](https://packagist.org/packages/smi2/phpClickHouse) - [8bitov/clickhouse-php-クライアント](https://packagist.org/packages/8bitov/clickhouse-php-client) From 2de812af0f7e27afbc93c1bb54667808dbc15fde Mon Sep 17 00:00:00 2001 From: long2ice Date: Thu, 7 Jan 2021 14:32:59 +0800 Subject: [PATCH 161/256] Update client-libraries.md --- docs/ru/interfaces/third-party/client-libraries.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/ru/interfaces/third-party/client-libraries.md b/docs/ru/interfaces/third-party/client-libraries.md index c07aab5826c..26e05b02509 100644 --- a/docs/ru/interfaces/third-party/client-libraries.md +++ b/docs/ru/interfaces/third-party/client-libraries.md @@ -13,6 +13,7 @@ toc_title: "\u041a\u043b\u0438\u0435\u043d\u0442\u0441\u043a\u0438\u0435\u0020\u - [clickhouse-driver](https://github.com/mymarilyn/clickhouse-driver) - [clickhouse-client](https://github.com/yurial/clickhouse-client) - [aiochclient](https://github.com/maximdanilchenko/aiochclient) + - [asynch](https://github.com/long2ice/asynch) - PHP - [smi2/phpclickhouse](https://packagist.org/packages/smi2/phpClickHouse) - [8bitov/clickhouse-php-client](https://packagist.org/packages/8bitov/clickhouse-php-client) From 36e1361cf8cb5195791414aa3831fae1c5c4875a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Jan 2021 15:29:34 +0300 Subject: [PATCH 162/256] Miscellaneous --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 104eedf060e..0b4297e37b3 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -840,7 +840,7 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ sync_guard.emplace(volume->getDisk(), to); if (!volume->getDisk()->exists(from)) - throw Exception("Part directory " + fullPath(volume->getDisk(), from) + " doesn't exist. Most likely it is logical error.", ErrorCodes::FILE_DOESNT_EXIST); + throw Exception("Part directory " + fullPath(volume->getDisk(), from) + " doesn't exist. Most likely it is a logical error.", ErrorCodes::FILE_DOESNT_EXIST); if (volume->getDisk()->exists(to)) { From c920a57097a1117ff0d384752b93a13fc8375399 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 7 Jan 2021 15:35:20 +0300 Subject: [PATCH 163/256] Update tests.md --- docs/en/development/tests.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index 5b096bcc5fa..1cb8c0fdcdb 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -203,10 +203,10 @@ For production builds, gcc is used (it still generates slightly more efficient c We run functional and integration tests under ASan on per-commit basis. ### Valgrind (Memcheck) -We run functional tests under Valgrind overnight. It takes multiple hours. Currently there is one known false positive in `re2` library, see [this article](https://research.swtch.com/sparse). +We used to run functional tests under Valgrind overnight, but don't do it anymore. It takes multiple hours. Currently there is one known false positive in `re2` library, see [this article](https://research.swtch.com/sparse). ### Undefined behaviour sanitizer -We run functional and integration tests under ASan on per-commit basis. +We run functional and integration tests under USan on per-commit basis. ### Thread sanitizer We run functional tests under TSan on per-commit basis. We still don’t run integration tests under TSan on per-commit basis. @@ -233,6 +233,8 @@ Google OSS-Fuzz can be found at `docker/fuzz`. We also use simple fuzz test to generate random SQL queries and to check that the server doesn’t die executing them. You can find it in `00746_sql_fuzzy.pl`. This test should be run continuously (overnight and longer). +We also use sophisticated AST-based query fuzzer that is able to find huge amount of corner cases. + ## Security Audit {#security-audit} People from Yandex Security Team do some basic overview of ClickHouse capabilities from the security standpoint. From c6c2536a2a274ee332c7300a53572b0345d4f7a9 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 7 Jan 2021 15:37:18 +0300 Subject: [PATCH 164/256] Update tests.md --- docs/en/development/tests.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index 1cb8c0fdcdb..598ed5d1a42 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -269,7 +269,7 @@ These tests are automated by separate team. Due to high number of moving parts, ## Test Coverage {#test-coverage} -As of July 2018 we don’t track test coverage. +We also track test coverage but only for functional tests and only for clickhouse-server. It is performed on daily basis. ## Test Automation {#test-automation} From fe96e81e7e258805ff6cd366564a591a517db8c2 Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Thu, 7 Jan 2021 13:57:39 +0100 Subject: [PATCH 165/256] Fix errors add examples --- .../functions/array-functions.md | 47 ++++++++++++++++++- 1 file changed, 45 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index ee4449d46fa..dc7727bdfd8 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -1290,22 +1290,65 @@ Note that the `arrayFirstIndex` is a [higher-order function](../../sql-reference ## arrayMin(\[func,\] arr1, …) {#array-min} -Returns the sum of the `func` values. If the function is omitted, it just returns the min of the array elements. +Returns the min of the `func` values. If the function is omitted, it just returns the min of the array elements. Note that the `arrayMin` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You can pass a lambda function to it as the first argument. +Examples: +```sql +SELECT arrayMin([1, 2, 4]) AS res +┌─res─┐ +│ 1 │ +└─────┘ + + +SELECT arrayMin(x -> (-x), [1, 2, 4]) AS res +┌─res─┐ +│ -4 │ +└─────┘ +``` + ## arrayMax(\[func,\] arr1, …) {#array-max} -Returns the sum of the `func` values. If the function is omitted, it just returns the min of the array elements. +Returns the max of the `func` values. If the function is omitted, it just returns the max of the array elements. Note that the `arrayMax` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You can pass a lambda function to it as the first argument. +Examples: +```sql +SELECT arrayMax([1, 2, 4]) AS res +┌─res─┐ +│ 4 │ +└─────┘ + + +SELECT arrayMax(x -> (-x), [1, 2, 4]) AS res +┌─res─┐ +│ -1 │ +└─────┘ +``` + ## arraySum(\[func,\] arr1, …) {#array-sum} Returns the sum of the `func` values. If the function is omitted, it just returns the sum of the array elements. Note that the `arraySum` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You can pass a lambda function to it as the first argument. +Examples: +```sql +SELECT arraySum([2,3]) AS res +┌─res─┐ +│ 5 │ +└─────┘ + + +SELECT arraySum(x -> x*x, [2, 3]) AS res +┌─res─┐ +│ 13 │ +└─────┘ +``` + + ## arrayAvg(\[func,\] arr1, …) {#array-avg} Returns the average of the `func` values. If the function is omitted, it just returns the average of the array elements. From 10478811f972a8ffc3cca3418a8b573757377ca7 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 7 Jan 2021 15:59:05 +0300 Subject: [PATCH 166/256] Update tests.md --- docs/en/development/tests.md | 76 ++++++++++++++++++++++++++++-------- 1 file changed, 59 insertions(+), 17 deletions(-) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index 598ed5d1a42..8a416027818 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -88,7 +88,7 @@ You can also place pair of files `.sh` and `.reference` along with the tool to r ## Miscellaneous Tests {#miscellaneous-tests} -There are tests for external dictionaries located at `tests/external_dictionaries` and for machine learned models in `tests/external_models`. These tests are not updated and must be transferred to integration tests. +There are tests for machine learned models in `tests/external_models`. These tests are not updated and must be transferred to integration tests. There is separate test for quorum inserts. This test run ClickHouse cluster on separate servers and emulate various failure cases: network split, packet drop (between ClickHouse nodes, between ClickHouse and ZooKeeper, between ClickHouse server and client, etc.), `kill -9`, `kill -STOP` and `kill -CONT` , like [Jepsen](https://aphyr.com/tags/Jepsen). Then the test checks that all acknowledged inserts was written and all rejected inserts was not. @@ -185,38 +185,43 @@ For example, build with system packages is bad practice, because we cannot guara Though we cannot run all tests on all variant of builds, we want to check at least that various build variants are not broken. For this purpose we use build tests. +We also test that there are no translation units that are too long to compile or require too much RAM. + +We also test that there are no too large stack frames. + ## Testing for Protocol Compatibility {#testing-for-protocol-compatibility} When we extend ClickHouse network protocol, we test manually that old clickhouse-client works with new clickhouse-server and new clickhouse-client works with old clickhouse-server (simply by running binaries from corresponding packages). +We also test some cases automatically with integrational tests: +- if data written by old version of ClickHouse can be successfully read by the new version; +- do distributed queries work in a cluster with different ClickHouse versions. + ## Help from the Compiler {#help-from-the-compiler} Main ClickHouse code (that is located in `dbms` directory) is built with `-Wall -Wextra -Werror` and with some additional enabled warnings. Although these options are not enabled for third-party libraries. Clang has even more useful warnings - you can look for them with `-Weverything` and pick something to default build. -For production builds, gcc is used (it still generates slightly more efficient code than clang). For development, clang is usually more convenient to use. You can build on your own machine with debug mode (to save battery of your laptop), but please note that compiler is able to generate more warnings with `-O3` due to better control flow and inter-procedure analysis. When building with clang in debug mode, debug version of `libc++` is used that allows to catch more errors at runtime. +For production builds, clang is used, but we also test make gcc builds. For development, clang is usually more convenient to use. You can build on your own machine with debug mode (to save battery of your laptop), but please note that compiler is able to generate more warnings with `-O3` due to better control flow and inter-procedure analysis. When building with clang in debug mode, debug version of `libc++` is used that allows to catch more errors at runtime. ## Sanitizers {#sanitizers} ### Address sanitizer -We run functional and integration tests under ASan on per-commit basis. +We run functional, integration, stress and unit tests under ASan on per-commit basis. + +### Thread sanitizer +We run functional, integration, stress and unit tests under TSan on per-commit basis. + +### Memory sanitizer +We run functional, integration, stress and unit tests under MSan on per-commit basis. + +### Undefined behaviour sanitizer +We run functional, integration, stress and unit tests under USan on per-commit basis. The code of some third-party libraries is not sanitized for UB. ### Valgrind (Memcheck) We used to run functional tests under Valgrind overnight, but don't do it anymore. It takes multiple hours. Currently there is one known false positive in `re2` library, see [this article](https://research.swtch.com/sparse). -### Undefined behaviour sanitizer -We run functional and integration tests under USan on per-commit basis. - -### Thread sanitizer -We run functional tests under TSan on per-commit basis. We still don’t run integration tests under TSan on per-commit basis. - -### Memory sanitizer -Currently we still don’t use MSan. - -### Debug allocator -Debug version of `jemalloc` is used for debug build. - ## Fuzzing {#fuzzing} ClickHouse fuzzing is implemented both using [libFuzzer](https://llvm.org/docs/LibFuzzer.html) and random SQL queries. @@ -235,19 +240,50 @@ You can find it in `00746_sql_fuzzy.pl`. This test should be run continuously (o We also use sophisticated AST-based query fuzzer that is able to find huge amount of corner cases. +## Stress test + +Stress tests are another case of fuzzing. It runs all functional tests in parallel in random order with a single server. Results of the tests are not checked. + +It is checked that: +- server does not crash, no debug or sanitizer traps are triggered; +- there are no deadlocks; +- the database structure is consistent; +- server can successfully stop after the test and start again without exceptions. + +There are five variants (Debug, ASan, TSan, MSan, UBSan). + +## Thread Fuzzer + +Thread Fuzzer (please don't mix up with Thread Sanitizer) is another kind of fuzzing that allows to randomize thread order of execution. It helps to find even more special cases. + ## Security Audit {#security-audit} People from Yandex Security Team do some basic overview of ClickHouse capabilities from the security standpoint. ## Static Analyzers {#static-analyzers} -We run `PVS-Studio` on per-commit basis. We have evaluated `clang-tidy`, `Coverity`, `cppcheck`, `PVS-Studio`, `tscancode`. You will find instructions for usage in `tests/instructions/` directory. Also you can read [the article in russian](https://habr.com/company/yandex/blog/342018/). +We run `clang-tidy` and `PVS-Studio` on per-commit basis. `clang-static-analyzer` checks are also enabled. `clang-tidy` is also used for some style checks. + +We have evaluated `clang-tidy`, `Coverity`, `cppcheck`, `PVS-Studio`, `tscancode`, `CodeQL`. You will find instructions for usage in `tests/instructions/` directory. Also you can read [the article in russian](https://habr.com/company/yandex/blog/342018/). If you use `CLion` as an IDE, you can leverage some `clang-tidy` checks out of the box. +We also use `shellcheck` for static analysis of shell scripts. + ## Hardening {#hardening} -`FORTIFY_SOURCE` is used by default. It is almost useless, but still makes sense in rare cases and we don’t disable it. +In debug build we are using custom allocator that does ASLR of user-level allocations. + +We also manually protect memory regions that are expected to be readonly after allocation. + +In debug build we also involve a customization of libc that ensures that no "harmful" (obsolete, insecure, not thread-safe) functions are called. + +Debug assertions are used extensively. + +In debug build, if exception with "logical error" code (implies a bug) is being thrown, the program is terminated prematurally. It allows to use exceptions in release build but make it an assertion in debug build. + +Debug version of jemalloc is used for debug builds. +Debug version of libc++ is used for debug builds. ## Code Style {#code-style} @@ -261,6 +297,8 @@ Alternatively you can try `uncrustify` tool to reformat your code. Configuration `CLion` has its own code formatter that has to be tuned for our code style. +We also use `codespell` to find typos in code. It is automated as well. + ## Metrica B2B Tests {#metrica-b2b-tests} Each ClickHouse release is tested with Yandex Metrica and AppMetrica engines. Testing and stable versions of ClickHouse are deployed on VMs and run with a small copy of Metrica engine that is processing fixed sample of input data. Then results of two instances of Metrica engine are compared together. @@ -271,6 +309,10 @@ These tests are automated by separate team. Due to high number of moving parts, We also track test coverage but only for functional tests and only for clickhouse-server. It is performed on daily basis. +## Tests for Tests + +There is automated check for flaky tests. It runs all new tests 100 times (for functional tests) or 10 times (for integration tests). If at least single time the test failed, it is considered flaky. + ## Test Automation {#test-automation} We run tests with Yandex internal CI and job automation system named “Sandbox”. From c4c10fddb22b832308d65523ac3d01900b159c0c Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 7 Jan 2021 15:59:59 +0300 Subject: [PATCH 167/256] Update tests.md --- docs/en/development/tests.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index 8a416027818..ba591244472 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -317,7 +317,7 @@ There is automated check for flaky tests. It runs all new tests 100 times (for f We run tests with Yandex internal CI and job automation system named “Sandbox”. -Build jobs and tests are run in Sandbox on per commit basis. Resulting packages and test results are published in GitHub and can be downloaded by direct links. Artifacts are stored eternally. When you send a pull request on GitHub, we tag it as “can be tested” and our CI system will build ClickHouse packages (release, debug, with address sanitizer, etc) for you. +Build jobs and tests are run in Sandbox on per commit basis. Resulting packages and test results are published in GitHub and can be downloaded by direct links. Artifacts are stored for several months. When you send a pull request on GitHub, we tag it as “can be tested” and our CI system will build ClickHouse packages (release, debug, with address sanitizer, etc) for you. We don’t use Travis CI due to the limit on time and computational power. We don’t use Jenkins. It was used before and now we are happy we are not using Jenkins. From 025ade6ef518eb3cb7c6c0e240efe88ab4630b5d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 7 Jan 2021 16:00:54 +0300 Subject: [PATCH 168/256] Update tests.md --- docs/en/development/tests.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index ba591244472..cc7d266aaa3 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -84,8 +84,6 @@ If you want to improve performance of ClickHouse in some scenario, and if improv Some programs in `tests` directory are not prepared tests, but are test tools. For example, for `Lexer` there is a tool `src/Parsers/tests/lexer` that just do tokenization of stdin and writes colorized result to stdout. You can use these kind of tools as a code examples and for exploration and manual testing. -You can also place pair of files `.sh` and `.reference` along with the tool to run it on some predefined input - then script result can be compared to `.reference` file. These kind of tests are not automated. - ## Miscellaneous Tests {#miscellaneous-tests} There are tests for machine learned models in `tests/external_models`. These tests are not updated and must be transferred to integration tests. From 9de3a2f892900815e54fa378b335d69028a3e790 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 7 Jan 2021 16:01:54 +0300 Subject: [PATCH 169/256] Update tests.md --- docs/en/development/tests.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index cc7d266aaa3..758e905a090 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -11,7 +11,7 @@ Functional tests are the most simple and convenient to use. Most of ClickHouse f Each functional test sends one or multiple queries to the running ClickHouse server and compares the result with reference. -Tests are located in `queries` directory. There are two subdirectories: `stateless` and `stateful`. Stateless tests run queries without any preloaded test data - they often create small synthetic datasets on the fly, within the test itself. Stateful tests require preloaded test data from Yandex.Metrica and not available to general public. We tend to use only `stateless` tests and avoid adding new `stateful` tests. +Tests are located in `queries` directory. There are two subdirectories: `stateless` and `stateful`. Stateless tests run queries without any preloaded test data - they often create small synthetic datasets on the fly, within the test itself. Stateful tests require preloaded test data from Yandex.Metrica and it is available to general public. Each test can be one of two types: `.sql` and `.sh`. `.sql` test is the simple SQL script that is piped to `clickhouse-client --multiquery --testmode`. `.sh` test is a script that is run by itself. SQL tests are generally preferable to `.sh` tests. You should use `.sh` tests only when you have to test some feature that cannot be exercised from pure SQL, such as piping some input data into `clickhouse-client` or testing `clickhouse-local`. From d0c72f83cce76c32b9b88aef7df585813865250a Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 7 Jan 2021 16:04:41 +0300 Subject: [PATCH 170/256] Update tests.md --- docs/en/development/tests.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index 758e905a090..46f9b0b10c7 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -236,7 +236,7 @@ Google OSS-Fuzz can be found at `docker/fuzz`. We also use simple fuzz test to generate random SQL queries and to check that the server doesn’t die executing them. You can find it in `00746_sql_fuzzy.pl`. This test should be run continuously (overnight and longer). -We also use sophisticated AST-based query fuzzer that is able to find huge amount of corner cases. +We also use sophisticated AST-based query fuzzer that is able to find huge amount of corner cases. It does random permutations and substitutions in queries AST. It remembers AST nodes from previous tests to use them for fuzzing of subsequent tests while processing them in random order. ## Stress test From 99fa3cbd3448de5b6df20635f6ae183448ca0f14 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 7 Jan 2021 16:06:03 +0300 Subject: [PATCH 171/256] Update tests.md --- docs/en/development/tests.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index 46f9b0b10c7..6c7620df1fb 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -215,7 +215,7 @@ We run functional, integration, stress and unit tests under TSan on per-commit b We run functional, integration, stress and unit tests under MSan on per-commit basis. ### Undefined behaviour sanitizer -We run functional, integration, stress and unit tests under USan on per-commit basis. The code of some third-party libraries is not sanitized for UB. +We run functional, integration, stress and unit tests under UBSan on per-commit basis. The code of some third-party libraries is not sanitized for UB. ### Valgrind (Memcheck) We used to run functional tests under Valgrind overnight, but don't do it anymore. It takes multiple hours. Currently there is one known false positive in `re2` library, see [this article](https://research.swtch.com/sparse). From 80bdc76a231a74c9f63fa09e87db64d3839ceeca Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 7 Jan 2021 16:13:48 +0300 Subject: [PATCH 172/256] Update tests.md --- docs/en/development/tests.md | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index 6c7620df1fb..21baab8312e 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -283,6 +283,14 @@ In debug build, if exception with "logical error" code (implies a bug) is being Debug version of jemalloc is used for debug builds. Debug version of libc++ is used for debug builds. +## Runtime Integrity Checks + +Data stored on disk is checksummed. Data in MergeTree tables is checksummed in three ways (compressed data blocks, uncompressed data blocks, the total checksum across blocks). Data transferred over network between client and server or between servers is also checksummed. Replication ensures bit-identical data on replicas. + +It is required to protect from faulty hardware (bit rot on storage media, bit flips in RAM on server, bit flips in RAM of network controller, bit flips in RAM of network switch, bit flips in RAM of client). Note that bit flips are common and likely to occur even for ECC RAM and in presense of TCP checksums (if you manage to run thousands of servers processing petabytes of data each day). [See video (russian)](https://www.youtube.com/watch?v=ooBAQIe0KlQ). + +ClickHouse provides diagnostics that will help ops engineers to find faulty hardware. + ## Code Style {#code-style} Code style rules are described [here](style.md). From bb4adf0293171fe1a4889c9bc0eaaabbb447ff57 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 7 Jan 2021 16:15:00 +0300 Subject: [PATCH 173/256] Update tests.md --- docs/en/development/tests.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index 21baab8312e..d0eedcad9e2 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -285,12 +285,14 @@ Debug version of libc++ is used for debug builds. ## Runtime Integrity Checks -Data stored on disk is checksummed. Data in MergeTree tables is checksummed in three ways (compressed data blocks, uncompressed data blocks, the total checksum across blocks). Data transferred over network between client and server or between servers is also checksummed. Replication ensures bit-identical data on replicas. +Data stored on disk is checksummed. Data in MergeTree tables is checksummed in three ways simultaneously* (compressed data blocks, uncompressed data blocks, the total checksum across blocks). Data transferred over network between client and server or between servers is also checksummed. Replication ensures bit-identical data on replicas. It is required to protect from faulty hardware (bit rot on storage media, bit flips in RAM on server, bit flips in RAM of network controller, bit flips in RAM of network switch, bit flips in RAM of client). Note that bit flips are common and likely to occur even for ECC RAM and in presense of TCP checksums (if you manage to run thousands of servers processing petabytes of data each day). [See video (russian)](https://www.youtube.com/watch?v=ooBAQIe0KlQ). ClickHouse provides diagnostics that will help ops engineers to find faulty hardware. +\* and it is not slow. + ## Code Style {#code-style} Code style rules are described [here](style.md). From 8d3e50ae1d1d1b99a33e5b68c65d2a22a2c23672 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 7 Jan 2021 16:15:44 +0300 Subject: [PATCH 174/256] Update tests.md --- docs/en/development/tests.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index d0eedcad9e2..ee92a6d9467 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -287,7 +287,7 @@ Debug version of libc++ is used for debug builds. Data stored on disk is checksummed. Data in MergeTree tables is checksummed in three ways simultaneously* (compressed data blocks, uncompressed data blocks, the total checksum across blocks). Data transferred over network between client and server or between servers is also checksummed. Replication ensures bit-identical data on replicas. -It is required to protect from faulty hardware (bit rot on storage media, bit flips in RAM on server, bit flips in RAM of network controller, bit flips in RAM of network switch, bit flips in RAM of client). Note that bit flips are common and likely to occur even for ECC RAM and in presense of TCP checksums (if you manage to run thousands of servers processing petabytes of data each day). [See video (russian)](https://www.youtube.com/watch?v=ooBAQIe0KlQ). +It is required to protect from faulty hardware (bit rot on storage media, bit flips in RAM on server, bit flips in RAM of network controller, bit flips in RAM of network switch, bit flips in RAM of client). Note that bit flips are common and likely to occur even for ECC RAM and in presense of TCP checksums (if you manage to run thousands of servers processing petabytes of data each day). [See the video (russian)](https://www.youtube.com/watch?v=ooBAQIe0KlQ). ClickHouse provides diagnostics that will help ops engineers to find faulty hardware. From 263c0a940873a1bdeb8c65f10ccbd4306f9c7199 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 7 Jan 2021 16:17:00 +0300 Subject: [PATCH 175/256] Update tests.md --- docs/en/development/tests.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index ee92a6d9467..0c7165916ed 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -287,7 +287,7 @@ Debug version of libc++ is used for debug builds. Data stored on disk is checksummed. Data in MergeTree tables is checksummed in three ways simultaneously* (compressed data blocks, uncompressed data blocks, the total checksum across blocks). Data transferred over network between client and server or between servers is also checksummed. Replication ensures bit-identical data on replicas. -It is required to protect from faulty hardware (bit rot on storage media, bit flips in RAM on server, bit flips in RAM of network controller, bit flips in RAM of network switch, bit flips in RAM of client). Note that bit flips are common and likely to occur even for ECC RAM and in presense of TCP checksums (if you manage to run thousands of servers processing petabytes of data each day). [See the video (russian)](https://www.youtube.com/watch?v=ooBAQIe0KlQ). +It is required to protect from faulty hardware (bit rot on storage media, bit flips in RAM on server, bit flips in RAM of network controller, bit flips in RAM of network switch, bit flips in RAM of client, bit flips on the wire). Note that bit flips are common and likely to occur even for ECC RAM and in presense of TCP checksums (if you manage to run thousands of servers processing petabytes of data each day). [See the video (russian)](https://www.youtube.com/watch?v=ooBAQIe0KlQ). ClickHouse provides diagnostics that will help ops engineers to find faulty hardware. From c69931d352f68715b9bcb929534c785cb2c0523c Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 7 Jan 2021 16:22:28 +0300 Subject: [PATCH 176/256] Update tests.md --- docs/en/development/tests.md | 17 +++++++---------- 1 file changed, 7 insertions(+), 10 deletions(-) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index 0c7165916ed..47c2d7f7115 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -167,17 +167,14 @@ Precise query execution timings are not recorded and not compared due to high va ## Build Tests {#build-tests} -Build tests allow to check that build is not broken on various alternative configurations and on some foreign systems. Tests are located at `ci` directory. They run build from source inside Docker, Vagrant, and sometimes with `qemu-user-static` inside Docker. These tests are under development and test runs are not automated. +Build tests allow to check that build is not broken on various alternative configurations and on some foreign systems. These tests are automated as well. -Motivation: - -Normally we release and run all tests on a single variant of ClickHouse build. But there are alternative build variants that are not thoroughly tested. Examples: - -- build on FreeBSD -- build on Debian with libraries from system packages -- build with shared linking of libraries -- build on AArch64 platform -- build on PowerPc platform +Examples: +- cross-compile for Darwin x86_64 (Mac OS X) +- cross-compile for FreeBSD x86_64 +- cross-compile for Linux AArch64 +- build on Ubuntu with libraries from system packages (discouraged) +- build with shared linking of libraries (discouraged) For example, build with system packages is bad practice, because we cannot guarantee what exact version of packages a system will have. But this is really needed by Debian maintainers. For this reason we at least have to support this variant of build. Another example: shared linking is a common source of trouble, but it is needed for some enthusiasts. From 9b4160bbd0d66d9c06ae321c00e16438ef85ba4a Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 7 Jan 2021 16:31:50 +0300 Subject: [PATCH 177/256] Update tests.md --- docs/en/development/tests.md | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index 47c2d7f7115..fb453e55417 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -318,6 +318,14 @@ We also track test coverage but only for functional tests and only for clickhous There is automated check for flaky tests. It runs all new tests 100 times (for functional tests) or 10 times (for integration tests). If at least single time the test failed, it is considered flaky. +## Testflows + +[Testflows](https://testflows.com/) is an enterprise-grade testing framework. It is used by Altinity for some of the tests and we run these tests in our CI. + +## Yandex Checks (only for Yandex employees) + +These checks are importing ClickHouse code into Yandex internal monorepository, so ClickHouse codebase can be used as a library by other products at Yandex (YT and YDB). Note that clickhouse-server itself is not being build from internal repo and unmodified open-source build is used for Yandex applications. + ## Test Automation {#test-automation} We run tests with Yandex internal CI and job automation system named “Sandbox”. From cea1b7dba5ef20de832aba09afee8103fc08c163 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Jan 2021 16:37:06 +0300 Subject: [PATCH 178/256] Fix build --- programs/server/Server.cpp | 1 - src/Common/getHashOfLoadedBinary.cpp | 20 +++++++++++++++++++- 2 files changed, 19 insertions(+), 2 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 63caec25099..2f8029fc39c 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -452,7 +452,6 @@ int Server::main(const std::vector & /*args*/) /// that are interpreted (not executed) but can alter the behaviour of the program as well. String calculated_binary_hash = getHashOfLoadedBinaryHex(); - String stored_binary_hash = getStoredBinaryHash(); if (stored_binary_hash.empty()) { diff --git a/src/Common/getHashOfLoadedBinary.cpp b/src/Common/getHashOfLoadedBinary.cpp index b3f69a0a6bb..d401b0311be 100644 --- a/src/Common/getHashOfLoadedBinary.cpp +++ b/src/Common/getHashOfLoadedBinary.cpp @@ -1,6 +1,9 @@ +#include + +#if defined(__linux__) + #include #include -#include #include @@ -38,3 +41,18 @@ std::string getHashOfLoadedBinaryHex() hash.get128(checksum[0], checksum[1]); return getHexUIntUppercase(checksum); } + +#else + +SipHash getHashOfLoadedBinary() +{ + return {}; +} + + +std::string getHashOfLoadedBinaryHex() +{ + return {}; +} + +#endif From b85bc3f5eba36c40a35d311e18848693580216a1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Jan 2021 16:37:55 +0300 Subject: [PATCH 179/256] Fix Arcadia --- src/Common/ya.make | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/ya.make b/src/Common/ya.make index 5d2534c3547..6c0bdef6c9c 100644 --- a/src/Common/ya.make +++ b/src/Common/ya.make @@ -98,6 +98,7 @@ SRCS( formatIPv6.cpp formatReadable.cpp getExecutablePath.cpp + getHashOfLoadedBinary.cpp getMappedArea.cpp getMultipleKeysFromConfig.cpp getNumberOfPhysicalCPUCores.cpp From a08db94343232617ce5ab9e0e69b25415e0c01c0 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 7 Jan 2021 16:40:52 +0300 Subject: [PATCH 180/256] Revert "Add metrics for part number in MergeTree in ClickHouse" --- src/Common/CurrentMetrics.cpp | 3 -- src/Storages/MergeTree/MergeTreeData.cpp | 44 +----------------- src/Storages/StorageMergeTree.cpp | 19 -------- src/Storages/StorageReplicatedMergeTree.cpp | 19 -------- .../01600_count_of_parts_metrics.reference | 4 -- .../01600_count_of_parts_metrics.sh | 46 ------------------- 6 files changed, 2 insertions(+), 133 deletions(-) delete mode 100644 tests/queries/0_stateless/01600_count_of_parts_metrics.reference delete mode 100755 tests/queries/0_stateless/01600_count_of_parts_metrics.sh diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 009ad7e0521..d3a4a41046e 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -5,9 +5,6 @@ #define APPLY_FOR_METRICS(M) \ M(Query, "Number of executing queries") \ M(Merge, "Number of executing background merges") \ - M(Parts, "Total number of data parts") \ - M(PartsActive, "Number of active data parts") \ - M(PartsInactive, "Number of inactive data parts") \ M(PartMutation, "Number of mutations (ALTER DELETE/UPDATE)") \ M(ReplicatedFetch, "Number of data parts being fetched from replica") \ M(ReplicatedSend, "Number of data parts being sent to replicas") \ diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 344d0e64cf9..897aea4363a 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -74,9 +74,6 @@ namespace CurrentMetrics { extern const Metric DelayedInserts; extern const Metric BackgroundMovePoolTask; - extern const Metric Parts; - extern const Metric PartsActive; - extern const Metric PartsInactive; } @@ -884,8 +881,6 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) throw Exception("Part " + part->name + " already exists", ErrorCodes::DUPLICATE_DATA_PART); addPartContributionToDataVolume(part); - CurrentMetrics::add(CurrentMetrics::Parts); - CurrentMetrics::add(CurrentMetrics::PartsActive); }); } @@ -902,10 +897,8 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) if (!data_parts_indexes.insert(part).second) throw Exception("Part " + part->name + " already exists", ErrorCodes::DUPLICATE_DATA_PART); - addPartContributionToDataVolume(part); - CurrentMetrics::add(CurrentMetrics::Parts); - CurrentMetrics::add(CurrentMetrics::PartsActive); + addPartContributionToDataVolume(part); } if (has_non_adaptive_parts && has_adaptive_parts && !settings->enable_mixed_granularity_parts) @@ -938,9 +931,6 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) (*it)->remove_time.store((*it)->modification_time, std::memory_order_relaxed); modifyPartState(it, DataPartState::Outdated); removePartContributionToDataVolume(*it); - - CurrentMetrics::sub(CurrentMetrics::PartsActive); - CurrentMetrics::add(CurrentMetrics::PartsInactive); }; (*prev_jt)->assertState({DataPartState::Committed}); @@ -1108,9 +1098,6 @@ void MergeTreeData::removePartsFinally(const MergeTreeData::DataPartsVector & pa (*it)->assertState({DataPartState::Deleting}); data_parts_indexes.erase(it); - - CurrentMetrics::sub(CurrentMetrics::Parts); - CurrentMetrics::sub(CurrentMetrics::PartsInactive); } } @@ -2011,9 +1998,6 @@ bool MergeTreeData::renameTempPartAndReplace( auto part_it = data_parts_indexes.insert(part).first; - CurrentMetrics::add(CurrentMetrics::Parts); - CurrentMetrics::add(CurrentMetrics::PartsInactive); - if (out_transaction) { out_transaction->precommitted_parts.insert(part); @@ -2029,13 +2013,9 @@ bool MergeTreeData::renameTempPartAndReplace( covered_part->remove_time.store(current_time, std::memory_order_relaxed); modifyPartState(covered_part, DataPartState::Outdated); removePartContributionToColumnSizes(covered_part); - reduce_bytes += covered_part->getBytesOnDisk(); reduce_rows += covered_part->rows_count; ++reduce_parts; - - CurrentMetrics::sub(CurrentMetrics::PartsActive); - CurrentMetrics::add(CurrentMetrics::PartsInactive); } decreaseDataVolume(reduce_bytes, reduce_rows, reduce_parts); @@ -2043,9 +2023,6 @@ bool MergeTreeData::renameTempPartAndReplace( modifyPartState(part_it, DataPartState::Committed); addPartContributionToColumnSizes(part); addPartContributionToDataVolume(part); - - CurrentMetrics::add(CurrentMetrics::PartsActive); - CurrentMetrics::sub(CurrentMetrics::PartsInactive); } auto part_in_memory = asInMemoryPart(part); @@ -2089,9 +2066,6 @@ void MergeTreeData::removePartsFromWorkingSet(const MergeTreeData::DataPartsVect { removePartContributionToColumnSizes(part); removePartContributionToDataVolume(part); - - CurrentMetrics::sub(CurrentMetrics::PartsActive); - CurrentMetrics::add(CurrentMetrics::PartsInactive); } if (part->state == IMergeTreeDataPart::State::Committed || clear_without_timeout) @@ -2118,9 +2092,6 @@ void MergeTreeData::removePartsFromWorkingSetImmediatelyAndSetTemporaryState(con modifyPartState(part, IMergeTreeDataPart::State::Temporary); /// Erase immediately data_parts_indexes.erase(it_part); - - CurrentMetrics::sub(CurrentMetrics::Parts); - CurrentMetrics::sub(CurrentMetrics::PartsInactive); } } @@ -2211,17 +2182,12 @@ restore_covered) { removePartContributionToDataVolume(part); removePartContributionToColumnSizes(part); - - CurrentMetrics::sub(CurrentMetrics::PartsActive); - CurrentMetrics::add(CurrentMetrics::PartsInactive); } modifyPartState(it_part, DataPartState::Deleting); part->renameToDetached(prefix); - data_parts_indexes.erase(it_part); - CurrentMetrics::sub(CurrentMetrics::Parts); - CurrentMetrics::sub(CurrentMetrics::PartsInactive); + data_parts_indexes.erase(it_part); if (restore_covered && part->info.level == 0) { @@ -3431,9 +3397,6 @@ MergeTreeData::DataPartsVector MergeTreeData::Transaction::commit(MergeTreeData: data.modifyPartState(covered_part, DataPartState::Outdated); data.removePartContributionToColumnSizes(covered_part); - - CurrentMetrics::sub(CurrentMetrics::PartsActive); - CurrentMetrics::add(CurrentMetrics::PartsInactive); } reduce_parts += covered_parts.size(); @@ -3443,9 +3406,6 @@ MergeTreeData::DataPartsVector MergeTreeData::Transaction::commit(MergeTreeData: data.modifyPartState(part, DataPartState::Committed); data.addPartContributionToColumnSizes(part); - - CurrentMetrics::add(CurrentMetrics::PartsActive); - CurrentMetrics::sub(CurrentMetrics::PartsInactive); } } data.decreaseDataVolume(reduce_bytes, reduce_rows, reduce_parts); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index c959ee613a8..8d1bee3e889 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -31,9 +31,6 @@ namespace CurrentMetrics { extern const Metric BackgroundPoolTask; - extern const Metric Parts; - extern const Metric PartsActive; - extern const Metric PartsInactive; } namespace DB @@ -162,22 +159,6 @@ void StorageMergeTree::shutdown() /// may have race condition between our remove call and background /// process. clearOldPartsFromFilesystem(true); - - auto lock = lockParts(); - DataPartsVector all_parts(data_parts_by_info.begin(), data_parts_by_info.end()); - - size_t committed_parts_count = 0; - for (const auto & parts_info : all_parts) - { - if (parts_info->state == DataPartState::Committed) - { - ++committed_parts_count; - } - } - - CurrentMetrics::sub(CurrentMetrics::Parts, all_parts.size()); - CurrentMetrics::sub(CurrentMetrics::PartsActive, committed_parts_count); - CurrentMetrics::sub(CurrentMetrics::PartsInactive, all_parts.size() - committed_parts_count); } catch (...) { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index f1990dfd8cb..8a802037f61 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -81,9 +81,6 @@ namespace ProfileEvents namespace CurrentMetrics { extern const Metric BackgroundFetchesPoolTask; - extern const Metric Parts; - extern const Metric PartsActive; - extern const Metric PartsInactive; } namespace DB @@ -3686,22 +3683,6 @@ void StorageReplicatedMergeTree::shutdown() /// which will remove themselves in their descrutors. If so, we may have /// race condition between our remove call and background process. clearOldPartsFromFilesystem(true); - - auto lock = lockParts(); - DataPartsVector all_parts(data_parts_by_info.begin(), data_parts_by_info.end()); - - size_t committed_parts_count = 0; - for (const auto & parts_info : all_parts) - { - if (parts_info->state == DataPartState::Committed) - { - committed_parts_count++; - } - } - - CurrentMetrics::sub(CurrentMetrics::Parts, all_parts.size()); - CurrentMetrics::sub(CurrentMetrics::PartsActive, committed_parts_count); - CurrentMetrics::sub(CurrentMetrics::PartsInactive, all_parts.size() - committed_parts_count); } diff --git a/tests/queries/0_stateless/01600_count_of_parts_metrics.reference b/tests/queries/0_stateless/01600_count_of_parts_metrics.reference deleted file mode 100644 index 98fb6a68656..00000000000 --- a/tests/queries/0_stateless/01600_count_of_parts_metrics.reference +++ /dev/null @@ -1,4 +0,0 @@ -1 -1 -1 -1 diff --git a/tests/queries/0_stateless/01600_count_of_parts_metrics.sh b/tests/queries/0_stateless/01600_count_of_parts_metrics.sh deleted file mode 100755 index 703fe1703a6..00000000000 --- a/tests/queries/0_stateless/01600_count_of_parts_metrics.sh +++ /dev/null @@ -1,46 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -verify_sql="SELECT COUNT(1) -FROM -(SELECT - SUM(IF(metric = 'Parts', value, 0)) AS Parts, - SUM(IF(metric = 'PartsActive', value, 0)) AS PartsActive, - SUM(IF(metric = 'PartsInactive', value, 0)) AS PartsInactive -FROM system.metrics) as a INNER JOIN -(SELECT - toInt64(SUM(1)) AS Parts, - toInt64(SUM(IF(active = 1, 1, 0))) AS PartsActive, - toInt64(SUM(IF(active = 0, 1, 0))) AS PartsInactive -FROM system.parts -) as b USING (Parts,PartsActive,PartsInactive)" - -verify(){ -for _ in $(seq 1 10) -do -result=$( $CLICKHOUSE_CLIENT -m --query="$verify_sql" ) -if [ "$result" = "1" ];then - echo 1 - return -fi -done -echo 0 -} - -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test_table" 2>/dev/null -$CLICKHOUSE_CLIENT --query="CREATE TABLE test_table(data Date) ENGINE = MergeTree PARTITION BY toYear(data) ORDER BY data;" 2>/dev/null - -$CLICKHOUSE_CLIENT --query="INSERT INTO test_table VALUES ('1992-01-01')" 2>/dev/null -verify - -$CLICKHOUSE_CLIENT --query="INSERT INTO test_table VALUES ('1992-01-02')" 2>/dev/null -verify - -$CLICKHOUSE_CLIENT --query="OPTIMIZE TABLE test_table FINAL" 2>/dev/null -verify - -$CLICKHOUSE_CLIENT --query="DROP TABLE test_table" 2>/dev/null -verify From ea047b951f272fb2ad5339403d5ee50dd04090d9 Mon Sep 17 00:00:00 2001 From: fastio Date: Thu, 7 Jan 2021 22:13:17 +0800 Subject: [PATCH 181/256] Expand macros for fetchPartition --- src/Storages/StorageReplicatedMergeTree.cpp | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 8a802037f61..d683c61215a 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4958,8 +4958,13 @@ void StorageReplicatedMergeTree::fetchPartition( const String & from_, const Context & query_context) { - String auxiliary_zookeeper_name = extractZooKeeperName(from_); - String from = extractZooKeeperPath(from_); + Macros::MacrosExpansionInfo info; + info.expand_special_macros_only = false; + info.table_id = getStorageID(); + info.table_id.uuid = UUIDHelpers::Nil; + auto expand_from = query_context.getMacros()->expand(from_, info); + String auxiliary_zookeeper_name = extractZooKeeperName(expand_from); + String from = extractZooKeeperPath(expand_from); if (from.empty()) throw Exception("ZooKeeper path should not be empty", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); From 46c3266c6a9f927ec5f914f42943cec15e65ea35 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 7 Jan 2021 15:53:01 +0300 Subject: [PATCH 182/256] Respect MINSIGSTKSZ for alternative stack to fix under aarch64 And acording to sigaltstack(2) alignment is not required: When a signal handler is invoked on the alternate stack, the kernel automatically aligns the address given in ss.ss_sp to a suitable address boundary for the underlying hardware architecture. Fixes: #18785 --- src/Common/ThreadStatus.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index 3809f84711e..5105fff03b2 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -25,7 +25,7 @@ thread_local ThreadStatus * current_thread = nullptr; thread_local ThreadStatus * main_thread = nullptr; #if !defined(SANITIZER) && !defined(ARCADIA_BUILD) - alignas(4096) static thread_local char alt_stack[4096]; + alignas(4096) static thread_local char alt_stack[std::max(MINSIGSTKSZ, 4096)]; static thread_local bool has_alt_stack = false; #endif From 8d3ec1cce9b378a7d1153ac5c83c770116173cf4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 7 Jan 2021 18:49:14 +0300 Subject: [PATCH 183/256] Fix fsync_part_directory (missing O_DIRECTORY) On linux you should pass O_DIRECTORY, otherwise open(2) will fail --- src/Common/FileSyncGuard.cpp | 32 ++++++++++++++++++++++++++++++++ src/Common/FileSyncGuard.h | 24 +++++++----------------- 2 files changed, 39 insertions(+), 17 deletions(-) create mode 100644 src/Common/FileSyncGuard.cpp diff --git a/src/Common/FileSyncGuard.cpp b/src/Common/FileSyncGuard.cpp new file mode 100644 index 00000000000..3c36b7c42e5 --- /dev/null +++ b/src/Common/FileSyncGuard.cpp @@ -0,0 +1,32 @@ +#include +#include +#include +#include // O_RDWR + +/// OSX does not have O_DIRECTORY +#ifndef O_DIRECTORY +#define O_DIRECTORY O_RDWR +#endif + +namespace DB +{ + +FileSyncGuard::FileSyncGuard(const DiskPtr & disk_, const String & path) + : disk(disk_) + , fd(disk_->open(path, O_DIRECTORY)) +{} + +FileSyncGuard::~FileSyncGuard() +{ + try + { + disk->sync(fd); + disk->close(fd); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} + +} diff --git a/src/Common/FileSyncGuard.h b/src/Common/FileSyncGuard.h index 486b02d0f24..f7ea00e0d36 100644 --- a/src/Common/FileSyncGuard.h +++ b/src/Common/FileSyncGuard.h @@ -1,10 +1,14 @@ #pragma once -#include +#include +#include namespace DB { +class IDisk; +using DiskPtr = std::shared_ptr; + /// Helper class, that receives file descriptor and does fsync for it in destructor. /// It's used to keep descriptor open, while doing some operations with it, and do fsync at the end. /// Guaranties of sequence 'close-reopen-fsync' may depend on kernel version. @@ -15,22 +19,8 @@ public: /// NOTE: If you have already opened descriptor, it's preferred to use /// this constructor instead of constructor with path. FileSyncGuard(const DiskPtr & disk_, int fd_) : disk(disk_), fd(fd_) {} - - FileSyncGuard(const DiskPtr & disk_, const String & path) - : disk(disk_), fd(disk_->open(path, O_RDWR)) {} - - ~FileSyncGuard() - { - try - { - disk->sync(fd); - disk->close(fd); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } + FileSyncGuard(const DiskPtr & disk_, const std::string & path); + ~FileSyncGuard(); private: DiskPtr disk; From 52f56cd6ad22ca67fb8e97365961d3ba24abf9d5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 7 Jan 2021 19:01:10 +0300 Subject: [PATCH 184/256] Add support of F_FULLSYNC on osx --- src/Common/FileSyncGuard.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Common/FileSyncGuard.cpp b/src/Common/FileSyncGuard.cpp index 3c36b7c42e5..53134bc3230 100644 --- a/src/Common/FileSyncGuard.cpp +++ b/src/Common/FileSyncGuard.cpp @@ -20,6 +20,10 @@ FileSyncGuard::~FileSyncGuard() { try { +#if defined(OS_DARWIN) + if (fcntl(fd, F_FULLFSYNC, 0)) + throwFromErrno("Cannot fcntl(F_FULLFSYNC)"); +#endif disk->sync(fd); disk->close(fd); } From 53268d9f0592c04bf0ca0feb1c80db903bcb34d1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 7 Jan 2021 19:17:02 +0300 Subject: [PATCH 185/256] Fix fsync_after_insert by not closing fd from finalize() But this may cause troubles, another option will be to split finalize() into finalize(bool sync) and finalizeImpl() This was introduced in #8356 --- src/IO/WriteBufferFromFile.h | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/IO/WriteBufferFromFile.h b/src/IO/WriteBufferFromFile.h index 77530c323d2..b7d58638113 100644 --- a/src/IO/WriteBufferFromFile.h +++ b/src/IO/WriteBufferFromFile.h @@ -50,11 +50,6 @@ public: /// Close file before destruction of object. void close(); - void finalize() override - { - close(); - } - std::string getFileName() const override { return file_name; From 513a824f30df80ff24abfb5935b8d5b85207e3a3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 7 Jan 2021 19:26:53 +0300 Subject: [PATCH 186/256] Fix fsync_part_directory for parts renames --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 0b4297e37b3..1fcdabd033a 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -835,10 +835,6 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ String from = getFullRelativePath(); String to = storage.relative_data_path + new_relative_path + "/"; - std::optional sync_guard; - if (storage.getSettings()->fsync_part_directory) - sync_guard.emplace(volume->getDisk(), to); - if (!volume->getDisk()->exists(from)) throw Exception("Part directory " + fullPath(volume->getDisk(), from) + " doesn't exist. Most likely it is a logical error.", ErrorCodes::FILE_DOESNT_EXIST); @@ -862,6 +858,10 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ volume->getDisk()->setLastModified(from, Poco::Timestamp::fromEpochTime(time(nullptr))); volume->getDisk()->moveFile(from, to); relative_path = new_relative_path; + + std::optional sync_guard; + if (storage.getSettings()->fsync_part_directory) + sync_guard.emplace(volume->getDisk(), to); } From 798fb4e842ed88bfe788c5de9ccac9d9986586d3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 7 Jan 2021 19:00:09 +0300 Subject: [PATCH 187/256] Smoke coverage of various fsync capabilities --- .../01643_merge_tree_fsync_smoke.reference | 12 ++++++ .../01643_merge_tree_fsync_smoke.sql | 37 +++++++++++++++++++ 2 files changed, 49 insertions(+) create mode 100644 tests/queries/0_stateless/01643_merge_tree_fsync_smoke.reference create mode 100644 tests/queries/0_stateless/01643_merge_tree_fsync_smoke.sql diff --git a/tests/queries/0_stateless/01643_merge_tree_fsync_smoke.reference b/tests/queries/0_stateless/01643_merge_tree_fsync_smoke.reference new file mode 100644 index 00000000000..654db9dbc86 --- /dev/null +++ b/tests/queries/0_stateless/01643_merge_tree_fsync_smoke.reference @@ -0,0 +1,12 @@ +default +1 +compact fsync_after_insert +1 +compact fsync_after_insert,fsync_part_directory +1 +wide fsync_after_insert +1 +wide fsync_after_insert,fsync_part_directory +1 +memory in_memory_parts_insert_sync +1 diff --git a/tests/queries/0_stateless/01643_merge_tree_fsync_smoke.sql b/tests/queries/0_stateless/01643_merge_tree_fsync_smoke.sql new file mode 100644 index 00000000000..21ebb607693 --- /dev/null +++ b/tests/queries/0_stateless/01643_merge_tree_fsync_smoke.sql @@ -0,0 +1,37 @@ +drop table if exists data_01643; + +select 'default'; +create table data_01643 (key Int) engine=MergeTree() order by key; +insert into data_01643 values (1); +select * from data_01643; +drop table data_01643; + +select 'compact fsync_after_insert'; +create table data_01643 (key Int) engine=MergeTree() order by key settings min_rows_for_wide_part=2, fsync_after_insert=1; +insert into data_01643 values (1); +select * from data_01643; +drop table data_01643; + +select 'compact fsync_after_insert,fsync_part_directory'; +create table data_01643 (key Int) engine=MergeTree() order by key settings min_rows_for_wide_part=2, fsync_after_insert=1, fsync_part_directory=1; +insert into data_01643 values (1); +select * from data_01643; +drop table data_01643; + +select 'wide fsync_after_insert'; +create table data_01643 (key Int) engine=MergeTree() order by key settings min_bytes_for_wide_part=0, fsync_after_insert=1; +insert into data_01643 values (1); +select * from data_01643; +drop table data_01643; + +select 'wide fsync_after_insert,fsync_part_directory'; +create table data_01643 (key Int) engine=MergeTree() order by key settings min_bytes_for_wide_part=0, fsync_after_insert=1, fsync_part_directory=1; +insert into data_01643 values (1); +select * from data_01643; +drop table data_01643; + +select 'memory in_memory_parts_insert_sync'; +create table data_01643 (key Int) engine=MergeTree() order by key settings min_rows_for_compact_part=2, in_memory_parts_insert_sync=1, fsync_after_insert=1, fsync_part_directory=1; +insert into data_01643 values (1); +select * from data_01643; +drop table data_01643; From 0fc2020f2abe1d69e8c5dff4f38740e5ebc09ea7 Mon Sep 17 00:00:00 2001 From: myrrc Date: Thu, 7 Jan 2021 19:38:58 +0300 Subject: [PATCH 188/256] fix gcc build --- contrib/CMakeLists.txt | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index be3d3f86348..12078cea263 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -18,7 +18,11 @@ if (WITH_COVERAGE) set (WITHOUT_COVERAGE_LIST ${WITHOUT_COVERAGE}) separate_arguments(WITHOUT_COVERAGE_LIST) # disable coverage for contib files and build with optimisations - add_compile_options(-O3 -DNDEBUG -finline-functions -finline-hint-functions ${WITHOUT_COVERAGE_LIST}) + if (COMPILER_CLANG) + add_compile_options(-O3 -DNDEBUG -finline-functions -finline-hint-functions ${WITHOUT_COVERAGE_LIST}) + else() + add_compile_options(-O3 -DNDEBUG -finline-functions ${WITHOUT_COVERAGE_LIST}) + endif() endif() if (SANITIZE STREQUAL "undefined") From b1f08f5c27435cf9dad36a771f6b3a2bf619c8b2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 7 Jan 2021 20:26:18 +0300 Subject: [PATCH 189/256] Rename FileSyncGuard to DirectorySyncGuard --- src/Common/{FileSyncGuard.cpp => DirectorySyncGuard.cpp} | 6 +++--- src/Common/{FileSyncGuard.h => DirectorySyncGuard.h} | 8 ++++---- src/Storages/MergeTree/DataPartsExchange.cpp | 4 ++-- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 6 +++--- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 4 ++-- 6 files changed, 16 insertions(+), 16 deletions(-) rename src/Common/{FileSyncGuard.cpp => DirectorySyncGuard.cpp} (76%) rename src/Common/{FileSyncGuard.h => DirectorySyncGuard.h} (76%) diff --git a/src/Common/FileSyncGuard.cpp b/src/Common/DirectorySyncGuard.cpp similarity index 76% rename from src/Common/FileSyncGuard.cpp rename to src/Common/DirectorySyncGuard.cpp index 53134bc3230..f8f7c153f03 100644 --- a/src/Common/FileSyncGuard.cpp +++ b/src/Common/DirectorySyncGuard.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include // O_RDWR @@ -11,12 +11,12 @@ namespace DB { -FileSyncGuard::FileSyncGuard(const DiskPtr & disk_, const String & path) +DirectorySyncGuard::DirectorySyncGuard(const DiskPtr & disk_, const String & path) : disk(disk_) , fd(disk_->open(path, O_DIRECTORY)) {} -FileSyncGuard::~FileSyncGuard() +DirectorySyncGuard::~DirectorySyncGuard() { try { diff --git a/src/Common/FileSyncGuard.h b/src/Common/DirectorySyncGuard.h similarity index 76% rename from src/Common/FileSyncGuard.h rename to src/Common/DirectorySyncGuard.h index f7ea00e0d36..062d20324ed 100644 --- a/src/Common/FileSyncGuard.h +++ b/src/Common/DirectorySyncGuard.h @@ -13,14 +13,14 @@ using DiskPtr = std::shared_ptr; /// It's used to keep descriptor open, while doing some operations with it, and do fsync at the end. /// Guaranties of sequence 'close-reopen-fsync' may depend on kernel version. /// Source: linux-fsdevel mailing-list https://marc.info/?l=linux-fsdevel&m=152535409207496 -class FileSyncGuard +class DirectorySyncGuard { public: /// NOTE: If you have already opened descriptor, it's preferred to use /// this constructor instead of constructor with path. - FileSyncGuard(const DiskPtr & disk_, int fd_) : disk(disk_), fd(fd_) {} - FileSyncGuard(const DiskPtr & disk_, const std::string & path); - ~FileSyncGuard(); + DirectorySyncGuard(const DiskPtr & disk_, int fd_) : disk(disk_), fd(fd_) {} + DirectorySyncGuard(const DiskPtr & disk_, const std::string & path); + ~DirectorySyncGuard(); private: DiskPtr disk; diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index f85f9501fcf..5d50f29756c 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include #include @@ -398,7 +398,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk( disk->createDirectories(part_download_path); - std::optional sync_guard; + std::optional sync_guard; if (data.getSettings()->fsync_part_directory) sync_guard.emplace(disk, part_download_path); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 1fcdabd033a..5d0f79f4679 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include #include #include @@ -859,7 +859,7 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ volume->getDisk()->moveFile(from, to); relative_path = new_relative_path; - std::optional sync_guard; + std::optional sync_guard; if (storage.getSettings()->fsync_part_directory) sync_guard.emplace(volume->getDisk(), to); } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 64a0e7329ee..83ad57fa6ab 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -29,7 +29,7 @@ #include #include #include -#include +#include #include #include @@ -780,7 +780,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor gathering_column_names.clear(); } - std::optional sync_guard; + std::optional sync_guard; if (data.getSettings()->fsync_part_directory) sync_guard.emplace(disk, new_part_tmp_path); @@ -1182,7 +1182,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor disk->createDirectories(new_part_tmp_path); - std::optional sync_guard; + std::optional sync_guard; if (data.getSettings()->fsync_part_directory) sync_guard.emplace(disk, new_part_tmp_path); diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index c93d4bceba0..de4d70d5e3e 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include @@ -362,7 +362,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa new_data_part->minmax_idx = std::move(minmax_idx); new_data_part->is_temp = true; - std::optional sync_guard; + std::optional sync_guard; if (new_data_part->isStoredOnDisk()) { /// The name could be non-unique in case of stale files from previous runs. From 9a09044da6b582538336521a218b61c8307fae06 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 7 Jan 2021 20:41:43 +0300 Subject: [PATCH 190/256] Add DirectorySyncGuard.cpp into ya.make --- src/Common/ya.make | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/ya.make b/src/Common/ya.make index 5d2534c3547..d60c16f4be4 100644 --- a/src/Common/ya.make +++ b/src/Common/ya.make @@ -37,6 +37,7 @@ SRCS( CurrentMetrics.cpp CurrentThread.cpp DNSResolver.cpp + DirectorySyncGuard.cpp Dwarf.cpp Elf.cpp ErrorCodes.cpp From 32d8785a26a99ab84059b46725f4893b07e76994 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Jan 2021 21:36:38 +0300 Subject: [PATCH 191/256] Better kill #18847 --- base/daemon/BaseDaemon.cpp | 5 +++-- base/daemon/BaseDaemon.h | 2 +- src/Interpreters/InterpreterSystemQuery.cpp | 8 +++++--- 3 files changed, 9 insertions(+), 6 deletions(-) diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index 1224d9171ea..0ea5dea6406 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -493,8 +493,9 @@ void BaseDaemon::kill() { dumpCoverageReportIfPossible(); pid_file.reset(); - if (::raise(SIGKILL) != 0) - throw Poco::SystemException("cannot kill process"); + /// Exit with the same code as it is usually set by shell when process is terminated by SIGKILL. + /// It's better than doing 'raise' or 'kill', because they have no effect for 'init' process (with pid = 0, usually in Docker). + _exit(128 + SIGKILL); } std::string BaseDaemon::getDefaultCorePath() const diff --git a/base/daemon/BaseDaemon.h b/base/daemon/BaseDaemon.h index 090d4997606..ed6b5d21434 100644 --- a/base/daemon/BaseDaemon.h +++ b/base/daemon/BaseDaemon.h @@ -60,7 +60,7 @@ public: static void terminate(); /// Forceful shutdown - void kill(); + [[noreturn]] void kill(); /// Cancellation request has been received. bool isCancelled() const diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index fd36f3a6fd6..e583c11c00d 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -227,9 +227,11 @@ BlockIO InterpreterSystemQuery::execute() break; case Type::KILL: context.checkAccess(AccessType::SYSTEM_SHUTDOWN); - if (kill(0, SIGKILL)) - throwFromErrno("System call kill(0, SIGKILL) failed", ErrorCodes::CANNOT_KILL); - break; + /// Exit with the same code as it is usually set by shell when process is terminated by SIGKILL. + /// It's better than doing 'raise' or 'kill', because they have no effect for 'init' process (with pid = 0, usually in Docker). + LOG_INFO(log, "Exit immediately as the SYSTEM KILL command has been issued."); + _exit(128 + SIGKILL); + // break; /// unreachable case Type::DROP_DNS_CACHE: context.checkAccess(AccessType::SYSTEM_DROP_DNS_CACHE); DNSResolver::instance().dropCache(); From ffa5bbd5fdf88a9dda6c715542173fdccb7c6e7d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Jan 2021 22:19:33 +0300 Subject: [PATCH 192/256] Add SYSTEM SUSPEND command #15979 --- src/Interpreters/InterpreterSystemQuery.cpp | 31 ++++++++++++++++++- src/Parsers/ASTSystemQuery.cpp | 15 ++++++++- src/Parsers/ASTSystemQuery.h | 4 ++- src/Parsers/ParserSystemQuery.cpp | 14 +++++++++ .../01643_system_suspend.reference | 1 + .../0_stateless/01643_system_suspend.sql | 5 +++ 6 files changed, 67 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/01643_system_suspend.reference create mode 100644 tests/queries/0_stateless/01643_system_suspend.sql diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index fd36f3a6fd6..2a9c4cdb259 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -221,21 +222,40 @@ BlockIO InterpreterSystemQuery::execute() switch (query.type) { case Type::SHUTDOWN: + { context.checkAccess(AccessType::SYSTEM_SHUTDOWN); if (kill(0, SIGTERM)) throwFromErrno("System call kill(0, SIGTERM) failed", ErrorCodes::CANNOT_KILL); break; + } case Type::KILL: + { context.checkAccess(AccessType::SYSTEM_SHUTDOWN); if (kill(0, SIGKILL)) throwFromErrno("System call kill(0, SIGKILL) failed", ErrorCodes::CANNOT_KILL); break; + } + case Type::SUSPEND: + { + auto command = fmt::format("kill -STOP {0} && sleep {1} && kill -CONT {0}", getpid(), query.seconds); + LOG_DEBUG(log, "Will run {}", command); + auto res = ShellCommand::execute(command); + res->in.close(); + WriteBufferFromOwnString out; + copyData(res->out, out); + copyData(res->err, out); + if (!out.str().empty()) + LOG_DEBUG(log, "The command returned output: {}", command, out.str()); + break; + } case Type::DROP_DNS_CACHE: + { context.checkAccess(AccessType::SYSTEM_DROP_DNS_CACHE); DNSResolver::instance().dropCache(); /// Reinitialize clusters to update their resolved_addresses system_context.reloadClusterConfig(); break; + } case Type::DROP_MARK_CACHE: context.checkAccess(AccessType::SYSTEM_DROP_MARK_CACHE); system_context.dropMarkCache(); @@ -251,12 +271,15 @@ BlockIO InterpreterSystemQuery::execute() break; #endif case Type::RELOAD_DICTIONARY: + { context.checkAccess(AccessType::SYSTEM_RELOAD_DICTIONARY); system_context.getExternalDictionariesLoader().loadOrReload( DatabaseCatalog::instance().resolveDictionaryName(query.target_dictionary)); ExternalDictionariesLoader::resetAll(); break; + } case Type::RELOAD_DICTIONARIES: + { context.checkAccess(AccessType::SYSTEM_RELOAD_DICTIONARY); executeCommandsAndThrowIfError( [&] () { system_context.getExternalDictionariesLoader().reloadAllTriedToLoad(); }, @@ -264,6 +287,7 @@ BlockIO InterpreterSystemQuery::execute() ); ExternalDictionariesLoader::resetAll(); break; + } case Type::RELOAD_EMBEDDED_DICTIONARIES: context.checkAccess(AccessType::SYSTEM_RELOAD_EMBEDDED_DICTIONARIES); system_context.getEmbeddedDictionaries().reload(); @@ -273,6 +297,7 @@ BlockIO InterpreterSystemQuery::execute() system_context.reloadConfig(); break; case Type::RELOAD_SYMBOLS: + { #if defined(__ELF__) && !defined(__FreeBSD__) context.checkAccess(AccessType::SYSTEM_RELOAD_SYMBOLS); (void)SymbolIndex::instance(true); @@ -280,6 +305,7 @@ BlockIO InterpreterSystemQuery::execute() #else throw Exception("SYSTEM RELOAD SYMBOLS is not supported on current platform", ErrorCodes::NOT_IMPLEMENTED); #endif + } case Type::STOP_MERGES: startStopAction(ActionLocks::PartsMerge, false); break; @@ -340,6 +366,7 @@ BlockIO InterpreterSystemQuery::execute() ErrorCodes::BAD_ARGUMENTS); break; case Type::FLUSH_LOGS: + { context.checkAccess(AccessType::SYSTEM_FLUSH_LOGS); executeCommandsAndThrowIfError( [&] () { if (auto query_log = context.getQueryLog()) query_log->flush(true); }, @@ -352,6 +379,7 @@ BlockIO InterpreterSystemQuery::execute() [&] () { if (auto opentelemetry_span_log = context.getOpenTelemetrySpanLog()) opentelemetry_span_log->flush(true); } ); break; + } case Type::STOP_LISTEN_QUERIES: case Type::START_LISTEN_QUERIES: throw Exception(String(ASTSystemQuery::typeToString(query.type)) + " is not supported yet", ErrorCodes::NOT_IMPLEMENTED); @@ -586,7 +614,8 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() switch (query.type) { case Type::SHUTDOWN: [[fallthrough]]; - case Type::KILL: + case Type::KILL: [[fallthrough]]; + case Type::SUSPEND: { required_access.emplace_back(AccessType::SYSTEM_SHUTDOWN); break; diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index 0d6e15a3d8c..f3a43d7f3fd 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -22,6 +22,8 @@ const char * ASTSystemQuery::typeToString(Type type) return "SHUTDOWN"; case Type::KILL: return "KILL"; + case Type::SUSPEND: + return "SUSPEND"; case Type::DROP_DNS_CACHE: return "DROP DNS CACHE"; case Type::DROP_MARK_CACHE: @@ -146,7 +148,7 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &, auto print_on_volume = [&] { - settings.ostr << " ON VOLUME " + settings.ostr << (settings.hilite ? hilite_keyword : "") << " ON VOLUME " << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(storage_policy) << (settings.hilite ? hilite_none : "") << "." @@ -182,9 +184,20 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &, print_database_table(); } else if (type == Type::RELOAD_DICTIONARY) + { print_database_dictionary(); + } else if (type == Type::DROP_REPLICA) + { print_drop_replica(); + } + else if (type == Type::SUSPEND) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " FOR " + << (settings.hilite ? hilite_none : "") << seconds + << (settings.hilite ? hilite_keyword : "") << " SECOND" + << (settings.hilite ? hilite_none : ""); + } } diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 756b5b52600..ad7eb664659 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -20,6 +20,7 @@ public: UNKNOWN, SHUTDOWN, KILL, + SUSPEND, DROP_DNS_CACHE, DROP_MARK_CACHE, DROP_UNCOMPRESSED_CACHE, @@ -65,9 +66,10 @@ public: String table; String replica; String replica_zk_path; - bool is_drop_whole_replica; + bool is_drop_whole_replica{}; String storage_policy; String volume; + UInt64 seconds{}; String getID(char) const override { return "SYSTEM query"; } diff --git a/src/Parsers/ParserSystemQuery.cpp b/src/Parsers/ParserSystemQuery.cpp index b6a90b348a0..491037da9a9 100644 --- a/src/Parsers/ParserSystemQuery.cpp +++ b/src/Parsers/ParserSystemQuery.cpp @@ -169,6 +169,20 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & parseDatabaseAndTableName(pos, expected, res->database, res->table); break; + case Type::SUSPEND: + { + ASTPtr seconds; + if (!(ParserKeyword{"FOR"}.ignore(pos, expected) + && ParserUnsignedInteger().parse(pos, seconds, expected) + && ParserKeyword{"SECOND"}.ignore(pos, expected))) /// SECOND, not SECONDS to be consistent with INTERVAL parsing in SQL + { + return false; + } + + res->seconds = seconds->as()->value.get(); + break; + } + default: /// There are no [db.table] after COMMAND NAME break; diff --git a/tests/queries/0_stateless/01643_system_suspend.reference b/tests/queries/0_stateless/01643_system_suspend.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/01643_system_suspend.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/01643_system_suspend.sql b/tests/queries/0_stateless/01643_system_suspend.sql new file mode 100644 index 00000000000..c2cd37e6156 --- /dev/null +++ b/tests/queries/0_stateless/01643_system_suspend.sql @@ -0,0 +1,5 @@ +CREATE TEMPORARY TABLE t (x DateTime); +INSERT INTO t VALUES (now()); +SYSTEM SUSPEND FOR 1 SECOND; +INSERT INTO t VALUES (now()); +SELECT max(x) - min(x) >= 1 FROM t; From 9cbb9c6d7c7bde5709f84efae61d343db7154868 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Jan 2021 22:22:41 +0300 Subject: [PATCH 193/256] Add SYSTEM SUSPEND command #15979, continued --- src/Interpreters/InterpreterSystemQuery.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 2a9c4cdb259..c0715409011 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -246,6 +246,7 @@ BlockIO InterpreterSystemQuery::execute() copyData(res->err, out); if (!out.str().empty()) LOG_DEBUG(log, "The command returned output: {}", command, out.str()); + res->wait(); break; } case Type::DROP_DNS_CACHE: From 7b9f670cc3daacabdb91ec1ba031261105d455cb Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 7 Jan 2021 23:34:08 +0300 Subject: [PATCH 194/256] Fix DirectorySyncGuard under OSX --- src/Common/DirectorySyncGuard.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Common/DirectorySyncGuard.cpp b/src/Common/DirectorySyncGuard.cpp index f8f7c153f03..f279a0d25d5 100644 --- a/src/Common/DirectorySyncGuard.cpp +++ b/src/Common/DirectorySyncGuard.cpp @@ -11,6 +11,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int CANNOT_FSYNC; +} + DirectorySyncGuard::DirectorySyncGuard(const DiskPtr & disk_, const String & path) : disk(disk_) , fd(disk_->open(path, O_DIRECTORY)) @@ -22,7 +27,7 @@ DirectorySyncGuard::~DirectorySyncGuard() { #if defined(OS_DARWIN) if (fcntl(fd, F_FULLFSYNC, 0)) - throwFromErrno("Cannot fcntl(F_FULLFSYNC)"); + throwFromErrno("Cannot fcntl(F_FULLFSYNC)", ErrorCodes::CANNOT_FSYNC); #endif disk->sync(fd); disk->close(fd); From f2ff785460ba5876fd608680a374861498c0bb91 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 8 Jan 2021 02:18:57 +0300 Subject: [PATCH 195/256] Do not check bit flips for big buffers (since the size can be corrupted) --- src/Compression/CompressedReadBufferBase.cpp | 27 ++++++++++++-------- 1 file changed, 16 insertions(+), 11 deletions(-) diff --git a/src/Compression/CompressedReadBufferBase.cpp b/src/Compression/CompressedReadBufferBase.cpp index 8b16b68a999..74c902ce492 100644 --- a/src/Compression/CompressedReadBufferBase.cpp +++ b/src/Compression/CompressedReadBufferBase.cpp @@ -67,20 +67,25 @@ static void validateChecksum(char * data, size_t size, const Checksum expected_c buf[pos / 8] ^= 1 << pos % 8; }; - /// Check if the difference caused by single bit flip in data. - for (size_t bit_pos = 0; bit_pos < size * 8; ++bit_pos) + /// If size is too huge, then this may be caused by corruption. + /// And anyway this is pretty heavy, so avoid burning too much CPU here. + if (size < 1<<20) { - flip_bit(data, bit_pos); - - auto checksum_of_data_with_flipped_bit = CityHash_v1_0_2::CityHash128(data, size); - if (expected_checksum == checksum_of_data_with_flipped_bit) + /// Check if the difference caused by single bit flip in data. + for (size_t bit_pos = 0; bit_pos < size * 8; ++bit_pos) { - message << ". The mismatch is caused by single bit flip in data block at byte " << (bit_pos / 8) << ", bit " << (bit_pos % 8) << ". " - << message_hardware_failure; - throw Exception(message.str(), ErrorCodes::CHECKSUM_DOESNT_MATCH); - } + flip_bit(data, bit_pos); - flip_bit(data, bit_pos); /// Restore + auto checksum_of_data_with_flipped_bit = CityHash_v1_0_2::CityHash128(data, size); + if (expected_checksum == checksum_of_data_with_flipped_bit) + { + message << ". The mismatch is caused by single bit flip in data block at byte " << (bit_pos / 8) << ", bit " << (bit_pos % 8) << ". " + << message_hardware_failure; + throw Exception(message.str(), ErrorCodes::CHECKSUM_DOESNT_MATCH); + } + + flip_bit(data, bit_pos); /// Restore + } } /// Check if the difference caused by single bit flip in stored checksum. From 4f0068bdde847c6126ec92f64110fec2279d6149 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Thu, 7 Jan 2021 18:42:54 -0500 Subject: [PATCH 196/256] Fixing inproper usage of And step. --- .../ldap/external_user_directory/tests/authentications.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/testflows/ldap/external_user_directory/tests/authentications.py b/tests/testflows/ldap/external_user_directory/tests/authentications.py index ced54e95115..b47a85e3e92 100644 --- a/tests/testflows/ldap/external_user_directory/tests/authentications.py +++ b/tests/testflows/ldap/external_user_directory/tests/authentications.py @@ -968,7 +968,7 @@ def verification_cooldown_performance(self, server, rbac=False, iterations=5000) no_vcd_time = repeat_requests(server=server, iterations=iterations, vcd_value="0", rbac=rbac) metric("login_with_vcd_value_0", units="seconds", value=no_vcd_time) - with And("Log the performance improvement as a percentage."): + with Then("Log the performance improvement as a percentage"): metric("percentage_improvement", units="%", value=100*(no_vcd_time - vcd_time)/vcd_time) @TestOutline From a3d19fa64df8fc70fc6b6e5a0bc434721ae571e7 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 8 Jan 2021 12:28:09 +0800 Subject: [PATCH 197/256] Correctly override default settings remotely --- src/Core/Settings.h | 2 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 +-- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- .../__init__.py | 0 .../test.py | 28 +++++++++++++++++++ 5 files changed, 32 insertions(+), 4 deletions(-) create mode 100644 tests/integration/test_passing_max_partitions_to_read_remotely/__init__.py create mode 100644 tests/integration/test_passing_max_partitions_to_read_remotely/test.py diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e2aa4e2f2d8..d181820c911 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -353,7 +353,7 @@ class IColumn; M(Bool, allow_introspection_functions, false, "Allow functions for introspection of ELF and DWARF for query profiling. These functions are slow and may impose security considerations.", 0) \ \ M(UInt64, max_partitions_per_insert_block, 100, "Limit maximum number of partitions in single INSERTed block. Zero means unlimited. Throw exception if the block contains too many partitions. This setting is a safety threshold, because using large number of partitions is a common misconception.", 0) \ - M(UInt64, max_partitions_to_read, 0, "Limit the max number of partitions that can be accessed in one query. 0 means unlimited.", 0) \ + M(Int64, max_partitions_to_read, -1, "Limit the max number of partitions that can be accessed in one query. <= 0 means unlimited.", 0) \ M(Bool, check_query_single_value_result, true, "Return check query result as single 1/0 value", 0) \ M(Bool, allow_drop_detached, false, "Allow ALTER TABLE ... DROP DETACHED PART[ITION] ... queries", 0) \ \ diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 75760e145d3..4e1f307137a 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -709,12 +709,12 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( auto max_partitions_to_read = settings.max_partitions_to_read.changed ? settings.max_partitions_to_read : data.getSettings()->max_partitions_to_read; - if (max_partitions_to_read) + if (max_partitions_to_read > 0) { std::set partitions; for (auto & part_with_ranges : parts_with_ranges) partitions.insert(part_with_ranges.data_part->info.partition_id); - if (partitions.size() > max_partitions_to_read) + if (partitions.size() > size_t(max_partitions_to_read)) throw Exception( ErrorCodes::TOO_MANY_PARTITIONS, "Too many partitions to read. Current {}, max {}", diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 97c6114a0be..3e811336fc6 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -110,7 +110,7 @@ struct Settings; M(Bool, allow_nullable_key, false, "Allow Nullable types as primary keys.", 0) \ M(Bool, remove_empty_parts, true, "Remove empty parts after they were pruned by TTL, mutation, or collapsing merge algorithm", 0) \ M(Bool, assign_part_uuids, false, "Generate UUIDs for parts. Before enabling check that all replicas support new format.", 0) \ - M(UInt64, max_partitions_to_read, 0, "Limit the max number of partitions that can be accessed in one query. 0 means unlimited. This setting is the default that can be overridden by the query-level setting with the same name.", 0) \ + M(Int64, max_partitions_to_read, -1, "Limit the max number of partitions that can be accessed in one query. <= 0 means unlimited. This setting is the default that can be overridden by the query-level setting with the same name.", 0) \ \ /** Obsolete settings. Kept for backward compatibility only. */ \ M(UInt64, min_relative_delay_to_yield_leadership, 120, "Obsolete setting, does nothing.", 0) \ diff --git a/tests/integration/test_passing_max_partitions_to_read_remotely/__init__.py b/tests/integration/test_passing_max_partitions_to_read_remotely/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_passing_max_partitions_to_read_remotely/test.py b/tests/integration/test_passing_max_partitions_to_read_remotely/test.py new file mode 100644 index 00000000000..45b3dd00b2a --- /dev/null +++ b/tests/integration/test_passing_max_partitions_to_read_remotely/test.py @@ -0,0 +1,28 @@ +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +ch1 = cluster.add_instance("ch1") +ch2 = cluster.add_instance("ch2") + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + ch1.query("CREATE DATABASE test_default_database;") + yield cluster + + finally: + cluster.shutdown() + + +def test_default_database_on_cluster(started_cluster): + ch1.query( + database="test_default_database", + sql="CREATE TABLE test_local_table ENGINE MergeTree PARTITION BY i ORDER BY tuple() SETTINGS max_partitions_to_read = 1 AS SELECT arrayJoin([1, 2]) i;", + ) + + assert ch2.query( + sql="SELECT * FROM remote('ch1:9000', test_default_database, test_local_table) ORDER BY i FORMAT TSV SETTINGS max_partitions_to_read = 0;", + ) == "1\n2\n" From 9e6cb0301532e691c81b5ae1ec2599b13157fedf Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 8 Jan 2021 07:31:29 +0300 Subject: [PATCH 198/256] Update index.md --- docs/zh/operations/system-tables/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/operations/system-tables/index.md b/docs/zh/operations/system-tables/index.md index 2dc1e2712e5..6aaca0a9630 100644 --- a/docs/zh/operations/system-tables/index.md +++ b/docs/zh/operations/system-tables/index.md @@ -22,7 +22,7 @@ toc_title: "\u7CFB\u7EDF\u8868" 大多数系统表将数据存储在RAM中。 ClickHouse服务器在开始时创建此类系统表。 -与其他系统表不同,系统日志表 [metric_log](../../operations/system-tables/metric_log.md#system_tables-metric_log), [query_log](../../operations/system-tables/query_log.md#system_tables-query_log), [query_thread_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log), [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log), [part_log](../../operations/system-tables/part_log.md#system.part_log), [crash_log](../../operations/system-tables/crash_log.md#system.crash_log) and [text_log](../../operations/system-tables/crash_log.md#system.text_log) 默认采用[MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) 引擎并将其数据存储在存储文件系统中。 如果从文件系统中删除表,ClickHouse服务器会在下一次写入数据时再次创建空表。 如果系统表架构在新版本中发生更改,则ClickHouse会重命名当前表并创建一个新表。 +与其他系统表不同,系统日志表 [metric_log](../../operations/system-tables/metric_log.md#system_tables-metric_log), [query_log](../../operations/system-tables/query_log.md#system_tables-query_log), [query_thread_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log), [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log), [part_log](../../operations/system-tables/part_log.md#system.part_log), crash_log and text_log 默认采用[MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) 引擎并将其数据存储在存储文件系统中。 如果从文件系统中删除表,ClickHouse服务器会在下一次写入数据时再次创建空表。 如果系统表架构在新版本中发生更改,则ClickHouse会重命名当前表并创建一个新表。 用户可以通过在`/etc/clickhouse-server/config.d/`下创建与系统表同名的配置文件, 或者在`/etc/clickhouse-server/config.xml`中设置相应配置项,来自定义系统日志表的结构。可以自定义的配置项如下: From 723d52e08d8aa59c86572446f6bcd89ea35df6ae Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 8 Jan 2021 07:32:19 +0300 Subject: [PATCH 199/256] Update index.md --- docs/en/operations/system-tables/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/index.md b/docs/en/operations/system-tables/index.md index 350ec7204c0..5ad95b48ea3 100644 --- a/docs/en/operations/system-tables/index.md +++ b/docs/en/operations/system-tables/index.md @@ -20,7 +20,7 @@ System tables: Most of system tables store their data in RAM. A ClickHouse server creates such system tables at the start. -Unlike other system tables, the system log tables [metric_log](../../operations/system-tables/metric-log.md), [query_log](../../operations/system-tables/query-log.md), [query_thread_log](../../operations/system-tables/query-thread-log.md), [trace_log](../../operations/system-tables/trace-log.md), [part_log](../../operations/system-tables/part-log.md), [crash_log](../../operations/system-tables/crash-log.md) and [text_log](../../operations/system-tables/text-log.md) are served by [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table engine and store their data in a storage filesystem by default. If you remove a table from a filesystem, the ClickHouse server creates the empty one again at the time of the next data writing. If system table schema changed in a new release, then ClickHouse renames the current table and creates a new one. +Unlike other system tables, the system log tables [metric_log](../../operations/system-tables/metric_log.md), [query_log](../../operations/system-tables/query_log.md), [query_thread_log](../../operations/system-tables/query_thread_log.md), [trace_log](../../operations/system-tables/trace_log.md), [part_log](../../operations/system-tables/part_log.md), [crash_log](../../operations/system-tables/crash_log.md) and [text_log](../../operations/system-tables/text_log.md) are served by [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table engine and store their data in a storage filesystem by default. If you remove a table from a filesystem, the ClickHouse server creates the empty one again at the time of the next data writing. If system table schema changed in a new release, then ClickHouse renames the current table and creates a new one. System log tables can be customized by creating a config file with the same name as the table under `/etc/clickhouse-server/config.d/`, or setting corresponding elements in `/etc/clickhouse-server/config.xml`. Elements can be customized are: From c9d37b194e099878556b0caa9eae3b9af0020ef4 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 8 Jan 2021 07:33:01 +0300 Subject: [PATCH 200/256] Update index.md --- docs/en/operations/system-tables/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/index.md b/docs/en/operations/system-tables/index.md index 5ad95b48ea3..479bed38bdd 100644 --- a/docs/en/operations/system-tables/index.md +++ b/docs/en/operations/system-tables/index.md @@ -20,7 +20,7 @@ System tables: Most of system tables store their data in RAM. A ClickHouse server creates such system tables at the start. -Unlike other system tables, the system log tables [metric_log](../../operations/system-tables/metric_log.md), [query_log](../../operations/system-tables/query_log.md), [query_thread_log](../../operations/system-tables/query_thread_log.md), [trace_log](../../operations/system-tables/trace_log.md), [part_log](../../operations/system-tables/part_log.md), [crash_log](../../operations/system-tables/crash_log.md) and [text_log](../../operations/system-tables/text_log.md) are served by [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table engine and store their data in a storage filesystem by default. If you remove a table from a filesystem, the ClickHouse server creates the empty one again at the time of the next data writing. If system table schema changed in a new release, then ClickHouse renames the current table and creates a new one. +Unlike other system tables, the system log tables [metric_log](../../operations/system-tables/metric_log.md), [query_log](../../operations/system-tables/query_log.md), [query_thread_log](../../operations/system-tables/query_thread_log.md), [trace_log](../../operations/system-tables/trace_log.md), [part_log](../../operations/system-tables/part_log.md), crash_log and [text_log](../../operations/system-tables/text_log.md) are served by [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table engine and store their data in a storage filesystem by default. If you remove a table from a filesystem, the ClickHouse server creates the empty one again at the time of the next data writing. If system table schema changed in a new release, then ClickHouse renames the current table and creates a new one. System log tables can be customized by creating a config file with the same name as the table under `/etc/clickhouse-server/config.d/`, or setting corresponding elements in `/etc/clickhouse-server/config.xml`. Elements can be customized are: From 10ea06edadf7bb0133075533f16c9c248a007ae3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Jan 2021 08:09:30 +0300 Subject: [PATCH 201/256] Import strsignal from Musl --- base/daemon/BaseDaemon.cpp | 30 +----- base/glibc-compatibility/musl/strsignal.c | 125 ++++++++++++++++++++++ base/harmful/harmful.c | 2 +- 3 files changed, 130 insertions(+), 27 deletions(-) create mode 100644 base/glibc-compatibility/musl/strsignal.c diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index 2c20eccf914..b35959d6b35 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -83,28 +83,6 @@ static void call_default_signal_handler(int sig) raise(sig); } -// Apparently strsignal is not instrumented by MemorySanitizer, so we -// have to unpoison it to avoid msan reports inside fmt library when we -// print it. -const char * msan_strsignal(int sig) -{ - // no glibc in osx/freebsd -#if !defined(__GLIBC_PREREQ) -#define __GLIBC_PREREQ(x, y) 0 -#endif - - // glibc 2.32+ deprecates sys_siglist[] - // newer glibc is a problem only for unbundled build. -#if __GLIBC_PREREQ(2, 32) - const char * signal_name = sigdescr_np(sig); -#else - const char * signal_name = sys_siglist[sig]; -#endif - - __msan_unpoison_string(signal_name); - return signal_name; -} - static constexpr size_t max_query_id_size = 127; static const size_t signal_pipe_buf_size = @@ -309,13 +287,13 @@ private: { LOG_FATAL(log, "(version {}{}, {}) (from thread {}) (no query) Received signal {} ({})", VERSION_STRING, VERSION_OFFICIAL, daemon.build_id_info, - thread_num, msan_strsignal(sig), sig); + thread_num, strsignal(sig), sig); } else { LOG_FATAL(log, "(version {}{}, {}) (from thread {}) (query_id: {}) Received signal {} ({})", VERSION_STRING, VERSION_OFFICIAL, daemon.build_id_info, - thread_num, query_id, msan_strsignal(sig), sig); + thread_num, query_id, strsignal(sig), sig); } String error_message; @@ -894,13 +872,13 @@ void BaseDaemon::handleSignal(int signal_id) onInterruptSignals(signal_id); } else - throw DB::Exception(std::string("Unsupported signal: ") + msan_strsignal(signal_id), 0); + throw DB::Exception(std::string("Unsupported signal: ") + strsignal(signal_id), 0); } void BaseDaemon::onInterruptSignals(int signal_id) { is_cancelled = true; - LOG_INFO(&logger(), "Received termination signal ({})", msan_strsignal(signal_id)); + LOG_INFO(&logger(), "Received termination signal ({})", strsignal(signal_id)); if (sigint_signals_counter >= 2) { diff --git a/base/glibc-compatibility/musl/strsignal.c b/base/glibc-compatibility/musl/strsignal.c new file mode 100644 index 00000000000..fee894e8550 --- /dev/null +++ b/base/glibc-compatibility/musl/strsignal.c @@ -0,0 +1,125 @@ +#include +#include + +#if (SIGHUP == 1) && (SIGINT == 2) && (SIGQUIT == 3) && (SIGILL == 4) \ + && (SIGTRAP == 5) && (SIGABRT == 6) && (SIGBUS == 7) && (SIGFPE == 8) \ + && (SIGKILL == 9) && (SIGUSR1 == 10) && (SIGSEGV == 11) && (SIGUSR2 == 12) \ + && (SIGPIPE == 13) && (SIGALRM == 14) && (SIGTERM == 15) && (SIGSTKFLT == 16) \ + && (SIGCHLD == 17) && (SIGCONT == 18) && (SIGSTOP == 19) && (SIGTSTP == 20) \ + && (SIGTTIN == 21) && (SIGTTOU == 22) && (SIGURG == 23) && (SIGXCPU == 24) \ + && (SIGXFSZ == 25) && (SIGVTALRM == 26) && (SIGPROF == 27) && (SIGWINCH == 28) \ + && (SIGPOLL == 29) && (SIGPWR == 30) && (SIGSYS == 31) + +#define sigmap(x) x + +#else + +static const char map[] = { + [SIGHUP] = 1, + [SIGINT] = 2, + [SIGQUIT] = 3, + [SIGILL] = 4, + [SIGTRAP] = 5, + [SIGABRT] = 6, + [SIGBUS] = 7, + [SIGFPE] = 8, + [SIGKILL] = 9, + [SIGUSR1] = 10, + [SIGSEGV] = 11, + [SIGUSR2] = 12, + [SIGPIPE] = 13, + [SIGALRM] = 14, + [SIGTERM] = 15, +#if defined(SIGSTKFLT) + [SIGSTKFLT] = 16, +#elif defined(SIGEMT) + [SIGEMT] = 16, +#endif + [SIGCHLD] = 17, + [SIGCONT] = 18, + [SIGSTOP] = 19, + [SIGTSTP] = 20, + [SIGTTIN] = 21, + [SIGTTOU] = 22, + [SIGURG] = 23, + [SIGXCPU] = 24, + [SIGXFSZ] = 25, + [SIGVTALRM] = 26, + [SIGPROF] = 27, + [SIGWINCH] = 28, + [SIGPOLL] = 29, + [SIGPWR] = 30, + [SIGSYS] = 31 +}; + +#define sigmap(x) ((x) >= sizeof map ? (x) : map[(x)]) + +#endif + +static const char strings[] = + "Unknown signal\0" + "Hangup\0" + "Interrupt\0" + "Quit\0" + "Illegal instruction\0" + "Trace/breakpoint trap\0" + "Aborted\0" + "Bus error\0" + "Arithmetic exception\0" + "Killed\0" + "User defined signal 1\0" + "Segmentation fault\0" + "User defined signal 2\0" + "Broken pipe\0" + "Alarm clock\0" + "Terminated\0" +#if defined(SIGSTKFLT) + "Stack fault\0" +#elif defined(SIGEMT) + "Emulator trap\0" +#else + "Unknown signal\0" +#endif + "Child process status\0" + "Continued\0" + "Stopped (signal)\0" + "Stopped\0" + "Stopped (tty input)\0" + "Stopped (tty output)\0" + "Urgent I/O condition\0" + "CPU time limit exceeded\0" + "File size limit exceeded\0" + "Virtual timer expired\0" + "Profiling timer expired\0" + "Window changed\0" + "I/O possible\0" + "Power failure\0" + "Bad system call\0" + "RT32" + "\0RT33\0RT34\0RT35\0RT36\0RT37\0RT38\0RT39\0RT40" + "\0RT41\0RT42\0RT43\0RT44\0RT45\0RT46\0RT47\0RT48" + "\0RT49\0RT50\0RT51\0RT52\0RT53\0RT54\0RT55\0RT56" + "\0RT57\0RT58\0RT59\0RT60\0RT61\0RT62\0RT63\0RT64" +#if _NSIG > 65 + "\0RT65\0RT66\0RT67\0RT68\0RT69\0RT70\0RT71\0RT72" + "\0RT73\0RT74\0RT75\0RT76\0RT77\0RT78\0RT79\0RT80" + "\0RT81\0RT82\0RT83\0RT84\0RT85\0RT86\0RT87\0RT88" + "\0RT89\0RT90\0RT91\0RT92\0RT93\0RT94\0RT95\0RT96" + "\0RT97\0RT98\0RT99\0RT100\0RT101\0RT102\0RT103\0RT104" + "\0RT105\0RT106\0RT107\0RT108\0RT109\0RT110\0RT111\0RT112" + "\0RT113\0RT114\0RT115\0RT116\0RT117\0RT118\0RT119\0RT120" + "\0RT121\0RT122\0RT123\0RT124\0RT125\0RT126\0RT127\0RT128" +#endif + ""; + +char *strsignal(int signum) +{ + const char *s = strings; + + signum = sigmap(signum); + if (signum - 1U >= _NSIG-1) signum = 0; + + for (; signum--; s++) for (; *s; s++); + + return (char *)s; +} diff --git a/base/harmful/harmful.c b/base/harmful/harmful.c index 4d08174170f..a99db6186ac 100644 --- a/base/harmful/harmful.c +++ b/base/harmful/harmful.c @@ -164,7 +164,7 @@ TRAP(sigsuspend) TRAP(sleep) TRAP(srand48) //TRAP(strerror) // Used by RocksDB and many other libraries, unfortunately. -TRAP(strsignal) +//TRAP(strsignal) // This function is imported from Musl and is thread safe. TRAP(strtok) TRAP(tcflow) TRAP(tcsendbreak) From a1d0c04e68b1bfed4b9e9219fbf61ae42b230489 Mon Sep 17 00:00:00 2001 From: fastio Date: Fri, 8 Jan 2021 13:10:00 +0800 Subject: [PATCH 202/256] fix build --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index d683c61215a..ebf1e43ca04 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4958,7 +4958,7 @@ void StorageReplicatedMergeTree::fetchPartition( const String & from_, const Context & query_context) { - Macros::MacrosExpansionInfo info; + Macros::MacroExpansionInfo info; info.expand_special_macros_only = false; info.table_id = getStorageID(); info.table_id.uuid = UUIDHelpers::Nil; From b4c6052dd98f81c49eb96fc2b4d9b0d15e72b861 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 8 Jan 2021 08:16:06 +0300 Subject: [PATCH 203/256] Update index.md --- docs/zh/operations/system-tables/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/operations/system-tables/index.md b/docs/zh/operations/system-tables/index.md index 6aaca0a9630..e1a56505569 100644 --- a/docs/zh/operations/system-tables/index.md +++ b/docs/zh/operations/system-tables/index.md @@ -50,7 +50,7 @@ toc_title: "\u7CFB\u7EDF\u8868" ``` -默认情况下,表增长是无限的。 要控制表的大小,可以使用 [TTL](../../sql-reference/statements/alter.md#manipulations-with-table-ttl) 删除过期日志记录的设置。 你也可以使用分区功能 `MergeTree`-发动机表。 +默认情况下,表增长是无限的。 要控制表的大小,可以使用 TTL 删除过期日志记录的设置。 你也可以使用分区功能 `MergeTree`-发动机表。 ## 系统指标的来源 {#system-tables-sources-of-system-metrics} From d0352d1cd6967634c800206fea0cf84a4e618672 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 8 Jan 2021 08:17:02 +0300 Subject: [PATCH 204/256] Update index.md --- docs/zh/operations/system-tables/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/operations/system-tables/index.md b/docs/zh/operations/system-tables/index.md index e1a56505569..56067bc5057 100644 --- a/docs/zh/operations/system-tables/index.md +++ b/docs/zh/operations/system-tables/index.md @@ -29,7 +29,7 @@ toc_title: "\u7CFB\u7EDF\u8868" - `database`: 系统日志表所在的数据库。这个选项目前已经废弃。所有的系统日表都位于`system`库中。 - `table`: 系统日志表名。 - `partition_by`: 指定[PARTITION BY](../../engines/table-engines/mergetree-family/custom-partitioning-key.md)表达式。 -- `ttl`: 指定系统日志表[TTL](../../sql-reference/statements/alter/ttl.md#manipulations-with-table-ttl)选项。 +- `ttl`: 指定系统日志表TTL选项。 - `flush_interval_milliseconds`: 指定系统日志表数据落盘时间。 - `engine`: 指定完整的表引擎定义。(以`ENGINE = `开始)。 这个选项与`partition_by`以及`ttl`冲突。如果两者一起设置,服务启动时会抛出异常并且退出。 From 4a2bf8b699d2c53ed9f054816b9e12dea5c5908e Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 8 Jan 2021 09:06:35 +0300 Subject: [PATCH 205/256] Update index.md --- docs/en/operations/system-tables/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/index.md b/docs/en/operations/system-tables/index.md index 479bed38bdd..5dc23aee686 100644 --- a/docs/en/operations/system-tables/index.md +++ b/docs/en/operations/system-tables/index.md @@ -27,7 +27,7 @@ System log tables can be customized by creating a config file with the same name - `database`: database the system log table belongs to. This option is deprecated now. All system log tables are under database `system`. - `table`: table to insert data. - `partition_by`: specify [PARTITION BY](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) expression. -- `ttl`: specify table [TTL](../../sql-reference/statements/alter.md) expression. +- `ttl`: specify table [TTL](../../sql-reference/statements/alter/ttl.md) expression. - `flush_interval_milliseconds`: interval of flushing data to disk. - `engine`: provide full engine expression (starting with `ENGINE =` ) with parameters. This option is contradict with `partition_by` and `ttl`. If set together, the server would raise an exception and exit. From 0506d4bf47c79f50024e12967155878e2968de21 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Fri, 8 Jan 2021 15:41:15 +0800 Subject: [PATCH 206/256] Update contrib from ClickHouse-Extras/rocksdb --- contrib/rocksdb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/rocksdb b/contrib/rocksdb index 8b966f0ca29..54a0decabbc 160000 --- a/contrib/rocksdb +++ b/contrib/rocksdb @@ -1 +1 @@ -Subproject commit 8b966f0ca298fc1475bd09d9775f32dff0fdce0a +Subproject commit 54a0decabbcf4c0bb5cf7befa9c597f28289bff5 From 64cb6405ac89d25e0cab19023b61629bd69b0485 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 8 Jan 2021 14:20:13 +0300 Subject: [PATCH 207/256] Drop IStorage::supportsSettings() (replaced with StorageFeatures::supports_settings) --- src/Storages/IStorage.h | 3 --- src/Storages/Kafka/StorageKafka.h | 1 - src/Storages/MergeTree/MergeTreeData.h | 1 - src/Storages/RabbitMQ/StorageRabbitMQ.h | 1 - src/Storages/StorageProxy.h | 1 - 5 files changed, 7 deletions(-) diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 42ece547e1c..cfb4c4e9646 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -120,9 +120,6 @@ public: /// Returns true if the storage supports deduplication of inserted data blocks. virtual bool supportsDeduplication() const { return false; } - /// Returns true if the storage supports settings. - virtual bool supportsSettings() const { return false; } - /// Returns true if the blocks shouldn't be pushed to associated views on insert. virtual bool noPushingToViews() const { return false; } diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index 8ec8e718011..53871990810 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -36,7 +36,6 @@ class StorageKafka final : public ext::shared_ptr_helper, public I public: std::string getName() const override { return "Kafka"; } - bool supportsSettings() const override { return true; } bool noPushingToViews() const override { return true; } void startup() override; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index e5ffe8c025b..52c0b61b977 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -357,7 +357,6 @@ public: || merging_params.mode == MergingParams::VersionedCollapsing; } - bool supportsSettings() const override { return true; } NamesAndTypesList getVirtuals() const override; bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context &, const StorageMetadataPtr & metadata_snapshot) const override; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index a46da6072af..893c5167a97 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -29,7 +29,6 @@ class StorageRabbitMQ final: public ext::shared_ptr_helper, pub public: std::string getName() const override { return "RabbitMQ"; } - bool supportsSettings() const override { return true; } bool noPushingToViews() const override { return true; } void startup() override; diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h index b7b948af4ba..fed9dd04e76 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -25,7 +25,6 @@ public: bool supportsReplication() const override { return getNested()->supportsReplication(); } bool supportsParallelInsert() const override { return getNested()->supportsParallelInsert(); } bool supportsDeduplication() const override { return getNested()->supportsDeduplication(); } - bool supportsSettings() const override { return getNested()->supportsSettings(); } bool noPushingToViews() const override { return getNested()->noPushingToViews(); } bool hasEvenlyDistributedRead() const override { return getNested()->hasEvenlyDistributedRead(); } From c7d0c2d075d86f6a04a8c691af9c0613d670a4b5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 8 Jan 2021 14:36:42 +0300 Subject: [PATCH 208/256] Add some comments for StorageFeatures --- src/Storages/StorageFactory.h | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Storages/StorageFactory.h b/src/Storages/StorageFactory.h index de9060769cb..b13d4d6f679 100644 --- a/src/Storages/StorageFactory.h +++ b/src/Storages/StorageFactory.h @@ -47,13 +47,17 @@ public: bool has_force_restore_data_flag; }; + /// Analog of the IStorage::supports*() helpers + /// (But the former cannot be replaced with StorageFeatures due to nesting) struct StorageFeatures { bool supports_settings = false; bool supports_skipping_indices = false; bool supports_sort_order = false; bool supports_ttl = false; + /// See also IStorage::supportsReplication() bool supports_replication = false; + /// See also IStorage::supportsDeduplication() bool supports_deduplication = false; AccessType source_access_type = AccessType::NONE; }; From 714d5a067a8bfb770925d05f7fa7f6577c5b5af6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 8 Jan 2021 14:42:17 +0300 Subject: [PATCH 209/256] Expose supports_parallel_insert via system.table_engines --- .../MergeTree/registerStorageMergeTree.cpp | 1 + .../RocksDB/StorageEmbeddedRocksDB.cpp | 1 + src/Storages/StorageBuffer.cpp | 3 ++ src/Storages/StorageDistributed.cpp | 1 + src/Storages/StorageFactory.h | 3 ++ src/Storages/StorageMemory.cpp | 3 ++ src/Storages/StorageNull.cpp | 3 ++ .../System/StorageSystemTableEngines.cpp | 33 +++++++++++-------- 8 files changed, 34 insertions(+), 14 deletions(-) diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index a2429cead3d..0854cc3653c 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -756,6 +756,7 @@ void registerStorageMergeTree(StorageFactory & factory) .supports_skipping_indices = true, .supports_sort_order = true, .supports_ttl = true, + .supports_parallel_insert = true, }; factory.registerStorage("MergeTree", create, features); diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 80b25793806..249026d1011 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -367,6 +367,7 @@ void registerStorageEmbeddedRocksDB(StorageFactory & factory) { StorageFactory::StorageFeatures features{ .supports_sort_order = true, + .supports_parallel_insert = true, }; factory.registerStorage("EmbeddedRocksDB", create, features); diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index e24db51688e..34787556649 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -996,6 +996,9 @@ void registerStorageBuffer(StorageFactory & factory) StorageBuffer::Thresholds{max_time, max_rows, max_bytes}, destination_id, static_cast(args.local_context.getSettingsRef().insert_allow_materialized_columns)); + }, + { + .supports_parallel_insert = true, }); } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 1390455edb1..dd99d0f0f27 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1005,6 +1005,7 @@ void registerStorageDistributed(StorageFactory & factory) args.attach); }, { + .supports_parallel_insert = true, .source_access_type = AccessType::REMOTE, }); } diff --git a/src/Storages/StorageFactory.h b/src/Storages/StorageFactory.h index b13d4d6f679..18dd24e10db 100644 --- a/src/Storages/StorageFactory.h +++ b/src/Storages/StorageFactory.h @@ -59,6 +59,8 @@ public: bool supports_replication = false; /// See also IStorage::supportsDeduplication() bool supports_deduplication = false; + /// See also IStorage::supportsParallelInsert() + bool supports_parallel_insert = false; AccessType source_access_type = AccessType::NONE; }; @@ -89,6 +91,7 @@ public: .supports_ttl = false, .supports_replication = false, .supports_deduplication = false, + .supports_parallel_insert = false, .source_access_type = AccessType::NONE, }); diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 93f00206e6b..8651caecdfa 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -303,6 +303,9 @@ void registerStorageMemory(StorageFactory & factory) ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); return StorageMemory::create(args.table_id, args.columns, args.constraints); + }, + { + .supports_parallel_insert = true, }); } diff --git a/src/Storages/StorageNull.cpp b/src/Storages/StorageNull.cpp index 499f7329cd9..f324d502834 100644 --- a/src/Storages/StorageNull.cpp +++ b/src/Storages/StorageNull.cpp @@ -29,6 +29,9 @@ void registerStorageNull(StorageFactory & factory) ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); return StorageNull::create(args.table_id, args.columns, args.constraints); + }, + { + .supports_parallel_insert = true, }); } diff --git a/src/Storages/System/StorageSystemTableEngines.cpp b/src/Storages/System/StorageSystemTableEngines.cpp index e63923f69b6..3f06faf6736 100644 --- a/src/Storages/System/StorageSystemTableEngines.cpp +++ b/src/Storages/System/StorageSystemTableEngines.cpp @@ -8,26 +8,31 @@ namespace DB NamesAndTypesList StorageSystemTableEngines::getNamesAndTypes() { - return {{"name", std::make_shared()}, - {"supports_settings", std::make_shared()}, - {"supports_skipping_indices", std::make_shared()}, - {"supports_sort_order", std::make_shared()}, - {"supports_ttl", std::make_shared()}, - {"supports_replication", std::make_shared()}, - {"supports_deduplication", std::make_shared()}}; + return { + {"name", std::make_shared()}, + {"supports_settings", std::make_shared()}, + {"supports_skipping_indices", std::make_shared()}, + {"supports_sort_order", std::make_shared()}, + {"supports_ttl", std::make_shared()}, + {"supports_replication", std::make_shared()}, + {"supports_deduplication", std::make_shared()}, + {"supports_parallel_insert", std::make_shared()}, + }; } void StorageSystemTableEngines::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const { for (const auto & pair : StorageFactory::instance().getAllStorages()) { - res_columns[0]->insert(pair.first); - res_columns[1]->insert(pair.second.features.supports_settings); - res_columns[2]->insert(pair.second.features.supports_skipping_indices); - res_columns[3]->insert(pair.second.features.supports_sort_order); - res_columns[4]->insert(pair.second.features.supports_ttl); - res_columns[5]->insert(pair.second.features.supports_replication); - res_columns[6]->insert(pair.second.features.supports_deduplication); + int i = 0; + res_columns[i++]->insert(pair.first); + res_columns[i++]->insert(pair.second.features.supports_settings); + res_columns[i++]->insert(pair.second.features.supports_skipping_indices); + res_columns[i++]->insert(pair.second.features.supports_sort_order); + res_columns[i++]->insert(pair.second.features.supports_ttl); + res_columns[i++]->insert(pair.second.features.supports_replication); + res_columns[i++]->insert(pair.second.features.supports_deduplication); + res_columns[i++]->insert(pair.second.features.supports_parallel_insert); } } From c312676368464c420fd58c298d3835d5e0d269af Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 8 Jan 2021 14:48:34 +0300 Subject: [PATCH 210/256] Add a smoke test for system.table_engines --- .../queries/0_stateless/01645_system_table_engines.reference | 5 +++++ tests/queries/0_stateless/01645_system_table_engines.sql | 1 + 2 files changed, 6 insertions(+) create mode 100644 tests/queries/0_stateless/01645_system_table_engines.reference create mode 100644 tests/queries/0_stateless/01645_system_table_engines.sql diff --git a/tests/queries/0_stateless/01645_system_table_engines.reference b/tests/queries/0_stateless/01645_system_table_engines.reference new file mode 100644 index 00000000000..00dbd906ee7 --- /dev/null +++ b/tests/queries/0_stateless/01645_system_table_engines.reference @@ -0,0 +1,5 @@ +┌─name──────────────────────────┬─supports_settings─┬─supports_skipping_indices─┬─supports_sort_order─┬─supports_ttl─┬─supports_replication─┬─supports_deduplication─┬─supports_parallel_insert─┐ +│ MergeTree │ 1 │ 1 │ 1 │ 1 │ 0 │ 0 │ 1 │ +│ Kafka │ 1 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ +│ ReplicatedCollapsingMergeTree │ 1 │ 1 │ 1 │ 1 │ 1 │ 1 │ 1 │ +└───────────────────────────────┴───────────────────┴───────────────────────────┴─────────────────────┴──────────────┴──────────────────────┴────────────────────────┴──────────────────────────┘ diff --git a/tests/queries/0_stateless/01645_system_table_engines.sql b/tests/queries/0_stateless/01645_system_table_engines.sql new file mode 100644 index 00000000000..35bfb9d29c9 --- /dev/null +++ b/tests/queries/0_stateless/01645_system_table_engines.sql @@ -0,0 +1 @@ +SELECT * FROM system.table_engines WHERE name in ('Kafka', 'MergeTree', 'ReplicatedCollapsingMergeTree') FORMAT PrettyCompactNoEscapes; From a8f13594edf7ca2072ef9ba5b908112320c53ef1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 8 Jan 2021 14:51:07 +0300 Subject: [PATCH 211/256] Update documentation for system.table_engines --- docs/en/operations/system-tables/table_engines.md | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/docs/en/operations/system-tables/table_engines.md b/docs/en/operations/system-tables/table_engines.md index 4ca1fc657ee..30122cb133e 100644 --- a/docs/en/operations/system-tables/table_engines.md +++ b/docs/en/operations/system-tables/table_engines.md @@ -11,6 +11,7 @@ This table contains the following columns (the column type is shown in brackets) - `supports_sort_order` (UInt8) — Flag that indicates if table engine supports clauses `PARTITION_BY`, `PRIMARY_KEY`, `ORDER_BY` and `SAMPLE_BY`. - `supports_replication` (UInt8) — Flag that indicates if table engine supports [data replication](../../engines/table-engines/mergetree-family/replication.md). - `supports_duduplication` (UInt8) — Flag that indicates if table engine supports data deduplication. +- `supports_parallel_insert` (UInt8) — Flag that indicates if table engine supports parallel insert (see [`max_insert_threads`](../../operations/settings/settings.md#settings-max-insert-threads) setting). Example: @@ -21,11 +22,11 @@ WHERE name in ('Kafka', 'MergeTree', 'ReplicatedCollapsingMergeTree') ``` ``` text -┌─name──────────────────────────┬─supports_settings─┬─supports_skipping_indices─┬─supports_sort_order─┬─supports_ttl─┬─supports_replication─┬─supports_deduplication─┐ -│ Kafka │ 1 │ 0 │ 0 │ 0 │ 0 │ 0 │ -│ MergeTree │ 1 │ 1 │ 1 │ 1 │ 0 │ 0 │ -│ ReplicatedCollapsingMergeTree │ 1 │ 1 │ 1 │ 1 │ 1 │ 1 │ -└───────────────────────────────┴───────────────────┴───────────────────────────┴─────────────────────┴──────────────┴──────────────────────┴────────────────────────┘ +┌─name──────────────────────────┬─supports_settings─┬─supports_skipping_indices─┬─supports_sort_order─┬─supports_ttl─┬─supports_replication─┬─supports_deduplication─┬─supports_parallel_insert─┐ +│ MergeTree │ 1 │ 1 │ 1 │ 1 │ 0 │ 0 │ 1 │ +│ Kafka │ 1 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ +│ ReplicatedCollapsingMergeTree │ 1 │ 1 │ 1 │ 1 │ 1 │ 1 │ 1 │ +└───────────────────────────────┴───────────────────┴───────────────────────────┴─────────────────────┴──────────────┴──────────────────────┴────────────────────────┴──────────────────────────┘ ``` **See also** From 6dba945a2edfd8c220e53cde534d175f282c43fb Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 8 Jan 2021 15:18:53 +0300 Subject: [PATCH 212/256] untrap setenv() call (used for timezone) This will also fix integrity check (#18811), the reason it does not pops up on CI during PR testing because CI does not set YANDEX_OFFICIAL_BUILD, while it does set it for commits merged into upstream commits, and hence once it was merged the special build was failed. Follow-up for: #16366 --- base/harmful/harmful.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/harmful/harmful.c b/base/harmful/harmful.c index 4d08174170f..51112eb4cb3 100644 --- a/base/harmful/harmful.c +++ b/base/harmful/harmful.c @@ -142,7 +142,7 @@ TRAP(qecvt) TRAP(qfcvt) TRAP(register_printf_function) TRAP(seed48) -TRAP(setenv) +//TRAP(setenv) TRAP(setfsent) TRAP(setgrent) TRAP(sethostent) From ed26ccf2175695447506dec9d94da6dfe331db97 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 8 Jan 2021 15:22:34 +0300 Subject: [PATCH 213/256] Do not run integrity check for cross compiled binaries Follow-up for: #18811 Refs: #18785 --- CMakeLists.txt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 95f95464857..853b2df7aca 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -222,7 +222,8 @@ endif () # Add a section with the hash of the compiled machine code for integrity checks. # Only for official builds, because adding a section can be time consuming (rewrite of several GB). -if (OBJCOPY_PATH AND YANDEX_OFFICIAL_BUILD) +# And cross compiled binaries are not supported (since you cannot execute clickhouse hash-binary) +if (OBJCOPY_PATH AND YANDEX_OFFICIAL_BUILD AND (NOT CMAKE_TOOLCHAIN_FILE)) set (USE_BINARY_HASH 1) endif () From 04c07d59bf3f9f78e2ec0ddf29542fc22d3d52d1 Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 8 Jan 2021 15:43:49 +0000 Subject: [PATCH 214/256] add performance test --- tests/performance/optimize_window_funnel.xml | 12 ++++++++++++ 1 file changed, 12 insertions(+) create mode 100644 tests/performance/optimize_window_funnel.xml diff --git a/tests/performance/optimize_window_funnel.xml b/tests/performance/optimize_window_funnel.xml new file mode 100644 index 00000000000..d28b21e2f5d --- /dev/null +++ b/tests/performance/optimize_window_funnel.xml @@ -0,0 +1,12 @@ + + CREATE TABLE action(uid UInt64, event String, time DateTime) ENGINE = MergeTree ORDER BY uid + + INSERT INTO action SELECT arrayJoin(groupArray(number)), 'a', now() from numbers(10000000) + INSERT INTO action SELECT arrayJoin(groupArray(number)), 'b', now() + INTERVAL 6 hour from numbers(10000000) + INSERT INTO action SELECT arrayJoin(groupArray(number)), 'c', now() + INTERVAL 12 hour from numbers(10000000) + INSERT INTO action SELECT arrayJoin(groupArray(number)), 'd', now() + INTERVAL 18 hour from numbers(10000000) + + SELECT level, count() from (select windowFunnel(86400)(time, event='a', event='b', event='c', event='d') level from action group by uid) group by level FORMAT Null + + DROP TABLE IF EXISTS action + From 6be8338584dacbf6d0f834256ce06a246289cb8c Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 8 Jan 2021 22:41:24 +0300 Subject: [PATCH 215/256] Update CompressedReadBufferBase.cpp --- src/Compression/CompressedReadBufferBase.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Compression/CompressedReadBufferBase.cpp b/src/Compression/CompressedReadBufferBase.cpp index 74c902ce492..5be31913140 100644 --- a/src/Compression/CompressedReadBufferBase.cpp +++ b/src/Compression/CompressedReadBufferBase.cpp @@ -69,7 +69,7 @@ static void validateChecksum(char * data, size_t size, const Checksum expected_c /// If size is too huge, then this may be caused by corruption. /// And anyway this is pretty heavy, so avoid burning too much CPU here. - if (size < 1<<20) + if (size < (1ULL << 20)) { /// Check if the difference caused by single bit flip in data. for (size_t bit_pos = 0; bit_pos < size * 8; ++bit_pos) From 5426eae79a3b7daadf43dbc6778bd1d8467ddbf9 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Sat, 9 Jan 2021 00:26:33 +0300 Subject: [PATCH 216/256] Minor fix --- utils/github/backport.py | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/utils/github/backport.py b/utils/github/backport.py index c51c84e6680..576e3b069c2 100644 --- a/utils/github/backport.py +++ b/utils/github/backport.py @@ -32,8 +32,8 @@ class Backport: branches.append(pull_request['headRefName']) return branches - def execute(self, repo, until_commit, number, run_cherrypick, find_lts=False): - repo = LocalRepo(repo, 'origin', self.default_branch_name) + def execute(self, repo, upstream, until_commit, number, run_cherrypick, find_lts=False): + repo = LocalRepo(repo, upstream, self.default_branch_name) all_branches = repo.get_release_branches() # [(branch_name, base_commit)] last_branches = set([branch[0] for branch in all_branches[-number:]]) @@ -42,7 +42,7 @@ class Backport: branches = [] # iterate over all branches to preserve their precedence. for branch in all_branches: - if branch in last_branches or branch in lts_branches: + if branch[0] in last_branches or branch[0] in lts_branches: branches.append(branch) if not branches: @@ -119,6 +119,7 @@ if __name__ == "__main__": parser.add_argument('--lts', action='store_true', help='consider branches with LTS') parser.add_argument('--dry-run', action='store_true', help='do not create or merge any PRs', default=False) parser.add_argument('--verbose', '-v', action='store_true', help='more verbose output', default=False) + parser.add_argument('--upstream', '-u', type=str, help='remote name of upstream in repository', default='origin') args = parser.parse_args() if args.verbose: @@ -128,4 +129,4 @@ if __name__ == "__main__": cherrypick_run = lambda token, pr, branch: CherryPick(token, 'ClickHouse', 'ClickHouse', 'core', pr, branch).execute(args.repo, args.dry_run) bp = Backport(args.token, 'ClickHouse', 'ClickHouse', 'core') - bp.execute(args.repo, args.til, args.number, cherrypick_run, args.lts) + bp.execute(args.repo, args.upstream, args.til, args.number, cherrypick_run, args.lts) From a24be01025ab2a3cfaf0b09f666e4f1833a7bcc7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 9 Jan 2021 00:59:03 +0300 Subject: [PATCH 217/256] Drop Kafka storage from 01645_system_table_engines for fasttest --- tests/queries/0_stateless/01645_system_table_engines.reference | 1 - tests/queries/0_stateless/01645_system_table_engines.sql | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01645_system_table_engines.reference b/tests/queries/0_stateless/01645_system_table_engines.reference index 00dbd906ee7..afe0584bea1 100644 --- a/tests/queries/0_stateless/01645_system_table_engines.reference +++ b/tests/queries/0_stateless/01645_system_table_engines.reference @@ -1,5 +1,4 @@ ┌─name──────────────────────────┬─supports_settings─┬─supports_skipping_indices─┬─supports_sort_order─┬─supports_ttl─┬─supports_replication─┬─supports_deduplication─┬─supports_parallel_insert─┐ │ MergeTree │ 1 │ 1 │ 1 │ 1 │ 0 │ 0 │ 1 │ -│ Kafka │ 1 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ │ ReplicatedCollapsingMergeTree │ 1 │ 1 │ 1 │ 1 │ 1 │ 1 │ 1 │ └───────────────────────────────┴───────────────────┴───────────────────────────┴─────────────────────┴──────────────┴──────────────────────┴────────────────────────┴──────────────────────────┘ diff --git a/tests/queries/0_stateless/01645_system_table_engines.sql b/tests/queries/0_stateless/01645_system_table_engines.sql index 35bfb9d29c9..5e8eef5508b 100644 --- a/tests/queries/0_stateless/01645_system_table_engines.sql +++ b/tests/queries/0_stateless/01645_system_table_engines.sql @@ -1 +1 @@ -SELECT * FROM system.table_engines WHERE name in ('Kafka', 'MergeTree', 'ReplicatedCollapsingMergeTree') FORMAT PrettyCompactNoEscapes; +SELECT * FROM system.table_engines WHERE name in ('MergeTree', 'ReplicatedCollapsingMergeTree') FORMAT PrettyCompactNoEscapes; From 9049599e367bcccd809f45af138452f965b3303a Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 9 Jan 2021 05:15:40 +0300 Subject: [PATCH 218/256] Update optimize_window_funnel.xml --- tests/performance/optimize_window_funnel.xml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/performance/optimize_window_funnel.xml b/tests/performance/optimize_window_funnel.xml index d28b21e2f5d..0d928fd0f4e 100644 --- a/tests/performance/optimize_window_funnel.xml +++ b/tests/performance/optimize_window_funnel.xml @@ -1,10 +1,10 @@ CREATE TABLE action(uid UInt64, event String, time DateTime) ENGINE = MergeTree ORDER BY uid - INSERT INTO action SELECT arrayJoin(groupArray(number)), 'a', now() from numbers(10000000) - INSERT INTO action SELECT arrayJoin(groupArray(number)), 'b', now() + INTERVAL 6 hour from numbers(10000000) - INSERT INTO action SELECT arrayJoin(groupArray(number)), 'c', now() + INTERVAL 12 hour from numbers(10000000) - INSERT INTO action SELECT arrayJoin(groupArray(number)), 'd', now() + INTERVAL 18 hour from numbers(10000000) + INSERT INTO action SELECT arrayJoin(groupArray(number)), 'a', now() from numbers(1000000) + INSERT INTO action SELECT arrayJoin(groupArray(number)), 'b', now() + INTERVAL 6 hour from numbers(1000000) + INSERT INTO action SELECT arrayJoin(groupArray(number)), 'c', now() + INTERVAL 12 hour from numbers(1000000) + INSERT INTO action SELECT arrayJoin(groupArray(number)), 'd', now() + INTERVAL 18 hour from numbers(1000000) SELECT level, count() from (select windowFunnel(86400)(time, event='a', event='b', event='c', event='d') level from action group by uid) group by level FORMAT Null From 8fb64472f28ba9d4491399b60af34672de13a17d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Jan 2021 06:28:54 +0300 Subject: [PATCH 219/256] External queries: fix the case of expr IN table #9756 --- ...t_transform_query_for_external_database.cpp | 18 ++++++++++++++++++ .../transformQueryForExternalDatabase.cpp | 6 ++++++ 2 files changed, 24 insertions(+) diff --git a/src/Storages/tests/gtest_transform_query_for_external_database.cpp b/src/Storages/tests/gtest_transform_query_for_external_database.cpp index 835aebab900..99dfc55ed69 100644 --- a/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -80,6 +80,24 @@ TEST(TransformQueryForExternalDatabase, InWithSingleElement) state.context, state.columns); } +TEST(TransformQueryForExternalDatabase, InWithTable) +{ + const State & state = State::instance(); + + check("SELECT column FROM test.table WHERE 1 IN external_table", + R"(SELECT "column" FROM "test"."table")", + state.context, state.columns); + check("SELECT column FROM test.table WHERE 1 IN (x)", + R"(SELECT "column" FROM "test"."table")", + state.context, state.columns); + check("SELECT column, field, value FROM test.table WHERE column IN (field, value)", + R"(SELECT "column", "field", "value" FROM "test"."table" WHERE "column" IN ("field", "value"))", + state.context, state.columns); + check("SELECT column FROM test.table WHERE column NOT IN hello AND column = 123", + R"(SELECT "column" FROM "test"."table" WHERE ("column" = 123))", + state.context, state.columns); +} + TEST(TransformQueryForExternalDatabase, Like) { const State & state = State::instance(); diff --git a/src/Storages/transformQueryForExternalDatabase.cpp b/src/Storages/transformQueryForExternalDatabase.cpp index f35fb1c8a34..42daf8cfc26 100644 --- a/src/Storages/transformQueryForExternalDatabase.cpp +++ b/src/Storages/transformQueryForExternalDatabase.cpp @@ -138,6 +138,12 @@ bool isCompatible(const IAST & node) if (name == "tuple" && function->arguments->children.size() <= 1) return false; + /// If the right hand side of IN is an identifier (example: x IN table), then it's not compatible. + if ((name == "in" || name == "notIn") + && (function->arguments->children.size() != 2 + || function->arguments->children[1]->as())) + return false; + for (const auto & expr : function->arguments->children) if (!isCompatible(*expr)) return false; From 32f10c38d963b4d12ada46bfdb96c99cd4e0a4e9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Jan 2021 08:30:23 +0300 Subject: [PATCH 220/256] Query Fuzzer: return fail fast semantics --- programs/client/Client.cpp | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 80ad8da837c..ca2a3db193f 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -801,7 +801,8 @@ private: connection->setDefaultDatabase(connection_parameters.default_database); ReadBufferFromFile in(queries_file); readStringUntilEOF(text, in); - processMultiQuery(text); + if (!processMultiQuery(text)) + break; } return; } @@ -984,7 +985,8 @@ private: if (query_fuzzer_runs) { - processWithFuzzing(full_query); + if (!processWithFuzzing(full_query)) + return false; } else { @@ -1034,7 +1036,8 @@ private: } - void processWithFuzzing(const String & text) + /// Returns false when server is not available. + bool processWithFuzzing(const String & text) { ASTPtr orig_ast; @@ -1052,7 +1055,7 @@ private: if (!orig_ast) { // Can't continue after a parsing error - return; + return true; } // Don't repeat inserts, the tables grow too big. Also don't repeat @@ -1147,7 +1150,7 @@ private: // Probably the server is dead because we found an assertion // failure. Fail fast. fmt::print(stderr, "Lost connection to the server\n"); - return; + return false; } // The server is still alive so we're going to continue fuzzing. @@ -1173,6 +1176,8 @@ private: fuzz_base = ast_to_process; } } + + return true; } void processTextAsSingleQuery(const String & text_) From 8a57f8f42a2d980ee7aba0855b841b17c582dcdf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Jan 2021 08:32:10 +0300 Subject: [PATCH 221/256] Remove useless headers --- src/Core/BackgroundSchedulePool.cpp | 1 - src/Core/MySQL/Authentication.cpp | 1 - src/Storages/MergeTree/ReplicatedFetchList.h | 2 +- 3 files changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Core/BackgroundSchedulePool.cpp b/src/Core/BackgroundSchedulePool.cpp index 6b568ea937f..b3aedd6c875 100644 --- a/src/Core/BackgroundSchedulePool.cpp +++ b/src/Core/BackgroundSchedulePool.cpp @@ -1,5 +1,4 @@ #include "BackgroundSchedulePool.h" -#include #include #include #include diff --git a/src/Core/MySQL/Authentication.cpp b/src/Core/MySQL/Authentication.cpp index b0f5f8ccae2..e685ad0394d 100644 --- a/src/Core/MySQL/Authentication.cpp +++ b/src/Core/MySQL/Authentication.cpp @@ -6,7 +6,6 @@ #include #include -#include #include #include diff --git a/src/Storages/MergeTree/ReplicatedFetchList.h b/src/Storages/MergeTree/ReplicatedFetchList.h index 81d538abf9c..0ab631e53b4 100644 --- a/src/Storages/MergeTree/ReplicatedFetchList.h +++ b/src/Storages/MergeTree/ReplicatedFetchList.h @@ -3,9 +3,9 @@ #include #include #include -#include #include + namespace CurrentMetrics { extern const Metric ReplicatedFetch; From b5ace2701442776bddcf7b6196860e3f7b797d0b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 7 Jan 2021 17:14:41 +0300 Subject: [PATCH 222/256] Add fsync support for Distributed engine. Two new settings (by analogy with MergeTree family) has been added: - `fsync_after_insert` - Do fsync for every inserted. Will decreases performance of inserts. - `fsync_tmp_directory` - Do fsync for temporary directory (that is used for async INSERT only) after all part operations (writes, renames, etc.). Refs: #17380 (p1) --- .../table-engines/special/distributed.md | 9 ++++ .../DistributedBlockOutputStream.cpp | 27 +++++++++--- .../Distributed/DistributedSettings.cpp | 42 +++++++++++++++++++ .../Distributed/DistributedSettings.h | 31 ++++++++++++++ src/Storages/StorageDistributed.cpp | 16 ++++++- src/Storages/StorageDistributed.h | 7 ++++ src/Storages/ya.make | 1 + src/TableFunctions/TableFunctionRemote.cpp | 2 + ...ributed_async_insert_fsync_smoke.reference | 6 +++ ...4_distributed_async_insert_fsync_smoke.sql | 24 +++++++++++ .../queries/0_stateless/arcadia_skip_list.txt | 1 + 11 files changed, 160 insertions(+), 6 deletions(-) create mode 100644 src/Storages/Distributed/DistributedSettings.cpp create mode 100644 src/Storages/Distributed/DistributedSettings.h create mode 100644 tests/queries/0_stateless/01644_distributed_async_insert_fsync_smoke.reference create mode 100644 tests/queries/0_stateless/01644_distributed_async_insert_fsync_smoke.sql diff --git a/docs/en/engines/table-engines/special/distributed.md b/docs/en/engines/table-engines/special/distributed.md index 9f96ca3fe8c..f8194ced23f 100644 --- a/docs/en/engines/table-engines/special/distributed.md +++ b/docs/en/engines/table-engines/special/distributed.md @@ -25,10 +25,19 @@ The Distributed engine accepts parameters: - [insert_distributed_sync](../../../operations/settings/settings.md#insert_distributed_sync) setting - [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) for the examples +Also it accept the following settings: + +- `fsync_after_insert` - Do fsync for every inserted. Will decreases performance of inserts (only for async INSERT, i.e. `insert_distributed_sync=false`), + +- `fsync_tmp_directory` - Do fsync for temporary directory (that is used for async INSERT only) after all part operations (writes, renames, etc.). + Example: ``` sql Distributed(logs, default, hits[, sharding_key[, policy_name]]) +SETTINGS + fsync_after_insert=0, + fsync_tmp_directory=0; ``` Data will be read from all servers in the `logs` cluster, from the default.hits table located on every server in the cluster. diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index 8d901028057..6c6399bf6d5 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -29,6 +29,7 @@ #include #include #include +#include #include #include #include @@ -588,6 +589,10 @@ void DistributedBlockOutputStream::writeToLocal(const Block & block, const size_ void DistributedBlockOutputStream::writeToShard(const Block & block, const std::vector & dir_names) { const auto & settings = context.getSettingsRef(); + const auto & distributed_settings = storage.getDistributedSettingsRef(); + + bool fsync = distributed_settings.fsync_after_insert; + bool dir_fsync = distributed_settings.fsync_tmp_directory; std::string compression_method = Poco::toUpper(settings.network_compression_method.toString()); std::optional compression_level; @@ -603,14 +608,15 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: std::string first_file_tmp_path{}; auto reservation = storage.getStoragePolicy()->reserveAndCheck(block.bytes()); - auto disk = reservation->getDisk()->getPath(); + const auto disk = reservation->getDisk(); + auto disk_path = disk->getPath(); auto data_path = storage.getRelativeDataPath(); auto it = dir_names.begin(); /// on first iteration write block to a temporary directory for subsequent /// hardlinking to ensure the inode is not freed until we're done { - const std::string path(disk + data_path + *it); + const std::string path(disk_path + data_path + *it); Poco::File(path).createDirectory(); const std::string tmp_path(path + "/tmp/"); @@ -622,6 +628,13 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: /// Write batch to temporary location { + std::optional tmp_path_sync_guard; + if (dir_fsync) + { + const std::string relative_tmp_path(data_path + *it + "/tmp/"); + tmp_path_sync_guard.emplace(disk, relative_tmp_path); + } + WriteBufferFromFile out{first_file_tmp_path}; CompressedWriteBuffer compress{out, compression_codec}; NativeBlockOutputStream stream{compress, DBMS_TCP_PROTOCOL_VERSION, block.cloneEmpty()}; @@ -647,6 +660,10 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: stream.writePrefix(); stream.write(block); stream.writeSuffix(); + + out.finalize(); + if (fsync) + out.sync(); } // Create hardlink here to reuse increment number @@ -658,10 +675,10 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: /// Make hardlinks for (; it != dir_names.end(); ++it) { - const std::string path(disk + data_path + *it); + const std::string path(disk_path + data_path + *it); Poco::File(path).createDirectory(); - const std::string block_file_path(path + '/' + toString(storage.file_names_increment.get()) + ".bin"); + const std::string block_file_path(path + '/' + toString(storage.file_names_increment.get()) + ".bin"); createHardLink(first_file_tmp_path, block_file_path); } @@ -673,7 +690,7 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: auto sleep_ms = context.getSettingsRef().distributed_directory_monitor_sleep_time_ms; for (const auto & dir_name : dir_names) { - auto & directory_monitor = storage.requireDirectoryMonitor(disk, dir_name); + auto & directory_monitor = storage.requireDirectoryMonitor(disk_path, dir_name); directory_monitor.scheduleAfter(sleep_ms.totalMilliseconds()); } } diff --git a/src/Storages/Distributed/DistributedSettings.cpp b/src/Storages/Distributed/DistributedSettings.cpp new file mode 100644 index 00000000000..555aeba7c58 --- /dev/null +++ b/src/Storages/Distributed/DistributedSettings.cpp @@ -0,0 +1,42 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_SETTING; +} + +IMPLEMENT_SETTINGS_TRAITS(DistributedSettingsTraits, LIST_OF_DISTRIBUTED_SETTINGS) + +void DistributedSettings::loadFromQuery(ASTStorage & storage_def) +{ + if (storage_def.settings) + { + try + { + applyChanges(storage_def.settings->changes); + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::UNKNOWN_SETTING) + e.addMessage("for storage " + storage_def.engine->name); + throw; + } + } + else + { + auto settings_ast = std::make_shared(); + settings_ast->is_standalone = false; + storage_def.set(storage_def.settings, settings_ast); + } +} + +} + diff --git a/src/Storages/Distributed/DistributedSettings.h b/src/Storages/Distributed/DistributedSettings.h new file mode 100644 index 00000000000..9d79feaa05a --- /dev/null +++ b/src/Storages/Distributed/DistributedSettings.h @@ -0,0 +1,31 @@ +#pragma once + +#include +#include + + +namespace Poco::Util +{ + class AbstractConfiguration; +} + + +namespace DB +{ +class ASTStorage; + +#define LIST_OF_DISTRIBUTED_SETTINGS(M) \ + M(Bool, fsync_after_insert, false, "Do fsync for every inserted. Will decreases performance of inserts (only for async INSERT, i.e. insert_distributed_sync=false)", 0) \ + M(Bool, fsync_tmp_directory, false, "Do fsync for temporary directory (that is used for async INSERT only) after all part operations (writes, renames, etc.).", 0) \ + +DECLARE_SETTINGS_TRAITS(DistributedSettingsTraits, LIST_OF_DISTRIBUTED_SETTINGS) + + +/** Settings for the Distributed family of engines. + */ +struct DistributedSettings : public BaseSettings +{ + void loadFromQuery(ASTStorage & storage_def); +}; + +} diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index dd99d0f0f27..af3c410872f 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -363,6 +363,7 @@ StorageDistributed::StorageDistributed( const ASTPtr & sharding_key_, const String & storage_policy_name_, const String & relative_data_path_, + const DistributedSettings & distributed_settings_, bool attach_, ClusterPtr owned_cluster_) : IStorage(id_) @@ -374,6 +375,7 @@ StorageDistributed::StorageDistributed( , cluster_name(global_context.getMacros()->expand(cluster_name_)) , has_sharding_key(sharding_key_) , relative_data_path(relative_data_path_) + , distributed_settings(distributed_settings_) , rng(randomSeed()) { StorageInMemoryMetadata storage_metadata; @@ -417,9 +419,10 @@ StorageDistributed::StorageDistributed( const ASTPtr & sharding_key_, const String & storage_policy_name_, const String & relative_data_path_, + const DistributedSettings & distributed_settings_, bool attach, ClusterPtr owned_cluster_) - : StorageDistributed(id_, columns_, constraints_, String{}, String{}, cluster_name_, context_, sharding_key_, storage_policy_name_, relative_data_path_, attach, std::move(owned_cluster_)) + : StorageDistributed(id_, columns_, constraints_, String{}, String{}, cluster_name_, context_, sharding_key_, storage_policy_name_, relative_data_path_, distributed_settings_, attach, std::move(owned_cluster_)) { remote_table_function_ptr = std::move(remote_table_function_ptr_); } @@ -954,6 +957,8 @@ void registerStorageDistributed(StorageFactory & factory) * - constant expression with string result, like currentDatabase(); * -- string literal as specific case; * - empty string means 'use default database from cluster'. + * + * Distributed engine also supports SETTINGS clause. */ ASTs & engine_args = args.engine_args; @@ -995,6 +1000,13 @@ void registerStorageDistributed(StorageFactory & factory) ", but should be one of integer type", ErrorCodes::TYPE_MISMATCH); } + /// TODO: move some arguments from the arguments to the SETTINGS. + DistributedSettings distributed_settings; + if (args.storage_def->settings) + { + distributed_settings.loadFromQuery(*args.storage_def); + } + return StorageDistributed::create( args.table_id, args.columns, args.constraints, remote_database, remote_table, cluster_name, @@ -1002,9 +1014,11 @@ void registerStorageDistributed(StorageFactory & factory) sharding_key, storage_policy, args.relative_data_path, + distributed_settings, args.attach); }, { + .supports_settings = true, .supports_parallel_insert = true, .source_access_type = AccessType::REMOTE, }); diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index ce7e48c85a9..083407666f6 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -4,6 +4,7 @@ #include #include +#include #include #include #include @@ -127,6 +128,8 @@ public: size_t getRandomShardIndex(const Cluster::ShardsInfo & shards); + const DistributedSettings & getDistributedSettingsRef() const { return distributed_settings; } + String remote_database; String remote_table; ASTPtr remote_table_function_ptr; @@ -162,6 +165,7 @@ protected: const ASTPtr & sharding_key_, const String & storage_policy_name_, const String & relative_data_path_, + const DistributedSettings & distributed_settings_, bool attach_, ClusterPtr owned_cluster_ = {}); @@ -175,6 +179,7 @@ protected: const ASTPtr & sharding_key_, const String & storage_policy_name_, const String & relative_data_path_, + const DistributedSettings & distributed_settings_, bool attach, ClusterPtr owned_cluster_ = {}); @@ -188,6 +193,8 @@ protected: /// Other volumes will be ignored. It's needed to allow using the same multi-volume policy both for Distributed and other engines. VolumePtr data_volume; + DistributedSettings distributed_settings; + struct ClusterNodeData { std::unique_ptr directory_monitor; diff --git a/src/Storages/ya.make b/src/Storages/ya.make index 27aa9e3ac3f..9169ff54b87 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -17,6 +17,7 @@ SRCS( ConstraintsDescription.cpp Distributed/DirectoryMonitor.cpp Distributed/DistributedBlockOutputStream.cpp + Distributed/DistributedSettings.cpp IStorage.cpp IndicesDescription.cpp JoinSettings.cpp diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index a031490b88b..914b7083fca 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -211,6 +211,7 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & /*ast_function*/, con ASTPtr{}, String{}, String{}, + DistributedSettings{}, false, cluster) : StorageDistributed::create( @@ -224,6 +225,7 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & /*ast_function*/, con ASTPtr{}, String{}, String{}, + DistributedSettings{}, false, cluster); diff --git a/tests/queries/0_stateless/01644_distributed_async_insert_fsync_smoke.reference b/tests/queries/0_stateless/01644_distributed_async_insert_fsync_smoke.reference new file mode 100644 index 00000000000..fc919524275 --- /dev/null +++ b/tests/queries/0_stateless/01644_distributed_async_insert_fsync_smoke.reference @@ -0,0 +1,6 @@ +no fsync +0 +90 +fsync +90 +180 diff --git a/tests/queries/0_stateless/01644_distributed_async_insert_fsync_smoke.sql b/tests/queries/0_stateless/01644_distributed_async_insert_fsync_smoke.sql new file mode 100644 index 00000000000..f01bddbcc4f --- /dev/null +++ b/tests/queries/0_stateless/01644_distributed_async_insert_fsync_smoke.sql @@ -0,0 +1,24 @@ +drop table if exists dist_01643; +drop table if exists data_01643; + +create table data_01643 (key Int) engine=Memory(); + +select 'no fsync'; +create table dist_01643 as data_01643 engine=Distributed(test_cluster_two_shards, currentDatabase(), data_01643, key); +system stop distributed sends dist_01643; +insert into dist_01643 select * from numbers(10) settings prefer_localhost_replica=0; +select sum(*) from dist_01643; +system flush distributed dist_01643; +select sum(*) from dist_01643; +drop table dist_01643; + +select 'fsync'; +create table dist_01643 as data_01643 engine=Distributed(test_cluster_two_shards, currentDatabase(), data_01643, key) settings fsync_after_insert=1, fsync_tmp_directory=1; +system stop distributed sends dist_01643; +insert into dist_01643 select * from numbers(10) settings prefer_localhost_replica=0; +select sum(*) from dist_01643; +system flush distributed dist_01643; +select sum(*) from dist_01643; +drop table dist_01643; + +drop table if exists data_01643; diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index 6c6636b923d..a4507555373 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -182,3 +182,4 @@ 01601_custom_tld 01636_nullable_fuzz2 01639_distributed_sync_insert_zero_rows +01644_distributed_async_insert_fsync_smoke From 4cabfa356e0adb93c3474ff60eeb0420df057c8b Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Sat, 9 Jan 2021 02:42:15 +0300 Subject: [PATCH 223/256] Update documentation for Distributed fsync settings. --- docs/en/engines/table-engines/special/distributed.md | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/special/distributed.md b/docs/en/engines/table-engines/special/distributed.md index f8194ced23f..e46a9a13086 100644 --- a/docs/en/engines/table-engines/special/distributed.md +++ b/docs/en/engines/table-engines/special/distributed.md @@ -27,9 +27,17 @@ The Distributed engine accepts parameters: Also it accept the following settings: -- `fsync_after_insert` - Do fsync for every inserted. Will decreases performance of inserts (only for async INSERT, i.e. `insert_distributed_sync=false`), +- `fsync_after_insert` - do the `fsync` for the file data after asynchronous insert to Distributed. Guarantees that the OS flushed the whole inserted data to a file **on the initiator node** disk. -- `fsync_tmp_directory` - Do fsync for temporary directory (that is used for async INSERT only) after all part operations (writes, renames, etc.). +- `fsync_tmp_directory` - do the `fsync` for directories. Guarantees that the OS refreshed directory metadata after operations related to asynchronous inserts on Distributed table (after insert, after sending the data to shard, etc). + +!!! note "Note" + + **Durability settings** (`fsync_...`): + + - Affect only asynchronous INSERTs (i.e. `insert_distributed_sync=false`) when data first stored on the initiator node disk and later asynchronously send to shards. + - May significantly decrease the inserts' performance + - Affect writing the data stored inside Distributed table folder into the **node which accepted your insert**. If you need to have guarantees of writing data to underlying MergeTree tables - see durability settings (`...fsync...`) in `system.merge_tree_settings` Example: From fbe5df809bacecad9d7b010c40c89affe1837ef9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 9 Jan 2021 02:42:46 +0300 Subject: [PATCH 224/256] Sync other temporary directories for Distributed fsync_tmp_directories --- .../DistributedBlockOutputStream.cpp | 20 +++++++++++++------ 1 file changed, 14 insertions(+), 6 deletions(-) diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index 6c6399bf6d5..6c808d02d19 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -612,6 +612,17 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: auto disk_path = disk->getPath(); auto data_path = storage.getRelativeDataPath(); + auto make_directory_sync_guard = [&](const std::string & current_path) + { + std::unique_ptr guard; + if (dir_fsync) + { + const std::string relative_path(data_path + current_path); + guard = std::make_unique(disk, relative_path); + } + return guard; + }; + auto it = dir_names.begin(); /// on first iteration write block to a temporary directory for subsequent /// hardlinking to ensure the inode is not freed until we're done @@ -628,12 +639,7 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: /// Write batch to temporary location { - std::optional tmp_path_sync_guard; - if (dir_fsync) - { - const std::string relative_tmp_path(data_path + *it + "/tmp/"); - tmp_path_sync_guard.emplace(disk, relative_tmp_path); - } + auto tmp_dir_sync_guard = make_directory_sync_guard(*it + "/tmp/"); WriteBufferFromFile out{first_file_tmp_path}; CompressedWriteBuffer compress{out, compression_codec}; @@ -669,6 +675,7 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: // Create hardlink here to reuse increment number const std::string block_file_path(path + '/' + file_name); createHardLink(first_file_tmp_path, block_file_path); + auto dir_sync_guard = make_directory_sync_guard(*it); } ++it; @@ -680,6 +687,7 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: const std::string block_file_path(path + '/' + toString(storage.file_names_increment.get()) + ".bin"); createHardLink(first_file_tmp_path, block_file_path); + auto dir_sync_guard = make_directory_sync_guard(*it); } /// remove the temporary file, enabling the OS to reclaim inode after all threads From dd669cb2b6be1923ddb7a2dfe545b87f51ff679e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 9 Jan 2021 11:29:49 +0300 Subject: [PATCH 225/256] Add fsync support for Distributed/DirectoryMonitor Note that there is no fsync_tmp_directory support in DirectoryMonitor since you cannot propagate the error to user anyway. --- src/Storages/Distributed/DirectoryMonitor.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index 5d089eb9f80..697f7dd66d9 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -6,8 +6,8 @@ #include #include #include -#include #include +#include #include #include #include @@ -464,6 +464,9 @@ struct StorageDistributedDirectoryMonitor::Batch CurrentMetrics::Increment metric_increment{CurrentMetrics::DistributedSend}; + const auto & distributed_settings = parent.storage.getDistributedSettingsRef(); + bool fsync = distributed_settings.fsync_after_insert; + if (!recovered) { /// For deduplication in Replicated tables to work, in case of error @@ -480,6 +483,10 @@ struct StorageDistributedDirectoryMonitor::Batch { WriteBufferFromFile out{tmp_file, O_WRONLY | O_TRUNC | O_CREAT}; writeText(out); + + out.finalize(); + if (fsync) + out.sync(); } Poco::File{tmp_file}.renameTo(parent.current_batch_file_path); From 2e55bd2285921423fa5b415e43d5dfae3fe613b9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 9 Jan 2021 15:26:37 +0300 Subject: [PATCH 226/256] Accept IDisk in DirectoryMonitor (for further fsync) --- src/Storages/Distributed/DirectoryMonitor.cpp | 27 ++++++++++++------- src/Storages/Distributed/DirectoryMonitor.h | 15 +++++++++-- .../DistributedBlockOutputStream.cpp | 2 +- src/Storages/StorageDistributed.cpp | 21 ++++++++------- src/Storages/StorageDistributed.h | 7 +++-- 5 files changed, 49 insertions(+), 23 deletions(-) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index 697f7dd66d9..77e9c623a2b 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include @@ -80,18 +81,25 @@ namespace StorageDistributedDirectoryMonitor::StorageDistributedDirectoryMonitor( - StorageDistributed & storage_, std::string path_, ConnectionPoolPtr pool_, ActionBlocker & monitor_blocker_, BackgroundSchedulePool & bg_pool) + StorageDistributed & storage_, + const DiskPtr & disk_, + const std::string & relative_path_, + ConnectionPoolPtr pool_, + ActionBlocker & monitor_blocker_, + BackgroundSchedulePool & bg_pool) : storage(storage_) , pool(std::move(pool_)) - , path{path_ + '/'} + , disk(disk_) + , relative_path(relative_path_) + , path(disk->getPath() + relative_path + '/') , 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} - , max_sleep_time{storage.global_context.getSettingsRef().distributed_directory_monitor_max_sleep_time_ms.totalMilliseconds()} - , log{&Poco::Logger::get(getLoggerName())} + , 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) + , max_sleep_time(storage.global_context.getSettingsRef().distributed_directory_monitor_max_sleep_time_ms.totalMilliseconds()) + , log(&Poco::Logger::get(getLoggerName())) , monitor_blocker(monitor_blocker_) , metric_pending_files(CurrentMetrics::DistributedFilesToInsert, 0) { @@ -788,14 +796,15 @@ std::string StorageDistributedDirectoryMonitor::getLoggerName() const return storage.getStorageID().getFullTableName() + ".DirectoryMonitor"; } -void StorageDistributedDirectoryMonitor::updatePath(const std::string & new_path) +void StorageDistributedDirectoryMonitor::updatePath(const std::string & new_relative_path) { task_handle->deactivate(); std::lock_guard lock{mutex}; { std::unique_lock metrics_lock(metrics_mutex); - path = new_path; + relative_path = new_relative_path; + path = disk->getPath() + relative_path + '/'; } current_batch_file_path = path + "current_batch.txt"; diff --git a/src/Storages/Distributed/DirectoryMonitor.h b/src/Storages/Distributed/DirectoryMonitor.h index 1d34357b3b1..eb746350858 100644 --- a/src/Storages/Distributed/DirectoryMonitor.h +++ b/src/Storages/Distributed/DirectoryMonitor.h @@ -14,6 +14,9 @@ namespace CurrentMetrics { class Increment; } namespace DB { +class IDisk; +using DiskPtr = std::shared_ptr; + class StorageDistributed; class ActionBlocker; class BackgroundSchedulePool; @@ -25,13 +28,18 @@ class StorageDistributedDirectoryMonitor { public: StorageDistributedDirectoryMonitor( - StorageDistributed & storage_, std::string path_, ConnectionPoolPtr pool_, ActionBlocker & monitor_blocker_, BackgroundSchedulePool & bg_pool); + StorageDistributed & storage_, + const DiskPtr & disk_, + const std::string & relative_path_, + ConnectionPoolPtr pool_, + ActionBlocker & monitor_blocker_, + BackgroundSchedulePool & bg_pool); ~StorageDistributedDirectoryMonitor(); static ConnectionPoolPtr createPool(const std::string & name, const StorageDistributed & storage); - void updatePath(const std::string & new_path); + void updatePath(const std::string & new_relative_path); void flushAllData(); @@ -70,6 +78,9 @@ private: StorageDistributed & storage; const ConnectionPoolPtr pool; + + DiskPtr disk; + std::string relative_path; std::string path; const bool should_batch_inserts = false; diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index 6c808d02d19..d390c54ad51 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -698,7 +698,7 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: auto sleep_ms = context.getSettingsRef().distributed_directory_monitor_sleep_time_ms; for (const auto & dir_name : dir_names) { - auto & directory_monitor = storage.requireDirectoryMonitor(disk_path, dir_name); + auto & directory_monitor = storage.requireDirectoryMonitor(disk, dir_name); directory_monitor.scheduleAfter(sleep_ms.totalMilliseconds()); } } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index af3c410872f..afd7d6b876e 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include @@ -604,7 +604,7 @@ void StorageDistributed::startup() return; for (const DiskPtr & disk : data_volume->getDisks()) - createDirectoryMonitors(disk->getPath()); + createDirectoryMonitors(disk); for (const String & path : getDataPaths()) { @@ -684,9 +684,9 @@ StoragePolicyPtr StorageDistributed::getStoragePolicy() const return storage_policy; } -void StorageDistributed::createDirectoryMonitors(const std::string & disk) +void StorageDistributed::createDirectoryMonitors(const DiskPtr & disk) { - const std::string path(disk + relative_data_path); + const std::string path(disk->getPath() + relative_data_path); Poco::File{path}.createDirectories(); std::filesystem::directory_iterator begin(path); @@ -717,10 +717,10 @@ void StorageDistributed::createDirectoryMonitors(const std::string & disk) } -StorageDistributedDirectoryMonitor& StorageDistributed::requireDirectoryMonitor(const std::string & disk, const std::string & name) +StorageDistributedDirectoryMonitor& StorageDistributed::requireDirectoryMonitor(const DiskPtr & disk, const std::string & name) { - const std::string path(disk + relative_data_path + name); - const std::string key(disk + name); + const std::string & disk_path = disk->getPath(); + const std::string key(disk_path + name); std::lock_guard lock(cluster_nodes_mutex); auto & node_data = cluster_nodes_data[key]; @@ -728,7 +728,10 @@ StorageDistributedDirectoryMonitor& StorageDistributed::requireDirectoryMonitor( { node_data.connection_pool = StorageDistributedDirectoryMonitor::createPool(name, *this); node_data.directory_monitor = std::make_unique( - *this, path, node_data.connection_pool, monitors_blocker, global_context.getDistributedSchedulePool()); + *this, disk, relative_data_path + name, + node_data.connection_pool, + monitors_blocker, + global_context.getDistributedSchedulePool()); } return *node_data.directory_monitor; } @@ -935,7 +938,7 @@ void StorageDistributed::renameOnDisk(const String & new_path_to_table_data) std::lock_guard lock(cluster_nodes_mutex); for (auto & node : cluster_nodes_data) - node.second.directory_monitor->updatePath(new_path); + node.second.directory_monitor->updatePath(new_path_to_table_data); } relative_data_path = new_path_to_table_data; diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 083407666f6..585efafddfb 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -24,6 +24,9 @@ class Context; class IVolume; using VolumePtr = std::shared_ptr; +class IDisk; +using DiskPtr = std::shared_ptr; + class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; @@ -104,9 +107,9 @@ public: std::string getClusterName() const { return cluster_name; } /// Returns empty string if tables is used by TableFunctionRemote /// create directory monitors for each existing subdirectory - void createDirectoryMonitors(const std::string & disk); + void createDirectoryMonitors(const DiskPtr & disk); /// ensure directory monitor thread and connectoin pool creation by disk and subdirectory name - StorageDistributedDirectoryMonitor & requireDirectoryMonitor(const std::string & disk, const std::string & name); + StorageDistributedDirectoryMonitor & requireDirectoryMonitor(const DiskPtr & disk, const std::string & name); /// Return list of metrics for all created monitors /// (note that monitors are created lazily, i.e. until at least one INSERT executed) std::vector getDirectoryMonitorsStatuses() const; From ae0b15455f2d40e77ab9720aac209918e21ea835 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 9 Jan 2021 15:42:21 +0300 Subject: [PATCH 227/256] Add fsync_tmp_directory support into DirectoryMonitor --- src/Storages/Distributed/DirectoryMonitor.cpp | 52 ++++++++++++++++--- src/Storages/Distributed/DirectoryMonitor.h | 1 + 2 files changed, 45 insertions(+), 8 deletions(-) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index 77e9c623a2b..c79c3543de7 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -93,6 +94,7 @@ StorageDistributedDirectoryMonitor::StorageDistributedDirectoryMonitor( , relative_path(relative_path_) , path(disk->getPath() + relative_path + '/') , should_batch_inserts(storage.global_context.getSettingsRef().distributed_directory_monitor_batch_inserts) + , dir_fsync(storage.getDistributedSettingsRef().fsync_tmp_directory) , 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") @@ -142,6 +144,10 @@ void StorageDistributedDirectoryMonitor::shutdownAndDropAllData() task_handle->deactivate(); } + std::optional dir_sync_guard; + if (dir_fsync) + dir_sync_guard.emplace(disk, relative_path); + Poco::File(path).remove(true); } @@ -345,6 +351,10 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa throw; } + std::optional dir_sync_guard; + if (dir_fsync) + dir_sync_guard.emplace(disk, relative_path); + Poco::File{file_path}.remove(); metric_pending_files.sub(); @@ -452,10 +462,16 @@ struct StorageDistributedDirectoryMonitor::Batch StorageDistributedDirectoryMonitor & parent; const std::map & file_index_to_path; + bool fsync = false; + bool dir_fsync = false; + Batch( StorageDistributedDirectoryMonitor & parent_, const std::map & file_index_to_path_) - : parent(parent_), file_index_to_path(file_index_to_path_) + : parent(parent_) + , file_index_to_path(file_index_to_path_) + , fsync(parent.storage.getDistributedSettingsRef().fsync_after_insert) + , dir_fsync(parent.dir_fsync) {} bool isEnoughSize() const @@ -472,9 +488,6 @@ struct StorageDistributedDirectoryMonitor::Batch CurrentMetrics::Increment metric_increment{CurrentMetrics::DistributedSend}; - const auto & distributed_settings = parent.storage.getDistributedSettingsRef(); - bool fsync = distributed_settings.fsync_after_insert; - if (!recovered) { /// For deduplication in Replicated tables to work, in case of error @@ -485,6 +498,10 @@ struct StorageDistributedDirectoryMonitor::Batch /// Temporary file is required for atomicity. String tmp_file{parent.current_batch_file_path + ".tmp"}; + std::optional dir_sync_guard; + if (dir_fsync) + dir_sync_guard.emplace(parent.disk, parent.relative_path); + if (Poco::File{tmp_file}.exists()) LOG_ERROR(parent.log, "Temporary file {} exists. Unclean shutdown?", backQuote(tmp_file)); @@ -552,6 +569,10 @@ struct StorageDistributedDirectoryMonitor::Batch { LOG_TRACE(parent.log, "Sent a batch of {} files.", file_indices.size()); + std::optional dir_sync_guard; + if (dir_fsync) + dir_sync_guard.emplace(parent.disk, parent.relative_path); + for (UInt64 file_index : file_indices) Poco::File{file_index_to_path.at(file_index)}.remove(); } @@ -749,10 +770,16 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map metric_pending_files.sub(batch.file_indices.size()); } - /// current_batch.txt will not exist if there was no send - /// (this is the case when all batches that was pending has been marked as pending) - if (Poco::File{current_batch_file_path}.exists()) - Poco::File{current_batch_file_path}.remove(); + { + std::optional dir_sync_guard; + if (dir_fsync) + dir_sync_guard.emplace(disk, relative_path); + + /// current_batch.txt will not exist if there was no send + /// (this is the case when all batches that was pending has been marked as pending) + if (Poco::File{current_batch_file_path}.exists()) + Poco::File{current_batch_file_path}.remove(); + } } bool StorageDistributedDirectoryMonitor::isFileBrokenErrorCode(int code) @@ -774,6 +801,15 @@ void StorageDistributedDirectoryMonitor::markAsBroken(const std::string & file_p const auto & broken_file_path = broken_path + file_name; Poco::File{broken_path}.createDirectory(); + + std::optional dir_sync_guard; + std::optional broken_dir_sync_guard; + if (dir_fsync) + { + broken_dir_sync_guard.emplace(disk, relative_path + "/broken/"); + dir_sync_guard.emplace(disk, relative_path); + } + Poco::File{file_path}.renameTo(broken_file_path); LOG_ERROR(log, "Renamed `{}` to `{}`", file_path, broken_file_path); diff --git a/src/Storages/Distributed/DirectoryMonitor.h b/src/Storages/Distributed/DirectoryMonitor.h index eb746350858..a6175b44d7b 100644 --- a/src/Storages/Distributed/DirectoryMonitor.h +++ b/src/Storages/Distributed/DirectoryMonitor.h @@ -84,6 +84,7 @@ private: std::string path; const bool should_batch_inserts = false; + const bool dir_fsync = false; const size_t min_batched_block_size_rows = 0; const size_t min_batched_block_size_bytes = 0; String current_batch_file_path; From 9cdb1d4ae6897e1d362d914a4b4fdc3232e83786 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 9 Jan 2021 15:55:55 +0300 Subject: [PATCH 228/256] Simple coverage of fsync_after_insert/fsync_tmp_directory with batching --- tests/integration/test_insert_into_distributed/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_insert_into_distributed/test.py b/tests/integration/test_insert_into_distributed/test.py index 52beaf06ec2..ebec1507975 100644 --- a/tests/integration/test_insert_into_distributed/test.py +++ b/tests/integration/test_insert_into_distributed/test.py @@ -36,7 +36,7 @@ CREATE TABLE distributed (x UInt32) ENGINE = Distributed('test_cluster', 'defaul remote.query("CREATE TABLE local2 (d Date, x UInt32, s String) ENGINE = MergeTree(d, x, 8192)") instance_test_inserts_batching.query(''' -CREATE TABLE distributed (d Date, x UInt32) ENGINE = Distributed('test_cluster', 'default', 'local2') +CREATE TABLE distributed (d Date, x UInt32) ENGINE = Distributed('test_cluster', 'default', 'local2') SETTINGS fsync_after_insert=1, fsync_tmp_directory=1 ''') instance_test_inserts_local_cluster.query( From dd52f88e548cd4fa55ed3a2a51673a9099bd1bd3 Mon Sep 17 00:00:00 2001 From: HuFuwang Date: Sat, 9 Jan 2021 22:37:49 +0800 Subject: [PATCH 229/256] fix the inconsistent result of windowFunnel in case of same timestamp for different events. --- src/AggregateFunctions/AggregateFunctionWindowFunnel.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionWindowFunnel.h b/src/AggregateFunctions/AggregateFunctionWindowFunnel.h index 96f0eb183be..ca1ffd3e292 100644 --- a/src/AggregateFunctions/AggregateFunctionWindowFunnel.h +++ b/src/AggregateFunctions/AggregateFunctionWindowFunnel.h @@ -24,7 +24,7 @@ struct ComparePairFirst final template bool operator()(const std::pair & lhs, const std::pair & rhs) const { - return lhs.first < rhs.first; + return lhs.first == rhs.first ? lhs.second < rhs.second : lhs.first < rhs.first; } }; From 471deab63a83db1940027b2e038409dfa2f32dcc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 9 Jan 2021 17:51:30 +0300 Subject: [PATCH 230/256] Rename fsync_tmp_directory to fsync_directories for Distributed engine --- docs/en/engines/table-engines/special/distributed.md | 4 ++-- src/Storages/Distributed/DirectoryMonitor.cpp | 2 +- src/Storages/Distributed/DistributedBlockOutputStream.cpp | 2 +- src/Storages/Distributed/DistributedSettings.h | 2 +- tests/integration/test_insert_into_distributed/test.py | 2 +- .../01644_distributed_async_insert_fsync_smoke.sql | 2 +- 6 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/en/engines/table-engines/special/distributed.md b/docs/en/engines/table-engines/special/distributed.md index e46a9a13086..7fffa962480 100644 --- a/docs/en/engines/table-engines/special/distributed.md +++ b/docs/en/engines/table-engines/special/distributed.md @@ -29,7 +29,7 @@ Also it accept the following settings: - `fsync_after_insert` - do the `fsync` for the file data after asynchronous insert to Distributed. Guarantees that the OS flushed the whole inserted data to a file **on the initiator node** disk. -- `fsync_tmp_directory` - do the `fsync` for directories. Guarantees that the OS refreshed directory metadata after operations related to asynchronous inserts on Distributed table (after insert, after sending the data to shard, etc). +- `fsync_directories` - do the `fsync` for directories. Guarantees that the OS refreshed directory metadata after operations related to asynchronous inserts on Distributed table (after insert, after sending the data to shard, etc). !!! note "Note" @@ -45,7 +45,7 @@ Example: Distributed(logs, default, hits[, sharding_key[, policy_name]]) SETTINGS fsync_after_insert=0, - fsync_tmp_directory=0; + fsync_directories=0; ``` Data will be read from all servers in the `logs` cluster, from the default.hits table located on every server in the cluster. diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index c79c3543de7..ade75506a38 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -94,7 +94,7 @@ StorageDistributedDirectoryMonitor::StorageDistributedDirectoryMonitor( , relative_path(relative_path_) , path(disk->getPath() + relative_path + '/') , should_batch_inserts(storage.global_context.getSettingsRef().distributed_directory_monitor_batch_inserts) - , dir_fsync(storage.getDistributedSettingsRef().fsync_tmp_directory) + , dir_fsync(storage.getDistributedSettingsRef().fsync_directories) , 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") diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index d390c54ad51..7299953c88e 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -592,7 +592,7 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: const auto & distributed_settings = storage.getDistributedSettingsRef(); bool fsync = distributed_settings.fsync_after_insert; - bool dir_fsync = distributed_settings.fsync_tmp_directory; + bool dir_fsync = distributed_settings.fsync_directories; std::string compression_method = Poco::toUpper(settings.network_compression_method.toString()); std::optional compression_level; diff --git a/src/Storages/Distributed/DistributedSettings.h b/src/Storages/Distributed/DistributedSettings.h index 9d79feaa05a..9df787428df 100644 --- a/src/Storages/Distributed/DistributedSettings.h +++ b/src/Storages/Distributed/DistributedSettings.h @@ -16,7 +16,7 @@ class ASTStorage; #define LIST_OF_DISTRIBUTED_SETTINGS(M) \ M(Bool, fsync_after_insert, false, "Do fsync for every inserted. Will decreases performance of inserts (only for async INSERT, i.e. insert_distributed_sync=false)", 0) \ - M(Bool, fsync_tmp_directory, false, "Do fsync for temporary directory (that is used for async INSERT only) after all part operations (writes, renames, etc.).", 0) \ + M(Bool, fsync_directories, false, "Do fsync for temporary directory (that is used for async INSERT only) after all part operations (writes, renames, etc.).", 0) \ DECLARE_SETTINGS_TRAITS(DistributedSettingsTraits, LIST_OF_DISTRIBUTED_SETTINGS) diff --git a/tests/integration/test_insert_into_distributed/test.py b/tests/integration/test_insert_into_distributed/test.py index ebec1507975..d71d1075c70 100644 --- a/tests/integration/test_insert_into_distributed/test.py +++ b/tests/integration/test_insert_into_distributed/test.py @@ -36,7 +36,7 @@ CREATE TABLE distributed (x UInt32) ENGINE = Distributed('test_cluster', 'defaul remote.query("CREATE TABLE local2 (d Date, x UInt32, s String) ENGINE = MergeTree(d, x, 8192)") instance_test_inserts_batching.query(''' -CREATE TABLE distributed (d Date, x UInt32) ENGINE = Distributed('test_cluster', 'default', 'local2') SETTINGS fsync_after_insert=1, fsync_tmp_directory=1 +CREATE TABLE distributed (d Date, x UInt32) ENGINE = Distributed('test_cluster', 'default', 'local2') SETTINGS fsync_after_insert=1, fsync_directories=1 ''') instance_test_inserts_local_cluster.query( diff --git a/tests/queries/0_stateless/01644_distributed_async_insert_fsync_smoke.sql b/tests/queries/0_stateless/01644_distributed_async_insert_fsync_smoke.sql index f01bddbcc4f..87ae96aa451 100644 --- a/tests/queries/0_stateless/01644_distributed_async_insert_fsync_smoke.sql +++ b/tests/queries/0_stateless/01644_distributed_async_insert_fsync_smoke.sql @@ -13,7 +13,7 @@ select sum(*) from dist_01643; drop table dist_01643; select 'fsync'; -create table dist_01643 as data_01643 engine=Distributed(test_cluster_two_shards, currentDatabase(), data_01643, key) settings fsync_after_insert=1, fsync_tmp_directory=1; +create table dist_01643 as data_01643 engine=Distributed(test_cluster_two_shards, currentDatabase(), data_01643, key) settings fsync_after_insert=1, fsync_directories=1; system stop distributed sends dist_01643; insert into dist_01643 select * from numbers(10) settings prefer_localhost_replica=0; select sum(*) from dist_01643; From 8e63328221dea80d46fdf45fd343d2651127fb76 Mon Sep 17 00:00:00 2001 From: HuFuwang Date: Sat, 9 Jan 2021 23:13:03 +0800 Subject: [PATCH 231/256] refine naming. --- src/AggregateFunctions/AggregateFunctionWindowFunnel.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionWindowFunnel.h b/src/AggregateFunctions/AggregateFunctionWindowFunnel.h index ca1ffd3e292..d112324f693 100644 --- a/src/AggregateFunctions/AggregateFunctionWindowFunnel.h +++ b/src/AggregateFunctions/AggregateFunctionWindowFunnel.h @@ -19,7 +19,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -struct ComparePairFirst final +struct ComparePair final { template bool operator()(const std::pair & lhs, const std::pair & rhs) const @@ -34,7 +34,7 @@ struct AggregateFunctionWindowFunnelData { using TimestampEvent = std::pair; using TimestampEvents = PODArray; - using Comparator = ComparePairFirst; + using Comparator = ComparePair; bool sorted = true; TimestampEvents events_list; From 0421ab6c808c8c9a7234c3d277b55be794dabf1a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 10 Jan 2021 02:21:10 +0300 Subject: [PATCH 232/256] Use time_macros over unset SOURCE_DATE_EPOCH in ccache 4.2 (unreleased) ccache 4.2+ will ignore SOURCE_DATE_EPOCH [1]. [1]: https://github.com/ccache/ccache/pull/755 --- cmake/find/ccache.cmake | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/cmake/find/ccache.cmake b/cmake/find/ccache.cmake index 2cdfed56de8..90f9fcdd568 100644 --- a/cmake/find/ccache.cmake +++ b/cmake/find/ccache.cmake @@ -32,12 +32,21 @@ if (CCACHE_FOUND AND NOT COMPILER_MATCHES_CCACHE) if (CCACHE_VERSION VERSION_GREATER "3.2.0" OR NOT CMAKE_CXX_COMPILER_ID STREQUAL "Clang") message(STATUS "Using ${CCACHE_FOUND} ${CCACHE_VERSION}") - # 4+ ccache respect SOURCE_DATE_EPOCH (always includes it into the hash - # of the manifest) and debian will extract these from d/changelog, and - # makes cache of ccache unusable + # debian (debhlpers) set SOURCE_DATE_EPOCH environment variable, that is + # filled from the debian/changelog or current time. # - # FIXME: once sloppiness will be introduced for this this can be removed. - if (CCACHE_VERSION VERSION_GREATER "4.0") + # - 4.0+ ccache always includes this environemtn variable into the hash + # of the manifest, which do not allow to use previous cache, + # - 4.2+ ccache ignores SOURCE_DATE_EPOCH under time_macros sloppiness. + # + # So for: + # - 4.2+ time_macros sloppiness is used, + # - 4.0+ will ignore SOURCE_DATE_EPOCH environment variable. + if (CCACHE_VERSION VERSION_GREATER_EQUAL "4.2") + message(STATUS "Use time_macros sloppiness for ccache") + set_property (GLOBAL PROPERTY RULE_LAUNCH_COMPILE "${CCACHE_FOUND} --set-config=sloppiness=time_macros") + set_property (GLOBAL PROPERTY RULE_LAUNCH_LINK "${CCACHE_FOUND} --set-config=sloppiness=time_macros") + elseif (CCACHE_VERSION VERSION_GREATER_EQUAL "4.0") message(STATUS "Ignore SOURCE_DATE_EPOCH for ccache") set_property (GLOBAL PROPERTY RULE_LAUNCH_COMPILE "env -u SOURCE_DATE_EPOCH ${CCACHE_FOUND}") set_property (GLOBAL PROPERTY RULE_LAUNCH_LINK "env -u SOURCE_DATE_EPOCH ${CCACHE_FOUND}") From d0378395c56524765eac8ba99ce864161cd84c2b Mon Sep 17 00:00:00 2001 From: HuFuwang Date: Sun, 10 Jan 2021 07:49:11 +0800 Subject: [PATCH 233/256] add tests 01646_fix_window_funnel_inconistency. --- ...6_fix_window_funnel_inconistency.reference | 3 ++ .../01646_fix_window_funnel_inconistency.sql | 49 +++++++++++++++++++ 2 files changed, 52 insertions(+) create mode 100644 tests/queries/0_stateless/01646_fix_window_funnel_inconistency.reference create mode 100644 tests/queries/0_stateless/01646_fix_window_funnel_inconistency.sql diff --git a/tests/queries/0_stateless/01646_fix_window_funnel_inconistency.reference b/tests/queries/0_stateless/01646_fix_window_funnel_inconistency.reference new file mode 100644 index 00000000000..3b39d6021a2 --- /dev/null +++ b/tests/queries/0_stateless/01646_fix_window_funnel_inconistency.reference @@ -0,0 +1,3 @@ +2 1 +--- +2 1 diff --git a/tests/queries/0_stateless/01646_fix_window_funnel_inconistency.sql b/tests/queries/0_stateless/01646_fix_window_funnel_inconistency.sql new file mode 100644 index 00000000000..51f599a2b09 --- /dev/null +++ b/tests/queries/0_stateless/01646_fix_window_funnel_inconistency.sql @@ -0,0 +1,49 @@ +DROP TABLE IF EXISTS trend; +CREATE TABLE trend +( + `event_date` Date, + `user_id` Int32, + `timestamp` DateTime, + `eventID` Int32, + `product` String +) +ENGINE = MergeTree() +PARTITION BY toYYYYMM(event_date) +ORDER BY user_id; + +insert into trend values ('2019-01-28', 1, '2019-01-29 10:00:00', 1004, 'phone') ('2019-01-28', 1, '2019-01-29 10:00:00', 1003, 'phone') ('2019-01-28', 1, '2019-01-28 10:00:00', 1002, 'phone'); + +SELECT + level, + count() AS c +FROM +( + SELECT + user_id, + windowFunnel(6048000000000000)(timestamp, eventID = 1004, eventID = 1003, eventID = 1002) AS level + FROM trend + GROUP BY user_id +) +GROUP BY level +ORDER BY level ASC; + +SELECT '---'; + +TRUNCATE TABLE trend; +insert into trend values ('2019-01-28', 1, '2019-01-29 10:00:00', 1003, 'phone') ('2019-01-28', 1, '2019-01-29 10:00:00', 1004, 'phone') ('2019-01-28', 1, '2019-01-28 10:00:00', 1002, 'phone'); + +SELECT + level, + count() AS c +FROM +( + SELECT + user_id, + windowFunnel(6048000000000000)(timestamp, eventID = 1004, eventID = 1003, eventID = 1002) AS level + FROM trend + GROUP BY user_id +) +GROUP BY level +ORDER BY level ASC; + +DROP TABLE trend; From 6eb5a5f4d94b06295525bdc29b38294061f31cfc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Jan 2021 03:28:59 +0300 Subject: [PATCH 234/256] Remove useless code --- src/Common/ProfileEvents.cpp | 3 - src/Dictionaries/SSDCacheDictionary.cpp | 2 + src/Dictionaries/SSDCacheDictionary.h | 1 - .../SSDComplexKeyCacheDictionary.cpp | 2 + .../SSDComplexKeyCacheDictionary.h | 1 - src/Disks/DiskCacheWrapper.cpp | 14 +- src/Disks/DiskCacheWrapper.h | 2 +- src/Disks/DiskDecorator.cpp | 4 +- src/Disks/DiskDecorator.h | 2 +- src/Disks/DiskLocal.cpp | 5 +- src/Disks/DiskLocal.h | 4 +- src/Disks/DiskMemory.cpp | 2 +- src/Disks/DiskMemory.h | 4 +- src/Disks/IDisk.h | 4 +- src/Disks/S3/DiskS3.cpp | 2 +- src/Disks/S3/DiskS3.h | 4 +- src/IO/WriteBufferAIO.cpp | 441 ---------------- src/IO/WriteBufferAIO.h | 102 ---- src/IO/createWriteBufferFromFileBase.cpp | 48 -- src/IO/createWriteBufferFromFileBase.h | 28 - src/IO/tests/CMakeLists.txt | 3 - src/IO/tests/write_buffer_aio.cpp | 498 ------------------ .../MergeTree/IMergedBlockOutputStream.cpp | 1 - .../MergeTree/MergeTreeDataMergerMutator.cpp | 4 +- .../MergeTreeDataPartWriterCompact.cpp | 4 +- .../MergeTreeDataPartWriterOnDisk.cpp | 9 +- .../MergeTree/MergeTreeDataPartWriterOnDisk.h | 4 +- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 9 +- .../MergeTree/MergeTreeDataPartWriterWide.h | 3 +- src/Storages/MergeTree/MergeTreeIOSettings.h | 7 - .../MergeTree/MergedBlockOutputStream.cpp | 20 - .../MergeTree/MergedBlockOutputStream.h | 9 - 32 files changed, 34 insertions(+), 1212 deletions(-) delete mode 100644 src/IO/WriteBufferAIO.cpp delete mode 100644 src/IO/WriteBufferAIO.h delete mode 100644 src/IO/createWriteBufferFromFileBase.cpp delete mode 100644 src/IO/createWriteBufferFromFileBase.h delete mode 100644 src/IO/tests/write_buffer_aio.cpp diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 0b0604cbe30..c459bf41352 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -45,9 +45,6 @@ M(CreatedReadBufferAIOFailed, "") \ M(CreatedReadBufferMMap, "") \ M(CreatedReadBufferMMapFailed, "") \ - M(CreatedWriteBufferOrdinary, "") \ - M(CreatedWriteBufferAIO, "") \ - M(CreatedWriteBufferAIOFailed, "") \ M(DiskReadElapsedMicroseconds, "Total time spent waiting for read syscall. This include reads from page cache.") \ M(DiskWriteElapsedMicroseconds, "Total time spent waiting for write syscall. This include writes to page cache.") \ M(NetworkReceiveElapsedMicroseconds, "") \ diff --git a/src/Dictionaries/SSDCacheDictionary.cpp b/src/Dictionaries/SSDCacheDictionary.cpp index 5547e34758f..1cf5946c95c 100644 --- a/src/Dictionaries/SSDCacheDictionary.cpp +++ b/src/Dictionaries/SSDCacheDictionary.cpp @@ -21,6 +21,8 @@ #include #include #include +#include + namespace ProfileEvents { diff --git a/src/Dictionaries/SSDCacheDictionary.h b/src/Dictionaries/SSDCacheDictionary.h index f4041e671ec..59df778e1f2 100644 --- a/src/Dictionaries/SSDCacheDictionary.h +++ b/src/Dictionaries/SSDCacheDictionary.h @@ -16,7 +16,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Dictionaries/SSDComplexKeyCacheDictionary.cpp b/src/Dictionaries/SSDComplexKeyCacheDictionary.cpp index 5ac821e5eda..58b117b3596 100644 --- a/src/Dictionaries/SSDComplexKeyCacheDictionary.cpp +++ b/src/Dictionaries/SSDComplexKeyCacheDictionary.cpp @@ -22,6 +22,8 @@ #include #include #include +#include + namespace ProfileEvents { diff --git a/src/Dictionaries/SSDComplexKeyCacheDictionary.h b/src/Dictionaries/SSDComplexKeyCacheDictionary.h index af9a0c0a7ee..4758d62f1df 100644 --- a/src/Dictionaries/SSDComplexKeyCacheDictionary.h +++ b/src/Dictionaries/SSDComplexKeyCacheDictionary.h @@ -19,7 +19,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Disks/DiskCacheWrapper.cpp b/src/Disks/DiskCacheWrapper.cpp index 7ce963380d4..652ea9935ed 100644 --- a/src/Disks/DiskCacheWrapper.cpp +++ b/src/Disks/DiskCacheWrapper.cpp @@ -146,7 +146,7 @@ DiskCacheWrapper::readFile(const String & path, size_t buf_size, size_t estimate auto tmp_path = path + ".tmp"; { auto src_buffer = DiskDecorator::readFile(path, buf_size, estimated_size, aio_threshold, mmap_threshold); - auto dst_buffer = cache_disk->writeFile(tmp_path, buf_size, WriteMode::Rewrite, estimated_size, aio_threshold); + auto dst_buffer = cache_disk->writeFile(tmp_path, buf_size, WriteMode::Rewrite); copyData(*src_buffer, *dst_buffer); } cache_disk->moveFile(tmp_path, path); @@ -175,10 +175,10 @@ DiskCacheWrapper::readFile(const String & path, size_t buf_size, size_t estimate } std::unique_ptr -DiskCacheWrapper::writeFile(const String & path, size_t buf_size, WriteMode mode, size_t estimated_size, size_t aio_threshold) +DiskCacheWrapper::writeFile(const String & path, size_t buf_size, WriteMode mode) { if (!cache_file_predicate(path)) - return DiskDecorator::writeFile(path, buf_size, mode, estimated_size, aio_threshold); + return DiskDecorator::writeFile(path, buf_size, mode); LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Write file {} to cache", backQuote(path)); @@ -187,12 +187,12 @@ DiskCacheWrapper::writeFile(const String & path, size_t buf_size, WriteMode mode cache_disk->createDirectories(dir_path); return std::make_unique( - cache_disk->writeFile(path, buf_size, mode, estimated_size, aio_threshold), - [this, path, buf_size, mode, estimated_size, aio_threshold]() + cache_disk->writeFile(path, buf_size, mode), + [this, path, buf_size, mode]() { /// Copy file from cache to actual disk when cached buffer is finalized. - auto src_buffer = cache_disk->readFile(path, buf_size, estimated_size, aio_threshold, 0); - auto dst_buffer = DiskDecorator::writeFile(path, buf_size, mode, estimated_size, aio_threshold); + auto src_buffer = cache_disk->readFile(path, buf_size, 0, 0, 0); + auto dst_buffer = DiskDecorator::writeFile(path, buf_size, mode); copyData(*src_buffer, *dst_buffer); dst_buffer->finalize(); }, diff --git a/src/Disks/DiskCacheWrapper.h b/src/Disks/DiskCacheWrapper.h index b0b373d900c..9e2b3c3b592 100644 --- a/src/Disks/DiskCacheWrapper.h +++ b/src/Disks/DiskCacheWrapper.h @@ -36,7 +36,7 @@ public: std::unique_ptr readFile(const String & path, size_t buf_size, size_t estimated_size, size_t aio_threshold, size_t mmap_threshold) const override; std::unique_ptr - writeFile(const String & path, size_t buf_size, WriteMode mode, size_t estimated_size, size_t aio_threshold) override; + writeFile(const String & path, size_t buf_size, WriteMode mode) override; void remove(const String & path) override; void removeRecursive(const String & path) override; void createHardLink(const String & src_path, const String & dst_path) override; diff --git a/src/Disks/DiskDecorator.cpp b/src/Disks/DiskDecorator.cpp index aaa54005f6f..e62f2737ec5 100644 --- a/src/Disks/DiskDecorator.cpp +++ b/src/Disks/DiskDecorator.cpp @@ -125,9 +125,9 @@ DiskDecorator::readFile(const String & path, size_t buf_size, size_t estimated_s } std::unique_ptr -DiskDecorator::writeFile(const String & path, size_t buf_size, WriteMode mode, size_t estimated_size, size_t aio_threshold) +DiskDecorator::writeFile(const String & path, size_t buf_size, WriteMode mode) { - return delegate->writeFile(path, buf_size, mode, estimated_size, aio_threshold); + return delegate->writeFile(path, buf_size, mode); } void DiskDecorator::remove(const String & path) diff --git a/src/Disks/DiskDecorator.h b/src/Disks/DiskDecorator.h index 1ce3c3ea773..61b7ee9ecee 100644 --- a/src/Disks/DiskDecorator.h +++ b/src/Disks/DiskDecorator.h @@ -38,7 +38,7 @@ public: std::unique_ptr readFile(const String & path, size_t buf_size, size_t estimated_size, size_t aio_threshold, size_t mmap_threshold) const override; std::unique_ptr - writeFile(const String & path, size_t buf_size, WriteMode mode, size_t estimated_size, size_t aio_threshold) override; + writeFile(const String & path, size_t buf_size, WriteMode mode) override; void remove(const String & path) override; void removeRecursive(const String & path) override; void setLastModified(const String & path, const Poco::Timestamp & timestamp) override; diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index cde9b3c5a41..e2d6a5410a3 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -7,7 +7,6 @@ #include #include -#include #include #include @@ -232,10 +231,10 @@ DiskLocal::readFile(const String & path, size_t buf_size, size_t estimated_size, } std::unique_ptr -DiskLocal::writeFile(const String & path, size_t buf_size, WriteMode mode, size_t estimated_size, size_t aio_threshold) +DiskLocal::writeFile(const String & path, size_t buf_size, WriteMode mode) { int flags = (mode == WriteMode::Append) ? (O_APPEND | O_CREAT | O_WRONLY) : -1; - return createWriteBufferFromFileBase(disk_path + path, estimated_size, aio_threshold, buf_size, flags); + return std::make_unique(disk_path + path, buf_size, flags); } void DiskLocal::remove(const String & path) diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index 762a8502faa..d862f72d4f8 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -83,9 +83,7 @@ public: std::unique_ptr writeFile( const String & path, size_t buf_size, - WriteMode mode, - size_t estimated_size, - size_t aio_threshold) override; + WriteMode mode) override; void remove(const String & path) override; diff --git a/src/Disks/DiskMemory.cpp b/src/Disks/DiskMemory.cpp index d185263d48c..aecdbf02f50 100644 --- a/src/Disks/DiskMemory.cpp +++ b/src/Disks/DiskMemory.cpp @@ -330,7 +330,7 @@ std::unique_ptr DiskMemory::readFile(const String & path return std::make_unique(path, iter->second.data); } -std::unique_ptr DiskMemory::writeFile(const String & path, size_t buf_size, WriteMode mode, size_t, size_t) +std::unique_ptr DiskMemory::writeFile(const String & path, size_t buf_size, WriteMode mode) { std::lock_guard lock(mutex); diff --git a/src/Disks/DiskMemory.h b/src/Disks/DiskMemory.h index 4d4b947098b..e24f3c9c184 100644 --- a/src/Disks/DiskMemory.h +++ b/src/Disks/DiskMemory.h @@ -74,9 +74,7 @@ public: std::unique_ptr writeFile( const String & path, size_t buf_size, - WriteMode mode, - size_t estimated_size, - size_t aio_threshold) override; + WriteMode mode) override; void remove(const String & path) override; diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index ac0f5a2ae8f..437718924db 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -148,9 +148,7 @@ public: virtual std::unique_ptr writeFile( const String & path, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, - WriteMode mode = WriteMode::Rewrite, - size_t estimated_size = 0, - size_t aio_threshold = 0) = 0; + WriteMode mode = WriteMode::Rewrite) = 0; /// Remove file or directory. Throws exception if file doesn't exists or if directory is not empty. virtual void remove(const String & path) = 0; diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 4786c05f8b0..d9b2eaefa81 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -665,7 +665,7 @@ std::unique_ptr DiskS3::readFile(const String & path, si return std::make_unique(std::move(reader), min_bytes_for_seek); } -std::unique_ptr DiskS3::writeFile(const String & path, size_t buf_size, WriteMode mode, size_t, size_t) +std::unique_ptr DiskS3::writeFile(const String & path, size_t buf_size, WriteMode mode) { bool exist = exists(path); if (exist && readMeta(path).read_only) diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index f62c603adda..900378f46cd 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -88,9 +88,7 @@ public: std::unique_ptr writeFile( const String & path, size_t buf_size, - WriteMode mode, - size_t estimated_size, - size_t aio_threshold) override; + WriteMode mode) override; void remove(const String & path) override; diff --git a/src/IO/WriteBufferAIO.cpp b/src/IO/WriteBufferAIO.cpp deleted file mode 100644 index 8e0224669f2..00000000000 --- a/src/IO/WriteBufferAIO.cpp +++ /dev/null @@ -1,441 +0,0 @@ -#if defined(OS_LINUX) || defined(__FreeBSD__) - -#include -#include -#include - -#include -#include -#include -#include - - -namespace ProfileEvents -{ - extern const Event FileOpen; - extern const Event WriteBufferAIOWrite; - extern const Event WriteBufferAIOWriteBytes; -} - -namespace CurrentMetrics -{ - extern const Metric Write; -} - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int FILE_DOESNT_EXIST; - extern const int CANNOT_OPEN_FILE; - extern const int LOGICAL_ERROR; - extern const int ARGUMENT_OUT_OF_BOUND; - extern const int AIO_READ_ERROR; - extern const int AIO_WRITE_ERROR; - extern const int CANNOT_IO_SUBMIT; - extern const int CANNOT_IO_GETEVENTS; - extern const int CANNOT_TRUNCATE_FILE; - extern const int CANNOT_FSYNC; -} - - -/// Note: an additional page is allocated that will contain data that -/// do not fit into the main buffer. -WriteBufferAIO::WriteBufferAIO(const std::string & filename_, size_t buffer_size_, int flags_, mode_t mode_, - char * existing_memory_) - : WriteBufferFromFileBase(buffer_size_ + DEFAULT_AIO_FILE_BLOCK_SIZE, existing_memory_, DEFAULT_AIO_FILE_BLOCK_SIZE), - flush_buffer(BufferWithOwnMemory(this->memory.size(), nullptr, DEFAULT_AIO_FILE_BLOCK_SIZE)), - filename(filename_) -{ - ProfileEvents::increment(ProfileEvents::FileOpen); - - /// Correct the buffer size information so that additional pages do not touch the base class `BufferBase`. - this->buffer().resize(this->buffer().size() - DEFAULT_AIO_FILE_BLOCK_SIZE); - this->internalBuffer().resize(this->internalBuffer().size() - DEFAULT_AIO_FILE_BLOCK_SIZE); - flush_buffer.buffer().resize(this->buffer().size() - DEFAULT_AIO_FILE_BLOCK_SIZE); - flush_buffer.internalBuffer().resize(this->internalBuffer().size() - DEFAULT_AIO_FILE_BLOCK_SIZE); - - int open_flags = (flags_ == -1) ? (O_RDWR | O_TRUNC | O_CREAT) : flags_; - open_flags |= O_DIRECT; - open_flags |= O_CLOEXEC; - - fd = ::open(filename.c_str(), open_flags, mode_); - if (fd == -1) - { - auto error_code = (errno == ENOENT) ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE; - throwFromErrnoWithPath("Cannot open file " + filename, filename, error_code); - } -} - -WriteBufferAIO::~WriteBufferAIO() -{ - if (!aio_failed) - { - try - { - flush(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } - - if (fd != -1) - ::close(fd); -} - -off_t WriteBufferAIO::getPositionInFile() -{ - return seek(0, SEEK_CUR); -} - -void WriteBufferAIO::sync() -{ - flush(); - - /// Ask OS to flush data to disk. - int res = ::fsync(fd); - if (res == -1) - throwFromErrnoWithPath("Cannot fsync " + getFileName(), getFileName(), ErrorCodes::CANNOT_FSYNC); -} - -void WriteBufferAIO::nextImpl() -{ - if (!offset()) - return; - - if (waitForAIOCompletion()) - finalize(); - - /// Create a request for asynchronous write. - prepare(); - -#if defined(__FreeBSD__) - request.aio.aio_lio_opcode = LIO_WRITE; - request.aio.aio_fildes = fd; - request.aio.aio_buf = reinterpret_cast(buffer_begin); - request.aio.aio_nbytes = region_aligned_size; - request.aio.aio_offset = region_aligned_begin; -#else - request.aio_lio_opcode = IOCB_CMD_PWRITE; - request.aio_fildes = fd; - request.aio_buf = reinterpret_cast(buffer_begin); - request.aio_nbytes = region_aligned_size; - request.aio_offset = region_aligned_begin; -#endif - - /// Send the request. - while (io_submit(aio_context.ctx, 1, &request_ptr) < 0) - { - if (errno != EINTR) - { - aio_failed = true; - throw Exception("Cannot submit request for asynchronous IO on file " + filename, ErrorCodes::CANNOT_IO_SUBMIT); - } - } - - is_pending_write = true; -} - -off_t WriteBufferAIO::seek(off_t off, int whence) -{ - flush(); - - if (whence == SEEK_SET) - { - if (off < 0) - throw Exception("SEEK_SET underflow", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - pos_in_file = off; - } - else if (whence == SEEK_CUR) - { - if (off >= 0) - { - if (off > (std::numeric_limits::max() - pos_in_file)) - throw Exception("SEEK_CUR overflow", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - } - else if (off < -pos_in_file) - throw Exception("SEEK_CUR underflow", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - pos_in_file += off; - } - else - throw Exception("WriteBufferAIO::seek expects SEEK_SET or SEEK_CUR as whence", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - - if (pos_in_file > max_pos_in_file) - max_pos_in_file = pos_in_file; - - return pos_in_file; -} - -void WriteBufferAIO::truncate(off_t length) -{ - flush(); - - int res = ::ftruncate(fd, length); - if (res == -1) - throwFromErrnoWithPath("Cannot truncate file " + filename, filename, ErrorCodes::CANNOT_TRUNCATE_FILE); -} - -void WriteBufferAIO::flush() -{ - next(); - if (waitForAIOCompletion()) - finalize(); -} - -bool WriteBufferAIO::waitForAIOCompletion() -{ - if (!is_pending_write) - return false; - - CurrentMetrics::Increment metric_increment_write{CurrentMetrics::Write}; - - io_event event; - while (io_getevents(aio_context.ctx, 1, 1, &event, nullptr) < 0) - { - if (errno != EINTR) - { - aio_failed = true; - throw Exception("Failed to wait for asynchronous IO completion on file " + filename, ErrorCodes::CANNOT_IO_GETEVENTS); - } - } - - // Unpoison the memory returned from an uninstrumented system function. - __msan_unpoison(&event, sizeof(event)); - - is_pending_write = false; -#if defined(__FreeBSD__) - bytes_written = aio_return(reinterpret_cast(event.udata)); -#else - bytes_written = event.res; -#endif - - ProfileEvents::increment(ProfileEvents::WriteBufferAIOWrite); - ProfileEvents::increment(ProfileEvents::WriteBufferAIOWriteBytes, bytes_written); - - return true; -} - -void WriteBufferAIO::prepare() -{ - /// Swap the main and duplicate buffers. - swap(flush_buffer); - - truncation_count = 0; - - /* - A page on disk or in memory - - start address (starting position in case of disk) is a multiply of DEFAULT_AIO_FILE_BLOCK_SIZE - : - : - +---------------+ - | | - | | - | | - | | - | | - | | - +---------------+ - <---------------> - : - : - DEFAULT_AIO_FILE_BLOCK_SIZE - - */ - - /* - Representation of data on a disk - - XXX : the data you want to write - ZZZ : data that is already on disk or zeros, if there is no data - - region_aligned_begin region_aligned_end - : region_begin region_end : - : : : : - : : : : - +---:-----------+---------------+---------------+---------------+--:------------+ - | : | | | | : | - | +-----------+---------------+---------------+---------------+--+ | - |ZZZ|XXXXXXXXXXX|XXXXXXXXXXXXXXX|XXXXXXXXXXXXXXX|XXXXXXXXXXXXXXX|XX|ZZZZZZZZZZZZ| - |ZZZ|XXXXXXXXXXX|XXXXXXXXXXXXXXX|XXXXXXXXXXXXXXX|XXXXXXXXXXXXXXX|XX|ZZZZZZZZZZZZ| - | +-----------+---------------+---------------+---------------+--+ | - | | | | | | - +---------------+---------------+---------------+---------------+---------------+ - - <--><--------------------------------------------------------------><-----------> - : : : - : : : - region_left_padding region_size region_right_padding - - <-------------------------------------------------------------------------------> - : - : - region_aligned_size - */ - - /// Region of the disk in which we want to write data. - const off_t region_begin = pos_in_file; - - if ((flush_buffer.offset() > static_cast(std::numeric_limits::max())) || - (pos_in_file > (std::numeric_limits::max() - static_cast(flush_buffer.offset())))) - throw Exception("An overflow occurred during file operation", ErrorCodes::LOGICAL_ERROR); - - const off_t region_end = pos_in_file + flush_buffer.offset(); - const size_t region_size = region_end - region_begin; - - /// The aligned region of the disk into which we want to write the data. - const size_t region_left_padding = region_begin % DEFAULT_AIO_FILE_BLOCK_SIZE; - const size_t region_right_padding = (DEFAULT_AIO_FILE_BLOCK_SIZE - (region_end % DEFAULT_AIO_FILE_BLOCK_SIZE)) % DEFAULT_AIO_FILE_BLOCK_SIZE; - - region_aligned_begin = region_begin - region_left_padding; - - if (region_end > (std::numeric_limits::max() - static_cast(region_right_padding))) - throw Exception("An overflow occurred during file operation", ErrorCodes::LOGICAL_ERROR); - - const off_t region_aligned_end = region_end + region_right_padding; - region_aligned_size = region_aligned_end - region_aligned_begin; - - bytes_to_write = region_aligned_size; - - /* - Representing data in the buffer before processing - - XXX : the data you want to write - - buffer_begin buffer_end - : : - : : - +---------------+---------------+---------------+-------------:-+ - | | | | : | - +---------------+---------------+---------------+-------------+ | - |XXXXXXXXXXXXXXX|XXXXXXXXXXXXXXX|XXXXXXXXXXXXXXX|XXXXXXXXXXXXX| | - |XXXXXXXXXXXXXXX|XXXXXXXXXXXXXXX|XXXXXXXXXXXXXXX|XXXXXXXXXXXXX| | - +---------------+---------------+---------------+-------------+ | - | | | | | - +---------------+---------------+---------------+---------------+ - - <-------------------------------------------------------------> - : - : - buffer_size - */ - - /// The buffer of data that we want to write to the disk. - buffer_begin = flush_buffer.buffer().begin(); - Position buffer_end = buffer_begin + region_size; - size_t buffer_size = buffer_end - buffer_begin; - - /// Process the buffer so that it reflects the structure of the disk region. - - /* - Representation of data in the buffer after processing - - XXX : the data you want to write - ZZZ : data from disk or zeros, if there is no data - - `buffer_begin` `buffer_end` extra page - : : : - : : : - +---:-----------+---------------+---------------+---------------+--:------------+ - | | | | | : | - | +-----------+---------------+---------------+---------------+--+ | - |ZZZ|XXXXXXXXXXX|XXXXXXXXXXXXXXX|XXXXXXXXXXXXXXX|XXXXXXXXXXXXXXX|XX|ZZZZZZZZZZZZ| - |ZZZ|XXXXXXXXXXX|XXXXXXXXXXXXXXX|XXXXXXXXXXXXXXX|XXXXXXXXXXXXXXX|XX|ZZZZZZZZZZZZ| - | +-----------+---------------+---------------+---------------+--+ | - | | | | | | - +---------------+---------------+---------------+---------------+---------------+ - - <--><--------------------------------------------------------------><-----------> - : : : - : : : - region_left_padding region_size region_right_padding - - <-------------------------------------------------------------------------------> - : - : - region_aligned_size - */ - - if ((region_left_padding > 0) || (region_right_padding > 0)) - { - char memory_page[DEFAULT_AIO_FILE_BLOCK_SIZE] __attribute__ ((aligned (DEFAULT_AIO_FILE_BLOCK_SIZE))); - - if (region_left_padding > 0) - { - /// Move the buffer data to the right. Complete the beginning of the buffer with data from the disk. - buffer_size += region_left_padding; - buffer_end = buffer_begin + buffer_size; - - ::memmove(buffer_begin + region_left_padding, buffer_begin, (buffer_size - region_left_padding) * sizeof(*buffer_begin)); - - ssize_t read_count = ::pread(fd, memory_page, DEFAULT_AIO_FILE_BLOCK_SIZE, region_aligned_begin); - if (read_count < 0) - throw Exception("Read error", ErrorCodes::AIO_READ_ERROR); - - size_t to_copy = std::min(static_cast(read_count), region_left_padding); - ::memcpy(buffer_begin, memory_page, to_copy * sizeof(*buffer_begin)); - ::memset(buffer_begin + to_copy, 0, (region_left_padding - to_copy) * sizeof(*buffer_begin)); - } - - if (region_right_padding > 0) - { - /// Add the end of the buffer with data from the disk. - ssize_t read_count = ::pread(fd, memory_page, DEFAULT_AIO_FILE_BLOCK_SIZE, region_aligned_end - DEFAULT_AIO_FILE_BLOCK_SIZE); - if (read_count < 0) - throw Exception("Read error", ErrorCodes::AIO_READ_ERROR); - - Position truncation_begin; - off_t offset = DEFAULT_AIO_FILE_BLOCK_SIZE - region_right_padding; - if (read_count > offset) - { - ::memcpy(buffer_end, memory_page + offset, (read_count - offset) * sizeof(*buffer_end)); - truncation_begin = buffer_end + (read_count - offset); - truncation_count = DEFAULT_AIO_FILE_BLOCK_SIZE - read_count; - } - else - { - truncation_begin = buffer_end; - truncation_count = region_right_padding; - } - - ::memset(truncation_begin, 0, truncation_count * sizeof(*truncation_begin)); - } - } -} - -void WriteBufferAIO::finalize() -{ - if (bytes_written < bytes_to_write) - throw Exception("Asynchronous write error on file " + filename, ErrorCodes::AIO_WRITE_ERROR); - - bytes_written -= truncation_count; - -#if defined(__FreeBSD__) - off_t aio_offset = request.aio.aio_offset; -#else - off_t aio_offset = request.aio_offset; -#endif - off_t pos_offset = bytes_written - (pos_in_file - aio_offset); - - if (pos_in_file > (std::numeric_limits::max() - pos_offset)) - throw Exception("An overflow occurred during file operation", ErrorCodes::LOGICAL_ERROR); - pos_in_file += pos_offset; - - if (pos_in_file > max_pos_in_file) - max_pos_in_file = pos_in_file; - - if (truncation_count > 0) - { - /// Truncate the file to remove unnecessary zeros from it. - int res = ::ftruncate(fd, max_pos_in_file); - if (res == -1) - throwFromErrnoWithPath("Cannot truncate file " + filename, filename, ErrorCodes::CANNOT_TRUNCATE_FILE); - } -} - -} - -#endif diff --git a/src/IO/WriteBufferAIO.h b/src/IO/WriteBufferAIO.h deleted file mode 100644 index f514acab359..00000000000 --- a/src/IO/WriteBufferAIO.h +++ /dev/null @@ -1,102 +0,0 @@ -#pragma once - -#if defined(OS_LINUX) || defined(__FreeBSD__) - -#include -#include -#include -#include -#include -#include - -#include -#include -#include - - -namespace CurrentMetrics -{ - extern const Metric OpenFileForWrite; -} - -namespace DB -{ - -/** Class for asynchronous data writing. - */ -class WriteBufferAIO final : public WriteBufferFromFileBase -{ -public: - WriteBufferAIO(const std::string & filename_, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE, int flags_ = -1, mode_t mode_ = 0666, - char * existing_memory_ = nullptr); - ~WriteBufferAIO() override; - - WriteBufferAIO(const WriteBufferAIO &) = delete; - WriteBufferAIO & operator=(const WriteBufferAIO &) = delete; - - off_t getPositionInFile(); - off_t seek(off_t off, int whence); - void truncate(off_t length); - void sync() override; - std::string getFileName() const override { return filename; } - int getFD() const { return fd; } - -private: - void nextImpl() override; - - /// If there's still data in the buffer, we'll write them. - void flush(); - /// Wait for the end of the current asynchronous task. - bool waitForAIOCompletion(); - /// Prepare an asynchronous request. - void prepare(); - /// - void finalize() override; - -private: - /// Buffer for asynchronous data writes. - BufferWithOwnMemory flush_buffer; - - /// Description of the asynchronous write request. - iocb request{}; - iocb * request_ptr{&request}; - - AIOContext aio_context{1}; - - const std::string filename; - - /// The number of bytes to be written to the disk. - off_t bytes_to_write = 0; - /// Number of bytes written with the last request. - off_t bytes_written = 0; - /// The number of zero bytes to be cut from the end of the file - /// after the data write operation completes. - off_t truncation_count = 0; - - /// The current position in the file. - off_t pos_in_file = 0; - /// The maximum position reached in the file. - off_t max_pos_in_file = 0; - - /// The starting position of the aligned region of the disk to which the data is written. - off_t region_aligned_begin = 0; - /// The size of the aligned region of the disk. - size_t region_aligned_size = 0; - - /// The file descriptor for writing. - int fd = -1; - - /// The data buffer that we want to write to the disk. - Position buffer_begin = nullptr; - - /// Is the asynchronous write operation still in progress? - bool is_pending_write = false; - /// Did the asynchronous operation fail? - bool aio_failed = false; - - CurrentMetrics::Increment metric_increment{CurrentMetrics::OpenFileForWrite}; -}; - -} - -#endif diff --git a/src/IO/createWriteBufferFromFileBase.cpp b/src/IO/createWriteBufferFromFileBase.cpp deleted file mode 100644 index 6022457f32e..00000000000 --- a/src/IO/createWriteBufferFromFileBase.cpp +++ /dev/null @@ -1,48 +0,0 @@ -#include -#include -#if defined(OS_LINUX) || defined(__FreeBSD__) -#include -#endif -#include - - -namespace ProfileEvents -{ - extern const Event CreatedWriteBufferOrdinary; - extern const Event CreatedWriteBufferAIO; - extern const Event CreatedWriteBufferAIOFailed; -} - -namespace DB -{ - -std::unique_ptr createWriteBufferFromFileBase(const std::string & filename_, size_t estimated_size, - size_t aio_threshold, size_t buffer_size_, int flags_, mode_t mode, char * existing_memory_, - size_t alignment) -{ -#if defined(OS_LINUX) || defined(__FreeBSD__) - if (aio_threshold && estimated_size >= aio_threshold) - { - /// Attempt to open a file with O_DIRECT - try - { - auto res = std::make_unique(filename_, buffer_size_, flags_, mode, existing_memory_); - ProfileEvents::increment(ProfileEvents::CreatedWriteBufferAIO); - return res; - } - catch (const ErrnoException &) - { - /// Fallback to cached IO if O_DIRECT is not supported. - ProfileEvents::increment(ProfileEvents::CreatedWriteBufferAIOFailed); - } - } -#else - (void)aio_threshold; - (void)estimated_size; -#endif - - ProfileEvents::increment(ProfileEvents::CreatedWriteBufferOrdinary); - return std::make_unique(filename_, buffer_size_, flags_, mode, existing_memory_, alignment); -} - -} diff --git a/src/IO/createWriteBufferFromFileBase.h b/src/IO/createWriteBufferFromFileBase.h deleted file mode 100644 index 42cad88303b..00000000000 --- a/src/IO/createWriteBufferFromFileBase.h +++ /dev/null @@ -1,28 +0,0 @@ -#pragma once - -#include -#include -#include - - -namespace DB -{ - -/** Create an object to write data to a file. - * estimated_size - number of bytes to write - * aio_threshold - the minimum number of bytes for asynchronous writes - * - * If aio_threshold = 0 or estimated_size < aio_threshold, the write operations are executed synchronously. - * Otherwise, write operations are performed asynchronously. - */ -std::unique_ptr createWriteBufferFromFileBase( - const std::string & filename_, - size_t estimated_size, - size_t aio_threshold, - size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE, - int flags_ = -1, - mode_t mode = 0666, - char * existing_memory_ = nullptr, - size_t alignment = 0); - -} diff --git a/src/IO/tests/CMakeLists.txt b/src/IO/tests/CMakeLists.txt index da4d330f0a9..fcd59d94cb0 100644 --- a/src/IO/tests/CMakeLists.txt +++ b/src/IO/tests/CMakeLists.txt @@ -55,9 +55,6 @@ add_executable (write_int write_int.cpp) target_link_libraries (write_int PRIVATE clickhouse_common_io) if (OS_LINUX OR OS_FREEBSD) - add_executable(write_buffer_aio write_buffer_aio.cpp) - target_link_libraries (write_buffer_aio PRIVATE clickhouse_common_io) - add_executable(read_buffer_aio read_buffer_aio.cpp) target_link_libraries (read_buffer_aio PRIVATE clickhouse_common_io) endif () diff --git a/src/IO/tests/write_buffer_aio.cpp b/src/IO/tests/write_buffer_aio.cpp deleted file mode 100644 index 9274e5abee5..00000000000 --- a/src/IO/tests/write_buffer_aio.cpp +++ /dev/null @@ -1,498 +0,0 @@ -#include -#include - -#include -#include -#include -#include -#include -#include - -namespace -{ - -namespace fs = std::filesystem; - -void run(); -[[noreturn]] void die(const std::string & msg); -void runTest(unsigned int num, const std::function & func); -std::string createTmpFile(); -std::string generateString(size_t n); - -bool test1(); -bool test2(); -bool test3(); -bool test4(); -bool test5(); -bool test6(); -bool test7(); -bool test8(); -bool test9(); -bool test10(); - -void run() -{ - const std::vector> tests = - { - test1, - test2, - test3, - test4, - test5, - test6, - test7, - test8, - test9, - test10 - }; - - unsigned int num = 0; - for (const auto & test : tests) - { - ++num; - runTest(num, test); - } -} - -void die(const std::string & msg) -{ - std::cout << msg; - ::exit(EXIT_FAILURE); -} - -void runTest(unsigned int num, const std::function & func) -{ - bool ok; - - try - { - ok = func(); - } - catch (const DB::Exception & ex) - { - ok = false; - std::cout << "Caught exception " << ex.displayText() << "\n"; - } - catch (const std::exception & ex) - { - ok = false; - std::cout << "Caught exception " << ex.what() << "\n"; - } - - if (ok) - std::cout << "Test " << num << " passed\n"; - else - std::cout << "Test " << num << " failed\n"; -} - -std::string createTmpFile() -{ - char pattern[] = "/tmp/fileXXXXXX"; - char * dir = ::mkdtemp(pattern); - if (dir == nullptr) - die("Could not create directory"); - - return std::string(dir) + "/foo"; -} - -std::string generateString(size_t n) -{ - static const std::string symbols = "ABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789"; - - std::string buf; - buf.reserve(n); - - for (size_t i = 0; i < n; ++i) - buf += symbols[i % symbols.length()]; - - return buf; -} - -bool test1() -{ - std::string filename = createTmpFile(); - - size_t n = 10 * DEFAULT_AIO_FILE_BLOCK_SIZE; - - std::string buf = generateString(n); - - { - DB::WriteBufferAIO out(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE); - - if (out.getFileName() != filename) - return false; - if (out.getFD() == -1) - return false; - - out.write(buf.data(), buf.length()); - } - - std::ifstream in(filename.c_str()); - if (!in.is_open()) - die("Could not open file"); - - std::string received{ std::istreambuf_iterator(in), std::istreambuf_iterator() }; - - in.close(); - fs::remove_all(fs::path(filename).parent_path().string()); - - return (received == buf); -} - -bool test2() -{ - std::string filename = createTmpFile(); - - size_t n = 10 * DEFAULT_AIO_FILE_BLOCK_SIZE; - - std::string buf = generateString(n); - - { - DB::WriteBufferAIO out(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE); - - if (out.getFileName() != filename) - return false; - if (out.getFD() == -1) - return false; - - out.write(buf.data(), buf.length() / 2); - out.seek(DEFAULT_AIO_FILE_BLOCK_SIZE, SEEK_CUR); - out.write(&buf[buf.length() / 2], buf.length() / 2); - } - - std::ifstream in(filename.c_str()); - if (!in.is_open()) - die("Could not open file"); - - std::string received{ std::istreambuf_iterator(in), std::istreambuf_iterator() }; - - in.close(); - fs::remove_all(fs::path(filename).parent_path().string()); - - if (received.substr(0, buf.length() / 2) != buf.substr(0, buf.length() / 2)) - return false; - if (received.substr(buf.length() / 2, DEFAULT_AIO_FILE_BLOCK_SIZE) != std::string(DEFAULT_AIO_FILE_BLOCK_SIZE, '\0')) - return false; - if (received.substr(buf.length() / 2 + DEFAULT_AIO_FILE_BLOCK_SIZE) != buf.substr(buf.length() / 2)) - return false; - - return true; -} - -bool test3() -{ - std::string filename = createTmpFile(); - - size_t n = 10 * DEFAULT_AIO_FILE_BLOCK_SIZE; - - std::string buf = generateString(n); - - { - DB::WriteBufferAIO out(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE); - - if (out.getFileName() != filename) - return false; - if (out.getFD() == -1) - return false; - - out.write(buf.data(), buf.length()); - - off_t pos1 = out.getPositionInFile(); - - out.truncate(buf.length() / 2); - - off_t pos2 = out.getPositionInFile(); - - if (pos1 != pos2) - return false; - } - - std::ifstream in(filename.c_str()); - if (!in.is_open()) - die("Could not open file"); - - std::string received{ std::istreambuf_iterator(in), std::istreambuf_iterator() }; - - in.close(); - fs::remove_all(fs::path(filename).parent_path().string()); - - return (received == buf.substr(0, buf.length() / 2)); -} - -bool test4() -{ - std::string filename = createTmpFile(); - - size_t n = 10 * DEFAULT_AIO_FILE_BLOCK_SIZE; - - std::string buf = generateString(n); - - { - DB::WriteBufferAIO out(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE); - - if (out.getFileName() != filename) - return false; - if (out.getFD() == -1) - return false; - - out.write(buf.data(), buf.length()); - - off_t pos1 = out.getPositionInFile(); - - out.truncate(3 * buf.length() / 2); - - off_t pos2 = out.getPositionInFile(); - - if (pos1 != pos2) - return false; - } - - std::ifstream in(filename.c_str()); - if (!in.is_open()) - die("Could not open file"); - - std::string received{ std::istreambuf_iterator(in), std::istreambuf_iterator() }; - - in.close(); - fs::remove_all(fs::path(filename).parent_path().string()); - - if (received.substr(0, buf.length()) != buf) - return false; - - if (received.substr(buf.length()) != std::string(buf.length() / 2, '\0')) - return false; - - return true; -} - -bool test5() -{ - std::string filename = createTmpFile(); - - size_t n = 10 * DEFAULT_AIO_FILE_BLOCK_SIZE; - - std::string buf = generateString(n); - - { - DB::WriteBufferAIO out(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE); - - if (out.getFileName() != filename) - return false; - if (out.getFD() == -1) - return false; - - out.seek(1, SEEK_SET); - out.write(buf.data(), buf.length()); - } - - std::ifstream in(filename.c_str()); - if (!in.is_open()) - die("Could not open file"); - - std::string received{ std::istreambuf_iterator(in), std::istreambuf_iterator() }; - - in.close(); - fs::remove_all(fs::path(filename).parent_path().string()); - - return received.substr(1) == buf; -} - -bool test6() -{ - std::string filename = createTmpFile(); - - size_t n = 10 * DEFAULT_AIO_FILE_BLOCK_SIZE; - - std::string buf = generateString(n); - - std::string buf2 = "1111111111"; - - { - DB::WriteBufferAIO out(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE); - - if (out.getFileName() != filename) - return false; - if (out.getFD() == -1) - return false; - - out.seek(3, SEEK_SET); - out.write(buf.data(), buf.length()); - out.seek(-2 * DEFAULT_AIO_FILE_BLOCK_SIZE, SEEK_CUR); - out.write(buf2.data(), buf2.length()); - } - - std::ifstream in(filename.c_str()); - if (!in.is_open()) - die("Could not open file"); - - std::string received{ std::istreambuf_iterator(in), std::istreambuf_iterator() }; - - in.close(); - fs::remove_all(fs::path(filename).parent_path().string()); - - if (received.substr(3, 8 * DEFAULT_AIO_FILE_BLOCK_SIZE) != buf.substr(0, 8 * DEFAULT_AIO_FILE_BLOCK_SIZE)) - return false; - - if (received.substr(3 + 8 * DEFAULT_AIO_FILE_BLOCK_SIZE, 10) != buf2) - return false; - - if (received.substr(13 + 8 * DEFAULT_AIO_FILE_BLOCK_SIZE) != buf.substr(10 + 8 * DEFAULT_AIO_FILE_BLOCK_SIZE)) - return false; - - return true; -} - -bool test7() -{ - std::string filename = createTmpFile(); - - std::string buf2 = "11111111112222222222"; - - { - DB::WriteBufferAIO out(filename, DEFAULT_AIO_FILE_BLOCK_SIZE); - - if (out.getFileName() != filename) - return false; - if (out.getFD() == -1) - return false; - - out.seek(DEFAULT_AIO_FILE_BLOCK_SIZE - (buf2.length() / 2), SEEK_SET); - out.write(buf2.data(), buf2.length()); - } - - std::ifstream in(filename.c_str()); - if (!in.is_open()) - die("Could not open file"); - - std::string received{ std::istreambuf_iterator(in), std::istreambuf_iterator() }; - - if (received.length() != 4106) - return false; - if (received.substr(0, 4086) != std::string(4086, '\0')) - return false; - if (received.substr(4086, 20) != buf2) - return false; - - in.close(); - fs::remove_all(fs::path(filename).parent_path().string()); - - return true; -} - -bool test8() -{ - std::string filename = createTmpFile(); - - std::string buf2 = "11111111112222222222"; - - { - DB::WriteBufferAIO out(filename, 2 * DEFAULT_AIO_FILE_BLOCK_SIZE); - - if (out.getFileName() != filename) - return false; - if (out.getFD() == -1) - return false; - - out.seek(2 * DEFAULT_AIO_FILE_BLOCK_SIZE - (buf2.length() / 2), SEEK_SET); - out.write(buf2.data(), buf2.length()); - } - - std::ifstream in(filename.c_str()); - if (!in.is_open()) - die("Could not open file"); - - std::string received{ std::istreambuf_iterator(in), std::istreambuf_iterator() }; - - if (received.length() != 8202) - return false; - if (received.substr(0, 8182) != std::string(8182, '\0')) - return false; - if (received.substr(8182, 20) != buf2) - return false; - - in.close(); - fs::remove_all(fs::path(filename).parent_path().string()); - - return true; -} - -bool test9() -{ - std::string filename = createTmpFile(); - - size_t n = 3 * DEFAULT_AIO_FILE_BLOCK_SIZE; - - std::string buf = generateString(n); - - std::string buf2(DEFAULT_AIO_FILE_BLOCK_SIZE + 10, '1'); - - { - DB::WriteBufferAIO out(filename, 2 * DEFAULT_AIO_FILE_BLOCK_SIZE); - - if (out.getFileName() != filename) - return false; - if (out.getFD() == -1) - return false; - - out.seek(3, SEEK_SET); - out.write(buf.data(), buf.length()); - out.seek(-DEFAULT_AIO_FILE_BLOCK_SIZE, SEEK_CUR); - out.write(buf2.data(), buf2.length()); - } - - std::ifstream in(filename.c_str()); - if (!in.is_open()) - die("Could not open file"); - - std::string received{ std::istreambuf_iterator(in), std::istreambuf_iterator() }; - - in.close(); - fs::remove_all(fs::path(filename).parent_path().string()); - - if (received.substr(3, 2 * DEFAULT_AIO_FILE_BLOCK_SIZE) != buf.substr(0, 2 * DEFAULT_AIO_FILE_BLOCK_SIZE)) - return false; - - if (received.substr(3 + 2 * DEFAULT_AIO_FILE_BLOCK_SIZE, DEFAULT_AIO_FILE_BLOCK_SIZE + 10) != buf2) - return false; - - return true; -} - -bool test10() -{ - std::string filename = createTmpFile(); - - size_t n = 10 * DEFAULT_AIO_FILE_BLOCK_SIZE + 3; - - std::string buf = generateString(n); - - { - DB::WriteBufferAIO out(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE); - - if (out.getFileName() != filename) - return false; - if (out.getFD() == -1) - return false; - - out.write(buf.data(), buf.length()); - } - - std::ifstream in(filename.c_str()); - if (!in.is_open()) - die("Could not open file"); - - std::string received{ std::istreambuf_iterator(in), std::istreambuf_iterator() }; - - in.close(); - fs::remove_all(fs::path(filename).parent_path().string()); - - return (received == buf); -} - -} - -int main() -{ - run(); - return 0; -} diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index 2a52d7d53f0..dd293bf2502 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -1,5 +1,4 @@ #include -#include #include #include diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 83ad57fa6ab..a735b939cd5 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -910,7 +910,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor if (metadata_snapshot->hasSecondaryIndices()) { const auto & indices = metadata_snapshot->getSecondaryIndices(); - merged_stream = std::make_shared(merged_stream, indices.getSingleExpressionForIndices(metadata_snapshot->getColumns(), data.global_context)); + merged_stream = std::make_shared( + merged_stream, indices.getSingleExpressionForIndices(metadata_snapshot->getColumns(), data.global_context)); merged_stream = std::make_shared(merged_stream); } @@ -921,7 +922,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor merging_columns, index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec, - data_settings->min_merge_bytes_to_use_direct_io, blocks_are_granules_size}; merged_stream->readPrefix(); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index d5e7009efd6..ef3b5eb7d24 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -24,9 +24,7 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( , plain_file(data_part->volume->getDisk()->writeFile( part_path + MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION, settings.max_compress_block_size, - WriteMode::Rewrite, - settings.estimated_size, - settings.aio_threshold)) + WriteMode::Rewrite)) , plain_hashing(*plain_file) , marks_file(data_part->volume->getDisk()->writeFile( part_path + MergeTreeDataPartCompact::DATA_FILE_NAME + marks_file_extension_, diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index 8e6ffe9ee68..fd3338c8a70 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -39,13 +39,11 @@ MergeTreeDataPartWriterOnDisk::Stream::Stream( const std::string & marks_path_, const std::string & marks_file_extension_, const CompressionCodecPtr & compression_codec_, - size_t max_compress_block_size_, - size_t estimated_size_, - size_t aio_threshold_) : + size_t max_compress_block_size_) : escaped_column_name(escaped_column_name_), data_file_extension{data_file_extension_}, marks_file_extension{marks_file_extension_}, - plain_file(disk_->writeFile(data_path_ + data_file_extension, max_compress_block_size_, WriteMode::Rewrite, estimated_size_, aio_threshold_)), + plain_file(disk_->writeFile(data_path_ + data_file_extension, max_compress_block_size_, WriteMode::Rewrite)), plain_hashing(*plain_file), compressed_buf(plain_hashing, compression_codec_), compressed(compressed_buf), marks_file(disk_->writeFile(marks_path_ + marks_file_extension, 4096, WriteMode::Rewrite)), marks(*marks_file) { @@ -164,8 +162,7 @@ void MergeTreeDataPartWriterOnDisk::initSkipIndices() data_part->volume->getDisk(), part_path + stream_name, INDEX_FILE_EXTENSION, part_path + stream_name, marks_file_extension, - default_codec, settings.max_compress_block_size, - 0, settings.aio_threshold)); + default_codec, settings.max_compress_block_size)); skip_indices_aggregators.push_back(index_helper->createIndexAggregator()); skip_index_accumulated_marks.push_back(0); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index a7b84c95e0a..704b38ba6d5 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -56,9 +56,7 @@ public: const std::string & marks_path_, const std::string & marks_file_extension_, const CompressionCodecPtr & compression_codec_, - size_t max_compress_block_size_, - size_t estimated_size_, - size_t aio_threshold_); + size_t max_compress_block_size_); String escaped_column_name; std::string data_file_extension; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index be735104e99..bb3b16d51e8 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -80,14 +80,13 @@ MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide( { const auto & columns = metadata_snapshot->getColumns(); for (const auto & it : columns_list) - addStreams(it.name, *it.type, columns.getCodecDescOrDefault(it.name, default_codec), settings.estimated_size); + addStreams(it.name, *it.type, columns.getCodecDescOrDefault(it.name, default_codec)); } void MergeTreeDataPartWriterWide::addStreams( const String & name, const IDataType & type, - const ASTPtr & effective_codec_desc, - size_t estimated_size) + const ASTPtr & effective_codec_desc) { IDataType::StreamCallback callback = [&] (const IDataType::SubstreamPath & substream_path, const IDataType & substream_type) { @@ -109,9 +108,7 @@ void MergeTreeDataPartWriterWide::addStreams( part_path + stream_name, DATA_FILE_EXTENSION, part_path + stream_name, marks_file_extension, compression_codec, - settings.max_compress_block_size, - estimated_size, - settings.aio_threshold); + settings.max_compress_block_size); }; IDataType::SubstreamPath stream_path; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h index 8c76c10abef..d897503a033 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h @@ -85,8 +85,7 @@ private: void addStreams( const String & name, const IDataType & type, - const ASTPtr & effective_codec_desc, - size_t estimated_size); + const ASTPtr & effective_codec_desc); /// Method for self check (used in debug-build only). Checks that written /// data and corresponding marks are consistent. Otherwise throws logical diff --git a/src/Storages/MergeTree/MergeTreeIOSettings.h b/src/Storages/MergeTree/MergeTreeIOSettings.h index 2dec16e7d10..9e315c08681 100644 --- a/src/Storages/MergeTree/MergeTreeIOSettings.h +++ b/src/Storages/MergeTree/MergeTreeIOSettings.h @@ -24,7 +24,6 @@ struct MergeTreeWriterSettings const Settings & global_settings, const MergeTreeSettingsPtr & storage_settings, bool can_use_adaptive_granularity_, - size_t aio_threshold_, bool rewrite_primary_key_, bool blocks_are_granules_size_ = false) : min_compress_block_size( @@ -32,7 +31,6 @@ struct MergeTreeWriterSettings , max_compress_block_size( storage_settings->max_compress_block_size ? storage_settings->max_compress_block_size : global_settings.max_compress_block_size) - , aio_threshold(aio_threshold_) , can_use_adaptive_granularity(can_use_adaptive_granularity_) , rewrite_primary_key(rewrite_primary_key_) , blocks_are_granules_size(blocks_are_granules_size_) @@ -41,14 +39,9 @@ struct MergeTreeWriterSettings size_t min_compress_block_size; size_t max_compress_block_size; - size_t aio_threshold; bool can_use_adaptive_granularity; bool rewrite_primary_key; bool blocks_are_granules_size; - - /// Used for AIO threshold comparison - /// FIXME currently doesn't work because WriteBufferAIO contain obscure bug(s) - size_t estimated_size = 0; }; } diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index feaf46194d1..1605ec693cb 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -21,25 +21,6 @@ MergedBlockOutputStream::MergedBlockOutputStream( const MergeTreeIndices & skip_indices, CompressionCodecPtr default_codec_, bool blocks_are_granules_size) - : MergedBlockOutputStream( - data_part, - metadata_snapshot_, - columns_list_, - skip_indices, - default_codec_, - data_part->storage.global_context.getSettings().min_bytes_to_use_direct_io, - blocks_are_granules_size) -{ -} - -MergedBlockOutputStream::MergedBlockOutputStream( - const MergeTreeDataPartPtr & data_part, - const StorageMetadataPtr & metadata_snapshot_, - const NamesAndTypesList & columns_list_, - const MergeTreeIndices & skip_indices, - CompressionCodecPtr default_codec_, - size_t aio_threshold, - bool blocks_are_granules_size) : IMergedBlockOutputStream(data_part, metadata_snapshot_) , columns_list(columns_list_) , default_codec(default_codec_) @@ -48,7 +29,6 @@ MergedBlockOutputStream::MergedBlockOutputStream( storage.global_context.getSettings(), storage.getSettings(), data_part->index_granularity_info.is_adaptive, - aio_threshold, /* rewrite_primary_key = */ true, blocks_are_granules_size); diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index 3db0e45d207..d04df598218 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -21,15 +21,6 @@ public: CompressionCodecPtr default_codec_, bool blocks_are_granules_size = false); - MergedBlockOutputStream( - const MergeTreeDataPartPtr & data_part, - const StorageMetadataPtr & metadata_snapshot_, - const NamesAndTypesList & columns_list_, - const MergeTreeIndices & skip_indices, - CompressionCodecPtr default_codec_, - size_t aio_threshold, - bool blocks_are_granules_size = false); - Block getHeader() const override { return metadata_snapshot->getSampleBlock(); } /// If the data is pre-sorted. From 3b11ad14da48a27c35fa2dac8176749c44bc771b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Jan 2021 03:33:45 +0300 Subject: [PATCH 235/256] Added issue template for fuzz reports --- .github/ISSUE_TEMPLATE/fuzzing-report.md | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) create mode 100644 .github/ISSUE_TEMPLATE/fuzzing-report.md diff --git a/.github/ISSUE_TEMPLATE/fuzzing-report.md b/.github/ISSUE_TEMPLATE/fuzzing-report.md new file mode 100644 index 00000000000..c27825ae006 --- /dev/null +++ b/.github/ISSUE_TEMPLATE/fuzzing-report.md @@ -0,0 +1,16 @@ +--- +name: Assertion found via fuzzing +about: Create a report to help us improve ClickHouse +title: '' +labels: fuzz +assignees: '' + +--- + +(you don't have to strictly follow this form) + +**Describe the bug** +A link to the report + +**How to reproduce** +Try to reproduce the report and copy the tables and queries involved. From 7dd9155a625e6f6a6c8c9893dca07c0837ea239c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Jan 2021 03:36:47 +0300 Subject: [PATCH 236/256] Reorder issue templates --- .github/ISSUE_TEMPLATE/{question.md => 10_question.md} | 0 .../ISSUE_TEMPLATE/{feature-request.md => 20_feature-request.md} | 0 .../{unexpected-behaviour.md => 30_unexpected-behaviour.md} | 0 .github/ISSUE_TEMPLATE/{bug-report.md => 40_bug-report.md} | 0 .../ISSUE_TEMPLATE/{usability-issue.md => 45_usability-issue.md} | 0 .github/ISSUE_TEMPLATE/{build-issue.md => 50_build-issue.md} | 0 .../{documentation-issue.md => 60_documentation-issue.md} | 0 .../{performance-issue.md => 70_performance-issue.md} | 0 .../{backward-compatibility.md => 80_backward-compatibility.md} | 0 .../ISSUE_TEMPLATE/{fuzzing-report.md => 90_fuzzing-report.md} | 0 10 files changed, 0 insertions(+), 0 deletions(-) rename .github/ISSUE_TEMPLATE/{question.md => 10_question.md} (100%) rename .github/ISSUE_TEMPLATE/{feature-request.md => 20_feature-request.md} (100%) rename .github/ISSUE_TEMPLATE/{unexpected-behaviour.md => 30_unexpected-behaviour.md} (100%) rename .github/ISSUE_TEMPLATE/{bug-report.md => 40_bug-report.md} (100%) rename .github/ISSUE_TEMPLATE/{usability-issue.md => 45_usability-issue.md} (100%) rename .github/ISSUE_TEMPLATE/{build-issue.md => 50_build-issue.md} (100%) rename .github/ISSUE_TEMPLATE/{documentation-issue.md => 60_documentation-issue.md} (100%) rename .github/ISSUE_TEMPLATE/{performance-issue.md => 70_performance-issue.md} (100%) rename .github/ISSUE_TEMPLATE/{backward-compatibility.md => 80_backward-compatibility.md} (100%) rename .github/ISSUE_TEMPLATE/{fuzzing-report.md => 90_fuzzing-report.md} (100%) diff --git a/.github/ISSUE_TEMPLATE/question.md b/.github/ISSUE_TEMPLATE/10_question.md similarity index 100% rename from .github/ISSUE_TEMPLATE/question.md rename to .github/ISSUE_TEMPLATE/10_question.md diff --git a/.github/ISSUE_TEMPLATE/feature-request.md b/.github/ISSUE_TEMPLATE/20_feature-request.md similarity index 100% rename from .github/ISSUE_TEMPLATE/feature-request.md rename to .github/ISSUE_TEMPLATE/20_feature-request.md diff --git a/.github/ISSUE_TEMPLATE/unexpected-behaviour.md b/.github/ISSUE_TEMPLATE/30_unexpected-behaviour.md similarity index 100% rename from .github/ISSUE_TEMPLATE/unexpected-behaviour.md rename to .github/ISSUE_TEMPLATE/30_unexpected-behaviour.md diff --git a/.github/ISSUE_TEMPLATE/bug-report.md b/.github/ISSUE_TEMPLATE/40_bug-report.md similarity index 100% rename from .github/ISSUE_TEMPLATE/bug-report.md rename to .github/ISSUE_TEMPLATE/40_bug-report.md diff --git a/.github/ISSUE_TEMPLATE/usability-issue.md b/.github/ISSUE_TEMPLATE/45_usability-issue.md similarity index 100% rename from .github/ISSUE_TEMPLATE/usability-issue.md rename to .github/ISSUE_TEMPLATE/45_usability-issue.md diff --git a/.github/ISSUE_TEMPLATE/build-issue.md b/.github/ISSUE_TEMPLATE/50_build-issue.md similarity index 100% rename from .github/ISSUE_TEMPLATE/build-issue.md rename to .github/ISSUE_TEMPLATE/50_build-issue.md diff --git a/.github/ISSUE_TEMPLATE/documentation-issue.md b/.github/ISSUE_TEMPLATE/60_documentation-issue.md similarity index 100% rename from .github/ISSUE_TEMPLATE/documentation-issue.md rename to .github/ISSUE_TEMPLATE/60_documentation-issue.md diff --git a/.github/ISSUE_TEMPLATE/performance-issue.md b/.github/ISSUE_TEMPLATE/70_performance-issue.md similarity index 100% rename from .github/ISSUE_TEMPLATE/performance-issue.md rename to .github/ISSUE_TEMPLATE/70_performance-issue.md diff --git a/.github/ISSUE_TEMPLATE/backward-compatibility.md b/.github/ISSUE_TEMPLATE/80_backward-compatibility.md similarity index 100% rename from .github/ISSUE_TEMPLATE/backward-compatibility.md rename to .github/ISSUE_TEMPLATE/80_backward-compatibility.md diff --git a/.github/ISSUE_TEMPLATE/fuzzing-report.md b/.github/ISSUE_TEMPLATE/90_fuzzing-report.md similarity index 100% rename from .github/ISSUE_TEMPLATE/fuzzing-report.md rename to .github/ISSUE_TEMPLATE/90_fuzzing-report.md From 1fa331183774d32ca693963c5267d9917646eb92 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Jan 2021 03:42:05 +0300 Subject: [PATCH 237/256] Reformulate some issue templates --- .github/ISSUE_TEMPLATE/10_question.md | 2 +- .github/ISSUE_TEMPLATE/30_unexpected-behaviour.md | 2 +- .github/ISSUE_TEMPLATE/45_usability-issue.md | 2 +- .github/ISSUE_TEMPLATE/80_backward-compatibility.md | 2 +- .github/ISSUE_TEMPLATE/90_fuzzing-report.md | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/.github/ISSUE_TEMPLATE/10_question.md b/.github/ISSUE_TEMPLATE/10_question.md index a5015de8217..6e23fbdc605 100644 --- a/.github/ISSUE_TEMPLATE/10_question.md +++ b/.github/ISSUE_TEMPLATE/10_question.md @@ -1,6 +1,6 @@ --- name: Question -about: Ask question about ClickHouse +about: Ask a question about ClickHouse title: '' labels: question assignees: '' diff --git a/.github/ISSUE_TEMPLATE/30_unexpected-behaviour.md b/.github/ISSUE_TEMPLATE/30_unexpected-behaviour.md index 27ab217ca33..3630d95ba33 100644 --- a/.github/ISSUE_TEMPLATE/30_unexpected-behaviour.md +++ b/.github/ISSUE_TEMPLATE/30_unexpected-behaviour.md @@ -1,6 +1,6 @@ --- name: Unexpected behaviour -about: Create a report to help us improve ClickHouse +about: Some feature is working in non-obvious way title: '' labels: unexpected behaviour assignees: '' diff --git a/.github/ISSUE_TEMPLATE/45_usability-issue.md b/.github/ISSUE_TEMPLATE/45_usability-issue.md index 6a084a72619..b03b11606c1 100644 --- a/.github/ISSUE_TEMPLATE/45_usability-issue.md +++ b/.github/ISSUE_TEMPLATE/45_usability-issue.md @@ -1,6 +1,6 @@ --- name: Usability issue -about: Create a report to help us improve ClickHouse +about: Report something can be made more convenient to use title: '' labels: usability assignees: '' diff --git a/.github/ISSUE_TEMPLATE/80_backward-compatibility.md b/.github/ISSUE_TEMPLATE/80_backward-compatibility.md index 8f87197e73d..a13e9508f70 100644 --- a/.github/ISSUE_TEMPLATE/80_backward-compatibility.md +++ b/.github/ISSUE_TEMPLATE/80_backward-compatibility.md @@ -1,6 +1,6 @@ --- name: Backward compatibility issue -about: Create a report to help us improve ClickHouse +about: Report the case when the behaviour of a new version can break existing use cases title: '' labels: backward compatibility assignees: '' diff --git a/.github/ISSUE_TEMPLATE/90_fuzzing-report.md b/.github/ISSUE_TEMPLATE/90_fuzzing-report.md index c27825ae006..1d9a8a75d28 100644 --- a/.github/ISSUE_TEMPLATE/90_fuzzing-report.md +++ b/.github/ISSUE_TEMPLATE/90_fuzzing-report.md @@ -1,6 +1,6 @@ --- name: Assertion found via fuzzing -about: Create a report to help us improve ClickHouse +about: Potential issue has been found via Fuzzer or Stress tests title: '' labels: fuzz assignees: '' From 8387ac86ef64de634f172c5ae057e2d05e7d210c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Jan 2021 03:44:10 +0300 Subject: [PATCH 238/256] Add another issue template --- .../35_incomplete_implementation.md | 30 +++++++++++++++++++ 1 file changed, 30 insertions(+) create mode 100644 .github/ISSUE_TEMPLATE/35_incomplete_implementation.md diff --git a/.github/ISSUE_TEMPLATE/35_incomplete_implementation.md b/.github/ISSUE_TEMPLATE/35_incomplete_implementation.md new file mode 100644 index 00000000000..6a014ce3c29 --- /dev/null +++ b/.github/ISSUE_TEMPLATE/35_incomplete_implementation.md @@ -0,0 +1,30 @@ +--- +name: Incomplete implementation +about: Implementation of existing feature is not finished +title: '' +labels: unfinished code +assignees: '' + +--- + +(you don't have to strictly follow this form) + +**Describe the unexpected behaviour** +A clear and concise description of what works not as it is supposed to. + +**How to reproduce** +* Which ClickHouse server version to use +* Which interface to use, if matters +* Non-default settings, if any +* `CREATE TABLE` statements for all tables involved +* Sample data for all these tables, use [clickhouse-obfuscator](https://github.com/ClickHouse/ClickHouse/blob/master/programs/obfuscator/Obfuscator.cpp#L42-L80) if necessary +* Queries to run that lead to unexpected result + +**Expected behavior** +A clear and concise description of what you expected to happen. + +**Error message and/or stacktrace** +If applicable, add screenshots to help explain your problem. + +**Additional context** +Add any other context about the problem here. From 7e21d4d70a1ea00eab230760a8206cfea2c981ce Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 10 Jan 2021 03:46:35 +0300 Subject: [PATCH 239/256] Update ccache.cmake --- cmake/find/ccache.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/find/ccache.cmake b/cmake/find/ccache.cmake index 90f9fcdd568..d8e9cf9588d 100644 --- a/cmake/find/ccache.cmake +++ b/cmake/find/ccache.cmake @@ -35,7 +35,7 @@ if (CCACHE_FOUND AND NOT COMPILER_MATCHES_CCACHE) # debian (debhlpers) set SOURCE_DATE_EPOCH environment variable, that is # filled from the debian/changelog or current time. # - # - 4.0+ ccache always includes this environemtn variable into the hash + # - 4.0+ ccache always includes this environment variable into the hash # of the manifest, which do not allow to use previous cache, # - 4.2+ ccache ignores SOURCE_DATE_EPOCH under time_macros sloppiness. # From 0421cc8dc2dcadf01e01e3f5a0fef21067912c6b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Jan 2021 03:49:29 +0300 Subject: [PATCH 240/256] Remove TestFlows due to timeouts --- tests/testflows/regression.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/testflows/regression.py b/tests/testflows/regression.py index 0e9a821cae0..ef2008aa173 100755 --- a/tests/testflows/regression.py +++ b/tests/testflows/regression.py @@ -14,10 +14,10 @@ def regression(self, local, clickhouse_binary_path, stress=None, parallel=None): """ args = {"local": local, "clickhouse_binary_path": clickhouse_binary_path, "stress": stress, "parallel": parallel} - Feature(test=load("example.regression", "regression"))(**args) - Feature(test=load("ldap.regression", "regression"))(**args) - Feature(test=load("rbac.regression", "regression"))(**args) - Feature(test=load("aes_encryption.regression", "regression"))(**args) +# Feature(test=load("example.regression", "regression"))(**args) +# Feature(test=load("ldap.regression", "regression"))(**args) +# Feature(test=load("rbac.regression", "regression"))(**args) +# Feature(test=load("aes_encryption.regression", "regression"))(**args) if main(): regression() From 76149947ef755878c0369ac1e93ca47b8a83d8d1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Jan 2021 05:48:57 +0300 Subject: [PATCH 241/256] Remove useless headers --- src/Databases/DatabaseAtomic.h | 3 +-- src/Databases/MySQL/MaterializeMySQLSyncThread.h | 1 - src/Interpreters/DNSCacheUpdater.cpp | 2 +- src/Storages/MergeTree/ReplicatedMergeTreeQueue.h | 1 - src/Storages/StorageMergeTree.h | 1 - 5 files changed, 2 insertions(+), 6 deletions(-) diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index 1b1c0cd4353..006d0e11434 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -1,10 +1,9 @@ #pragma once #include -#include - #include + namespace DB { diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.h b/src/Databases/MySQL/MaterializeMySQLSyncThread.h index 54f148026ad..26934b87511 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.h +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.h @@ -7,7 +7,6 @@ #if USE_MYSQL # include -# include # include # include # include diff --git a/src/Interpreters/DNSCacheUpdater.cpp b/src/Interpreters/DNSCacheUpdater.cpp index 248c0ffa4dd..fb0298f480f 100644 --- a/src/Interpreters/DNSCacheUpdater.cpp +++ b/src/Interpreters/DNSCacheUpdater.cpp @@ -1,7 +1,7 @@ #include "DNSCacheUpdater.h" #include #include -#include + namespace DB { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 6d5fab744a5..00ef3ee7292 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -12,7 +12,6 @@ #include #include -#include namespace DB diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 3263f124afa..9dd62439814 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -14,7 +14,6 @@ #include #include #include -#include #include From c38dca155c815dce26ff13d18cf5675addb4beef Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Jan 2021 05:51:54 +0300 Subject: [PATCH 242/256] Fix clang-tidy --- src/IO/ya.make | 2 -- src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp | 1 - 2 files changed, 3 deletions(-) diff --git a/src/IO/ya.make b/src/IO/ya.make index 4dc3afb2f11..2ef8bd0a986 100644 --- a/src/IO/ya.make +++ b/src/IO/ya.make @@ -51,7 +51,6 @@ SRCS( ReadHelpers.cpp SeekAvoidingReadBuffer.cpp UseSSL.cpp - WriteBufferAIO.cpp WriteBufferFromFile.cpp WriteBufferFromFileBase.cpp WriteBufferFromFileDescriptor.cpp @@ -69,7 +68,6 @@ SRCS( ZstdInflatingReadBuffer.cpp copyData.cpp createReadBufferFromFileBase.cpp - createWriteBufferFromFileBase.cpp parseDateTimeBestEffort.cpp readFloatText.cpp diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 87f67ee92a5..41479f104f3 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -28,7 +28,6 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( global_settings, storage_settings, index_granularity_info ? index_granularity_info->is_adaptive : data_part->storage.canUseAdaptiveGranularity(), - global_settings.min_bytes_to_use_direct_io, /* rewrite_primary_key = */false); writer = data_part->getWriter( From 833054251b1a4dc53df6dfc030ac5eb4234d852b Mon Sep 17 00:00:00 2001 From: HuFuwang Date: Sun, 10 Jan 2021 11:31:44 +0800 Subject: [PATCH 243/256] update sorted logic in AggregateFunctionWindowFunnelData::add --- src/AggregateFunctions/AggregateFunctionWindowFunnel.h | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionWindowFunnel.h b/src/AggregateFunctions/AggregateFunctionWindowFunnel.h index a59e1c8b26a..2be9d874a05 100644 --- a/src/AggregateFunctions/AggregateFunctionWindowFunnel.h +++ b/src/AggregateFunctions/AggregateFunctionWindowFunnel.h @@ -47,8 +47,13 @@ struct AggregateFunctionWindowFunnelData void add(T timestamp, UInt8 event) { // Since most events should have already been sorted by timestamp. - if (sorted && events_list.size() > 0 && events_list.back().first > timestamp) - sorted = false; + if (sorted && events_list.size() > 0) + { + if (events_list.back().first == timestamp) + sorted = events_list.back().second <= event; + else + sorted = events_list.back().first <= timestamp; + } events_list.emplace_back(timestamp, event); } From 79a38c4c6ea5254f9e53a5cdd81600bd269b7d6b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Jan 2021 06:35:05 +0300 Subject: [PATCH 244/256] Fix hang at shutdown in clickhouse-local --- programs/local/LocalServer.cpp | 7 ++++--- .../0_stateless/01647_clickhouse_local_hung.reference | 1 + .../queries/0_stateless/01647_clickhouse_local_hung.sh | 10 ++++++++++ 3 files changed, 15 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/01647_clickhouse_local_hung.reference create mode 100755 tests/queries/0_stateless/01647_clickhouse_local_hung.sh diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index dbf153eeb81..1fee37985a7 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -273,11 +273,12 @@ try global_context->setCurrentDatabase(default_database); applyCmdOptions(*global_context); - String path = global_context->getPath(); - if (!path.empty()) + if (config().has("path")) { + String path = global_context->getPath(); + /// Lock path directory before read - status.emplace(global_context->getPath() + "status", StatusFile::write_full_info); + status.emplace(path + "status", StatusFile::write_full_info); LOG_DEBUG(log, "Loading metadata from {}", path); Poco::File(path + "data/").createDirectories(); diff --git a/tests/queries/0_stateless/01647_clickhouse_local_hung.reference b/tests/queries/0_stateless/01647_clickhouse_local_hung.reference new file mode 100644 index 00000000000..8925955b88e --- /dev/null +++ b/tests/queries/0_stateless/01647_clickhouse_local_hung.reference @@ -0,0 +1 @@ + 100 14 diff --git a/tests/queries/0_stateless/01647_clickhouse_local_hung.sh b/tests/queries/0_stateless/01647_clickhouse_local_hung.sh new file mode 100755 index 00000000000..04f32055ab6 --- /dev/null +++ b/tests/queries/0_stateless/01647_clickhouse_local_hung.sh @@ -0,0 +1,10 @@ +#!/usr/bin/env bash + +set -e + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +for _ in {1..100}; do echo 'Hello, world!' | ${CLICKHOUSE_LOCAL} --query "SELECT * FROM table" --structure 's String' | wc -c; done | uniq -c From 44758935dfcd018e704ad51a7b1793b8321c2def Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 10 Jan 2021 17:40:47 +0800 Subject: [PATCH 245/256] correct index analysis of WITH aliases --- src/Parsers/ASTWithAlias.cpp | 5 +++++ src/Parsers/ASTWithAlias.h | 1 + src/Parsers/IAST.cpp | 8 ++++++++ src/Parsers/IAST.h | 7 +++++++ src/Storages/MergeTree/KeyCondition.cpp | 8 ++++---- .../01649_with_alias_key_condition.reference | 1 + .../0_stateless/01649_with_alias_key_condition.sql | 11 +++++++++++ 7 files changed, 37 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/01649_with_alias_key_condition.reference create mode 100644 tests/queries/0_stateless/01649_with_alias_key_condition.sql diff --git a/src/Parsers/ASTWithAlias.cpp b/src/Parsers/ASTWithAlias.cpp index 20f647fb575..88f6568a719 100644 --- a/src/Parsers/ASTWithAlias.cpp +++ b/src/Parsers/ASTWithAlias.cpp @@ -48,4 +48,9 @@ void ASTWithAlias::appendColumnName(WriteBuffer & ostr) const appendColumnNameImpl(ostr); } +void ASTWithAlias::appendColumnNameWithoutAlias(WriteBuffer & ostr) const +{ + appendColumnNameImpl(ostr); +} + } diff --git a/src/Parsers/ASTWithAlias.h b/src/Parsers/ASTWithAlias.h index 7a272a157e2..ea4419402b0 100644 --- a/src/Parsers/ASTWithAlias.h +++ b/src/Parsers/ASTWithAlias.h @@ -21,6 +21,7 @@ public: using IAST::IAST; void appendColumnName(WriteBuffer & ostr) const final; + void appendColumnNameWithoutAlias(WriteBuffer & ostr) const final; String getAliasOrColumnName() const override { return alias.empty() ? getColumnName() : alias; } String tryGetAlias() const override { return alias; } void setAlias(const String & to) override { alias = to; } diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index e223235b8e4..6451a9d0c0d 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -109,6 +109,14 @@ String IAST::getColumnName() const } +String IAST::getColumnNameWithoutAlias() const +{ + WriteBufferFromOwnString write_buffer; + appendColumnNameWithoutAlias(write_buffer); + return write_buffer.str(); +} + + void IAST::FormatSettings::writeIdentifier(const String & name) const { switch (identifier_quoting_style) diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index bed6c5bcdf9..54e08b2700e 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -41,11 +41,18 @@ public: /** Get the canonical name of the column if the element is a column */ String getColumnName() const; + /** Same as the above but ensure no alias names are used. This is for index analysis */ + String getColumnNameWithoutAlias() const; virtual void appendColumnName(WriteBuffer &) const { throw Exception("Trying to get name of not a column: " + getID(), ErrorCodes::LOGICAL_ERROR); } + virtual void appendColumnNameWithoutAlias(WriteBuffer &) const + { + throw Exception("Trying to get name of not a column: " + getID(), ErrorCodes::LOGICAL_ERROR); + } + /** Get the alias, if any, or the canonical name of the column, if it is not. */ virtual String getAliasOrColumnName() const { return getColumnName(); } diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index ead93a6e1d9..eb58b2911c9 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -444,7 +444,7 @@ bool KeyCondition::addCondition(const String & column, const Range & range) */ bool KeyCondition::getConstant(const ASTPtr & expr, Block & block_with_constants, Field & out_value, DataTypePtr & out_type) { - String column_name = expr->getColumnName(); + String column_name = expr->getColumnNameWithoutAlias(); if (const auto * lit = expr->as()) { @@ -607,7 +607,7 @@ bool KeyCondition::canConstantBeWrappedByMonotonicFunctions( if (strict) return false; - String expr_name = node->getColumnName(); + String expr_name = node->getColumnNameWithoutAlias(); const auto & sample_block = key_expr->getSampleBlock(); if (!sample_block.has(expr_name)) return false; @@ -675,7 +675,7 @@ bool KeyCondition::canConstantBeWrappedByFunctions( if (strict) return false; - String expr_name = ast->getColumnName(); + String expr_name = ast->getColumnNameWithoutAlias(); const auto & sample_block = key_expr->getSampleBlock(); if (!sample_block.has(expr_name)) return false; @@ -934,7 +934,7 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl( * Therefore, use the full name of the expression for search. */ const auto & sample_block = key_expr->getSampleBlock(); - String name = node->getColumnName(); + String name = node->getColumnNameWithoutAlias(); auto it = key_columns.find(name); if (key_columns.end() != it) diff --git a/tests/queries/0_stateless/01649_with_alias_key_condition.reference b/tests/queries/0_stateless/01649_with_alias_key_condition.reference new file mode 100644 index 00000000000..5816b4eb49b --- /dev/null +++ b/tests/queries/0_stateless/01649_with_alias_key_condition.reference @@ -0,0 +1 @@ +3 4 diff --git a/tests/queries/0_stateless/01649_with_alias_key_condition.sql b/tests/queries/0_stateless/01649_with_alias_key_condition.sql new file mode 100644 index 00000000000..b813e6ee84f --- /dev/null +++ b/tests/queries/0_stateless/01649_with_alias_key_condition.sql @@ -0,0 +1,11 @@ +drop table if exists alias_key_condition; + +create table alias_key_condition ( i int, j int ) engine MergeTree order by i; + +insert into alias_key_condition values (1, 2), (3, 4); + +set force_primary_key = 1; + +with i as k select * from alias_key_condition where k = 3; + +drop table if exists alias_key_condition; From 24ded3c1c1bfc950a58c5b1c7385388f4c6f01c8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Jan 2021 14:12:02 +0300 Subject: [PATCH 246/256] Allow clickhouse-local to run without system tables --- programs/local/LocalServer.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 1fee37985a7..5a8d35e204d 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -289,7 +289,7 @@ try DatabaseCatalog::instance().loadDatabases(); LOG_DEBUG(log, "Loaded metadata."); } - else + else if (!config().has("no-system-tables")) { attachSystemTables(*global_context); } @@ -541,6 +541,7 @@ void LocalServer::init(int argc, char ** argv) ("logger.log", po::value(), "Log file name") ("logger.level", po::value(), "Log level") ("ignore-error", "do not stop processing if a query failed") + ("no-system-tables", "do not attach system tables (better startup time)") ("version,V", "print version information and exit") ; @@ -603,6 +604,8 @@ void LocalServer::init(int argc, char ** argv) config().setString("logger.level", options["logger.level"].as()); if (options.count("ignore-error")) config().setBool("ignore-error", true); + if (options.count("no-system-tables")) + config().setBool("no-system-tables", true); std::vector arguments; for (int arg_num = 1; arg_num < argc; ++arg_num) From 33cf7351b064f55c8702c9ca98a1eaadde7b9044 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Sun, 10 Jan 2021 23:21:32 +0300 Subject: [PATCH 247/256] DOCSUP-5169: Document the min_compress_block_size and max_compress_block_size settings (#18248) Co-authored-by: Alexander Kazakov --- .../mergetree-family/mergetree.md | 4 +++- docs/en/operations/settings/settings.md | 14 ++++++++----- .../mergetree-family/mergetree.md | 20 ++++++++++--------- docs/ru/operations/settings/settings.md | 16 +++++++++------ 4 files changed, 33 insertions(+), 21 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 4f81a679b8e..80769fe9954 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -98,7 +98,9 @@ For a description of parameters, see the [CREATE query description](../../../sql - `merge_max_block_size` — Maximum number of rows in block for merge operations. Default value: 8192. - `storage_policy` — Storage policy. See [Using Multiple Block Devices for Data Storage](#table_engine-mergetree-multiple-volumes). - `min_bytes_for_wide_part`, `min_rows_for_wide_part` — Minimum number of bytes/rows in a data part that can be stored in `Wide` format. You can set one, both or none of these settings. See [Data Storage](#mergetree-data-storage). - - `max_parts_in_total` — Maximum number of parts in all partitions. + - `max_parts_in_total` — Maximum number of parts in all partitions. + - `max_compress_block_size` — Maximum size of blocks of uncompressed data before compressing for writing to a table. You can also specify this setting in the global settings (see [max_compress_block_size](../../../operations/settings/settings.md#max-compress-block-size) setting). The value specified when table is created overrides the global value for this setting. + - `min_compress_block_size` — Minimum size of blocks of uncompressed data required for compression when writing the next mark. You can also specify this setting in the global settings (see [min_compress_block_size](../../../operations/settings/settings.md#min-compress-block-size) setting). The value specified when table is created overrides the global value for this setting. **Example of Sections Setting** diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index e5f836e3b5a..82537ba637a 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -844,23 +844,27 @@ Higher values will lead to higher memory usage. ## max_compress_block_size {#max-compress-block-size} -The maximum size of blocks of uncompressed data before compressing for writing to a table. By default, 1,048,576 (1 MiB). If the size is reduced, the compression rate is significantly reduced, the compression and decompression speed increases slightly due to cache locality, and memory consumption is reduced. There usually isn’t any reason to change this setting. +The maximum size of blocks of uncompressed data before compressing for writing to a table. By default, 1,048,576 (1 MiB). Specifying smaller block size generally leads to slightly reduced compression ratio, the compression and decompression speed increases slightly due to cache locality, and memory consumption is reduced. + +!!! note "Warning" + This is an expert-level setting, and you shouldn't change it if you're just getting started with Clickhouse. Don’t confuse blocks for compression (a chunk of memory consisting of bytes) with blocks for query processing (a set of rows from a table). ## min_compress_block_size {#min-compress-block-size} -For [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md)" tables. In order to reduce latency when processing queries, a block is compressed when writing the next mark if its size is at least ‘min_compress_block_size’. By default, 65,536. +For [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) tables. In order to reduce latency when processing queries, a block is compressed when writing the next mark if its size is at least `min_compress_block_size`. By default, 65,536. -The actual size of the block, if the uncompressed data is less than ‘max_compress_block_size’, is no less than this value and no less than the volume of data for one mark. +The actual size of the block, if the uncompressed data is less than `max_compress_block_size`, is no less than this value and no less than the volume of data for one mark. -Let’s look at an example. Assume that ‘index_granularity’ was set to 8192 during table creation. +Let’s look at an example. Assume that `index_granularity` was set to 8192 during table creation. We are writing a UInt32-type column (4 bytes per value). When writing 8192 rows, the total will be 32 KB of data. Since min_compress_block_size = 65,536, a compressed block will be formed for every two marks. We are writing a URL column with the String type (average size of 60 bytes per value). When writing 8192 rows, the average will be slightly less than 500 KB of data. Since this is more than 65,536, a compressed block will be formed for each mark. In this case, when reading data from the disk in the range of a single mark, extra data won’t be decompressed. -There usually isn’t any reason to change this setting. +!!! note "Warning" + This is an expert-level setting, and you shouldn't change it if you're just getting started with Clickhouse. ## max_query_size {#settings-max_query_size} diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 501f7732745..c7bd64c4ab1 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -77,17 +77,19 @@ ORDER BY expr - `SETTINGS` — дополнительные параметры, регулирующие поведение `MergeTree` (необязательные): - - `index_granularity` — максимальное количество строк данных между засечками индекса. По умолчанию — 8192. Смотрите [Хранение данных](#mergetree-data-storage). - - `index_granularity_bytes` — максимальный размер гранул данных в байтах. По умолчанию — 10Mb. Чтобы ограничить размер гранул только количеством строк, установите значение 0 (не рекомендовано). Смотрите [Хранение данных](#mergetree-data-storage). + - `index_granularity` — максимальное количество строк данных между засечками индекса. По умолчанию — 8192. Смотрите [Хранение данных](#mergetree-data-storage). + - `index_granularity_bytes` — максимальный размер гранул данных в байтах. По умолчанию — 10Mb. Чтобы ограничить размер гранул только количеством строк, установите значение 0 (не рекомендовано). Смотрите [Хранение данных](#mergetree-data-storage). - `min_index_granularity_bytes` — минимально допустимый размер гранул данных в байтах. Значение по умолчанию — 1024b. Для обеспечения защиты от случайного создания таблиц с очень низким значением `index_granularity_bytes`. Смотрите [Хранение данных](#mergetree-data-storage). - - `enable_mixed_granularity_parts` — включает или выключает переход к ограничению размера гранул с помощью настройки `index_granularity_bytes`. Настройка `index_granularity_bytes` улучшает производительность ClickHouse при выборке данных из таблиц с большими (десятки и сотни мегабайтов) строками. Если у вас есть таблицы с большими строками, можно включить эту настройку, чтобы повысить эффективность запросов `SELECT`. - - `use_minimalistic_part_header_in_zookeeper` — Способ хранения заголовков кусков данных в ZooKeeper. Если `use_minimalistic_part_header_in_zookeeper = 1`, то ZooKeeper хранит меньше данных. Подробнее читайте в [описании настройки](../../../operations/server-configuration-parameters/settings.md#server-settings-use_minimalistic_part_header_in_zookeeper) в разделе "Конфигурационные параметры сервера". - - `min_merge_bytes_to_use_direct_io` — минимальный объём данных при слиянии, необходимый для прямого (небуферизованного) чтения/записи (direct I/O) на диск. При слиянии частей данных ClickHouse вычисляет общий объём хранения всех данных, подлежащих слиянию. Если общий объём хранения всех данных для чтения превышает `min_bytes_to_use_direct_io` байт, тогда ClickHouse использует флаг `O_DIRECT` при чтении данных с диска. Если `min_merge_bytes_to_use_direct_io = 0`, тогда прямой ввод-вывод отключен. Значение по умолчанию: `10 * 1024 * 1024 * 1024` байтов. - - `merge_with_ttl_timeout` — минимальное время в секундах перед повторным слиянием с TTL. По умолчанию — 86400 (1 день). - - `write_final_mark` — включает или отключает запись последней засечки индекса в конце куска данных, указывающей за последний байт. По умолчанию — 1. Не отключайте её. - - `merge_max_block_size` — максимальное количество строк в блоке для операций слияния. Значение по умолчанию: 8192. - - `storage_policy` — политика хранения данных. Смотрите [Хранение данных таблицы на нескольких блочных устройствах](#table_engine-mergetree-multiple-volumes). + - `enable_mixed_granularity_parts` — включает или выключает переход к ограничению размера гранул с помощью настройки `index_granularity_bytes`. Настройка `index_granularity_bytes` улучшает производительность ClickHouse при выборке данных из таблиц с большими (десятки и сотни мегабайтов) строками. Если у вас есть таблицы с большими строками, можно включить эту настройку, чтобы повысить эффективность запросов `SELECT`. + - `use_minimalistic_part_header_in_zookeeper` — Способ хранения заголовков кусков данных в ZooKeeper. Если `use_minimalistic_part_header_in_zookeeper = 1`, то ZooKeeper хранит меньше данных. Подробнее читайте в [описании настройки](../../../operations/server-configuration-parameters/settings.md#server-settings-use_minimalistic_part_header_in_zookeeper) в разделе "Конфигурационные параметры сервера". + - `min_merge_bytes_to_use_direct_io` — минимальный объём данных при слиянии, необходимый для прямого (небуферизованного) чтения/записи (direct I/O) на диск. При слиянии частей данных ClickHouse вычисляет общий объём хранения всех данных, подлежащих слиянию. Если общий объём хранения всех данных для чтения превышает `min_bytes_to_use_direct_io` байт, тогда ClickHouse использует флаг `O_DIRECT` при чтении данных с диска. Если `min_merge_bytes_to_use_direct_io = 0`, тогда прямой ввод-вывод отключен. Значение по умолчанию: `10 * 1024 * 1024 * 1024` байтов. + - `merge_with_ttl_timeout` — минимальное время в секундах перед повторным слиянием с TTL. По умолчанию — 86400 (1 день). + - `write_final_mark` — включает или отключает запись последней засечки индекса в конце куска данных, указывающей за последний байт. По умолчанию — 1. Не отключайте её. + - `merge_max_block_size` — максимальное количество строк в блоке для операций слияния. Значение по умолчанию: 8192. + - `storage_policy` — политика хранения данных. Смотрите [Хранение данных таблицы на нескольких блочных устройствах](#table_engine-mergetree-multiple-volumes). - `min_bytes_for_wide_part`, `min_rows_for_wide_part` — минимальное количество байт/строк в куске данных для хранения в формате `Wide`. Можно задать одну или обе настройки или не задавать ни одной. Подробнее см. в разделе [Хранение данных](#mergetree-data-storage). + - `max_compress_block_size` — максимальный размер блоков несжатых данных перед сжатием для записи в таблицу. Вы также можете задать этот параметр в глобальных настройках (смотрите [max_compress_block_size](../../../operations/settings/settings.md#max-compress-block-size)). Настройка, которая задается при создании таблицы, имеет более высокий приоритет, чем глобальная. + - `min_compress_block_size` — минимальный размер блоков несжатых данных, необходимых для сжатия при записи следующей засечки. Вы также можете задать этот параметр в глобальных настройках (смотрите [min_compress_block_size](../../../operations/settings/settings.md#min-compress-block-size)). Настройка, которая задается при создании таблицы, имеет более высокий приоритет, чем глобальная. **Пример задания секций** diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index c4f5cdaf2ca..a3bf13bfed3 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -811,23 +811,27 @@ log_query_threads=1 ## max_compress_block_size {#max-compress-block-size} -Максимальный размер блоков не сжатых данных перед сжатием при записи в таблицу. По умолчанию - 1 048 576 (1 MiB). При уменьшении размера, незначительно уменьшается коэффициент сжатия, незначительно возрастает скорость сжатия и разжатия за счёт кэш-локальности, и уменьшается потребление оперативки. Как правило, не имеет смысла менять эту настройку. +Максимальный размер блоков несжатых данных перед сжатием при записи в таблицу. По умолчанию - 1 048 576 (1 MiB). При уменьшении размера, незначительно уменьшается коэффициент сжатия, незначительно возрастает скорость сжатия и разжатия за счёт кэш-локальности, и уменьшается потребление оперативной памяти. + +!!! note "Предупреждение" + Эта настройка экспертного уровня, не используйте ее, если вы только начинаете работать с Clickhouse. Не путайте блоки для сжатия (кусок памяти, состоящий из байт) и блоки для обработки запроса (пачка строк из таблицы). ## min_compress_block_size {#min-compress-block-size} -Для таблиц типа [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). В целях уменьшения задержек при обработке запросов, блок сжимается при записи следующей засечки, если его размер не меньше min_compress_block_size. По умолчанию - 65 536. +Для таблиц типа [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). В целях уменьшения задержек при обработке запросов, блок сжимается при записи следующей засечки, если его размер не меньше `min_compress_block_size`. По умолчанию - 65 536. -Реальный размер блока, если несжатых данных меньше max_compress_block_size, будет не меньше этого значения и не меньше объёма данных на одну засечку. +Реальный размер блока, если несжатых данных меньше `max_compress_block_size`, будет не меньше этого значения и не меньше объёма данных на одну засечку. -Рассмотрим пример. Пусть index_granularity, указанная при создании таблицы - 8192. +Рассмотрим пример. Пусть `index_granularity`, указанная при создании таблицы - 8192. -Пусть мы записываем столбец типа UInt32 (4 байта на значение). При записи 8192 строк, будет всего 32 КБ данных. Так как min_compress_block_size = 65 536, сжатый блок будет сформирован на каждые две засечки. +Пусть мы записываем столбец типа UInt32 (4 байта на значение). При записи 8192 строк, будет всего 32 КБ данных. Так как `min_compress_block_size` = 65 536, сжатый блок будет сформирован на каждые две засечки. Пусть мы записываем столбец URL типа String (средний размер - 60 байт на значение). При записи 8192 строк, будет, в среднем, чуть меньше 500 КБ данных. Так как это больше 65 536 строк, то сжатый блок будет сформирован на каждую засечку. В этом случае, при чтении с диска данных из диапазона в одну засечку, не будет разжато лишних данных. -Как правило, не имеет смысла менять эту настройку. +!!! note "Предупреждение" + Эта настройка экспертного уровня, не используйте ее, если вы только начинаете работать с Clickhouse. ## max_query_size {#settings-max_query_size} From c7d4d12f72bff2929877e918b98fea30783bd03e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Jan 2021 03:51:08 +0300 Subject: [PATCH 248/256] Auto version update to [21.1.0] [54445] --- cmake/autogenerated_versions.txt | 14 +++--- debian/changelog | 4 +- docker/client/Dockerfile | 2 +- docker/server/Dockerfile | 2 +- docker/test/Dockerfile | 2 +- .../StorageSystemContributors.generated.cpp | 49 ++++++++++++++++++- 6 files changed, 60 insertions(+), 13 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index 87a30c9effc..dabac0cf317 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -1,9 +1,9 @@ # This strings autochanged from release_lib.sh: -SET(VERSION_REVISION 54444) -SET(VERSION_MAJOR 20) -SET(VERSION_MINOR 13) -SET(VERSION_PATCH 1) -SET(VERSION_GITHASH e581f9ccfc5c64867b0f488cce72412fd2966471) -SET(VERSION_DESCRIBE v20.13.1.1-prestable) -SET(VERSION_STRING 20.13.1.1) +SET(VERSION_REVISION 54445) +SET(VERSION_MAJOR 21) +SET(VERSION_MINOR 1) +SET(VERSION_PATCH 0) +SET(VERSION_GITHASH c2a3c6391437db2a05fe815a9d6faf81464d39ec) +SET(VERSION_DESCRIBE v21.1.0-testing) +SET(VERSION_STRING 21.1.0) # end of autochange diff --git a/debian/changelog b/debian/changelog index 5ea6b472e46..3a267a83c69 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,5 +1,5 @@ -clickhouse (20.13.1.1) unstable; urgency=low +clickhouse (21.1.0) unstable; urgency=low * Modified source code - -- clickhouse-release Mon, 23 Nov 2020 10:29:24 +0300 + -- Alexey Milovidov Mon, 11 Jan 2021 03:51:08 +0300 diff --git a/docker/client/Dockerfile b/docker/client/Dockerfile index 3ef6b8c8b32..ddfe3cd177b 100644 --- a/docker/client/Dockerfile +++ b/docker/client/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/" -ARG version=20.13.1.* +ARG version=21.1.0 RUN apt-get update \ && apt-get install --yes --no-install-recommends \ diff --git a/docker/server/Dockerfile b/docker/server/Dockerfile index f7e107a2fc9..890aa35fe92 100644 --- a/docker/server/Dockerfile +++ b/docker/server/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:20.04 ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/" -ARG version=20.13.1.* +ARG version=21.1.0 ARG gosu_ver=1.10 RUN apt-get update \ diff --git a/docker/test/Dockerfile b/docker/test/Dockerfile index 8e3b5193874..2e17151b31f 100644 --- a/docker/test/Dockerfile +++ b/docker/test/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/" -ARG version=20.13.1.* +ARG version=21.1.0 RUN apt-get update && \ apt-get install -y apt-transport-https dirmngr && \ diff --git a/src/Storages/System/StorageSystemContributors.generated.cpp b/src/Storages/System/StorageSystemContributors.generated.cpp index 39dc74822e0..ee39390a0f5 100644 --- a/src/Storages/System/StorageSystemContributors.generated.cpp +++ b/src/Storages/System/StorageSystemContributors.generated.cpp @@ -1,4 +1,4 @@ -// autogenerated by ./StorageSystemContributors.sh +// autogenerated by src/Storages/System/StorageSystemContributors.sh const char * auto_contributors[] { "0xflotus", "20018712", @@ -11,6 +11,7 @@ const char * auto_contributors[] { "Alberto", "Aleksandr Karo", "Aleksandra (Ася)", + "Aleksandrov Vladimir", "Aleksei Levushkin", "Aleksey", "Aleksey Akulovich", @@ -117,6 +118,7 @@ const char * auto_contributors[] { "BanyRule", "Baudouin Giard", "BayoNet", + "Bertrand Junqua", "Bharat Nallan", "Big Elephant", "Bill", @@ -132,6 +134,7 @@ const char * auto_contributors[] { "Carbyn", "Chao Wang", "Chen Yufei", + "Chienlung Cheung", "Ciprian Hacman", "Clement Rodriguez", "Clément Rodriguez", @@ -178,6 +181,8 @@ const char * auto_contributors[] { "Elizaveta Mironyuk", "Emmanuel Donin de Rosière", "Eric", + "Eric Daniel", + "Erixonich", "Ernest Poletaev", "Eugene Klimov", "Eugene Konkov", @@ -187,6 +192,7 @@ const char * auto_contributors[] { "Evgeniy Gatov", "Evgeniy Udodov", "Evgeny Konkov", + "Evgeny Markov", "Ewout", "Fabian Stäber", "Fabiano Francesconi", @@ -218,6 +224,7 @@ const char * auto_contributors[] { "Hamoon", "Hasnat", "Hiroaki Nakamura", + "HuFuwang", "Hui Wang", "Igor", "Igor Hatarist", @@ -239,6 +246,7 @@ const char * auto_contributors[] { "Ilya Skrypitsa", "Ilya Yatsishin", "ImgBotApp", + "Islam Israfilov (Islam93)", "Ivan", "Ivan A. Torgashov", "Ivan Babrou", @@ -264,6 +272,8 @@ const char * auto_contributors[] { "Kang Liu", "Karl Pietrzak", "Keiji Yoshida", + "Ken Chen", + "Kevin Chiang", "Kiran", "Kirill Danshin", "Kirill Malev", @@ -299,12 +309,14 @@ const char * auto_contributors[] { "Marek Vavrusa", "Marek Vavruša", "Marek Vavruša", + "Mark Andreev", "Mark Papadakis", "Maroun Maroun", "Marsel Arduanov", "Marti Raudsepp", "Martijn Bakker", "Masha", + "Matthew Peveler", "Matwey V. Kornilov", "Max", "Max Akhmedov", @@ -349,6 +361,7 @@ const char * auto_contributors[] { "Milad Arabi", "Mohammad Hossein Sekhavat", "MovElb", + "Mr.General", "Murat Kabilov", "MyroTk", "NIKITA MIKHAILOV", @@ -359,6 +372,7 @@ const char * auto_contributors[] { "Nico Piderman", "Nicolae Vartolomei", "Nik", + "Nikhil Nadig", "Nikhil Raman", "Nikita Lapkov", "Nikita Mikhailov", @@ -384,6 +398,7 @@ const char * auto_contributors[] { "Orivej Desh", "Oskar Wojciski", "OuO", + "PHO", "Paramtamtam", "Patrick Zippenfenig", "Pavel", @@ -442,6 +457,7 @@ const char * auto_contributors[] { "Sergey Zaikin", "Sergi Vladykin", "SevaCode", + "Sherry Wang", "Silviu Caragea", "Simon Liu", "Simon Podlipsky", @@ -452,13 +468,17 @@ const char * auto_contributors[] { "Stanislav Pavlovichev", "Stas Pavlovichev", "Stefan Thies", + "Stepan", "Stepan Herold", + "Steve-金勇", + "Stig Bakken", "Stupnikov Andrey", "SuperBot", "Sébastien Launay", "TAC", "TCeason", "Tagir Kuskarov", + "Tai White", "Tangaev", "Tema Novikov", "The-Alchemist", @@ -467,6 +487,7 @@ const char * auto_contributors[] { "Tsarkova Anastasia", "Ubuntu", "Ubus", + "UnamedRus", "V", "VDimir", "Vadim", @@ -499,6 +520,7 @@ const char * auto_contributors[] { "Vladimir Chebotarev", "Vladimir Golovchenko", "Vladimir Goncharov", + "Vladimir Klimontovich", "Vladimir Kolobaev", "Vladimir Kopysov", "Vladimir Kozbin", @@ -564,16 +586,20 @@ const char * auto_contributors[] { "asiana21", "avasiliev", "avsharapov", + "awesomeleo", "benamazing", "bgranvea", "bharatnc", "blazerer", "bluebirddm", "bobrovskij artemij", + "booknouse", "bseng", "cekc", "champtar", + "chang.chen", "chengy8934", + "chenqi", "chenxing-xc", "chenxing.xc", "chertus", @@ -591,6 +617,7 @@ const char * auto_contributors[] { "dgrr", "dimarub2000", "dinosaur", + "dkxiaohei", "dmi-feo", "dmitrii", "dmitriiut", @@ -605,8 +632,10 @@ const char * auto_contributors[] { "exprmntr", "ezhaka", "f1yegor", + "fastio", "favstovol", "felixoid", + "felixxdu", "feng lv", "fenglv", "fessmage", @@ -622,8 +651,10 @@ const char * auto_contributors[] { "ggerogery", "giordyb", "glockbender", + "glushkovds", "gyuton", "hao.he", + "hchen9", "hcz", "heng zhao", "hexiaoting", @@ -640,6 +671,8 @@ const char * auto_contributors[] { "javartisan", "javi", "javi santana", + "jetgm", + "jianmei zhang", "kmeaw", "koshachy", "kreuzerkrieg", @@ -653,6 +686,7 @@ const char * auto_contributors[] { "levysh", "liangqian", "libenwang", + "lichengxiang", "linceyou", "litao91", "liu-bov", @@ -685,6 +719,8 @@ const char * auto_contributors[] { "moscas", "myrrc", "nagorny", + "nauta", + "nautaa", "never lee", "nicelulu", "nikitamikhaylov", @@ -693,6 +729,7 @@ const char * auto_contributors[] { "nvartolomei", "oandrew", "objatie_groba", + "ocadaruma", "ogorbacheva", "olegkv", "olgarev", @@ -700,6 +737,7 @@ const char * auto_contributors[] { "palasonicq", "peshkurov", "philip.han", + "pingyu", "potya", "proller", "pufit", @@ -716,7 +754,9 @@ const char * auto_contributors[] { "roman", "romanzhukov", "root", + "roverxu", "santaux", + "satanson", "sdk2", "serebrserg", "sev7e0", @@ -725,6 +765,7 @@ const char * auto_contributors[] { "shangshujie", "shedx", "simon-says", + "spongedc", "spyros87", "stavrolia", "stepenhu", @@ -738,6 +779,7 @@ const char * auto_contributors[] { "taiyang-li", "tao jiang", "tavplubix", + "tiger.yan", "topvisor", "tyrionhuang", "ubuntu", @@ -755,10 +797,13 @@ const char * auto_contributors[] { "vxider", "vzakaznikov", "wangchao", + "weeds085490", "xPoSx", + "yangshuai", "yhgcn", "ylchou", "yonesko", + "yuefoo", "yulu86", "yuluxu", "zamulla", @@ -768,6 +813,7 @@ const char * auto_contributors[] { "zhen ni", "zhukai", "zvrr", + "zvvr", "zzsmdfj", "Šimon Podlipský", "Артем Стрельцов", @@ -781,6 +827,7 @@ const char * auto_contributors[] { "小路", "张健", "张风啸", + "徐炘", "极客青年", "谢磊", "贾顺名(Jarvis)", From db46af68c109247dcf24b4a4229a46eef682b8d2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Jan 2021 03:56:41 +0300 Subject: [PATCH 249/256] Remove outdated code --- utils/release/release_lib.sh | 9 --------- 1 file changed, 9 deletions(-) diff --git a/utils/release/release_lib.sh b/utils/release/release_lib.sh index c4dd177b931..b8898d96b02 100644 --- a/utils/release/release_lib.sh +++ b/utils/release/release_lib.sh @@ -126,15 +126,6 @@ function gen_revision_author { fi fi - - # Reset testing branch to current commit. - git checkout testing - git reset --hard "$tag" - - if [ -z $NO_PUSH ]; then - git push - fi - else get_version echo reusing old version $VERSION_STRING From 9fafb0861e402fd653274ae35aad0eea82455e41 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Jan 2021 06:31:21 +0300 Subject: [PATCH 250/256] Fix build after wrong "auto version update" --- cmake/autogenerated_versions.txt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index dabac0cf317..49516ab3207 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -2,8 +2,8 @@ SET(VERSION_REVISION 54445) SET(VERSION_MAJOR 21) SET(VERSION_MINOR 1) -SET(VERSION_PATCH 0) +SET(VERSION_PATCH 1) SET(VERSION_GITHASH c2a3c6391437db2a05fe815a9d6faf81464d39ec) -SET(VERSION_DESCRIBE v21.1.0-testing) -SET(VERSION_STRING 21.1.0) +SET(VERSION_DESCRIBE v21.1.1-testing) +SET(VERSION_STRING 21.1.1) # end of autochange From 53d0c9fa7255aa1dc48991d19f4246ff71cc2fd7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Jan 2021 06:35:07 +0300 Subject: [PATCH 251/256] Fix bug in "release" script --- utils/release/release_lib.sh | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/utils/release/release_lib.sh b/utils/release/release_lib.sh index b8898d96b02..896fa7f08a0 100644 --- a/utils/release/release_lib.sh +++ b/utils/release/release_lib.sh @@ -48,11 +48,12 @@ function gen_revision_author { VERSION_REVISION=$(($VERSION_REVISION + 1)) VERSION_MAJOR=$(($VERSION_MAJOR + 1)) VERSION_MINOR=1 - VERSION_PATCH=0 + # Version cannot be zero, otherwise is breaks CMake + VERSION_PATCH=1 elif [ "$TYPE" == "minor" ] || [ "$TYPE" == "" ]; then VERSION_REVISION=$(($VERSION_REVISION + 1)) VERSION_MINOR=$(($VERSION_MINOR + 1)) - VERSION_PATCH=0 + VERSION_PATCH=1 elif [ "$TYPE" == "patch" ] || [ "$TYPE" == "bugfix" ]; then # VERSION_REVISION not incremented in new scheme. if [ "$VERSION_MAJOR" -eq "1" ] && [ "$VERSION_MINOR" -eq "1" ]; then From 46b50d02dd07ad93a28f3c7273718c52cf613137 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 11 Jan 2021 10:21:48 +0300 Subject: [PATCH 252/256] DOCSUP-4261: Document the aggregate_functions_null_for_empty setting (#18333) Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> Co-authored-by: Alexander Kazakov --- docs/en/operations/settings/settings.md | 41 +++++++++++++++++++++++++ docs/ru/operations/settings/settings.md | 40 ++++++++++++++++++++++++ 2 files changed, 81 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 82537ba637a..d3a4d50d21c 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -2474,6 +2474,45 @@ Possible values: Default value: `0`. + +## aggregate_functions_null_for_empty {#aggregate_functions_null_for_empty} + +Enables or disables rewriting all aggregate functions in a query, adding [-OrNull](../../sql-reference/aggregate-functions/combinators.md#agg-functions-combinator-ornull) suffix to them. Enable it for SQL standard compatibility. +It is implemented via query rewrite (similar to [count_distinct_implementation](#settings-count_distinct_implementation) setting) to get consistent results for distributed queries. + +Possible values: + +- 0 — Disabled. +- 1 — Enabled. + +Default value: 0. + +**Example** + +Consider the following query with aggregate functions: +```sql +SELECT + SUM(-1), + MAX(0) +FROM system.one +WHERE 0 +``` + +With `aggregate_functions_null_for_empty = 0` it would produce: +```text +┌─SUM(-1)─┬─MAX(0)─┐ +│ 0 │ 0 │ +└─────────┴────────┘ +``` + +With `aggregate_functions_null_for_empty = 1` the result would be: +```text +┌─SUMOrNull(-1)─┬─MAXOrNull(0)─┐ +│ NULL │ NULL │ +└───────────────┴──────────────┘ +``` + + ## union_default_mode {#union-default-mode} Sets a mode for combining `SELECT` query results. The setting is only used when shared with [UNION](../../sql-reference/statements/select/union.md) without explicitly specifying the `UNION ALL` or `UNION DISTINCT`. @@ -2488,6 +2527,7 @@ Default value: `''`. See examples in [UNION](../../sql-reference/statements/select/union.md). + ## data_type_default_nullable {#data_type_default_nullable} Allows data types without explicit modifiers [NULL or NOT NULL](../../sql-reference/statements/create/table.md#null-modifiers) in column definition will be [Nullable](../../sql-reference/data-types/nullable.md#data_type-nullable). @@ -2499,6 +2539,7 @@ Possible values: Default value: `0`. + ## execute_merges_on_single_replica_time_threshold {#execute-merges-on-single-replica-time-threshold} Enables special logic to perform merges on replicas. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index a3bf13bfed3..aa549fc5776 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -2343,6 +2343,45 @@ SELECT number FROM numbers(3) FORMAT JSONEachRow; Значение по умолчанию: `0`. + +## aggregate_functions_null_for_empty {#aggregate_functions_null_for_empty} + +Включает или отключает перезапись всех агрегатных функций в запросе, с добавлением к ним суффикса [-OrNull](../../sql-reference/aggregate-functions/combinators.md#agg-functions-combinator-ornull). Включите для совместимости со стандартом SQL. +Реализуется с помощью перезаписи запросов (аналогично настройке [count_distinct_implementation](#settings-count_distinct_implementation)), чтобы получить согласованные результаты для распределенных запросов. + +Возможные значения: + +- 0 — выключена. +- 1 — включена. + +Значение по умолчанию: 0. + +**Пример** + +Рассмотрим запрос с агрегирующими функциями: +```sql +SELECT + SUM(-1), + MAX(0) +FROM system.one +WHERE 0 +``` + +Результат запроса с настройкой `aggregate_functions_null_for_empty = 0`: +```text +┌─SUM(-1)─┬─MAX(0)─┐ +│ 0 │ 0 │ +└─────────┴────────┘ +``` + +Результат запроса с настройкой `aggregate_functions_null_for_empty = 1`: +```text +┌─SUMOrNull(-1)─┬─MAXOrNull(0)─┐ +│ NULL │ NULL │ +└───────────────┴──────────────┘ +``` + + ## union_default_mode {#union-default-mode} Устанавливает режим объединения результатов `SELECT` запросов. Настройка используется только при совместном использовании с [UNION](../../sql-reference/statements/select/union.md) без явного указания `UNION ALL` или `UNION DISTINCT`. @@ -2357,6 +2396,7 @@ SELECT number FROM numbers(3) FORMAT JSONEachRow; Смотрите примеры в разделе [UNION](../../sql-reference/statements/select/union.md). + ## execute_merges_on_single_replica_time_threshold {#execute-merges-on-single-replica-time-threshold} Включает особую логику выполнения слияний на репликах. From d87ed6e242f197e9800b740f5b2feda96dd755b1 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 11 Jan 2021 11:09:35 +0300 Subject: [PATCH 253/256] Auto version update to [21.1.1.5643] [54445] --- cmake/autogenerated_versions.txt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index 49516ab3207..e4251752cc0 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -3,7 +3,7 @@ SET(VERSION_REVISION 54445) SET(VERSION_MAJOR 21) SET(VERSION_MINOR 1) SET(VERSION_PATCH 1) -SET(VERSION_GITHASH c2a3c6391437db2a05fe815a9d6faf81464d39ec) -SET(VERSION_DESCRIBE v21.1.1-testing) -SET(VERSION_STRING 21.1.1) +SET(VERSION_GITHASH 53d0c9fa7255aa1dc48991d19f4246ff71cc2fd7) +SET(VERSION_DESCRIBE v21.1.1.5643-prestable) +SET(VERSION_STRING 21.1.1.5643) # end of autochange From 5538aaf3749ba2845fd22ef9ac4ee517d8dc25d2 Mon Sep 17 00:00:00 2001 From: Mark Frost Date: Mon, 11 Jan 2021 13:11:52 +0500 Subject: [PATCH 254/256] Fixed type in metrics.md --- docs/ru/operations/system-tables/metrics.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/system-tables/metrics.md b/docs/ru/operations/system-tables/metrics.md index 39ebab58624..db4016687d6 100644 --- a/docs/ru/operations/system-tables/metrics.md +++ b/docs/ru/operations/system-tables/metrics.md @@ -8,7 +8,7 @@ - `value` ([Int64](../../sql-reference/data-types/int-uint.md)) — значение метрики. - `description` ([String](../../sql-reference/data-types/string.md)) — описание метрики. -Список поддержанных метрик смотрите в файле [src/Common/CurrentMetrics.cpp](https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/CurrentMetrics.cpp). +Список поддерживаемых метрик смотрите в файле [src/Common/CurrentMetrics.cpp](https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/CurrentMetrics.cpp). **Пример** From bc2593688ea671f6f8a875d92ff53b07604085fb Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 11 Jan 2021 11:12:11 +0300 Subject: [PATCH 255/256] Auto version update to [21.2.1.1] [54446] --- cmake/autogenerated_versions.txt | 8 ++++---- debian/changelog | 4 ++-- docker/client/Dockerfile | 2 +- docker/server/Dockerfile | 2 +- docker/test/Dockerfile | 2 +- .../System/StorageSystemContributors.generated.cpp | 2 +- 6 files changed, 10 insertions(+), 10 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index e4251752cc0..1c2e4c1f55e 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -1,9 +1,9 @@ # This strings autochanged from release_lib.sh: -SET(VERSION_REVISION 54445) +SET(VERSION_REVISION 54446) SET(VERSION_MAJOR 21) -SET(VERSION_MINOR 1) +SET(VERSION_MINOR 2) SET(VERSION_PATCH 1) SET(VERSION_GITHASH 53d0c9fa7255aa1dc48991d19f4246ff71cc2fd7) -SET(VERSION_DESCRIBE v21.1.1.5643-prestable) -SET(VERSION_STRING 21.1.1.5643) +SET(VERSION_DESCRIBE v21.2.1.1-prestable) +SET(VERSION_STRING 21.2.1.1) # end of autochange diff --git a/debian/changelog b/debian/changelog index 3a267a83c69..1cec020f026 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,5 +1,5 @@ -clickhouse (21.1.0) unstable; urgency=low +clickhouse (21.2.1.1) unstable; urgency=low * Modified source code - -- Alexey Milovidov Mon, 11 Jan 2021 03:51:08 +0300 + -- clickhouse-release Mon, 11 Jan 2021 11:12:08 +0300 diff --git a/docker/client/Dockerfile b/docker/client/Dockerfile index ddfe3cd177b..5022687c47b 100644 --- a/docker/client/Dockerfile +++ b/docker/client/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/" -ARG version=21.1.0 +ARG version=21.2.1.* RUN apt-get update \ && apt-get install --yes --no-install-recommends \ diff --git a/docker/server/Dockerfile b/docker/server/Dockerfile index 890aa35fe92..a0d2d3bd55c 100644 --- a/docker/server/Dockerfile +++ b/docker/server/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:20.04 ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/" -ARG version=21.1.0 +ARG version=21.2.1.* ARG gosu_ver=1.10 RUN apt-get update \ diff --git a/docker/test/Dockerfile b/docker/test/Dockerfile index 2e17151b31f..df918928f99 100644 --- a/docker/test/Dockerfile +++ b/docker/test/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/" -ARG version=21.1.0 +ARG version=21.2.1.* RUN apt-get update && \ apt-get install -y apt-transport-https dirmngr && \ diff --git a/src/Storages/System/StorageSystemContributors.generated.cpp b/src/Storages/System/StorageSystemContributors.generated.cpp index ee39390a0f5..0c50e452e95 100644 --- a/src/Storages/System/StorageSystemContributors.generated.cpp +++ b/src/Storages/System/StorageSystemContributors.generated.cpp @@ -1,4 +1,4 @@ -// autogenerated by src/Storages/System/StorageSystemContributors.sh +// autogenerated by ./StorageSystemContributors.sh const char * auto_contributors[] { "0xflotus", "20018712", From 42f63e14b5b5adaaf72f32a19ec04b9599880605 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Jan 2021 11:32:52 +0300 Subject: [PATCH 256/256] Revert "Auto version update to [21.2.1.1] [54446]" This reverts commit bc2593688ea671f6f8a875d92ff53b07604085fb. --- cmake/autogenerated_versions.txt | 8 ++++---- debian/changelog | 4 ++-- docker/client/Dockerfile | 2 +- docker/server/Dockerfile | 2 +- docker/test/Dockerfile | 2 +- .../System/StorageSystemContributors.generated.cpp | 2 +- 6 files changed, 10 insertions(+), 10 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index 1c2e4c1f55e..e4251752cc0 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -1,9 +1,9 @@ # This strings autochanged from release_lib.sh: -SET(VERSION_REVISION 54446) +SET(VERSION_REVISION 54445) SET(VERSION_MAJOR 21) -SET(VERSION_MINOR 2) +SET(VERSION_MINOR 1) SET(VERSION_PATCH 1) SET(VERSION_GITHASH 53d0c9fa7255aa1dc48991d19f4246ff71cc2fd7) -SET(VERSION_DESCRIBE v21.2.1.1-prestable) -SET(VERSION_STRING 21.2.1.1) +SET(VERSION_DESCRIBE v21.1.1.5643-prestable) +SET(VERSION_STRING 21.1.1.5643) # end of autochange diff --git a/debian/changelog b/debian/changelog index 1cec020f026..3a267a83c69 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,5 +1,5 @@ -clickhouse (21.2.1.1) unstable; urgency=low +clickhouse (21.1.0) unstable; urgency=low * Modified source code - -- clickhouse-release Mon, 11 Jan 2021 11:12:08 +0300 + -- Alexey Milovidov Mon, 11 Jan 2021 03:51:08 +0300 diff --git a/docker/client/Dockerfile b/docker/client/Dockerfile index 5022687c47b..ddfe3cd177b 100644 --- a/docker/client/Dockerfile +++ b/docker/client/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/" -ARG version=21.2.1.* +ARG version=21.1.0 RUN apt-get update \ && apt-get install --yes --no-install-recommends \ diff --git a/docker/server/Dockerfile b/docker/server/Dockerfile index a0d2d3bd55c..890aa35fe92 100644 --- a/docker/server/Dockerfile +++ b/docker/server/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:20.04 ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/" -ARG version=21.2.1.* +ARG version=21.1.0 ARG gosu_ver=1.10 RUN apt-get update \ diff --git a/docker/test/Dockerfile b/docker/test/Dockerfile index df918928f99..2e17151b31f 100644 --- a/docker/test/Dockerfile +++ b/docker/test/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/" -ARG version=21.2.1.* +ARG version=21.1.0 RUN apt-get update && \ apt-get install -y apt-transport-https dirmngr && \ diff --git a/src/Storages/System/StorageSystemContributors.generated.cpp b/src/Storages/System/StorageSystemContributors.generated.cpp index 0c50e452e95..ee39390a0f5 100644 --- a/src/Storages/System/StorageSystemContributors.generated.cpp +++ b/src/Storages/System/StorageSystemContributors.generated.cpp @@ -1,4 +1,4 @@ -// autogenerated by ./StorageSystemContributors.sh +// autogenerated by src/Storages/System/StorageSystemContributors.sh const char * auto_contributors[] { "0xflotus", "20018712",