From 00c2a3d2c26da00c56ee1bbe8b71a38bf21db75d Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 26 Sep 2023 10:01:37 +0000 Subject: [PATCH 001/264] per-column compress min-max block sizes Signed-off-by: Duc Canh Le --- src/Interpreters/InterpreterCreateQuery.cpp | 19 +++++++++++ src/Parsers/ASTColumnDeclaration.cpp | 12 +++++++ src/Parsers/ASTColumnDeclaration.h | 1 + src/Parsers/ParserCreateQuery.h | 16 ++++++++++ src/Storages/ColumnsDescription.cpp | 17 ++++++++++ src/Storages/ColumnsDescription.h | 1 + .../MergeTree/MergeTreeDataPartWriterWide.cpp | 15 +++++++-- .../02870_per_column_compress_block.reference | 11 +++++++ .../02870_per_column_compress_block.sql | 32 +++++++++++++++++++ 9 files changed, 122 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02870_per_column_compress_block.reference create mode 100644 tests/queries/0_stateless/02870_per_column_compress_block.sql diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index a0635f18214..6e27e2eb598 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -436,6 +436,14 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns) column_declaration->children.push_back(column_declaration->codec); } + if (column.compress_block_sizes.first || column.compress_block_sizes.second) + { + Tuple value; + value.push_back(column.compress_block_sizes.first); + value.push_back(column.compress_block_sizes.second); + column_declaration->compress_block_sizes = std::make_shared(Field(value)); + } + if (column.ttl) { column_declaration->ttl = column.ttl; @@ -638,6 +646,17 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( col_decl.codec, column.type, sanity_check_compression_codecs, allow_experimental_codecs, enable_deflate_qpl_codec); } + if (col_decl.compress_block_sizes) + { + auto sizes = col_decl.compress_block_sizes->as().value.safeGet(); + if (sizes.size() != 2 || sizes[0].getType() != Field::Types::UInt64 || sizes[0].getType() != Field::Types::UInt64) + throw Exception(ErrorCodes::SYNTAX_ERROR, "Column {}: COMPRESSION BLOCK must be tuple of two unsigned integer", column.name); + column.compress_block_sizes.first = sizes[0].safeGet(); + column.compress_block_sizes.second = sizes[1].safeGet(); + if (column.compress_block_sizes.first > column.compress_block_sizes.second) + throw Exception(ErrorCodes::SYNTAX_ERROR, "Column {}: min compress block size must smaller than max compress block size", column.name); + } + if (col_decl.ttl) column.ttl = col_decl.ttl; diff --git a/src/Parsers/ASTColumnDeclaration.cpp b/src/Parsers/ASTColumnDeclaration.cpp index e9b490a1be3..99cb17b0bce 100644 --- a/src/Parsers/ASTColumnDeclaration.cpp +++ b/src/Parsers/ASTColumnDeclaration.cpp @@ -39,6 +39,12 @@ ASTPtr ASTColumnDeclaration::clone() const res->children.push_back(res->codec); } + if (compress_block_sizes) + { + res->compress_block_sizes = compress_block_sizes->clone(); + res->children.push_back(res->compress_block_sizes); + } + if (ttl) { res->ttl = ttl->clone(); @@ -99,6 +105,12 @@ void ASTColumnDeclaration::formatImpl(const FormatSettings & settings, FormatSta codec->formatImpl(settings, state, frame); } + if (compress_block_sizes) + { + settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "COMPRESS BLOCK" << (settings.hilite ? hilite_none : "") << ' '; + compress_block_sizes->formatImpl(settings, state, frame); + } + if (ttl) { settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "TTL" << (settings.hilite ? hilite_none : "") << ' '; diff --git a/src/Parsers/ASTColumnDeclaration.h b/src/Parsers/ASTColumnDeclaration.h index 9d486667911..6e73b52ce71 100644 --- a/src/Parsers/ASTColumnDeclaration.h +++ b/src/Parsers/ASTColumnDeclaration.h @@ -20,6 +20,7 @@ public: ASTPtr comment; ASTPtr codec; ASTPtr ttl; + ASTPtr compress_block_sizes; ASTPtr collation; bool primary_key_specifier = false; diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index 4062ed25c6b..8ba81b66fda 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -136,10 +136,12 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E ParserKeyword s_type{"TYPE"}; ParserKeyword s_collate{"COLLATE"}; ParserKeyword s_primary_key{"PRIMARY KEY"}; + ParserKeyword s_compress_block{"COMPRESS BLOCK"}; ParserExpression expr_parser; ParserStringLiteral string_literal_parser; ParserLiteral literal_parser; ParserCodec codec_parser; + ParserTupleOfLiterals compress_block_parser; ParserCollation collation_parser; ParserExpression expression_parser; @@ -176,6 +178,7 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E ASTPtr default_expression; ASTPtr comment_expression; ASTPtr codec_expression; + ASTPtr compress_block_sizes; ASTPtr ttl_expression; ASTPtr collation_expression; bool primary_key_specifier = false; @@ -301,6 +304,12 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E return false; } + if (s_compress_block.ignore(pos, expected)) + { + if (!compress_block_parser.parse(pos, compress_block_sizes, expected)) + return false; + } + if (s_ttl.ignore(pos, expected)) { if (!expression_parser.parse(pos, ttl_expression, expected)) @@ -342,11 +351,18 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E column_declaration->children.push_back(std::move(codec_expression)); } + if (compress_block_sizes) + { + column_declaration->compress_block_sizes = compress_block_sizes; + column_declaration->children.push_back(std::move(compress_block_sizes)); + } + if (ttl_expression) { column_declaration->ttl = ttl_expression; column_declaration->children.push_back(std::move(ttl_expression)); } + if (collation_expression) { column_declaration->collation = collation_expression; diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 0d5508b8164..6c0940463a8 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -62,6 +62,7 @@ bool ColumnDescription::operator==(const ColumnDescription & other) const && default_desc == other.default_desc && comment == other.comment && ast_to_str(codec) == ast_to_str(other.codec) + && compress_block_sizes == other.compress_block_sizes && ast_to_str(ttl) == ast_to_str(other.ttl); } @@ -94,6 +95,16 @@ void ColumnDescription::writeText(WriteBuffer & buf) const writeEscapedString(queryToString(codec), buf); } + if (compress_block_sizes.first || compress_block_sizes.second) + { + writeChar('\t', buf); + DB::writeText("COMPRESSION BLOCK ", buf); + Tuple value; + value.push_back(compress_block_sizes.first); + value.push_back(compress_block_sizes.second); + writeEscapedString(queryToString(ASTLiteral(Field(value))), buf); + } + if (ttl) { writeChar('\t', buf); @@ -138,6 +149,12 @@ void ColumnDescription::readText(ReadBuffer & buf) if (col_ast->ttl) ttl = col_ast->ttl; + if (col_ast->compress_block_sizes) + { + auto sizes = col_ast->compress_block_sizes->as().value.safeGet(); + compress_block_sizes.first = sizes[0].safeGet(); + compress_block_sizes.second = sizes[1].safeGet(); + } } else throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse column description"); diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index 2d7536765ff..6b7ec279835 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -82,6 +82,7 @@ struct ColumnDescription ColumnDefault default_desc; String comment; ASTPtr codec; + std::pair compress_block_sizes; ASTPtr ttl; ColumnDescription() = default; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 135cafbca21..74b00bb5ee7 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB { @@ -142,13 +143,19 @@ void MergeTreeDataPartWriterWide::addStreams( auto ast = parseQuery(codec_parser, "(" + Poco::toUpper(settings.marks_compression_codec) + ")", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); CompressionCodecPtr marks_compression_codec = CompressionCodecFactory::instance().get(ast, nullptr); + const auto column_desc = metadata_snapshot->columns.tryGetColumnDescription(GetColumnsOptions(GetColumnsOptions::AllPhysical), column.getNameInStorage()); + + auto max_compress_block_size = column_desc ? column_desc->compress_block_sizes.second : 0; + if (!max_compress_block_size) + max_compress_block_size = settings.max_compress_block_size; + column_streams[stream_name] = std::make_unique( stream_name, data_part->getDataPartStoragePtr(), stream_name, DATA_FILE_EXTENSION, stream_name, marks_file_extension, compression_codec, - settings.max_compress_block_size, + max_compress_block_size, marks_compression_codec, settings.marks_compress_block_size, settings.query_write_settings); @@ -321,6 +328,10 @@ StreamsWithMarks MergeTreeDataPartWriterWide::getCurrentMarksForColumn( WrittenOffsetColumns & offset_columns) { StreamsWithMarks result; + const auto column_desc = metadata_snapshot->columns.tryGetColumnDescription(GetColumnsOptions(GetColumnsOptions::AllPhysical), column.getNameInStorage()); + auto min_compress_block_size = column_desc ? column_desc->compress_block_sizes.first : 0; + if (!min_compress_block_size) + min_compress_block_size = settings.min_compress_block_size; data_part->getSerialization(column.name)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) { bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes; @@ -333,7 +344,7 @@ StreamsWithMarks MergeTreeDataPartWriterWide::getCurrentMarksForColumn( Stream & stream = *column_streams[stream_name]; /// There could already be enough data to compress into the new block. - if (stream.compressed_hashing.offset() >= settings.min_compress_block_size) + if (stream.compressed_hashing.offset() >= min_compress_block_size) stream.compressed_hashing.next(); StreamNameAndMark stream_with_mark; diff --git a/tests/queries/0_stateless/02870_per_column_compress_block.reference b/tests/queries/0_stateless/02870_per_column_compress_block.reference new file mode 100644 index 00000000000..ab98fba1668 --- /dev/null +++ b/tests/queries/0_stateless/02870_per_column_compress_block.reference @@ -0,0 +1,11 @@ +1000 +(0,0) 0 +(1,1) 1 +(2,2) 2 +(3,3) 3 +(4,4) 4 +(5,5) 5 +(6,6) 6 +(7,7) 7 +(8,8) 8 +(9,9) 9 diff --git a/tests/queries/0_stateless/02870_per_column_compress_block.sql b/tests/queries/0_stateless/02870_per_column_compress_block.sql new file mode 100644 index 00000000000..5ff0e4fe0c4 --- /dev/null +++ b/tests/queries/0_stateless/02870_per_column_compress_block.sql @@ -0,0 +1,32 @@ +-- Tags: no-random-merge-tree-settings +CREATE TABLE t +( + `id` UInt64 CODEC(ZSTD(1)), + `long_string` String CODEC(ZSTD(9, 24)) COMPRESS BLOCK (67108864, 67108864), + `v1` String CODEC(ZSTD(1)), + `v2` UInt64 CODEC(ZSTD(1)), + `v3` Float32 CODEC(ZSTD(1)), + `v4` Float64 CODEC(ZSTD(1)) +) +ENGINE = MergeTree +ORDER BY id +SETTINGS min_bytes_for_wide_part = 1; + +INSERT INTO TABLE t SELECT number, randomPrintableASCII(1000), randomPrintableASCII(10), rand(number), rand(number+1), rand(number+2) FROM numbers(1000); + +SELECT count() FROM t; + +SET allow_experimental_object_type = 1; + +CREATE TABLE t2 +( + `id` UInt64 CODEC(ZSTD(1)), + `tup` Tuple(UInt64, UInt64) CODEC(ZSTD(1)) COMPRESS BLOCK (1024, 8192), + `json` JSON CODEC(ZSTD(9, 24)) COMPRESS BLOCK (671088, 671088), +) +ENGINE = MergeTree +ORDER BY id +SETTINGS min_bytes_for_wide_part = 1; + +INSERT INTO TABLE t2 SELECT number, tuple(number, number), concat('{"key": ', toString(number), ' ,"value": ', toString(rand(number+1)), '}') FROM numbers(1000); +SELECT tup, json.key AS key FROM t2 ORDER BY key LIMIT 10; From bb9521542da3c880eed5acb6796a7a1eb98e7f9c Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Thu, 28 Sep 2023 09:46:37 +0000 Subject: [PATCH 002/264] change syntax to per-column settings Signed-off-by: Duc Canh Le --- src/Interpreters/InterpreterCreateQuery.cpp | 31 +++++++-------- src/Parsers/ASTColumnDeclaration.cpp | 19 ++++----- src/Parsers/ASTColumnDeclaration.h | 2 +- src/Parsers/ParserCreateQuery.h | 39 +++++++++++++------ src/Storages/ColumnsDescription.cpp | 25 ++++++------ src/Storages/ColumnsDescription.h | 3 +- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 10 ++++- .../02870_per_column_compress_block.sql | 16 ++++++-- 8 files changed, 88 insertions(+), 57 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 6e27e2eb598..e3e718c4c15 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -10,6 +10,8 @@ #include #include #include +#include +#include #include #include @@ -436,12 +438,12 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns) column_declaration->children.push_back(column_declaration->codec); } - if (column.compress_block_sizes.first || column.compress_block_sizes.second) + if (!column.settings.empty()) { - Tuple value; - value.push_back(column.compress_block_sizes.first); - value.push_back(column.compress_block_sizes.second); - column_declaration->compress_block_sizes = std::make_shared(Field(value)); + auto per_column_settings = std::make_shared(); + per_column_settings->is_standalone = false; + per_column_settings->changes = column.settings; + column_declaration->per_column_settings = std::move(per_column_settings); } if (column.ttl) @@ -646,20 +648,17 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( col_decl.codec, column.type, sanity_check_compression_codecs, allow_experimental_codecs, enable_deflate_qpl_codec); } - if (col_decl.compress_block_sizes) - { - auto sizes = col_decl.compress_block_sizes->as().value.safeGet(); - if (sizes.size() != 2 || sizes[0].getType() != Field::Types::UInt64 || sizes[0].getType() != Field::Types::UInt64) - throw Exception(ErrorCodes::SYNTAX_ERROR, "Column {}: COMPRESSION BLOCK must be tuple of two unsigned integer", column.name); - column.compress_block_sizes.first = sizes[0].safeGet(); - column.compress_block_sizes.second = sizes[1].safeGet(); - if (column.compress_block_sizes.first > column.compress_block_sizes.second) - throw Exception(ErrorCodes::SYNTAX_ERROR, "Column {}: min compress block size must smaller than max compress block size", column.name); - } - if (col_decl.ttl) column.ttl = col_decl.ttl; + if (col_decl.per_column_settings) + { + column.settings = col_decl.per_column_settings->as().changes; + /// Sanity check here, assume mergetree + MergeTreeSettings dummy; + dummy.applyChanges(column.settings); + } + res.add(std::move(column)); } diff --git a/src/Parsers/ASTColumnDeclaration.cpp b/src/Parsers/ASTColumnDeclaration.cpp index 99cb17b0bce..80cae040893 100644 --- a/src/Parsers/ASTColumnDeclaration.cpp +++ b/src/Parsers/ASTColumnDeclaration.cpp @@ -39,10 +39,10 @@ ASTPtr ASTColumnDeclaration::clone() const res->children.push_back(res->codec); } - if (compress_block_sizes) + if (per_column_settings) { - res->compress_block_sizes = compress_block_sizes->clone(); - res->children.push_back(res->compress_block_sizes); + res->per_column_settings = per_column_settings->clone(); + res->children.push_back(res->per_column_settings); } if (ttl) @@ -105,12 +105,6 @@ void ASTColumnDeclaration::formatImpl(const FormatSettings & settings, FormatSta codec->formatImpl(settings, state, frame); } - if (compress_block_sizes) - { - settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "COMPRESS BLOCK" << (settings.hilite ? hilite_none : "") << ' '; - compress_block_sizes->formatImpl(settings, state, frame); - } - if (ttl) { settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "TTL" << (settings.hilite ? hilite_none : "") << ' '; @@ -122,6 +116,13 @@ void ASTColumnDeclaration::formatImpl(const FormatSettings & settings, FormatSta settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "COLLATE" << (settings.hilite ? hilite_none : "") << ' '; collation->formatImpl(settings, state, frame); } + + if (per_column_settings) + { + settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "SETTINGS" << (settings.hilite ? hilite_none : "") << ' ' << '('; + per_column_settings->formatImpl(settings, state, frame); + settings.ostr << ')'; + } } } diff --git a/src/Parsers/ASTColumnDeclaration.h b/src/Parsers/ASTColumnDeclaration.h index 6e73b52ce71..e311b18e896 100644 --- a/src/Parsers/ASTColumnDeclaration.h +++ b/src/Parsers/ASTColumnDeclaration.h @@ -20,8 +20,8 @@ public: ASTPtr comment; ASTPtr codec; ASTPtr ttl; - ASTPtr compress_block_sizes; ASTPtr collation; + ASTPtr per_column_settings; bool primary_key_specifier = false; String getID(char delim) const override { return "ColumnDeclaration" + (delim + name); } diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index 8ba81b66fda..76e6c174567 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -10,6 +10,7 @@ #include #include #include +#include #include namespace DB @@ -136,14 +137,14 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E ParserKeyword s_type{"TYPE"}; ParserKeyword s_collate{"COLLATE"}; ParserKeyword s_primary_key{"PRIMARY KEY"}; - ParserKeyword s_compress_block{"COMPRESS BLOCK"}; + ParserKeyword s_settings("SETTINGS"); ParserExpression expr_parser; ParserStringLiteral string_literal_parser; ParserLiteral literal_parser; ParserCodec codec_parser; - ParserTupleOfLiterals compress_block_parser; ParserCollation collation_parser; ParserExpression expression_parser; + ParserSetQuery settings_parser(true); /// mandatory column name ASTPtr name; @@ -178,7 +179,7 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E ASTPtr default_expression; ASTPtr comment_expression; ASTPtr codec_expression; - ASTPtr compress_block_sizes; + ASTPtr per_column_settings; ASTPtr ttl_expression; ASTPtr collation_expression; bool primary_key_specifier = false; @@ -304,12 +305,6 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E return false; } - if (s_compress_block.ignore(pos, expected)) - { - if (!compress_block_parser.parse(pos, compress_block_sizes, expected)) - return false; - } - if (s_ttl.ignore(pos, expected)) { if (!expression_parser.parse(pos, ttl_expression, expected)) @@ -321,6 +316,26 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E primary_key_specifier = true; } + auto old_pos = pos; + if (s_settings.ignore(pos, expected)) + { + ParserToken parser_opening_bracket(TokenType::OpeningRoundBracket); + if (parser_opening_bracket.ignore(pos, expected)) + { + if (!settings_parser.parse(pos, per_column_settings, expected)) + return false; + ParserToken parser_closing_bracket(TokenType::ClosingRoundBracket); + if (!parser_closing_bracket.ignore(pos, expected)) + return false; + } + else + { + /// This could be settings in alter query + /// E.g: ALTER TABLE alter_enum_array MODIFY COLUMN x String SETTINGS mutations_sync=2; + pos = old_pos; + } + } + node = column_declaration; if (type) @@ -351,10 +366,10 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E column_declaration->children.push_back(std::move(codec_expression)); } - if (compress_block_sizes) + if (per_column_settings) { - column_declaration->compress_block_sizes = compress_block_sizes; - column_declaration->children.push_back(std::move(compress_block_sizes)); + column_declaration->per_column_settings = per_column_settings; + column_declaration->children.push_back(std::move(per_column_settings)); } if (ttl_expression) diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 6c0940463a8..46eee56afc9 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -24,6 +24,7 @@ #include #include #include +#include "Parsers/ASTSetQuery.h" #include #include #include @@ -62,7 +63,7 @@ bool ColumnDescription::operator==(const ColumnDescription & other) const && default_desc == other.default_desc && comment == other.comment && ast_to_str(codec) == ast_to_str(other.codec) - && compress_block_sizes == other.compress_block_sizes + && settings == other.settings && ast_to_str(ttl) == ast_to_str(other.ttl); } @@ -95,14 +96,15 @@ void ColumnDescription::writeText(WriteBuffer & buf) const writeEscapedString(queryToString(codec), buf); } - if (compress_block_sizes.first || compress_block_sizes.second) + if (!settings.empty()) { writeChar('\t', buf); - DB::writeText("COMPRESSION BLOCK ", buf); - Tuple value; - value.push_back(compress_block_sizes.first); - value.push_back(compress_block_sizes.second); - writeEscapedString(queryToString(ASTLiteral(Field(value))), buf); + DB::writeText("SETTINGS ", buf); + DB::writeText("(", buf); + ASTSetQuery ast; + ast.changes = settings; + writeEscapedString(queryToString(ast), buf); + DB::writeText(")", buf); } if (ttl) @@ -149,12 +151,9 @@ void ColumnDescription::readText(ReadBuffer & buf) if (col_ast->ttl) ttl = col_ast->ttl; - if (col_ast->compress_block_sizes) - { - auto sizes = col_ast->compress_block_sizes->as().value.safeGet(); - compress_block_sizes.first = sizes[0].safeGet(); - compress_block_sizes.second = sizes[1].safeGet(); - } + + if (col_ast->per_column_settings) + settings = col_ast->per_column_settings->as().changes; } else throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse column description"); diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index 6b7ec279835..2f2d0c7d77c 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -7,6 +7,7 @@ #include #include #include +#include #include #include @@ -82,7 +83,7 @@ struct ColumnDescription ColumnDefault default_desc; String comment; ASTPtr codec; - std::pair compress_block_sizes; + SettingsChanges settings; ASTPtr ttl; ColumnDescription() = default; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 74b00bb5ee7..b7d4d7895c3 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -145,7 +145,10 @@ void MergeTreeDataPartWriterWide::addStreams( const auto column_desc = metadata_snapshot->columns.tryGetColumnDescription(GetColumnsOptions(GetColumnsOptions::AllPhysical), column.getNameInStorage()); - auto max_compress_block_size = column_desc ? column_desc->compress_block_sizes.second : 0; + UInt64 max_compress_block_size = 0; + if (column_desc) + if (const auto * value = column_desc->settings.tryGet("max_compress_block_size")) + max_compress_block_size = value->safeGet(); if (!max_compress_block_size) max_compress_block_size = settings.max_compress_block_size; @@ -329,7 +332,10 @@ StreamsWithMarks MergeTreeDataPartWriterWide::getCurrentMarksForColumn( { StreamsWithMarks result; const auto column_desc = metadata_snapshot->columns.tryGetColumnDescription(GetColumnsOptions(GetColumnsOptions::AllPhysical), column.getNameInStorage()); - auto min_compress_block_size = column_desc ? column_desc->compress_block_sizes.first : 0; + UInt64 min_compress_block_size = 0; + if (column_desc) + if (const auto * value = column_desc->settings.tryGet("min_compress_block_size")) + min_compress_block_size = value->safeGet(); if (!min_compress_block_size) min_compress_block_size = settings.min_compress_block_size; data_part->getSerialization(column.name)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) diff --git a/tests/queries/0_stateless/02870_per_column_compress_block.sql b/tests/queries/0_stateless/02870_per_column_compress_block.sql index 5ff0e4fe0c4..9b0c0210cf5 100644 --- a/tests/queries/0_stateless/02870_per_column_compress_block.sql +++ b/tests/queries/0_stateless/02870_per_column_compress_block.sql @@ -2,7 +2,7 @@ CREATE TABLE t ( `id` UInt64 CODEC(ZSTD(1)), - `long_string` String CODEC(ZSTD(9, 24)) COMPRESS BLOCK (67108864, 67108864), + `long_string` String CODEC(ZSTD(9, 24)) SETTINGS (min_compress_block_size = 81920, max_compress_block_size = 163840), `v1` String CODEC(ZSTD(1)), `v2` UInt64 CODEC(ZSTD(1)), `v3` Float32 CODEC(ZSTD(1)), @@ -21,8 +21,8 @@ SET allow_experimental_object_type = 1; CREATE TABLE t2 ( `id` UInt64 CODEC(ZSTD(1)), - `tup` Tuple(UInt64, UInt64) CODEC(ZSTD(1)) COMPRESS BLOCK (1024, 8192), - `json` JSON CODEC(ZSTD(9, 24)) COMPRESS BLOCK (671088, 671088), + `tup` Tuple(UInt64, UInt64) CODEC(ZSTD(1)) SETTINGS (min_compress_block_size = 81920, max_compress_block_size = 163840), + `json` JSON CODEC(ZSTD(9, 24)) SETTINGS (min_compress_block_size = 81920, max_compress_block_size = 163840), ) ENGINE = MergeTree ORDER BY id @@ -30,3 +30,13 @@ SETTINGS min_bytes_for_wide_part = 1; INSERT INTO TABLE t2 SELECT number, tuple(number, number), concat('{"key": ', toString(number), ' ,"value": ', toString(rand(number+1)), '}') FROM numbers(1000); SELECT tup, json.key AS key FROM t2 ORDER BY key LIMIT 10; + + +CREATE TABLE t3 +( + `id` UInt64 CODEC(ZSTD(1)), + `long_string` String CODEC(ZSTD(1)) SETTINGS (min_block_size = 81920, max_compress_block_size = 163840), +) +ENGINE = MergeTree +ORDER BY id +SETTINGS min_bytes_for_wide_part = 1; -- {serverError 115} From 4ff4b0a84bccd492d6f09b2cabf64809efa26cab Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Fri, 29 Sep 2023 04:25:34 +0000 Subject: [PATCH 003/264] column-level settings: alter-able Signed-off-by: Duc Canh Le --- src/Interpreters/InterpreterCreateQuery.cpp | 4 +--- src/Parsers/ParserAlterQuery.cpp | 3 +++ src/Storages/AlterCommands.cpp | 22 +++++++++++++++++++- src/Storages/AlterCommands.h | 5 +++-- src/Storages/MergeTree/MergeTreeSettings.cpp | 16 ++++++++++++++ src/Storages/MergeTree/MergeTreeSettings.h | 6 ++++++ 6 files changed, 50 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index e3e718c4c15..15a4efff857 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -654,9 +654,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( if (col_decl.per_column_settings) { column.settings = col_decl.per_column_settings->as().changes; - /// Sanity check here, assume mergetree - MergeTreeSettings dummy; - dummy.applyChanges(column.settings); + MergeTreeColumnSettings::validate(column.settings); } res.add(std::move(column)); diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index 8292b52f092..2f332e507e3 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -103,6 +103,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_comment("COMMENT"); ParserKeyword s_codec("CODEC"); ParserKeyword s_ttl("TTL"); + ParserKeyword s_column_settings("COLUMN SETTINGS"); ParserKeyword s_remove_ttl("REMOVE TTL"); ParserKeyword s_remove_sample_by("REMOVE SAMPLE BY"); @@ -636,6 +637,8 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->remove_property = "CODEC"; else if (s_ttl.ignore(pos, expected)) command->remove_property = "TTL"; + else if (s_column_settings.ignore(pos, expected)) + command->remove_property = "COLUMN SETTINGS"; else return false; } diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 3ade4474b6b..52fe6067c84 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -32,6 +32,7 @@ #include #include #include +#include namespace DB { @@ -65,6 +66,8 @@ AlterCommand::RemoveProperty removePropertyFromString(const String & property) return AlterCommand::RemoveProperty::CODEC; else if (property == "TTL") return AlterCommand::RemoveProperty::TTL; + else if (property == "COLUMN SETTINGS") + return AlterCommand::RemoveProperty::SETTINGS; throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot remove unknown property '{}'", property); } @@ -164,6 +167,9 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ if (ast_col_decl.codec) command.codec = ast_col_decl.codec; + if (ast_col_decl.per_column_settings) + command.settings_changes = ast_col_decl.per_column_settings->as().changes; + if (command_ast->column) command.after_column = getIdentifierName(command_ast->column); @@ -428,6 +434,10 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) { column.ttl.reset(); } + else if (to_remove == RemoveProperty::SETTINGS) + { + column.settings.clear(); + } else { if (codec) @@ -442,6 +452,12 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) if (data_type) column.type = data_type; + if (!settings_changes.empty()) + { + MergeTreeColumnSettings::validate(settings_changes); + column.settings = settings_changes; + } + /// User specified default expression or changed /// datatype. We have to replace default. if (default_expression || data_type) @@ -1189,7 +1205,11 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const ErrorCodes::BAD_ARGUMENTS, "Column {} doesn't have COMMENT, cannot remove it", backQuote(column_name)); - + if (command.to_remove == AlterCommand::RemoveProperty::SETTINGS && column_from_table.settings.empty()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Column {} doesn't have SETTINGS, cannot remove it", + backQuote(column_name)); } modified_columns.emplace(column_name); diff --git a/src/Storages/AlterCommands.h b/src/Storages/AlterCommands.h index c06872f9757..8afcddc5954 100644 --- a/src/Storages/AlterCommands.h +++ b/src/Storages/AlterCommands.h @@ -61,7 +61,8 @@ struct AlterCommand /// Other properties COMMENT, CODEC, - TTL + TTL, + SETTINGS }; Type type = UNKNOWN; @@ -130,7 +131,7 @@ struct AlterCommand /// For ADD and MODIFY ASTPtr codec = nullptr; - /// For MODIFY SETTING + /// For MODIFY SETTING or MODIFY COLUMN SETTINGS SettingsChanges settings_changes; /// For RESET SETTING diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 1906f130101..33b137b60da 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -212,4 +212,20 @@ void MergeTreeSettings::sanityCheck(size_t background_pool_tasks) const merge_selecting_sleep_slowdown_factor); } } + +void MergeTreeColumnSettings::validate(const SettingsChanges & changes) +{ + static MergeTreeSettings merge_tree_settings; + static std::set allowed_column_level_settings = {"min_compress_block_size", "max_compress_block_size"}; + for (const auto & change : changes) + { + if (!allowed_column_level_settings.contains(change.name)) + throw Exception( + ErrorCodes::UNKNOWN_SETTING, + "Setting {} is unknown or not supported at column level, supported settings: {}", + change.name, + fmt::join(allowed_column_level_settings, ", ")); + merge_tree_settings.checkCanSet(change.name, change.value); + } +} } diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index dbae87b0c5e..16a8d726abc 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -245,6 +245,12 @@ struct MergeTreeSettings : public BaseSettings void sanityCheck(size_t background_pool_tasks) const; }; + using MergeTreeSettingsPtr = std::shared_ptr; +namespace MergeTreeColumnSettings +{ + void validate(const SettingsChanges & changes); +} + } From bb62b91f94f29b799cdba178096ae5ed9738414d Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 2 Oct 2023 05:55:39 +0000 Subject: [PATCH 004/264] only allow column level settings for MergeTree family Signed-off-by: Duc Canh Le --- src/Interpreters/InterpreterCreateQuery.cpp | 41 ++++++++++++------- src/Storages/MergeTree/MergeTreeSettings.cpp | 9 +++- .../02870_per_column_compress_block.sql | 10 ++++- 3 files changed, 42 insertions(+), 18 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 15a4efff857..10f435ceac4 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -841,22 +841,33 @@ void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & creat throw Exception(ErrorCodes::DUPLICATE_COLUMN, "Column {} already exists", backQuoteIfNeed(column.name)); } - /// Check if _row_exists for lightweight delete column in column_lists for merge tree family. - if (create.storage && create.storage->engine && endsWith(create.storage->engine->name, "MergeTree")) + if (create.storage && create.storage->engine) { - auto search = all_columns.find(LightweightDeleteDescription::FILTER_COLUMN.name); - if (search != all_columns.end()) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "Cannot create table with column '{}' for *MergeTree engines because it " - "is reserved for lightweight delete feature", - LightweightDeleteDescription::FILTER_COLUMN.name); - - auto search_block_number = all_columns.find(BlockNumberColumn::name); - if (search_block_number != all_columns.end()) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "Cannot create table with column '{}' for *MergeTree engines because it " - "is reserved for storing block number", - BlockNumberColumn::name); + /// Check if _row_exists for lightweight delete column in column_lists for merge tree family. + if (endsWith(create.storage->engine->name, "MergeTree")) + { + auto search = all_columns.find(LightweightDeleteDescription::FILTER_COLUMN.name); + if (search != all_columns.end()) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Cannot create table with column '{}' for *MergeTree engines because it " + "is reserved for lightweight delete feature", + LightweightDeleteDescription::FILTER_COLUMN.name); + auto search_block_number = all_columns.find(BlockNumberColumn::name); + if (search_block_number != all_columns.end()) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Cannot create table with column '{}' for *MergeTree engines because it " + "is reserved for storing block number", + BlockNumberColumn::name); + } + else + { + /// Only merge tree family supports column with custom column setting + if (std::any_of( + properties.columns.begin(), + properties.columns.end(), + [](const ColumnDescription & column) { return !column.settings.empty(); })) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot create table with column level settings for non-MergeTree engines"); + } } const auto & settings = getContext()->getSettingsRef(); diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 33b137b60da..4042d2d5d33 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -215,8 +215,12 @@ void MergeTreeSettings::sanityCheck(size_t background_pool_tasks) const void MergeTreeColumnSettings::validate(const SettingsChanges & changes) { - static MergeTreeSettings merge_tree_settings; - static std::set allowed_column_level_settings = {"min_compress_block_size", "max_compress_block_size"}; + static const MergeTreeSettings merge_tree_settings; + static const std::set allowed_column_level_settings = + { + "min_compress_block_size", + "max_compress_block_size" + }; for (const auto & change : changes) { if (!allowed_column_level_settings.contains(change.name)) @@ -228,4 +232,5 @@ void MergeTreeColumnSettings::validate(const SettingsChanges & changes) merge_tree_settings.checkCanSet(change.name, change.value); } } + } diff --git a/tests/queries/0_stateless/02870_per_column_compress_block.sql b/tests/queries/0_stateless/02870_per_column_compress_block.sql index 9b0c0210cf5..589124ac1b4 100644 --- a/tests/queries/0_stateless/02870_per_column_compress_block.sql +++ b/tests/queries/0_stateless/02870_per_column_compress_block.sql @@ -8,7 +8,7 @@ CREATE TABLE t `v3` Float32 CODEC(ZSTD(1)), `v4` Float64 CODEC(ZSTD(1)) ) -ENGINE = MergeTree +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/t/2870', 'r1') ORDER BY id SETTINGS min_bytes_for_wide_part = 1; @@ -40,3 +40,11 @@ CREATE TABLE t3 ENGINE = MergeTree ORDER BY id SETTINGS min_bytes_for_wide_part = 1; -- {serverError 115} + +CREATE TABLE t4 +( + `id` UInt64 CODEC(ZSTD(1)), + `long_string` String CODEC(ZSTD(1)) SETTINGS (min_compress_block_size = 81920, max_compress_block_size = 163840), +) +ENGINE = TinyLog +ORDER BY id; -- {serverError 44} \ No newline at end of file From 50ad6457e4aa696a12ec6dd82f1bf3fa56136601 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 2 Oct 2023 05:56:56 +0000 Subject: [PATCH 005/264] fix wrong column record on zookeeper Signed-off-by: Duc Canh Le --- src/Storages/ColumnsDescription.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 46eee56afc9..6edc12c33dc 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -102,6 +102,7 @@ void ColumnDescription::writeText(WriteBuffer & buf) const DB::writeText("SETTINGS ", buf); DB::writeText("(", buf); ASTSetQuery ast; + ast.is_standalone = false; ast.changes = settings; writeEscapedString(queryToString(ast), buf); DB::writeText(")", buf); From 8e374882ccfc8474b916962e7cd4bf0901c13cf9 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 3 Oct 2023 07:27:55 +0000 Subject: [PATCH 006/264] support `ALTER table MODIFY column REMOVE SETTING ..` Signed-off-by: Duc Canh Le --- src/Parsers/ParserAlterQuery.cpp | 10 ++++-- src/Storages/AlterCommands.cpp | 33 ++++++++++++++----- src/Storages/AlterCommands.h | 6 ++-- .../02870_per_column_compress_block.reference | 3 ++ .../02870_per_column_compress_block.sql | 21 ++++++++++-- 5 files changed, 56 insertions(+), 17 deletions(-) diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index 2f332e507e3..cdf012d752d 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -103,7 +103,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_comment("COMMENT"); ParserKeyword s_codec("CODEC"); ParserKeyword s_ttl("TTL"); - ParserKeyword s_column_settings("COLUMN SETTINGS"); + ParserKeyword s_setting("SETTING"); ParserKeyword s_remove_ttl("REMOVE TTL"); ParserKeyword s_remove_sample_by("REMOVE SAMPLE BY"); @@ -637,8 +637,12 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->remove_property = "CODEC"; else if (s_ttl.ignore(pos, expected)) command->remove_property = "TTL"; - else if (s_column_settings.ignore(pos, expected)) - command->remove_property = "COLUMN SETTINGS"; + else if (s_setting.ignore(pos, expected)) + { + command->remove_property = "SETTING"; + if (!parser_reset_setting.parse(pos, command->settings_resets, expected)) + return false; + } else return false; } diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 52fe6067c84..515bca2d9b7 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -66,8 +66,8 @@ AlterCommand::RemoveProperty removePropertyFromString(const String & property) return AlterCommand::RemoveProperty::CODEC; else if (property == "TTL") return AlterCommand::RemoveProperty::TTL; - else if (property == "COLUMN SETTINGS") - return AlterCommand::RemoveProperty::SETTINGS; + else if (property == "SETTING") + return AlterCommand::RemoveProperty::SETTING; throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot remove unknown property '{}'", property); } @@ -143,6 +143,16 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ const auto & ast_col_decl = command_ast->col_decl->as(); command.column_name = ast_col_decl.name; command.to_remove = removePropertyFromString(command_ast->remove_property); + if (command.to_remove == RemoveProperty::SETTING) + { + for (const ASTPtr & identifier_ast : command_ast->settings_resets->children) + { + const auto & identifier = identifier_ast->as(); + auto insertion = command.settings_resets.emplace(identifier.name()); + if (!insertion.second) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Duplicate setting name {}", backQuote(identifier.name())); + } + } if (ast_col_decl.type) { @@ -434,9 +444,10 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) { column.ttl.reset(); } - else if (to_remove == RemoveProperty::SETTINGS) + else if (to_remove == RemoveProperty::SETTING) { - column.settings.clear(); + for (const auto & setting : settings_resets) + column.settings.removeSetting(setting); } else { @@ -1205,11 +1216,15 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const ErrorCodes::BAD_ARGUMENTS, "Column {} doesn't have COMMENT, cannot remove it", backQuote(column_name)); - if (command.to_remove == AlterCommand::RemoveProperty::SETTINGS && column_from_table.settings.empty()) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Column {} doesn't have SETTINGS, cannot remove it", - backQuote(column_name)); + if (command.to_remove == AlterCommand::RemoveProperty::SETTING) + { + for (const auto & setting : command.settings_resets) + { + if (!column_from_table.settings.tryGet(setting)) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Column {} doesn't have SETTINGS, cannot remove it", backQuote(column_name)); + } + } } modified_columns.emplace(column_name); diff --git a/src/Storages/AlterCommands.h b/src/Storages/AlterCommands.h index 8afcddc5954..3149d134a99 100644 --- a/src/Storages/AlterCommands.h +++ b/src/Storages/AlterCommands.h @@ -62,7 +62,7 @@ struct AlterCommand COMMENT, CODEC, TTL, - SETTINGS + SETTING }; Type type = UNKNOWN; @@ -131,10 +131,10 @@ struct AlterCommand /// For ADD and MODIFY ASTPtr codec = nullptr; - /// For MODIFY SETTING or MODIFY COLUMN SETTINGS + /// For MODIFY SETTING or MODIFY COLUMN with SETTINGS (...) SettingsChanges settings_changes; - /// For RESET SETTING + /// For RESET SETTING or MODIFY COLUMN REMOVE SETTING (...) std::set settings_resets; /// For MODIFY_QUERY diff --git a/tests/queries/0_stateless/02870_per_column_compress_block.reference b/tests/queries/0_stateless/02870_per_column_compress_block.reference index ab98fba1668..96e01669414 100644 --- a/tests/queries/0_stateless/02870_per_column_compress_block.reference +++ b/tests/queries/0_stateless/02870_per_column_compress_block.reference @@ -1,4 +1,7 @@ +CREATE TABLE db_02780.t\n(\n `id` UInt64 CODEC(ZSTD(1)),\n `long_string` String CODEC(ZSTD(9, 24)) SETTINGS (min_compress_block_size = 163840, max_compress_block_size = 163840),\n `v1` String CODEC(ZSTD(1)),\n `v2` UInt64 CODEC(ZSTD(1)),\n `v3` Float32 CODEC(ZSTD(1)),\n `v4` Float64 CODEC(ZSTD(1))\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/db_02780/t/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192 1000 +CREATE TABLE db_02780.t\n(\n `id` UInt64 CODEC(ZSTD(1)),\n `long_string` String CODEC(ZSTD(9, 24)),\n `v1` String CODEC(ZSTD(1)),\n `v2` UInt64 CODEC(ZSTD(1)),\n `v3` Float32 CODEC(ZSTD(1)),\n `v4` Float64 CODEC(ZSTD(1))\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/db_02780/t/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192 +CREATE TABLE db_02780.t\n(\n `id` UInt64 CODEC(ZSTD(1)),\n `long_string` String CODEC(ZSTD(9, 24)) SETTINGS (min_compress_block_size = 163840, max_compress_block_size = 163840),\n `v1` String CODEC(ZSTD(1)),\n `v2` UInt64 CODEC(ZSTD(1)),\n `v3` Float32 CODEC(ZSTD(1)),\n `v4` Float64 CODEC(ZSTD(1))\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/db_02780/t/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192 (0,0) 0 (1,1) 1 (2,2) 2 diff --git a/tests/queries/0_stateless/02870_per_column_compress_block.sql b/tests/queries/0_stateless/02870_per_column_compress_block.sql index 589124ac1b4..65556edeeb8 100644 --- a/tests/queries/0_stateless/02870_per_column_compress_block.sql +++ b/tests/queries/0_stateless/02870_per_column_compress_block.sql @@ -1,8 +1,11 @@ -- Tags: no-random-merge-tree-settings +DROP DATABASE IF EXISTS db_02780; +CREATE DATABASE db_02780; +USE db_02780; CREATE TABLE t ( `id` UInt64 CODEC(ZSTD(1)), - `long_string` String CODEC(ZSTD(9, 24)) SETTINGS (min_compress_block_size = 81920, max_compress_block_size = 163840), + `long_string` String CODEC(ZSTD(9, 24)) SETTINGS (min_compress_block_size = 163840, max_compress_block_size = 163840), `v1` String CODEC(ZSTD(1)), `v2` UInt64 CODEC(ZSTD(1)), `v3` Float32 CODEC(ZSTD(1)), @@ -12,10 +15,22 @@ ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/t/2870', 'r1') ORDER BY id SETTINGS min_bytes_for_wide_part = 1; +SHOW CREATE t; + INSERT INTO TABLE t SELECT number, randomPrintableASCII(1000), randomPrintableASCII(10), rand(number), rand(number+1), rand(number+2) FROM numbers(1000); SELECT count() FROM t; +ALTER TABLE t MODIFY COLUMN long_string REMOVE SETTING min_compress_block_size, max_compress_block_size; + +SHOW CREATE t; + +ALTER TABLE t MODIFY COLUMN long_string String CODEC(ZSTD(9, 24)) SETTINGS (min_compress_block_size = 163840, max_compress_block_size = 163840); + +SHOW CREATE t; + +DROP TABLE t; + SET allow_experimental_object_type = 1; CREATE TABLE t2 @@ -47,4 +62,6 @@ CREATE TABLE t4 `long_string` String CODEC(ZSTD(1)) SETTINGS (min_compress_block_size = 81920, max_compress_block_size = 163840), ) ENGINE = TinyLog -ORDER BY id; -- {serverError 44} \ No newline at end of file +ORDER BY id; -- {serverError 44} + +DROP DATABASE db_02780; \ No newline at end of file From 3fd9f85021173bedf4be26baa4e7abc1c9144da1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 7 Dec 2023 13:19:46 +0100 Subject: [PATCH 007/264] File cache small optimization improvement --- src/Interpreters/Cache/FileCache.cpp | 6 +++ src/Interpreters/Cache/FileSegment.cpp | 54 ++++++++++++++++---------- src/Interpreters/Cache/Metadata.cpp | 9 +++-- 3 files changed, 46 insertions(+), 23 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 2f2d199b8b6..916d917353d 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -959,12 +959,18 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size, FileCa void FileCache::removeKey(const Key & key) { assertInitialized(); +#ifdef ABORT_ON_LOGICAL_ERROR + assertCacheCorrectness(); +#endif metadata.removeKey(key, /* if_exists */false, /* if_releasable */true); } void FileCache::removeKeyIfExists(const Key & key) { assertInitialized(); +#ifdef ABORT_ON_LOGICAL_ERROR + assertCacheCorrectness(); +#endif metadata.removeKey(key, /* if_exists */true, /* if_releasable */true); } diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 3fa3da19e5a..e71fbe89d18 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -331,30 +331,31 @@ void FileSegment::write(const char * from, size_t size, size_t offset) if (!size) throw Exception(ErrorCodes::LOGICAL_ERROR, "Writing zero size is not allowed"); + { + auto lock = lockFileSegment(); + assertIsDownloaderUnlocked("write", lock); + assertNotDetachedUnlocked(lock); + } + const auto file_segment_path = getPathInLocalCache(); { - auto lock = lockFileSegment(); - - assertIsDownloaderUnlocked("write", lock); - assertNotDetachedUnlocked(lock); - if (download_state != State::DOWNLOADING) throw Exception( ErrorCodes::LOGICAL_ERROR, "Expected DOWNLOADING state, got {}", stateToString(download_state)); - size_t first_non_downloaded_offset = getCurrentWriteOffset(); + const size_t first_non_downloaded_offset = getCurrentWriteOffset(); if (offset != first_non_downloaded_offset) throw Exception( ErrorCodes::LOGICAL_ERROR, "Attempt to write {} bytes to offset: {}, but current write offset is {}", size, offset, first_non_downloaded_offset); - size_t current_downloaded_size = getDownloadedSize(); + const size_t current_downloaded_size = getDownloadedSize(); chassert(reserved_size >= current_downloaded_size); - size_t free_reserved_size = reserved_size - current_downloaded_size; + const size_t free_reserved_size = reserved_size - current_downloaded_size; if (free_reserved_size < size) throw Exception( ErrorCodes::LOGICAL_ERROR, @@ -363,20 +364,18 @@ void FileSegment::write(const char * from, size_t size, size_t offset) if (!is_unbound && current_downloaded_size == range().size()) throw Exception(ErrorCodes::LOGICAL_ERROR, "File segment is already fully downloaded"); - if (!cache_writer) - { - if (current_downloaded_size > 0) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Cache writer was finalized (downloaded size: {}, state: {})", - current_downloaded_size, stateToString(download_state)); - - cache_writer = std::make_unique(file_segment_path); - } + if (!cache_writer && current_downloaded_size > 0) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cache writer was finalized (downloaded size: {}, state: {})", + current_downloaded_size, stateToString(download_state)); } try { + if (!cache_writer) + cache_writer = std::make_unique(file_segment_path); + cache_writer->write(from, size); cache_writer->next(); @@ -385,16 +384,24 @@ void FileSegment::write(const char * from, size_t size, size_t offset) } catch (ErrnoException & e) { + const int code = e.getErrno(); + const bool is_no_space_left_error = code == /* No space left on device */28 || code == /* Quota exceeded */122; + auto lock = lockFileSegment(); e.addMessage(fmt::format("{}, current cache state: {}", e.what(), getInfoForLogUnlocked(lock))); - int code = e.getErrno(); - if (code == /* No space left on device */28 || code == /* Quota exceeded */122) + if (downloaded_size == 0 && fs::exists(file_segment_path)) + { + fs::remove(file_segment_path); + } + else if (is_no_space_left_error) { const auto file_size = fs::file_size(file_segment_path); + chassert(downloaded_size <= file_size); chassert(reserved_size >= file_size); chassert(file_size <= range().size()); + if (downloaded_size != file_size) downloaded_size = file_size; } @@ -786,6 +793,13 @@ bool FileSegment::assertCorrectnessUnlocked(const FileSegmentGuard::Lock &) cons chassert(entry.offset == offset()); }; + auto lk = lockFileSegment(); + + if (downloaded_size == 0) + chassert(!fs::exists(getPathInLocalCache())); + else + chassert(fs::exists(getPathInLocalCache())); + if (download_state == State::DOWNLOADED) { chassert(downloader_id.empty()); diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 6d3927c3f36..7c93acbff76 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -783,8 +783,11 @@ KeyMetadata::iterator LockedKey::removeFileSegmentImpl(KeyMetadata::iterator it, try { const auto path = key_metadata->getFileSegmentPath(*file_segment); - bool exists = fs::exists(path); - if (exists) + if (file_segment->downloaded_size == 0) + { + chassert(!fs::exists(path)); + } + else if (fs::exists(path)) { fs::remove(path); @@ -797,7 +800,7 @@ KeyMetadata::iterator LockedKey::removeFileSegmentImpl(KeyMetadata::iterator it, LOG_TEST(key_metadata->log, "Removed file segment at path: {}", path); } - else if (file_segment->downloaded_size && !can_be_broken) + else if (!can_be_broken) { #ifdef ABORT_ON_LOGICAL_ERROR throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected path {} to exist", path); From 629c27d368c39e8a2e2e63df4b48de9cfbe12a00 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 8 Dec 2023 11:00:40 +0100 Subject: [PATCH 008/264] Fxi --- src/Interpreters/Cache/FileSegment.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index e71fbe89d18..06e7271c2df 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -793,8 +793,6 @@ bool FileSegment::assertCorrectnessUnlocked(const FileSegmentGuard::Lock &) cons chassert(entry.offset == offset()); }; - auto lk = lockFileSegment(); - if (downloaded_size == 0) chassert(!fs::exists(getPathInLocalCache())); else From f80f61cd9678b87ab0a7f8fb2c4f721f3e736ce9 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 8 Dec 2023 13:00:01 +0100 Subject: [PATCH 009/264] Try fxi --- src/Interpreters/Cache/FileSegment.cpp | 17 +++++++++++++---- src/Interpreters/Cache/FileSegment.h | 1 + 2 files changed, 14 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 06e7271c2df..f46bbbc3630 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -376,6 +376,12 @@ void FileSegment::write(const char * from, size_t size, size_t offset) if (!cache_writer) cache_writer = std::make_unique(file_segment_path); +#ifdef ABORT_ON_LOGICAL_ERROR + /// This mutex is only needed to have a valid assertion in assertCacheCorrectness(), + /// which is only executed in debug/sanitizer builds (under ABORT_ON_LOGICAL_ERROR). + std::lock_guard lock(write_mutex); +#endif + cache_writer->write(from, size); cache_writer->next(); @@ -793,10 +799,13 @@ bool FileSegment::assertCorrectnessUnlocked(const FileSegmentGuard::Lock &) cons chassert(entry.offset == offset()); }; - if (downloaded_size == 0) - chassert(!fs::exists(getPathInLocalCache())); - else - chassert(fs::exists(getPathInLocalCache())); + { + std::lock_guard lock(write_mutex); + if (downloaded_size == 0) + chassert(!fs::exists(getPathInLocalCache())); + else + chassert(fs::exists(getPathInLocalCache())); + } if (download_state == State::DOWNLOADED) { diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index ff5c03c18f3..2d82676a485 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -323,6 +323,7 @@ private: /// downloaded_size should always be less or equal to reserved_size std::atomic downloaded_size = 0; std::atomic reserved_size = 0; + mutable std::mutex write_mutex; mutable FileSegmentGuard segment_guard; std::weak_ptr key_metadata; From 002b4066371c3b26c2772c6cd2cecdb347f7c030 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 8 Dec 2023 15:35:00 +0100 Subject: [PATCH 010/264] Debug info --- src/Interpreters/Cache/FileSegment.cpp | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index f46bbbc3630..192ba635e16 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -784,7 +784,7 @@ bool FileSegment::assertCorrectness() const return assertCorrectnessUnlocked(lockFileSegment()); } -bool FileSegment::assertCorrectnessUnlocked(const FileSegmentGuard::Lock &) const +bool FileSegment::assertCorrectnessUnlocked(const FileSegmentGuard::Lock & lock) const { auto check_iterator = [this](const Priority::Iterator & it) { @@ -799,12 +799,19 @@ bool FileSegment::assertCorrectnessUnlocked(const FileSegmentGuard::Lock &) cons chassert(entry.offset == offset()); }; + const auto file_path = getPathInLocalCache(); { - std::lock_guard lock(write_mutex); + std::lock_guard lk(write_mutex); if (downloaded_size == 0) - chassert(!fs::exists(getPathInLocalCache())); + { + if (fs::exists(file_path)) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected file {} not to exist ({})", + file_path, getInfoForLogUnlocked(lock)); + } + } else - chassert(fs::exists(getPathInLocalCache())); + chassert(fs::exists(file_path)); } if (download_state == State::DOWNLOADED) From 5c5c4622d848480ef7c3a3a5a61d35b627607d23 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 11 Dec 2023 13:44:39 +0100 Subject: [PATCH 011/264] Fxi --- src/Interpreters/Cache/WriteBufferToFileSegment.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp index 15a80667cc4..bf13f501994 100644 --- a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp +++ b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp @@ -57,6 +57,9 @@ void WriteBufferToFileSegment::nextImpl() reserve_stat_msg += fmt::format("{} hold {}, can release {}; ", toString(kind), ReadableSize(stat.non_releasable_size), ReadableSize(stat.releasable_size)); + if (std::filesystem::exists(file_segment->getPathInLocalCache())) + std::filesystem::remove(file_segment->getPathInLocalCache()); + throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, "Failed to reserve {} bytes for {}: {}(segment info: {})", bytes_to_write, file_segment->getKind() == FileSegmentKind::Temporary ? "temporary file" : "the file in cache", From bef27366c6648830089e39db939e9fdc71497883 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 11 Dec 2023 20:14:01 +0100 Subject: [PATCH 012/264] Increase log level to debug --- tests/config/install.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/install.sh b/tests/config/install.sh index b4e2b2484b9..a7a3c226c5d 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -42,7 +42,7 @@ ln -sf $SRC_PATH/config.d/transactions.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/encryption.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/CORS.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/zookeeper_log.xml $DEST_SERVER_PATH/config.d/ -ln -sf $SRC_PATH/config.d/logger_trace.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/logger_test.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/named_collection.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/ssl_certs.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/filesystem_cache_log.xml $DEST_SERVER_PATH/config.d/ From 2d5a5c6192df3cf5368339c9d7f806d22016ea77 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 18 Dec 2023 12:32:32 +0100 Subject: [PATCH 013/264] Fxi --- src/Interpreters/Cache/FileSegment.cpp | 24 +++++++++++++++--------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 192ba635e16..8e58a41bffb 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -786,32 +786,38 @@ bool FileSegment::assertCorrectness() const bool FileSegment::assertCorrectnessUnlocked(const FileSegmentGuard::Lock & lock) const { - auto check_iterator = [this](const Priority::Iterator & it) + auto throw_logical = [&](const std::string & error) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "{}. File segment info: {}", error, getInfoForLogUnlocked(lock)); + }; + + auto check_iterator = [&](const Priority::Iterator & it) { UNUSED(this); if (!it) return; const auto & entry = it->getEntry(); - UNUSED(entry); - chassert(entry.size == reserved_size); + if (entry.size != reserved_size) + throw_logical(fmt::format("Expected entry.size == reserved_size ({} == {})", entry.size, reserved_size)); + chassert(entry.key == key()); chassert(entry.offset == offset()); }; const auto file_path = getPathInLocalCache(); + if (segment_kind != FileSegmentKind::Temporary) { std::lock_guard lk(write_mutex); if (downloaded_size == 0) { if (fs::exists(file_path)) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected file {} not to exist ({})", - file_path, getInfoForLogUnlocked(lock)); - } + throw_logical("Expected file " + file_path + " not to exist"); + } + else if (!fs::exists(file_path)) + { + throw_logical("Expected file " + file_path + " to exist"); } - else - chassert(fs::exists(file_path)); } if (download_state == State::DOWNLOADED) From f2da31f7120d81ab9a70c45c506da3c9b7c49b05 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 18 Dec 2023 12:36:57 +0100 Subject: [PATCH 014/264] Safer --- src/Interpreters/Cache/Metadata.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 7c93acbff76..ceb54ab4955 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -783,7 +783,7 @@ KeyMetadata::iterator LockedKey::removeFileSegmentImpl(KeyMetadata::iterator it, try { const auto path = key_metadata->getFileSegmentPath(*file_segment); - if (file_segment->downloaded_size == 0) + if (file_segment->downloaded_size == 0 && file_segment->segment_kind != FileSegmentKind::Temporary) { chassert(!fs::exists(path)); } From 95a097864881a5c9440905bcbe8149d38bb355a1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 18 Dec 2023 18:42:52 +0100 Subject: [PATCH 015/264] Fxi --- src/Interpreters/Cache/Metadata.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index ceb54ab4955..0aca2343fdc 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -783,7 +783,14 @@ KeyMetadata::iterator LockedKey::removeFileSegmentImpl(KeyMetadata::iterator it, try { const auto path = key_metadata->getFileSegmentPath(*file_segment); - if (file_segment->downloaded_size == 0 && file_segment->segment_kind != FileSegmentKind::Temporary) + if (file_segment->segment_kind == FileSegmentKind::Temporary) + { + /// FIXME: For temporary file segment the requirement is not as strong because + /// the implementation of "temporary data in cache" creates files in advance. + if (fs::exists(path)) + fs::remove(path); + } + else if (file_segment->downloaded_size == 0) { chassert(!fs::exists(path)); } From 67d3f844093e26e15a6739685a29c2b4cb4f3fcf Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 19 Dec 2023 10:11:13 +0100 Subject: [PATCH 016/264] Try fxi --- src/Interpreters/Cache/FileSegment.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index a7ab0e85637..ca2112e526b 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -402,7 +402,9 @@ void FileSegment::write(const char * from, size_t size, size_t offset) const bool is_no_space_left_error = code == /* No space left on device */28 || code == /* Quota exceeded */122; auto lock = lockFileSegment(); + e.addMessage(fmt::format("{}, current cache state: {}", e.what(), getInfoForLogUnlocked(lock))); + setDownloadFailedUnlocked(lock); if (downloaded_size == 0 && fs::exists(file_segment_path)) { @@ -420,7 +422,6 @@ void FileSegment::write(const char * from, size_t size, size_t offset) downloaded_size = file_size; } - setDownloadFailedUnlocked(lock); throw; } From 319ae440b6ba09b1dc21b355fab22a99d073592c Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 19 Dec 2023 16:43:30 +0000 Subject: [PATCH 017/264] Implement Variant data type --- docs/en/operations/settings/settings.md | 52 + docs/en/sql-reference/data-types/variant.md | 217 ++ .../functions/other-functions.md | 36 + src/Columns/ColumnNullable.cpp | 22 +- src/Columns/ColumnNullable.h | 1 + src/Columns/ColumnVariant.cpp | 1360 +++++++++ src/Columns/ColumnVariant.h | 306 ++ src/Columns/IColumn.cpp | 6 + src/Columns/IColumn.h | 14 + src/Columns/MaskOperations.cpp | 6 +- src/Columns/MaskOperations.h | 2 +- src/Columns/tests/gtest_column_variant.cpp | 688 +++++ src/Core/Settings.h | 2 + src/Core/TypeId.h | 1 + src/DataTypes/DataTypeFactory.cpp | 1 + src/DataTypes/DataTypeFactory.h | 1 + src/DataTypes/DataTypeNullable.cpp | 28 + src/DataTypes/DataTypeNullable.h | 3 + src/DataTypes/DataTypeTuple.cpp | 9 +- src/DataTypes/DataTypeVariant.cpp | 197 ++ src/DataTypes/DataTypeVariant.h | 64 + src/DataTypes/EnumValues.cpp | 21 + src/DataTypes/EnumValues.h | 8 +- src/DataTypes/IDataType.cpp | 23 +- src/DataTypes/IDataType.h | 3 + .../Serializations/ISerialization.cpp | 61 +- src/DataTypes/Serializations/ISerialization.h | 13 + .../Serializations/SerializationArray.cpp | 138 +- .../Serializations/SerializationArray.h | 3 + .../Serializations/SerializationBool.cpp | 179 +- .../Serializations/SerializationBool.h | 8 +- .../SerializationCustomSimpleText.cpp | 56 + .../SerializationCustomSimpleText.h | 6 + .../Serializations/SerializationDate.cpp | 46 + .../Serializations/SerializationDate.h | 5 + .../Serializations/SerializationDate32.cpp | 45 + .../Serializations/SerializationDate32.h | 5 + .../Serializations/SerializationDateTime.cpp | 157 +- .../Serializations/SerializationDateTime.h | 5 + .../SerializationDateTime64.cpp | 112 + .../Serializations/SerializationDateTime64.h | 6 + .../Serializations/SerializationDecimal.cpp | 46 +- .../Serializations/SerializationDecimal.h | 6 +- .../Serializations/SerializationEnum.cpp | 97 + .../Serializations/SerializationEnum.h | 13 + .../SerializationFixedString.cpp | 56 + .../Serializations/SerializationFixedString.h | 6 + .../SerializationIPv4andIPv6.cpp | 188 ++ .../Serializations/SerializationIPv4andIPv6.h | 129 +- .../SerializationLowCardinality.cpp | 47 +- .../SerializationLowCardinality.h | 12 + .../Serializations/SerializationMap.cpp | 108 +- .../Serializations/SerializationMap.h | 7 +- .../Serializations/SerializationNamed.cpp | 1 + .../Serializations/SerializationNothing.h | 1 + .../Serializations/SerializationNullable.cpp | 532 +++- .../Serializations/SerializationNullable.h | 53 +- .../Serializations/SerializationNumber.cpp | 80 +- .../Serializations/SerializationNumber.h | 3 + .../Serializations/SerializationString.cpp | 101 +- .../Serializations/SerializationString.h | 5 + .../Serializations/SerializationTuple.cpp | 318 ++- .../Serializations/SerializationTuple.h | 12 + .../Serializations/SerializationUUID.cpp | 41 +- .../Serializations/SerializationUUID.h | 6 +- .../Serializations/SerializationVariant.cpp | 828 ++++++ .../Serializations/SerializationVariant.h | 116 + .../SerializationVariantElement.cpp | 241 ++ .../SerializationVariantElement.h | 87 + .../Serializations/SerializationWrapper.cpp | 25 + .../Serializations/SerializationWrapper.h | 5 + .../Serializations/SimpleTextSerialization.h | 38 + src/DataTypes/Utils.cpp | 1 + src/Databases/DatabaseReplicated.cpp | 1 + src/Formats/EscapingRuleUtils.cpp | 10 +- src/Formats/JSONUtils.cpp | 4 +- src/Formats/SchemaInferenceUtils.cpp | 2 +- src/Functions/FunctionsConversion.h | 262 +- src/Functions/if.cpp | 50 +- src/Functions/isNotNull.cpp | 13 + src/Functions/isNull.cpp | 13 + src/Functions/multiIf.cpp | 10 + src/Functions/variantElement.cpp | 238 ++ src/IO/ReadHelpers.cpp | 298 +- src/IO/ReadHelpers.h | 198 +- src/IO/readDecimalText.h | 20 + src/Interpreters/InterpreterCreateQuery.cpp | 14 + src/Interpreters/InterpreterInsertQuery.cpp | 2 +- src/Interpreters/inplaceBlockConversions.cpp | 35 +- .../parseColumnsListForTableFunction.cpp | 11 + .../parseColumnsListForTableFunction.h | 2 + src/Parsers/ExpressionElementParsers.cpp | 2 +- .../Formats/Impl/CSVRowInputFormat.cpp | 2 +- .../Formats/Impl/MySQLDumpRowInputFormat.cpp | 2 +- .../Formats/Impl/TSKVRowInputFormat.cpp | 2 +- .../Impl/TabSeparatedRowInputFormat.cpp | 4 +- .../Formats/Impl/ValuesBlockInputFormat.cpp | 2 +- ...940_variant_text_deserialization.reference | 516 ++++ .../02940_variant_text_deserialization.sql | 266 ++ .../02941_variant_type_1.reference | 2472 +++++++++++++++++ .../0_stateless/02941_variant_type_1.sh | 124 + .../02941_variant_type_2.reference | 51 + .../0_stateless/02941_variant_type_2.sh | 71 + .../02941_variant_type_3.reference | 51 + .../0_stateless/02941_variant_type_3.sh | 71 + .../02941_variant_type_4.reference | 56 + .../0_stateless/02941_variant_type_4.sh | 66 + .../0_stateless/02942_variant_cast.reference | 25 + .../0_stateless/02942_variant_cast.sql | 23 + .../02943_variant_element.reference | 44 + .../0_stateless/02943_variant_element.sql | 16 + ...44_variant_as_if_multi_if_result.reference | 96 + .../02944_variant_as_if_multi_if_result.sql | 64 + 113 files changed, 11750 insertions(+), 584 deletions(-) create mode 100644 docs/en/sql-reference/data-types/variant.md create mode 100644 src/Columns/ColumnVariant.cpp create mode 100644 src/Columns/ColumnVariant.h create mode 100644 src/Columns/tests/gtest_column_variant.cpp create mode 100644 src/DataTypes/DataTypeVariant.cpp create mode 100644 src/DataTypes/DataTypeVariant.h create mode 100644 src/DataTypes/Serializations/SerializationIPv4andIPv6.cpp create mode 100644 src/DataTypes/Serializations/SerializationVariant.cpp create mode 100644 src/DataTypes/Serializations/SerializationVariant.h create mode 100644 src/DataTypes/Serializations/SerializationVariantElement.cpp create mode 100644 src/DataTypes/Serializations/SerializationVariantElement.h create mode 100644 src/Functions/variantElement.cpp create mode 100644 tests/queries/0_stateless/02940_variant_text_deserialization.reference create mode 100644 tests/queries/0_stateless/02940_variant_text_deserialization.sql create mode 100644 tests/queries/0_stateless/02941_variant_type_1.reference create mode 100755 tests/queries/0_stateless/02941_variant_type_1.sh create mode 100644 tests/queries/0_stateless/02941_variant_type_2.reference create mode 100755 tests/queries/0_stateless/02941_variant_type_2.sh create mode 100644 tests/queries/0_stateless/02941_variant_type_3.reference create mode 100755 tests/queries/0_stateless/02941_variant_type_3.sh create mode 100644 tests/queries/0_stateless/02941_variant_type_4.reference create mode 100755 tests/queries/0_stateless/02941_variant_type_4.sh create mode 100644 tests/queries/0_stateless/02942_variant_cast.reference create mode 100644 tests/queries/0_stateless/02942_variant_cast.sql create mode 100644 tests/queries/0_stateless/02943_variant_element.reference create mode 100644 tests/queries/0_stateless/02943_variant_element.sql create mode 100644 tests/queries/0_stateless/02944_variant_as_if_multi_if_result.reference create mode 100644 tests/queries/0_stateless/02944_variant_as_if_multi_if_result.sql diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index dc46a3f0dcd..dbf5bc341cc 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5134,3 +5134,55 @@ When set to `true` than for all s3 requests first two attempts are made with low When set to `false` than all attempts are made with identical timeouts. Default value: `true`. + +## allow_experimental_variant_type {#allow_experimental_variant_type} + +Allows creation of experimental [Variant](../../sql-reference/data-types/variant.md). + +Default value: `false`. + +## use_variant_when_no_common_type_in_if {#use_variant_when_no_common_type_in_if} + +Allows to use `Variant` type as a result type for [if](../../sql-reference/functions/conditional-functions.md/#if)/[multiIf](../../sql-reference/functions/conditional-functions.md/#multiif) functions when there is no common type for argument types. + +Example: + +```sql +SET use_variant_when_no_common_type_in_if = 1; +SELECT toTypeName(if(number % 2, number, range(number))) as variant_type FROM numbers(1); +SELECT if(number % 2, number, range(number)) as variant FROM numbers(5); +``` + +```text +┌─variant_type───────────────────┐ +│ Variant(Array(UInt64), UInt64) │ +└────────────────────────────────┘ +┌─variant───┐ +│ [] │ +│ 1 │ +│ [0,1] │ +│ 3 │ +│ [0,1,2,3] │ +└───────────┘ +``` + +```sql +SET use_variant_when_no_common_type_in_if = 1; +SELECT toTypeName(multiIf((number % 4) = 0, 42, (number % 4) = 1, [1, 2, 3], (number % 4) = 2, 'Hello, World!', NULL)) AS variant_type FROM numbers(1); +SELECT multiIf((number % 4) = 0, 42, (number % 4) = 1, [1, 2, 3], (number % 4) = 2, 'Hello, World!', NULL) AS variant FROM numbers(4); +``` + +```text +─variant_type─────────────────────────┐ +│ Variant(Array(UInt8), String, UInt8) │ +└──────────────────────────────────────┘ + +┌─variant───────┐ +│ 42 │ +│ [1,2,3] │ +│ Hello, World! │ +│ ᴺᵁᴸᴸ │ +└───────────────┘ +``` + +Default value: `false`. diff --git a/docs/en/sql-reference/data-types/variant.md b/docs/en/sql-reference/data-types/variant.md new file mode 100644 index 00000000000..34966d79079 --- /dev/null +++ b/docs/en/sql-reference/data-types/variant.md @@ -0,0 +1,217 @@ +--- +slug: /en/sql-reference/data-types/json +sidebar_position: 55 +sidebar_label: Variant +--- + +# Variant(T1, T2, T3, ...) + +This type represents a union of other data types. Type `Variant(T1, T2, ..., TN)` means that each row of this type +has a value of either type `T1` or `T2` or ... or `TN` or none of them (`NULL` value). + +The order of nested types doesn't matter: Variant(T1, T2) = Variant(T2, T1). +Nested types can be arbitrary types except Nullable(...), LowCardinality(Nullable(...)) and Variant(...) types. + +:::note +The Variant data type is an experimental feature. To use it, set `allow_experimental_variant_type = 1`. +::: + +## Creating Variant + +Using `Variant` type in table column definition: + +```sql +CREATE TABLE test (v Variant(UInt64, String, Array(UInt64))) ENGINE = Memory; +INSERT INTO test VALUES (NULL), (42), ('Hello, World!'), ([1, 2, 3]); +SELECT v FROM test; +``` + +```text +┌─v─────────────┐ +│ ᴺᵁᴸᴸ │ +│ 42 │ +│ Hello, World! │ +│ [1,2,3] │ +└───────────────┘ +``` + +Using CAST from ordinary columns: + +```sql +SELECT toTypeName(variant) as type_name, 'Hello, World!'::Variant(UInt64, String, Array(UInt64)) as variant; +``` + +```text +┌─type_name──────────────────────────────┬─variant───────┐ +│ Variant(Array(UInt64), String, UInt64) │ Hello, World! │ +└────────────────────────────────────────┴───────────────┘ +``` + +Using functions `if/multiIf` when arguments doesn't have common type (setting `use_variant_when_no_common_type_in_if` should be enabled for it): + +```sql +SET use_variant_when_no_common_type_in_if = 1; +SELECT if(number % 2, number, range(number)) as variant FROM numbers(5); +``` + +```text +┌─variant───┐ +│ [] │ +│ 1 │ +│ [0,1] │ +│ 3 │ +│ [0,1,2,3] │ +└───────────┘ +``` + +```sql +SET use_variant_when_no_common_type_in_if = 1; +SELECT multiIf((number % 4) = 0, 42, (number % 4) = 1, [1, 2, 3], (number % 4) = 2, 'Hello, World!', NULL) AS variant FROM numbers(4); +``` + +```text +┌─variant───────┐ +│ 42 │ +│ [1,2,3] │ +│ Hello, World! │ +│ ᴺᵁᴸᴸ │ +└───────────────┘ +``` + +## Reading Variant nested types as subcolumns + +Variant type supports reading a single nested type from a Variant column using the type name as a subcolumn. +So, if you have column `variant Variant(T1, T2, T3)` you can read a subcolumn of type `T2` using syntax `variant.T2`, +this subcolumn will have type `Nullable(T2)` if `T2` can be inside `Nullable` and `T2` otherwise. This subcolumn will +be the same size as original `Variant` column and will contain `NULL` values (or empty values if `T2` cannot be inside `Nullable`) +in all rows in which original `Variant` column doesn't have type `T2`. + +Variant subcolumns can be also read using function `variantElement(variant_column, type_name)`. + +Examples: + +```sql +CREATE TABLE test (v Variant(UInt64, String, Array(UInt64))) ENGINE = Memory; +INSERT INTO test VALUES (NULL), (42), ('Hello, World!'), ([1, 2, 3]); +SELECT v, v.String, v.UInt64, v.`Array(UInt64)` FROM test; +``` + +```text +┌─v─────────────┬─v.String──────┬─v.UInt64─┬─v.Array(UInt64)─┐ +│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ [] │ +│ 42 │ ᴺᵁᴸᴸ │ 42 │ [] │ +│ Hello, World! │ Hello, World! │ ᴺᵁᴸᴸ │ [] │ +│ [1,2,3] │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ [1,2,3] │ +└───────────────┴───────────────┴──────────┴─────────────────┘ +``` + +```sql +SELECT toTypeName(v.String), toTypeName(v.UInt64), toTypeName(v.`Array(UInt64)`) FROM test LIMIT 1; +``` + +```text +┌─toTypeName(v.String)─┬─toTypeName(v.UInt64)─┬─toTypeName(v.Array(UInt64))─┐ +│ Nullable(String) │ Nullable(UInt64) │ Array(UInt64) │ +└──────────────────────┴──────────────────────┴─────────────────────────────┘ +``` + +```sql +SELECT v, variantElement(v, 'String'), variantElement(v, 'UInt64'), variantElement(v, 'Array(UInt64)') FROM test; +``` + +```text +┌─v─────────────┬─variantElement(v, 'String')─┬─variantElement(v, 'UInt64')─┬─variantElement(v, 'Array(UInt64)')─┐ +│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ [] │ +│ 42 │ ᴺᵁᴸᴸ │ 42 │ [] │ +│ Hello, World! │ Hello, World! │ ᴺᵁᴸᴸ │ [] │ +│ [1,2,3] │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ [1,2,3] │ +└───────────────┴─────────────────────────────┴─────────────────────────────┴────────────────────────────────────┘ +``` + +## Conversion between Variant column and other columns + +There are 3 possible conversions that can be performed with Variant column. + +### Converting an ordinary column to a Variant column + +It is possible to convert ordinary column with type `T` to a `Variant` column containing this type: + +```sql +SELECT toTypeName(variant) as type_name, 'Hello, World!'::Variant(UInt64, String, Array(UInt64)) as variant; +``` + +```text +┌─type_name──────────────────────────────┬─variant───────┐ +│ Variant(Array(UInt64), String, UInt64) │ Hello, World! │ +└────────────────────────────────────────┴───────────────┘ +``` + +### Converting a Variant column to an ordinary column + +It is possible to convert a `Variant` column to an ordinary column. In this case all nested variants will be converted to a destination type: + +```sql +CREATE TABLE test (v Variant(UInt64, String)) ENGINE = Memory; +INSERT INTO test VALUES (NULL), (42), ('42.42'); +SELECT v::Nullable(Float64) FROM test; +``` + +```text +┌─CAST(v, 'Nullable(Float64)')─┐ +│ ᴺᵁᴸᴸ │ +│ 42 │ +│ 42.42 │ +└──────────────────────────────┘ +``` + +### Converting a Variant to another Variant + +It is possible to convert a `Variant` column to another `Variant` column, but only if the destination `Variant` column contains all nested types from the original `Variant`: + +```sql +CREATE TABLE test (v Variant(UInt64, String)) ENGINE = Memory; +INSERT INTO test VALUES (NULL), (42), ('String'); +SELECT v::Variant(UInt64, String, Array(UInt64)) FROM test; +``` + +```text +┌─CAST(v, 'Variant(UInt64, String, Array(UInt64))')─┐ +│ ᴺᵁᴸᴸ │ +│ 42 │ +│ String │ +└───────────────────────────────────────────────────┘ +``` + + +## Reading Variant type from the data + +All text formats (TSV, CSV, CustomSeparated, Values, JSONEachRow, etc) supports reading `Variant` type. During data parsing ClickHouse tries to insert value into most appropriate variant type. + +Example: + +```sql +SELECT + v, + variantElement(v, 'String') AS str, + variantElement(v, 'UInt64') AS num, + variantElement(v, 'Float64') AS float, + variantElement(v, 'DateTime') AS date, + variantElement(v, 'Array(UInt64)') AS arr +FROM format(JSONEachRow, 'v Variant(String, UInt64, Float64, DateTime, Array(UInt64))', $$ +{"v" : "Hello, World!"}, +{"v" : 42}, +{"v" : 42.42}, +{"v" : "2020-01-01 00:00:00"}, +{"v" : [1, 2, 3]} +$$) +``` + +```text +┌─v───────────────────┬─str───────────┬──num─┬─float─┬────────────────date─┬─arr─────┐ +│ Hello, World! │ Hello, World! │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ [] │ +│ 42 │ ᴺᵁᴸᴸ │ 42 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ [] │ +│ 42.42 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 42.42 │ ᴺᵁᴸᴸ │ [] │ +│ 2020-01-01 00:00:00 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 2020-01-01 00:00:00 │ [] │ +│ [1,2,3] │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ [1,2,3] │ +└─────────────────────┴───────────────┴──────┴───────┴─────────────────────┴─────────┘ +``` diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 5b9d01985dd..47b5ac7b724 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2831,3 +2831,39 @@ Result: │ SELECT a, b FROM tab WHERE (a > 3) AND (b < 3) │ └─────────────────────────────────────────────────────────────────────────┘ ``` + +## variantElement + +Extracts a column with specified type from a `Variant` column. + +**Syntax** + +``` sql +tupleElement(variant, type_name, [, default_value]) +``` + +- `variant` — Variant column. [Variant](../../sql-reference/data-types/variant.md). +- `type_name` — The name of the variant type to extract. [String](../../sql-reference/data-types/string.md). +- `default_value` - The default value that will be used if variant doesn't have variant with specified type. Can be any type. Optional. + +**Returned value** + +- Subcolumn of a `Variant` column with specified type. + +**Example** + +```sql +CREATE TABLE test (v Variant(UInt64, String, Array(UInt64))) ENGINE = Memory; +INSERT INTO test VALUES (NULL), (42), ('Hello, World!'), ([1, 2, 3]); +SELECT v, variantElement(v, 'String'), variantElement(v, 'UInt64'), variantElement(v, 'Array(UInt64)') FROM test; +``` + +```text +┌─v─────────────┬─variantElement(v, 'String')─┬─variantElement(v, 'UInt64')─┬─variantElement(v, 'Array(UInt64)')─┐ +│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ [] │ +│ 42 │ ᴺᵁᴸᴸ │ 42 │ [] │ +│ Hello, World! │ Hello, World! │ ᴺᵁᴸᴸ │ [] │ +│ [1,2,3] │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ [1,2,3] │ +└───────────────┴─────────────────────────────┴─────────────────────────────┴────────────────────────────────────┘ +``` + diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 4ee6bb3d586..d2a579d6800 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -890,10 +890,7 @@ ColumnPtr makeNullable(const ColumnPtr & column) ColumnPtr makeNullableOrLowCardinalityNullable(const ColumnPtr & column) { - if (isColumnNullable(*column)) - return column; - - if (isColumnLowCardinalityNullable(*column)) + if (isColumnNullableOrLowCardinalityNullable(*column)) return column; if (isColumnConst(*column)) @@ -919,4 +916,21 @@ ColumnPtr makeNullableSafe(const ColumnPtr & column) return column; } +ColumnPtr makeNullableOrLowCardinalityNullableSafe(const ColumnPtr & column) +{ + if (isColumnNullableOrLowCardinalityNullable(*column)) + return column; + + if (isColumnConst(*column)) + return ColumnConst::create(makeNullableOrLowCardinalityNullableSafe(assert_cast(*column).getDataColumnPtr()), column->size()); + + if (column->lowCardinality()) + return assert_cast(*column).cloneNullable(); + + if (column->canBeInsideNullable()) + return makeNullableSafe(column); + + return column; +} + } diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index b57fdf3064d..60c7750f8fc 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -231,5 +231,6 @@ private: ColumnPtr makeNullable(const ColumnPtr & column); ColumnPtr makeNullableSafe(const ColumnPtr & column); ColumnPtr makeNullableOrLowCardinalityNullable(const ColumnPtr & column); +ColumnPtr makeNullableOrLowCardinalityNullableSafe(const ColumnPtr & column); } diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp new file mode 100644 index 00000000000..67754e77992 --- /dev/null +++ b/src/Columns/ColumnVariant.cpp @@ -0,0 +1,1360 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int ILLEGAL_COLUMN; + extern const int NOT_IMPLEMENTED; + extern const int LOGICAL_ERROR; + extern const int PARAMETER_OUT_OF_BOUND; + extern const int SIZES_OF_NESTED_COLUMNS_ARE_INCONSISTENT; + extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; +} + +std::string ColumnVariant::getName() const +{ + WriteBufferFromOwnString res; + res << "Variant("; + bool is_first = true; + for (const auto & local_variant : global_to_local_discriminators) + { + if (!is_first) + res << ", "; + is_first = false; + res << variants[local_variant]->getName(); + } + res << ")"; + return res.str(); +} + + +void ColumnVariant::initIdentityGlobalToLocalDiscriminatorsMapping() +{ + local_to_global_discriminators.reserve(variants.size()); + global_to_local_discriminators.reserve(variants.size()); + for (size_t i = 0; i != variants.size(); ++i) + { + local_to_global_discriminators.push_back(i); + global_to_local_discriminators.push_back(i); + } +} + +ColumnVariant::ColumnVariant(MutableColumns && variants_) : ColumnVariant(std::move(variants_), {}) +{ +} + +ColumnVariant::ColumnVariant(MutableColumns && variants_, const std::vector & local_to_global_discriminators_) +{ + /// Empty local_to_global_discriminators mapping means that variants are already in the global order. + if (!local_to_global_discriminators_.empty() && local_to_global_discriminators_.size() != variants_.size()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "The number of values in local_to_global_discriminators mapping ({}) doesn't match the number of variants ({})", + local_to_global_discriminators_.size(), + variants_.size()); + + /// As variants are empty, column with local discriminators will be also empty and we can reorder variants according to global discriminators. + variants.resize(variants_.size()); + for (size_t i = 0; i != variants_.size(); ++i) + { + if (isColumnConst(*variants_[i])) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "ColumnVariant cannot have ColumnConst as its element"); + + if (!variants_[i]->empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Not empty column passed to ColumnVariant, but no local_discriminators passed"); + + if (!local_to_global_discriminators_.empty() && local_to_global_discriminators_[i] > variants_.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid global discriminator {}. The number of variants: {}", UInt64(local_to_global_discriminators_[i]), variants_.size()); + + if (local_to_global_discriminators_.empty()) + variants[i] = std::move(variants_[i]); + else + variants[local_to_global_discriminators_[i]] = std::move(variants_[i]); + } + + local_discriminators = ColumnDiscriminators::create(); + offsets = ColumnOffsets::create(); + + /// Now global and local discriminators are the same. + initIdentityGlobalToLocalDiscriminatorsMapping(); +} + +ColumnVariant::ColumnVariant(MutableColumnPtr local_discriminators_, MutableColumns && variants_) : ColumnVariant(std::move(local_discriminators_), nullptr, std::move(variants_), {}) +{ +} + +ColumnVariant::ColumnVariant(MutableColumnPtr local_discriminators_, MutableColumns && variants_, const std::vector & global_discriminators) : ColumnVariant(std::move(local_discriminators_), nullptr, std::move(variants_), global_discriminators) +{ +} + +ColumnVariant::ColumnVariant(DB::MutableColumnPtr local_discriminators_, DB::MutableColumnPtr offsets_, DB::MutableColumns && variants_) : ColumnVariant(std::move(local_discriminators_), std::move(offsets_), std::move(variants_), {}) +{ +} + +ColumnVariant::ColumnVariant(DB::MutableColumnPtr local_discriminators_, DB::MutableColumnPtr offsets_, DB::MutableColumns && variants_, const std::vector & local_to_global_discriminators_) +{ + if (variants_.size() > MAX_NESTED_COLUMNS) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Variant type with more than {} nested types is not allowed", ColumnVariant::MAX_NESTED_COLUMNS); + + local_discriminators = std::move(local_discriminators_); + const ColumnDiscriminators * discriminators_concrete = typeid_cast(local_discriminators.get()); + if (!discriminators_concrete) + throw Exception(ErrorCodes::LOGICAL_ERROR, "discriminator column must be a ColumnUInt8"); + + variants.reserve(variants_.size()); + size_t total_size = 0; + for (auto & variant : variants_) + { + if (isColumnConst(*variant)) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "ColumnVariant cannot have ColumnConst as its element"); + + total_size += variant->size(); + variants.push_back(std::move(variant)); + } + + /// We can have more discriminators than values in columns + /// (because of NULL discriminators), but not less. + if (total_size > local_discriminators->size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Nested columns sizes are inconsistent with local_discriminators column size. Total column sizes: {}, local_discriminators size: {}", total_size, local_discriminators->size()); + + if (offsets_) + { + if (!typeid_cast(offsets_.get())) + throw Exception(ErrorCodes::LOGICAL_ERROR, "offsets column must be a ColumnUInt64"); + + offsets = std::move(offsets_); + } + else + { + /// If no offsets column was provided, construct offsets based on discriminators. + offsets = ColumnOffsets::create(); + Offsets & offsets_data = typeid_cast(offsets.get())->getData(); + offsets_data.reserve(discriminators_concrete->size()); + /// If we have only NULLs, offsets column will not contain any real offsets. + if (hasOnlyNulls()) + { + offsets_data.resize(discriminators_concrete->size()); + } + /// If we have only one non empty variant and no NULLs, + /// offsets column will contain just sequential offsets 0, 1, 2, ... + else if (getLocalDiscriminatorOfOneNoneEmptyVariantNoNulls()) + { + for (size_t i = 0; i != discriminators_concrete->size(); ++i) + offsets_data.push_back(i); + } + /// Otherwise we should iterate through discriminators and + /// remember current offset for each variant column. + else + { + std::vector nested_offsets; + nested_offsets.resize(variants.size()); + for (Discriminator discr : discriminators_concrete->getData()) + { + if (discr == NULL_DISCRIMINATOR) + offsets_data.emplace_back(); + else + offsets_data.push_back(nested_offsets[discr]++); + } + } + } + + /// Empty global_discriminators means that variants are already in global order. + if (local_to_global_discriminators_.empty()) + { + initIdentityGlobalToLocalDiscriminatorsMapping(); + } + else + { + if (local_to_global_discriminators_.size() != variants.size()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "The number of values in local_to_global_discriminators mapping ({}) doesn't match the number of variants ({})", + local_to_global_discriminators_.size(), + variants.size()); + + local_to_global_discriminators = local_to_global_discriminators_; + global_to_local_discriminators.resize(local_to_global_discriminators.size()); + /// Create mapping global discriminator -> local discriminator + for (size_t i = 0; i != local_to_global_discriminators.size(); ++i) + { + if (local_to_global_discriminators[i] > variants.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid global discriminator {}. The number of variants: {}", UInt64(local_to_global_discriminators[i]), variants_.size()); + + global_to_local_discriminators[local_to_global_discriminators[i]] = i; + } + } +} + +ColumnVariant::Ptr ColumnVariant::create(const Columns & variants, const std::vector & local_to_global_discriminators) +{ + MutableColumns mutable_variants; + mutable_variants.reserve(variants.size()); + for (const auto & variant : variants) + { + if (isColumnConst(*variant)) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "ColumnVariant cannot have ColumnConst as its element"); + mutable_variants.emplace_back(variant->assumeMutable()); + } + + return ColumnVariant::create(std::move(mutable_variants), local_to_global_discriminators); +} + +ColumnVariant::Ptr ColumnVariant::create(const DB::ColumnPtr & local_discriminators, const DB::Columns & variants, const std::vector & local_to_global_discriminators) +{ + MutableColumns mutable_variants; + mutable_variants.reserve(variants.size()); + for (const auto & variant : variants) + { + if (isColumnConst(*variant)) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "ColumnVariant cannot have ColumnConst as its element"); + mutable_variants.emplace_back(variant->assumeMutable()); + } + + return ColumnVariant::create(local_discriminators->assumeMutable(), std::move(mutable_variants), local_to_global_discriminators); +} + +ColumnVariant::Ptr ColumnVariant::create(const DB::ColumnPtr & local_discriminators, const DB::ColumnPtr & offsets, const DB::Columns & variants, const std::vector & local_to_global_discriminators) +{ + MutableColumns mutable_variants; + mutable_variants.reserve(variants.size()); + for (const auto & variant : variants) + { + if (isColumnConst(*variant)) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "ColumnVariant cannot have ColumnConst as its element"); + mutable_variants.emplace_back(variant->assumeMutable()); + } + + return ColumnVariant::create(local_discriminators->assumeMutable(), offsets->assumeMutable(), std::move(mutable_variants), local_to_global_discriminators); +} + +MutableColumnPtr ColumnVariant::cloneEmpty() const +{ + MutableColumns new_variants; + new_variants.reserve(variants.size()); + for (const auto & variant : variants) + new_variants.emplace_back(variant->cloneEmpty()); + + return ColumnVariant::create(std::move(new_variants), local_to_global_discriminators); +} + +MutableColumnPtr ColumnVariant::cloneResized(size_t new_size) const +{ + if (new_size == 0) + return cloneEmpty(); + + const size_t num_variants = variants.size(); + size_t size = local_discriminators->size(); + /// If new size is bigger than the old one, just clone column and append default values. + if (new_size >= size) + { + MutableColumns new_variants; + new_variants.reserve(num_variants); + for (const auto & variant : variants) + new_variants.emplace_back(IColumn::mutate(variant)); + + auto res = ColumnVariant::create(IColumn::mutate(local_discriminators), IColumn::mutate(offsets), std::move(new_variants), local_to_global_discriminators); + res->insertManyDefaults(new_size - size); + return res; + } + + /// If new size is less than current size, we should find the new size for all variants. + + /// Optimization for case when we have only NULLs. In this case we should just resize discriminators and offsets. + if (hasOnlyNulls()) + { + MutableColumns new_variants; + new_variants.reserve(num_variants); + for (const auto & variant : variants) + new_variants.emplace_back(IColumn::mutate(variant)); + + return ColumnVariant::create(local_discriminators->cloneResized(new_size), offsets->cloneResized(new_size), std::move(new_variants), local_to_global_discriminators); + } + + /// Optimization for case when there is only 1 non-empty variant and no NULLs. + /// In this case we can simply call cloneResized on this single variant, discriminators and offsets. + if (auto non_empty_local_discr = getLocalDiscriminatorOfOneNoneEmptyVariantNoNulls()) + { + MutableColumns new_variants; + new_variants.reserve(num_variants); + for (size_t i = 0; i != variants.size(); ++i) + { + if (i == *non_empty_local_discr) + new_variants.emplace_back(variants[i]->cloneResized(new_size)); + else + new_variants.emplace_back(variants[i]->cloneEmpty()); + } + + return ColumnVariant::create(local_discriminators->cloneResized(new_size), offsets->cloneResized(new_size), std::move(new_variants), local_to_global_discriminators); + } + + const auto & local_discriminators_data = getLocalDiscriminators(); + const auto & offsets_data = getOffsets(); + + /// We can find all variants sizes by scanning all new_size local_discriminators and calculating + /// sizes for all new variants. This code is below and commented. + +// std::vector new_nested_sizes(num_variants, 0); +// for (size_t i = 0; i != new_size; ++i) +// { +// Discriminator discr = local_discriminators_data[i]; +// if (discr != NULL_DISCRIMINATOR) +// ++new_nested_sizes[discr]; +// } +// +// MutableColumns new_variants; +// new_variants.reserve(num_variants); +// for (size_t i = 0; i != num_variants; ++i) +// { +// if (new_nested_sizes[i]) +// new_variants.emplace_back(variants[i]->cloneResized(new_nested_sizes[i])); +// else +// new_variants.emplace_back(variants[i]->cloneEmpty()); +// } +// +// return ColumnVariant::create(local_discriminators->cloneResized(new_size), offsets->cloneResized(new_size), std::move(new_variants), local_to_global_discriminators); + + /// But instead we are trying to optimize it using offsets column: + /// For all non-empty variants we are trying to find last occurrence of its discriminator in local_discriminators[:new_size] or + /// first occurrence in local_discriminators[new_size:]. The same row in offsets column will contain the desired size (or size - 1) of variant. + /// All empty variants will remain empty. + /// Not sure how good this optimization is, feel free to remove it and use simpler version above. + + MutableColumns new_variants(num_variants); + std::unordered_set seen_variants; + /// First, check which variants are empty. They will remain empty. + for (Discriminator i = 0; i != num_variants; ++i) + { + if (variants[i]->empty()) + { + seen_variants.insert(i); + new_variants[i] = variants[i]->cloneEmpty(); + } + } + + /// Now, iterate through local discriminators using two pointers. + /// First will go from new_size - 1 to 0, second from new_size to size. + /// Finish when we find all variants or hit lower or upper bound. + ssize_t i = new_size - 1; + size_t j = new_size; + while (i != -1 && j != size) + { + Discriminator i_discr = local_discriminators_data[i]; + if (i_discr != NULL_DISCRIMINATOR) + { + auto [_, inserted] = seen_variants.insert(i_discr); + /// If this is the first occurrence of this discriminator, + /// we can get new size for this variant. + if (inserted) + { + new_variants[i_discr] = variants[i_discr]->cloneResized(offsets_data[i] + 1); + if (seen_variants.size() == num_variants) + break; + } + } + + Discriminator j_discr = local_discriminators_data[j]; + if (j_discr != NULL_DISCRIMINATOR) + { + auto [_, inserted] = seen_variants.insert(j_discr); + /// If this is the first occurrence of this discriminator, + /// we can get new size for this variant. + if (inserted) + { + new_variants[j_discr] = variants[j_discr]->cloneResized(offsets_data[j]); + if (seen_variants.size() == num_variants) + break; + } + } + + --i; + ++j; + } + + /// We can finish in 3 cases: + /// 1) seen_variants.size() == num_variants - we found local_discriminators of all variants, nothing to do. + /// 2) i == -1 - we scanned all values in local_discriminators[:new_size]. Not found variants doesn't have + /// values in local_discriminators[:new_size], so they should be empty in the resized version. + /// 3) j == size - we scanned all values in local_discriminators[new_size:]. Not found variants doesn't have + /// values in local_discriminators[new_size:], so, we should use the full variant in the resized version. + if (seen_variants.size() != num_variants) + { + for (size_t discr = 0; discr != num_variants; ++discr) + { + if (!seen_variants.contains(discr)) + { + if (i == -1) + new_variants[discr] = variants[discr]->cloneEmpty(); + else + new_variants[discr] = IColumn::mutate(variants[discr]); + } + } + } + + return ColumnVariant::create(local_discriminators->cloneResized(new_size), offsets->cloneResized(new_size), std::move(new_variants), local_to_global_discriminators); +} + +Field ColumnVariant::operator[](size_t n) const +{ + Discriminator discr = localDiscriminatorAt(n); + if (discr == NULL_DISCRIMINATOR) + return Null(); + return (*variants[discr])[offsetAt(n)]; +} + +void ColumnVariant::get(size_t n, Field & res) const +{ + Discriminator discr = localDiscriminatorAt(n); + if (discr == NULL_DISCRIMINATOR) + res = Null(); + else + variants[discr]->get(offsetAt(n), res); +} + +bool ColumnVariant::isDefaultAt(size_t n) const +{ + return localDiscriminatorAt(n) == NULL_DISCRIMINATOR; +} + +bool ColumnVariant::isNullAt(size_t n) const +{ + return localDiscriminatorAt(n) == NULL_DISCRIMINATOR; +} + +StringRef ColumnVariant::getDataAt(size_t) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getDataAt is not supported for {}", getName()); +} + +void ColumnVariant::insertData(const char *, size_t) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertData is not supported for {}", getName()); +} + +void ColumnVariant::insert(const Field & field) +{ + if (field.isNull()) + insertDefault(); + else + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot insert field {} to column {}", toString(field), getName()); +} + +void ColumnVariant::insertFrom(const IColumn & src_, size_t n) +{ + const ColumnVariant & src = assert_cast(src_); + + const size_t num_variants = variants.size(); + if (src.variants.size() != num_variants) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert value of Variant type with different number of types"); + + /// Remember that src column can have different local variants order. + Discriminator global_discr = src.globalDiscriminatorAt(n); + Discriminator local_discr = localDiscriminatorByGlobal(global_discr); + getLocalDiscriminators().push_back(local_discr); + if (local_discr == NULL_DISCRIMINATOR) + { + getOffsets().emplace_back(); + } + else + { + getOffsets().push_back(variants[local_discr]->size()); + variants[local_discr]->insertFrom(src.getVariantByGlobalDiscriminator(global_discr), src.offsetAt(n)); + } +} + +void ColumnVariant::insertRangeFrom(const IColumn & src_, size_t start, size_t length) +{ + const size_t num_variants = variants.size(); + const auto & src = assert_cast(src_); + if (src.variants.size() != num_variants) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert value of Variant type with different number of types"); + + if (start + length > src.getLocalDiscriminators().size()) + throw Exception(ErrorCodes::PARAMETER_OUT_OF_BOUND, "Parameter out of bound in ColumnVariant::insertRangeFrom method. " + "[start({}) + length({}) > local_discriminators.size({})]", start, length, src.getLocalDiscriminators().size()); + + /// If src column contains only NULLs, just insert NULLs. + if (src.hasOnlyNulls()) + { + insertManyDefaults(length); + return; + } + + /// Optimization for case when there is only 1 non-empty variant and no NULLs in src column. + /// In this case we can simply call insertRangeFrom on this single variant. + if (auto non_empty_src_local_discr = src.getLocalDiscriminatorOfOneNoneEmptyVariantNoNulls()) + { + auto local_discr = localDiscriminatorByGlobal(src.globalDiscriminatorByLocal(*non_empty_src_local_discr)); + size_t offset = variants[local_discr]->size(); + variants[local_discr]->insertRangeFrom(*src.variants[*non_empty_src_local_discr], start, length); + getLocalDiscriminators().resize_fill(local_discriminators->size() + length, local_discr); + auto & offsets_data = getOffsets(); + offsets_data.reserve(offsets_data.size() + length); + for (size_t i = 0; i != length; ++i) + offsets_data.push_back(offset++); + return; + } + + /// Iterate through src local_discriminators in range [start, start + length], + /// collect ranges we need to insert for all variants and update offsets. + /// nested_ranges[i].first - offset in src.variants[i] + /// nested_ranges[i].second - length in src.variants[i] + std::vector> nested_ranges(num_variants, {0, 0}); + auto & offsets_data = getOffsets(); + offsets_data.reserve(offsets_data.size() + length); + auto & local_discriminators_data = getLocalDiscriminators(); + local_discriminators_data.reserve(local_discriminators_data.size() + length); + const auto & src_offsets_data = src.getOffsets(); + const auto & src_local_discriminators_data = src.getLocalDiscriminators(); + for (size_t i = start; i != start + length; ++i) + { + /// We insert from src.variants[src_local_discr] to variants[local_discr] + Discriminator src_local_discr = src_local_discriminators_data[i]; + Discriminator local_discr = localDiscriminatorByGlobal(src.globalDiscriminatorByLocal(src_local_discr)); + local_discriminators_data.push_back(local_discr); + if (local_discr == NULL_DISCRIMINATOR) + { + offsets_data.emplace_back(); + } + else + { + /// If we see this discriminator for the first time, set its range start. + if (!nested_ranges[src_local_discr].second) + nested_ranges[src_local_discr].first = src_offsets_data[i]; + /// Update offsets column with correct offset. + offsets_data.push_back(variants[local_discr]->size() + nested_ranges[src_local_discr].second); + ++nested_ranges[src_local_discr].second; + } + } + + for (size_t src_local_discr = 0; src_local_discr != nested_ranges.size(); ++src_local_discr) + { + auto [nested_start, nested_length] = nested_ranges[src_local_discr]; + auto local_discr = localDiscriminatorByGlobal(src.globalDiscriminatorByLocal(src_local_discr)); + if (nested_length) + variants[local_discr]->insertRangeFrom(*src.variants[src_local_discr], nested_start, nested_length); + } +} + +void ColumnVariant::insertManyFrom(const DB::IColumn & src_, size_t position, size_t length) +{ + const size_t num_variants = variants.size(); + const auto & src = assert_cast(src_); + if (src.variants.size() != num_variants) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert value of Variant type with different number of types"); + + /// Remember that src column can have different local variants order. + Discriminator src_local_discr = src.localDiscriminatorAt(position); + Discriminator local_discr = localDiscriminatorByGlobal(src.globalDiscriminatorByLocal(src_local_discr)); + auto & local_discriminators_data = getLocalDiscriminators(); + local_discriminators_data.resize_fill(local_discriminators_data.size() + length, local_discr); + + auto & offsets_data = getOffsets(); + if (local_discr == NULL_DISCRIMINATOR) + { + offsets_data.resize_fill(offsets_data.size() + length); + } + else + { + size_t prev_offset = variants[local_discr]->size(); + offsets_data.reserve(offsets_data.size() + length); + for (size_t i = 0; i != length; ++i) + offsets_data.push_back(prev_offset + i); + + variants[local_discr]->insertManyFrom(*src.variants[src_local_discr], src.offsetAt(position), length); + } +} + +void ColumnVariant::insertDefault() +{ + getLocalDiscriminators().push_back(NULL_DISCRIMINATOR); + getOffsets().emplace_back(); +} + +void ColumnVariant::insertManyDefaults(size_t length) +{ + size_t size = local_discriminators->size(); + getLocalDiscriminators().resize_fill(size + length, NULL_DISCRIMINATOR); + getOffsets().resize_fill(size + length); +} + +void ColumnVariant::popBack(size_t n) +{ + /// If we have only NULLs, just pop back from local_discriminators and offsets. + if (hasOnlyNulls()) + { + local_discriminators->popBack(n); + offsets->popBack(n); + return; + } + + /// Optimization for case when there is only 1 non-empty variant and no NULLs. + /// In this case we can just popBack n elements from this variant. + if (auto non_empty_local_discr = getLocalDiscriminatorOfOneNoneEmptyVariantNoNulls()) + { + variants[*non_empty_local_discr]->popBack(n); + local_discriminators->popBack(n); + offsets->popBack(n); + return; + } + + /// Calculate how many rows we need to pop from each variant + auto & local_discriminators_data = getLocalDiscriminators(); + size_t size = local_discriminators_data.size(); + const size_t num_variants = variants.size(); + std::vector nested_n(num_variants, 0); + for (size_t i = 0; i != n; ++i) + { + Discriminator discr = local_discriminators_data[size - i - 1]; + if (discr != NULL_DISCRIMINATOR) + ++nested_n[discr]; + } + + for (size_t i = 0; i != num_variants; ++i) + { + if (nested_n[i]) + variants[i]->popBack(nested_n[i]); + } + + local_discriminators->popBack(n); + offsets->popBack(n); +} + +StringRef ColumnVariant::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin, const UInt8 *) const +{ + /// During any serialization/deserialization we should always use global discriminators. + Discriminator global_discr = globalDiscriminatorAt(n); + char * pos = arena.allocContinue(sizeof(global_discr), begin); + memcpy(pos, &global_discr, sizeof(global_discr)); + StringRef res(pos, sizeof(global_discr)); + + if (global_discr == NULL_DISCRIMINATOR) + return res; + + auto value_ref = variants[localDiscriminatorByGlobal(global_discr)]->serializeValueIntoArena(offsetAt(n), arena, begin); + res.data = value_ref.data - res.size; + res.size += value_ref.size; + + return res; +} + +const char * ColumnVariant::deserializeAndInsertFromArena(const char * pos) +{ + /// During any serialization/deserialization we should always use global discriminators. + Discriminator global_discr = unalignedLoad(pos); + pos += sizeof(global_discr); + Discriminator local_discr = localDiscriminatorByGlobal(global_discr); + getLocalDiscriminators().push_back(local_discr); + if (local_discr == NULL_DISCRIMINATOR) + { + getOffsets().emplace_back(); + return pos; + } + + getOffsets().push_back(variants[local_discr]->size()); + return variants[local_discr]->deserializeAndInsertFromArena(pos); +} + +const char * ColumnVariant::skipSerializedInArena(const char * pos) const +{ + Discriminator global_discr = unalignedLoad(pos); + pos += sizeof(global_discr); + if (global_discr == NULL_DISCRIMINATOR) + return pos; + + return variants[localDiscriminatorByGlobal(global_discr)]->skipSerializedInArena(pos); +} + +void ColumnVariant::updateHashWithValue(size_t n, SipHash & hash) const +{ + Discriminator global_discr = globalDiscriminatorAt(n); + hash.update(global_discr); + if (global_discr != NULL_DISCRIMINATOR) + variants[localDiscriminatorByGlobal(global_discr)]->updateHashWithValue(offsetAt(n), hash); +} + +void ColumnVariant::updateWeakHash32(WeakHash32 & hash) const +{ + auto s = size(); + + if (hash.getData().size() != s) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match size of column: " + "column size is {}, hash size is {}", std::to_string(s), std::to_string(hash.getData().size())); + + /// If we have only NULLs, keep hash unchanged. + if (hasOnlyNulls()) + return; + + /// Optimization for case when there is only 1 non-empty variant and no NULLs. + /// In this case we can just calculate weak hash for this variant. + if (auto non_empty_local_discr = getLocalDiscriminatorOfOneNoneEmptyVariantNoNulls()) + { + variants[*non_empty_local_discr]->updateWeakHash32(hash); + return; + } + + /// Calculate weak hash for all variants. + std::vector nested_hashes; + for (const auto & variant : variants) + { + WeakHash32 nested_hash(variant->size()); + variant->updateWeakHash32(nested_hash); + nested_hashes.emplace_back(std::move(nested_hash)); + } + + /// For each row hash is a hash of corresponding row from corresponding variant. + auto & hash_data = hash.getData(); + const auto & local_discriminators_data = getLocalDiscriminators(); + const auto & offsets_data = getOffsets(); + for (size_t i = 0; i != local_discriminators_data.size(); ++i) + { + Discriminator discr = local_discriminators_data[i]; + /// Update hash only for non-NULL values + if (discr != NULL_DISCRIMINATOR) + { + auto nested_hash = nested_hashes[local_discriminators_data[i]].getData()[offsets_data[i]]; + hash_data[i] = static_cast(hashCRC32(nested_hash, hash_data[i])); + } + } +} + +void ColumnVariant::updateHashFast(SipHash & hash) const +{ + local_discriminators->updateHashFast(hash); + for (const auto & variant : variants) + variant->updateHashFast(hash); +} + +ColumnPtr ColumnVariant::filter(const Filter & filt, ssize_t result_size_hint) const +{ + if (size() != filt.size()) + throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "Size of filter ({}) doesn't match size of column ({})", filt.size(), size()); + + /// If we have only NULLs, just filter local_discriminators column. + if (hasOnlyNulls()) + { + Columns new_variants(variants.begin(), variants.end()); + auto new_discriminators = local_discriminators->filter(filt, result_size_hint); + /// In case of all NULL values offsets doesn't contain any useful values, just resize it. + ColumnPtr new_offsets = offsets->cloneResized(new_discriminators->size()); + return ColumnVariant::create(new_discriminators, new_offsets, new_variants, local_to_global_discriminators); + } + + /// Optimization for case when there is only 1 non-empty variant and no NULLs. + /// In this case we can just filter this variant and resize discriminators/offsets. + if (auto non_empty_discr = getLocalDiscriminatorOfOneNoneEmptyVariantNoNulls()) + { + Columns new_variants(variants.begin(), variants.end()); + new_variants[*non_empty_discr] = variants[*non_empty_discr]->filter(filt, result_size_hint); + size_t new_size = new_variants[*non_empty_discr]->size(); + ColumnPtr new_discriminators = local_discriminators->cloneResized(new_size); + ColumnPtr new_offsets = offsets->cloneResized(new_size); + return ColumnVariant::create(new_discriminators, new_offsets, new_variants, local_to_global_discriminators); + } + + /// We should create filter for each variant + /// according to local_discriminators and given filter. + const size_t num_variants = variants.size(); + std::vector nested_filters(num_variants); + for (size_t i = 0; i != num_variants; ++i) + nested_filters[i].reserve(variants[i]->size()); + + /// As we will iterate through local_discriminators anyway, we can count + /// result size for each variant. + std::vector variant_result_size_hints(num_variants); + + const auto & local_discriminators_data = getLocalDiscriminators(); + for (size_t i = 0; i != local_discriminators_data.size(); ++i) + { + Discriminator discr = local_discriminators_data[i]; + if (discr != NULL_DISCRIMINATOR) + { + nested_filters[local_discriminators_data[i]].push_back(filt[i]); + variant_result_size_hints[local_discriminators_data[i]] += !!(filt[i]); + } + } + + Columns new_variants; + new_variants.reserve(num_variants); + for (size_t i = 0; i != num_variants; ++i) + { + /// It make sense to call filter() on variant only if the result size is not 0. + if (variant_result_size_hints[i]) + new_variants.emplace_back(variants[i]->filter(nested_filters[i], variant_result_size_hints[i])); + else + new_variants.emplace_back(variants[i]->cloneEmpty()); + } + + /// We cannot use filtered offsets column, as it will be incorrect. + /// It will be reconstructed on ColumnVariant creation according to new local_discriminators. + return ColumnVariant::create(local_discriminators->filter(filt, result_size_hint), new_variants, local_to_global_discriminators); +} + +void ColumnVariant::expand(const Filter & mask, bool inverted) +{ + /// Expand local_discriminators using NULL_DISCRIMINATOR for 0-rows. + expandDataByMask(getLocalDiscriminators(), mask, inverted, NULL_DISCRIMINATOR); + expandDataByMask(getOffsets(), mask, inverted); +} + +ColumnPtr ColumnVariant::permute(const Permutation & perm, size_t limit) const +{ + /// If we have only NULLs, permutation will take no effect, just return resized column. + if (hasOnlyNulls()) + return cloneResized(limit); + + /// Optimization when we have only one non empty variant and no NULLs. + /// In this case local_discriminators column is filled with identical values and offsets column + /// filled with sequential numbers. In this case we can just apply permutation to this + /// single non-empty variant and cut local_discriminators and offsets columns to the result size. + if (auto non_empty_local_discr = getLocalDiscriminatorOfOneNoneEmptyVariantNoNulls()) + { + MutableColumns new_variants; + const size_t num_variants = variants.size(); + new_variants.reserve(num_variants); + for (size_t i = 0; i != num_variants; ++i) + { + if (i == *non_empty_local_discr) + new_variants.emplace_back(variants[*non_empty_local_discr]->permute(perm, limit)->assumeMutable()); + else + new_variants.emplace_back(variants[i]->assumeMutable()); + } + + size_t new_size = new_variants[*non_empty_local_discr]->size(); + return ColumnVariant::create(local_discriminators->cloneResized(new_size), offsets->cloneResized(new_size), std::move(new_variants), local_to_global_discriminators); + } + + return permuteImpl(*this, perm, limit); +} + +ColumnPtr ColumnVariant::index(const IColumn & indexes, size_t limit) const +{ + /// If we have only NULLs, index will take no effect, just return resized column. + if (hasOnlyNulls()) + return cloneResized(limit); + + /// Optimization when we have only one non empty variant and no NULLs. + /// In this case local_discriminators column is filled with identical values and offsets column + /// filled with sequential numbers. So we can just apply indexes to this + /// single non-empty variant and cut local_discriminators and offsets columns to the result size. + if (auto non_empty_local_discr = getLocalDiscriminatorOfOneNoneEmptyVariantNoNulls()) + { + MutableColumns new_variants; + const size_t num_variants = variants.size(); + new_variants.reserve(num_variants); + for (size_t i = 0; i != num_variants; ++i) + { + if (i == *non_empty_local_discr) + new_variants.emplace_back(variants[*non_empty_local_discr]->index(indexes, limit)->assumeMutable()); + else + new_variants.emplace_back(variants[i]->assumeMutable()); + } + + size_t new_size = new_variants[*non_empty_local_discr]->size(); + return ColumnVariant::create(local_discriminators->cloneResized(new_size), offsets->cloneResized(new_size), std::move(new_variants), local_to_global_discriminators); + } + + return selectIndexImpl(*this, indexes, limit); +} + +template +ColumnPtr ColumnVariant::indexImpl(const PaddedPODArray & indexes, size_t limit) const +{ + /// First, apply indexes for local_discriminators and offsets. + ColumnPtr new_local_discriminators = assert_cast(*local_discriminators).indexImpl(indexes, limit); + ColumnPtr new_offsets = assert_cast(*offsets).indexImpl(indexes, limit); + const auto & new_local_discriminators_data = assert_cast(*new_local_discriminators).getData(); + const auto & new_offsets_data = assert_cast(*new_offsets).getData(); + /// Then, create permutation for each variant. + const size_t num_variants = variants.size(); + std::vector nested_perms(num_variants); + /// If there is no limit, we know the size of each permutation + /// in advance and can use reserve. + if (limit == 0) + { + for (size_t i = 0; i != num_variants; ++i) + nested_perms[i].reserve(variants[i]->size()); + } + + for (size_t i = 0; i != new_local_discriminators_data.size(); ++i) + { + Discriminator discr = new_local_discriminators_data[i]; + if (discr != NULL_DISCRIMINATOR) + nested_perms[discr].push_back(new_offsets_data[i]); + } + + Columns new_variants; + new_variants.reserve(num_variants); + for (size_t i = 0; i != num_variants; ++i) + { + size_t nested_limit = nested_perms[i].size() == variants[i]->size() ? 0 : nested_perms[i].size(); + new_variants.emplace_back(variants[i]->permute(nested_perms[i], nested_limit)); + } + + /// We cannot use new_offsets column as an offset column, because it became invalid after variants permutation. + /// New offsets column will be created in constructor. + return ColumnVariant::create(new_local_discriminators, new_variants, local_to_global_discriminators); +} + +ColumnPtr ColumnVariant::replicate(const Offsets & replicate_offsets) const +{ + if (size() != replicate_offsets.size()) + throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "Size of offsets {} doesn't match size of column {}", replicate_offsets.size(), size()); + + if (empty()) + return cloneEmpty(); + + /// If we have only NULLs, just resize column to the new size. + if (hasOnlyNulls()) + return cloneResized(replicate_offsets.back()); + + const size_t num_variants = variants.size(); + + /// Optimization when we have only one non empty variant and no NULLs. + /// In this case local_discriminators column is filled with identical values and offsets column + /// filled with sequential numbers. So we can just replicate this one non empty variant, + /// then resize local_discriminators to the result size and fill offsets column. + if (auto non_empty_local_discr = getLocalDiscriminatorOfOneNoneEmptyVariantNoNulls()) + { + MutableColumns new_variants; + new_variants.reserve(num_variants); + for (size_t i = 0; i != num_variants; ++i) + { + if (i == *non_empty_local_discr) + new_variants.emplace_back(variants[*non_empty_local_discr]->replicate(replicate_offsets)->assumeMutable()); + else + new_variants.emplace_back(variants[i]->cloneEmpty()); + } + + size_t new_size = new_variants[*non_empty_local_discr]->size(); + /// Create and fill new local_discriminators column with non_empty_index discriminator. + auto new_local_discriminators = IColumn::mutate(local_discriminators); + assert_cast(*new_local_discriminators).getData().resize_fill(new_size, *non_empty_local_discr); + /// Create and fill new offsets column with sequential indexes. + auto new_offsets = IColumn::mutate(offsets); + auto & new_offsets_data = assert_cast(*new_offsets).getData(); + size_t old_size = offsets->size(); + if (new_size > old_size) + { + new_offsets_data.reserve(new_size); + for (size_t i = old_size; i < new_size; ++i) + new_offsets_data.push_back(new_offsets_data[i - 1] + 1); + } + else + { + new_offsets_data.resize(new_size); + } + + return ColumnVariant::create(std::move(new_local_discriminators), std::move(new_offsets), std::move(new_variants), local_to_global_discriminators); + } + + /// Create replicate offsets for each variant according to + /// local_discriminators column. + std::vector nested_replicated_offsets(num_variants); + for (size_t i = 0; i != num_variants; ++i) + nested_replicated_offsets[i].reserve(variants[i]->size()); + + const auto & local_discriminators_data = getLocalDiscriminators(); + for (size_t i = 0; i != local_discriminators_data.size(); ++i) + { + Discriminator discr = local_discriminators_data[i]; + if (discr != NULL_DISCRIMINATOR) + { + size_t repeat_count = replicate_offsets[i] - replicate_offsets[i - 1]; + nested_replicated_offsets[discr].push_back(nested_replicated_offsets[discr].back() + repeat_count); + } + } + + auto new_local_discriminators = local_discriminators->replicate(replicate_offsets); + Columns new_variants; + new_variants.reserve(num_variants); + for (size_t i = 0; i != num_variants; ++i) + new_variants.emplace_back(variants[i]->replicate(nested_replicated_offsets[i])); + + /// New offsets column will be created in constructor. + return ColumnVariant::create(new_local_discriminators, new_variants, local_to_global_discriminators); +} + +MutableColumns ColumnVariant::scatter(ColumnIndex num_columns, const Selector & selector) const +{ + const size_t num_variants = variants.size(); + + /// If we have only NULLs, we need to scatter only local_discriminators. + if (hasOnlyNulls()) + { + auto scattered_local_discriminators = local_discriminators->scatter(num_columns, selector); + MutableColumns result; + result.reserve(num_columns); + for (size_t i = 0; i != num_columns; ++i) + { + MutableColumns new_variants; + new_variants.reserve(num_variants); + for (const auto & variant : variants) + new_variants.emplace_back(IColumn::mutate(variant)); + + result.emplace_back(ColumnVariant::create(std::move(scattered_local_discriminators[i]), std::move(new_variants), local_to_global_discriminators)); + } + + return result; + } + + /// Optimization when we have only one non empty variant and no NULLs. + /// In this case we can just scatter local_discriminators and this non empty variant. + if (auto non_empty_local_discr = getLocalDiscriminatorOfOneNoneEmptyVariantNoNulls()) + { + auto scattered_local_discriminators = local_discriminators->scatter(num_columns, selector); + auto scattered_non_empty_variant = variants[*non_empty_local_discr]->scatter(num_columns, selector); + MutableColumns result; + result.reserve(num_columns); + for (size_t i = 0; i != num_columns; ++i) + { + MutableColumns scattered_nested_variants(num_variants); + for (size_t j = 0; j != num_variants; ++j) + { + if (j == *non_empty_local_discr) + scattered_nested_variants[j] = std::move(scattered_non_empty_variant[i]); + else + scattered_nested_variants[j] = IColumn::mutate(variants[j]); + } + + result.emplace_back(ColumnVariant::create(std::move(scattered_local_discriminators[i]), std::move(scattered_nested_variants), local_to_global_discriminators)); + } + + return result; + } + + /// Create selector for each variant according to local_discriminators. + std::vector nested_selectors(num_variants); + for (size_t i = 0; i != num_variants; ++i) + nested_selectors[i].reserve(variants[i]->size()); + + const auto & local_discriminators_data = getLocalDiscriminators(); + for (size_t i = 0; i != local_discriminators_data.size(); ++i) + { + Discriminator discr = local_discriminators_data[i]; + if (discr != NULL_DISCRIMINATOR) + nested_selectors[discr].push_back(selector[i]); + } + + auto scattered_local_discriminators = local_discriminators->scatter(num_columns, selector); + std::vector nested_scattered_variants; + nested_scattered_variants.reserve(num_variants); + for (size_t i = 0; i != num_variants; ++i) + nested_scattered_variants.emplace_back(variants[i]->scatter(num_columns, nested_selectors[i])); + + MutableColumns result; + result.reserve(num_columns); + for (size_t i = 0; i != num_columns; ++i) + { + MutableColumns new_variants; + new_variants.reserve(num_variants); + for (size_t j = 0; j != num_variants; ++j) + new_variants.emplace_back(std::move(nested_scattered_variants[j][i])); + result.emplace_back(ColumnVariant::create(std::move(scattered_local_discriminators[i]), std::move(new_variants), local_to_global_discriminators)); + } + + return result; +} + +void ColumnVariant::gather(ColumnGathererStream & gatherer) +{ + gatherer.gather(*this); +} + +bool ColumnVariant::hasEqualValues() const +{ + if (local_discriminators->empty() || hasOnlyNulls()) + return true; + + return local_discriminators->hasEqualValues() && variants[localDiscriminatorAt(0)]->hasEqualValues(); +} + +void ColumnVariant::getPermutation(IColumn::PermutationSortDirection, IColumn::PermutationSortStability, size_t, int, IColumn::Permutation & res) const +{ + size_t s = local_discriminators->size(); + res.resize(s); + for (size_t i = 0; i < s; ++i) + res[i] = i; +} + +void ColumnVariant::updatePermutation(IColumn::PermutationSortDirection, IColumn::PermutationSortStability, size_t, int, IColumn::Permutation &, DB::EqualRanges &) const +{ +} + +void ColumnVariant::reserve(size_t n) +{ + local_discriminators->reserve(n); + offsets->reserve(n); +} + +void ColumnVariant::ensureOwnership() +{ + const size_t num_variants = variants.size(); + for (size_t i = 0; i < num_variants; ++i) + getVariantByLocalDiscriminator(i).ensureOwnership(); +} + +size_t ColumnVariant::byteSize() const +{ + size_t res = local_discriminators->byteSize() + offsets->byteSize(); + for (const auto & variant : variants) + res += variant->byteSize(); + return res; +} + +size_t ColumnVariant::byteSizeAt(size_t n) const +{ + size_t res = sizeof(Offset) + sizeof(Discriminator); + Discriminator discr = localDiscriminatorAt(n); + if (discr == NULL_DISCRIMINATOR) + return res; + + return res + variants[discr]->byteSizeAt(offsetAt(n)); +} + +size_t ColumnVariant::allocatedBytes() const +{ + size_t res = local_discriminators->allocatedBytes() + offsets->allocatedBytes(); + for (const auto & variant : variants) + res += variant->allocatedBytes(); + return res; +} + +void ColumnVariant::protect() +{ + local_discriminators->protect(); + offsets->protect(); + for (auto & variant : variants) + variant->protect(); +} + +void ColumnVariant::getExtremes(Field & min, Field & max) const +{ + min = Null(); + max = Null(); +} + +void ColumnVariant::forEachSubcolumn(MutableColumnCallback callback) +{ + callback(local_discriminators); + callback(offsets); + for (auto & variant : variants) + callback(variant); +} + +void ColumnVariant::forEachSubcolumnRecursively(RecursiveMutableColumnCallback callback) +{ + callback(*local_discriminators); + local_discriminators->forEachSubcolumnRecursively(callback); + callback(*offsets); + offsets->forEachSubcolumnRecursively(callback); + + for (auto & variant : variants) + { + callback(*variant); + variant->forEachSubcolumnRecursively(callback); + } +} + +bool ColumnVariant::structureEquals(const IColumn & rhs) const +{ + const auto * rhs_variant = typeid_cast(&rhs); + if (!rhs_variant) + return false; + + const size_t num_variants = variants.size(); + if (num_variants != rhs_variant->variants.size()) + return false; + + for (size_t i = 0; i < num_variants; ++i) + if (!variants[i]->structureEquals(rhs_variant->getVariantByGlobalDiscriminator(globalDiscriminatorByLocal(i)))) + return false; + + return true; +} + +ColumnPtr ColumnVariant::compress() const +{ + ColumnPtr local_discriminators_compressed = local_discriminators->compress(); + ColumnPtr offsets_compressed = offsets->compress(); + size_t byte_size = local_discriminators_compressed->byteSize() + offsets_compressed->byteSize(); + Columns compressed; + compressed.reserve(variants.size()); + for (const auto & variant : variants) + { + auto compressed_variant = variant->compress(); + byte_size += compressed_variant->byteSize(); + compressed.emplace_back(std::move(compressed_variant)); + } + + return ColumnCompressed::create(size(), byte_size, + [my_local_discriminators_compressed = std::move(local_discriminators_compressed), my_offsets_compressed = std::move(offsets_compressed), my_compressed = std::move(compressed), my_local_to_global_discriminators = this->local_to_global_discriminators]() mutable + { + for (auto & variant : my_compressed) + variant = variant->decompress(); + return ColumnVariant::create(my_local_discriminators_compressed->decompress(), my_offsets_compressed->decompress(), my_compressed, my_local_to_global_discriminators); + }); +} + +double ColumnVariant::getRatioOfDefaultRows(double) const +{ + UInt64 num_defaults = getNumberOfDefaultRows(); + return static_cast(num_defaults) / local_discriminators->size(); +} + +UInt64 ColumnVariant::getNumberOfDefaultRows() const +{ + size_t total_variant_sizes = 0; + for (const auto & variant : variants) + total_variant_sizes += variant->size(); + return local_discriminators->size() - total_variant_sizes; +} + +void ColumnVariant::getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const +{ + return getIndicesOfNonDefaultRowsImpl(indices, from, limit); +} + +void ColumnVariant::finalize() +{ + for (auto & variant : variants) + variant->finalize(); +} + +bool ColumnVariant::isFinalized() const +{ + return std::all_of(variants.begin(), variants.end(), [](const auto & variant) { return variant->isFinalized(); }); +} + +std::optional ColumnVariant::getLocalDiscriminatorOfOneNoneEmptyVariantNoNulls() const +{ + for (size_t i = 0; i != variants.size(); ++i) + { + if (variants[i]->size() == local_discriminators->size()) + return i; + } + + return std::nullopt; +} + +void ColumnVariant::applyNullMap(const ColumnVector::Container & null_map) +{ + applyNullMapImpl(null_map); +} + +void ColumnVariant::applyNegatedNullMap(const ColumnVector::Container & null_map) +{ + applyNullMapImpl(null_map); +} + +template +void ColumnVariant::applyNullMapImpl(const ColumnVector::Container & null_map) +{ + if (null_map.size() != local_discriminators->size()) + throw Exception(ErrorCodes::SIZES_OF_NESTED_COLUMNS_ARE_INCONSISTENT, + "Logical error: Sizes of discriminators column and null map data are not equal"); + + /// If we have only NULLs, nothing to do. + if (hasOnlyNulls()) + { + return; + } + + /// If we have only 1 non empty column and no NULLs, we can just filter that + /// variant according to the null_map. + if (auto non_empty_local_discr = getLocalDiscriminatorOfOneNoneEmptyVariantNoNulls()) + { + auto & local_discriminators_data = getLocalDiscriminators(); + auto & offsets_data = getOffsets(); + size_t size_hint = 0; + + if constexpr (inverted) + { + for (size_t i = 0; i != local_discriminators_data.size(); ++i) + { + if (null_map[i]) + offsets_data[i] = size_hint++; + else + local_discriminators_data[i] = NULL_DISCRIMINATOR; + } + variants[*non_empty_local_discr] = variants[*non_empty_local_discr]->filter(null_map, size_hint); + } + else + { + ColumnVector::Container filter; + filter.reserve(null_map.size()); + for (size_t i = 0; i != local_discriminators_data.size(); ++i) + { + if (null_map[i]) + { + filter.push_back(0); + local_discriminators_data[i] = NULL_DISCRIMINATOR; + } + else + { + filter.push_back(1); + offsets_data[i] = size_hint++; + } + } + variants[*non_empty_local_discr] = variants[*non_empty_local_discr]->filter(filter, size_hint); + } + + return; + } + + /// In general case we should iterate through null_map + discriminators, + /// create filter for each variant and update offsets column. + std::vector variant_filters; + variant_filters.resize(variants.size()); + std::vector variant_new_sizes; + variant_new_sizes.resize(variants.size(), 0); + + auto & local_discriminators_data = getLocalDiscriminators(); + auto & offsets_data = getOffsets(); + for (size_t i = 0; i != local_discriminators_data.size(); ++i) + { + auto & discr = local_discriminators_data[i]; + if (discr != NULL_DISCRIMINATOR) + { + if (null_map[i] ^ inverted) + { + auto & variant_filter = variant_filters[discr]; + /// We create filters lazily. + if (variant_filter.empty()) + variant_filter.resize_fill(variants[discr]->size(), 1); + variant_filter[offsets_data[i]] = 0; + discr = NULL_DISCRIMINATOR; + } + else + { + offsets_data[i] = variant_new_sizes[discr]++; + } + } + } + + for (size_t i = 0; i != variants.size(); ++i) + { + if (!variant_filters[i].empty()) + variants[i] = variants[i]->filter(variant_filters[i], variant_new_sizes[i]); + } +} + +} diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h new file mode 100644 index 00000000000..692fdd1709e --- /dev/null +++ b/src/Columns/ColumnVariant.h @@ -0,0 +1,306 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +/** + * Column for storing Variant(...) type values. + * Variant type represents a union of other data types. + * For example, type Variant(T1, T2, ..., TN) means that each row of this type + * has a value of either type T1 or T2 or ... or TN or none of them (NULL value) + * + * ColumnVariant stores: + * - The discriminators column, which determines which variant is stored in each row. + * - The offsets column, which determines the offset in the corresponding variant column in each row. + * - The list of variant columns with only real values (so the sizes of variant columns can be different). + * Discriminator is an index of a variant in the variants list, it also has special value called NULL_DISCRIMINATOR + * that indicates that the value in the row is NULL. + * + * We want to be able to extend Variant column for free without rewriting the data, but as we don't care about the + * order of variants during Variant creation (we want Variant(T1, T2) to be the same as Variant(T2, T1)), we support + * some global order of nested types inside Variant during type creation, so after extension the order of variant types + * (and so their discriminators) can change. For example: Variant(T1, T3) -> Variant(T1, T2, T3). + * To avoid full rewrite of discriminators column on Variant extension, we differentiate local order of variants + * inside a column and global order of variants created during type creation. So, ColumnVariant stores only local + * discriminators and additionally stores the mapping between global and local discriminators. + * So, when we need to extend Variant column with new variant, we can just append it to a list of variant columns + * with new local discriminator and update mapping from global to local orders. + * + * Note that two instances of ColumnVariant can have different local orders, so we should always use global + * discriminators during inter-column interactions. + * + * Let's take an example with type Variant(UInt32, String, Array(UInt32)): + * During type creation we will sort types by their names and get the global order: Array(UInt32), String, UInt32. + * So, type Array(UInt32) will have global discriminator 0, String - 1 and UInt32 - 2. + * Let's say we have a column with local order (String, UInt32, Array(UInt32)) and values: + * 'Hello', 42, NULL, 'World', 43, [1, 2, 3], NULL, 44 + * + * Let's see how these values will be stored in ColumnVariant: + * + * local_to_global_discriminators: {0 : 1, 1 : 2, 2 : 0} + * global_to_local_discriminators: {0 : 2, 1 : 0, 2 : 1} + * local_discriminators offsets String UInt32 Array(UInt32) + * 0 0 'Hello' 42 [1, 2, 3] + * 1 0 'World' 43 + * NULL_DISCRIMINATOR 0 44 + * 0 1 + * 1 1 + * 2 0 + * NULL_DISCRIMINATOR 0 + * 1 2 + * + */ +class ColumnVariant final : public COWHelper +{ +public: + using Discriminator = UInt8; + using Discriminators = PaddedPODArray; + using ColumnDiscriminators = ColumnVector; + using ColumnOffsets = ColumnVector; + + static constexpr UInt8 NULL_DISCRIMINATOR = std::numeric_limits::max(); /// 255 + static constexpr size_t MAX_NESTED_COLUMNS = std::numeric_limits::max(); /// 255 + +private: + friend class COWHelper; + + using NestedColumns = std::vector; + + /// Create an empty column with provided variants. + /// Variants are in global order. + explicit ColumnVariant(MutableColumns && variants_); + /// Variants are in local order according to provided mapping. + explicit ColumnVariant(MutableColumns && variants_, const std::vector & local_to_global_discriminators_); + + /// Create column from discriminators column and list of variant columns. + /// Offsets column should be constructed according to the discriminators. + /// Variants are in global order. + ColumnVariant(MutableColumnPtr local_discriminators_, MutableColumns && variants_); + /// Variants are in local order according to provided mapping. + ColumnVariant(MutableColumnPtr local_discriminators_, MutableColumns && variants_, const std::vector & local_to_global_discriminators_); + + /// Create column from discriminators column, offsets column and list of variant columns. + /// Variants are in global order. + ColumnVariant(MutableColumnPtr local_discriminators_, MutableColumnPtr offsets_, MutableColumns && variants_); + /// Variants are in local order according to provided mapping. + ColumnVariant(MutableColumnPtr local_discriminators_, MutableColumnPtr offsets_, MutableColumns && variants_, const std::vector & local_to_global_discriminators_); + + ColumnVariant(const ColumnVariant &) = default; + +public: + /** Create immutable column using immutable arguments. This arguments may be shared with other variants. + * Use IColumn::mutate in order to make mutable column and mutate shared nested variants. + */ + using Base = COWHelper; + static Ptr create(const Columns & variants_) { return create(variants_, {}); } + static Ptr create(const Columns & variants_, const std::vector & local_to_global_discriminators_); + static Ptr create(const ColumnPtr & local_discriminators_, const Columns & variants_) { return create(local_discriminators_, variants_, {}); } + static Ptr create(const ColumnPtr & local_discriminators_, const Columns & variants_, const std::vector & local_to_global_discriminators_); + static Ptr create(const ColumnPtr & local_discriminators_, const DB::ColumnPtr & offsets_, const Columns & variants_) { return create(local_discriminators_, offsets_, variants_, {}); } + static Ptr create(const ColumnPtr & local_discriminators_, const DB::ColumnPtr & offsets_, const Columns & variants_, const std::vector & local_to_global_discriminators_); + + static MutablePtr create(MutableColumns && variants_) + { + return Base::create(std::move(variants_)); + } + + static MutablePtr create(MutableColumns && variants_, const std::vector & local_to_global_discriminators_) + { + return Base::create(std::move(variants_), local_to_global_discriminators_); + } + + static MutablePtr create(MutableColumnPtr local_discriminators_, MutableColumns && variants_) + { + return Base::create(std::move(local_discriminators_), std::move(variants_)); + } + + static MutablePtr create(MutableColumnPtr local_discriminators_, MutableColumns && variants_, const std::vector & local_to_global_discriminators_) + { + return Base::create(std::move(local_discriminators_), std::move(variants_), local_to_global_discriminators_); + } + + static MutablePtr create(MutableColumnPtr local_discriminators_, MutableColumnPtr offsets_, MutableColumns && variants_) + { + return Base::create(std::move(local_discriminators_), std::move(offsets_), std::move(variants_)); + } + + static MutablePtr create(MutableColumnPtr local_discriminators_, MutableColumnPtr offsets_, MutableColumns && variants_, const std::vector & local_to_global_discriminators_) + { + return Base::create(std::move(local_discriminators_), std::move(offsets_), std::move(variants_), local_to_global_discriminators_); + } + + std::string getName() const override; + const char * getFamilyName() const override { return "Variant"; } + TypeIndex getDataType() const override { return TypeIndex::Variant; } + + MutableColumnPtr cloneEmpty() const override; + MutableColumnPtr cloneResized(size_t size) const override; + + size_t ALWAYS_INLINE offsetAt(size_t i) const { return getOffsets()[i]; } + Discriminator ALWAYS_INLINE localDiscriminatorAt(size_t i) const { return getLocalDiscriminators()[i]; } + Discriminator ALWAYS_INLINE globalDiscriminatorAt(size_t i) const { return globalDiscriminatorByLocal(getLocalDiscriminators()[i]); } + + Discriminator ALWAYS_INLINE globalDiscriminatorByLocal(Discriminator local_discr) const + { + /// NULL_DISCRIMINATOR is always the same in local and global orders. + return local_discr == NULL_DISCRIMINATOR ? NULL_DISCRIMINATOR : local_to_global_discriminators[local_discr]; + } + + Discriminator ALWAYS_INLINE localDiscriminatorByGlobal(Discriminator global_discr) const + { + /// NULL_DISCRIMINATOR is always the same in local and global orders. + return global_discr == NULL_DISCRIMINATOR ? NULL_DISCRIMINATOR : global_to_local_discriminators[global_discr]; + } + + size_t size() const override + { + return local_discriminators->size(); + } + + Field operator[](size_t n) const override; + void get(size_t n, Field & res) const override; + + bool isDefaultAt(size_t n) const override; + bool isNullAt(size_t n) const override; + StringRef getDataAt(size_t n) const override; + void insertData(const char * pos, size_t length) override; + void insert(const Field & x) override; + void insertFrom(const IColumn & src_, size_t n) override; + void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; + void insertManyFrom(const IColumn & src, size_t position, size_t length) override; + void insertDefault() override; + void insertManyDefaults(size_t length) override; + void popBack(size_t n) override; + StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin, const UInt8 *) const override; + const char * deserializeAndInsertFromArena(const char * pos) override; + const char * skipSerializedInArena(const char * pos) const override; + void updateHashWithValue(size_t n, SipHash & hash) const override; + void updateWeakHash32(WeakHash32 & hash) const override; + void updateHashFast(SipHash & hash) const override; + ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; + void expand(const Filter & mask, bool inverted) override; + ColumnPtr permute(const Permutation & perm, size_t limit) const override; + ColumnPtr index(const IColumn & indexes, size_t limit) const override; + template + ColumnPtr indexImpl(const PaddedPODArray & indexes, size_t limit) const; + ColumnPtr replicate(const Offsets & replicate_offsets) const override; + MutableColumns scatter(ColumnIndex num_variants, const Selector & selector) const override; + void gather(ColumnGathererStream & gatherer_stream) override; + + /// Variant type is not comparable. + int compareAt(size_t, size_t, const IColumn &, int) const override + { + return 0; + } + + void compareColumn(const IColumn &, size_t, PaddedPODArray *, PaddedPODArray &, int, int) const override + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method compareColumn is not supported for ColumnAggregateFunction"); + } + + bool hasEqualValues() const override; + void getExtremes(Field & min, Field & max) const override; + void getPermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, + size_t limit, int nan_direction_hint, IColumn::Permutation & res) const override; + void updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, + size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const override; + + void reserve(size_t n) override; + void ensureOwnership() override; + size_t byteSize() const override; + size_t byteSizeAt(size_t n) const override; + size_t allocatedBytes() const override; + void protect() override; + void forEachSubcolumn(MutableColumnCallback callback) override; + void forEachSubcolumnRecursively(RecursiveMutableColumnCallback callback) override; + bool structureEquals(const IColumn & rhs) const override; + ColumnPtr compress() const override; + double getRatioOfDefaultRows(double sample_ratio) const override; + UInt64 getNumberOfDefaultRows() const override; + void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const override; + void finalize() override; + bool isFinalized() const override; + + const IColumn & getVariantByLocalDiscriminator(size_t discr) const { return *variants[discr]; } + const IColumn & getVariantByGlobalDiscriminator(size_t discr) const { return *variants[global_to_local_discriminators.at(discr)]; } + IColumn & getVariantByLocalDiscriminator(size_t discr) { return *variants[discr]; } + IColumn & getVariantByGlobalDiscriminator(size_t discr) { return *variants[global_to_local_discriminators.at(discr)]; } + + const ColumnPtr & getVariantPtrByLocalDiscriminator(size_t discr) const { return variants[discr]; } + const ColumnPtr & getVariantPtrByGlobalDiscriminator(size_t discr) const { return variants[global_to_local_discriminators.at(discr)]; } + ColumnPtr & getVariantPtrByLocalDiscriminator(size_t discr) { return variants[discr]; } + ColumnPtr & getVariantPtrByGlobalDiscriminator(size_t discr) { return variants[global_to_local_discriminators.at(discr)]; } + + const IColumn & getLocalDiscriminatorsColumn() const { return *local_discriminators; } + IColumn & getLocalDiscriminatorsColumn() { return *local_discriminators; } + + const ColumnPtr & getLocalDiscriminatorsPtr() const { return local_discriminators; } + ColumnPtr & getLocalDiscriminatorsPtr() { return local_discriminators; } + + const Discriminators & ALWAYS_INLINE getLocalDiscriminators() const { return assert_cast(*local_discriminators).getData(); } + Discriminators & ALWAYS_INLINE getLocalDiscriminators() { return assert_cast(*local_discriminators).getData(); } + + const IColumn & getOffsetsColumn() const { return *offsets; } + IColumn & getOffsetsColumn() { return *offsets; } + + const ColumnPtr & getOffsetsPtr() const { return offsets; } + ColumnPtr & getOffsetsPtr() { return offsets; } + + const Offsets & ALWAYS_INLINE getOffsets() const { return assert_cast(*offsets).getData(); } + Offsets & ALWAYS_INLINE getOffsets() { return assert_cast(*offsets).getData(); } + + size_t getNumVariants() const { return variants.size(); } + + bool hasOnlyNulls() const + { + /// If all variants are empty, we have only NULL values. + return std::all_of(variants.begin(), variants.end(), [](const auto & v){ return v->empty(); } ); + } + + /// Check if local and global order is the same. + bool hasGlobalVariantsOrder() const + { + for (size_t i = 0; i != local_to_global_discriminators.size(); ++i) + { + if (local_to_global_discriminators[i] != i) + return false; + } + + return true; + } + + /// Check if we have only 1 non-empty variant and no NULL values, + /// and if so, return the discriminator of this non-empty column. + std::optional getLocalDiscriminatorOfOneNoneEmptyVariantNoNulls() const; + + /// Apply null map to a Variant column. + /// Replace corresponding discriminators with NULL_DISCRIMINATOR + /// and filter out rows in variants if needed. + void applyNullMap(const ColumnVector::Container & null_map); + void applyNegatedNullMap(const ColumnVector::Container & null_map); + +private: + void initIdentityGlobalToLocalDiscriminatorsMapping(); + + template + void applyNullMapImpl(const ColumnVector::Container & null_map); + + WrappedPtr local_discriminators; + WrappedPtr offsets; + NestedColumns variants; + + std::vector global_to_local_discriminators; + std::vector local_to_global_discriminators; +}; + + +} diff --git a/src/Columns/IColumn.cpp b/src/Columns/IColumn.cpp index 7923bca6354..82dc82e0bd9 100644 --- a/src/Columns/IColumn.cpp +++ b/src/Columns/IColumn.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -83,6 +84,11 @@ bool isColumnNullable(const IColumn & column) return checkColumn(column); } +bool isColumnNullableOrLowCardinalityNullable(const IColumn & column) +{ + return isColumnNullable(column) || isColumnLowCardinalityNullable(column); +} + bool isColumnConst(const IColumn & column) { return checkColumn(column); diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index f012eeca61f..0dcba5b310c 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -631,6 +631,17 @@ struct IsMutableColumns template <> struct IsMutableColumns<> { static const bool value = true; }; +template +struct IsMutableColumnsOrRvalueReferences; + +template +struct IsMutableColumnsOrRvalueReferences +{ + static const bool value = (std::is_assignable::value || std::is_rvalue_reference_v) && IsMutableColumnsOrRvalueReferences::value; +}; + +template <> +struct IsMutableColumnsOrRvalueReferences<> { static const bool value = true; }; template const Type * checkAndGetColumn(const IColumn & column) @@ -662,4 +673,7 @@ bool isColumnConst(const IColumn & column); /// True if column's an ColumnNullable instance. It's just a syntax sugar for type check. bool isColumnNullable(const IColumn & column); +/// True if column's is ColumnNullable or ColumnLowCardinality with nullable nested column. +bool isColumnNullableOrLowCardinalityNullable(const IColumn & column); + } diff --git a/src/Columns/MaskOperations.cpp b/src/Columns/MaskOperations.cpp index b84268356a7..518269e1728 100644 --- a/src/Columns/MaskOperations.cpp +++ b/src/Columns/MaskOperations.cpp @@ -17,7 +17,7 @@ namespace ErrorCodes } template -void expandDataByMask(PaddedPODArray & data, const PaddedPODArray & mask, bool inverted) +void expandDataByMask(PaddedPODArray & data, const PaddedPODArray & mask, bool inverted, T default_value) { if (mask.size() < data.size()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mask size should be no less than data size."); @@ -38,7 +38,7 @@ void expandDataByMask(PaddedPODArray & data, const PaddedPODArray & ma --from; } else - data[index] = T(); + data[index] = default_value; --index; } @@ -49,7 +49,7 @@ void expandDataByMask(PaddedPODArray & data, const PaddedPODArray & ma /// Explicit instantiations - not to place the implementation of the function above in the header file. #define INSTANTIATE(TYPE) \ -template void expandDataByMask(PaddedPODArray &, const PaddedPODArray &, bool); +template void expandDataByMask(PaddedPODArray &, const PaddedPODArray &, bool, TYPE); INSTANTIATE(UInt8) INSTANTIATE(UInt16) diff --git a/src/Columns/MaskOperations.h b/src/Columns/MaskOperations.h index e43b4588258..cc5226bf0c1 100644 --- a/src/Columns/MaskOperations.h +++ b/src/Columns/MaskOperations.h @@ -13,7 +13,7 @@ namespace DB /// If inverted is true, we will work with inverted mask. This function is used in implementations of /// expand() method in IColumn interface. template -void expandDataByMask(PaddedPODArray & data, const PaddedPODArray & mask, bool inverted); +void expandDataByMask(PaddedPODArray & data, const PaddedPODArray & mask, bool inverted, T default_value = T()); struct MaskInfo { diff --git a/src/Columns/tests/gtest_column_variant.cpp b/src/Columns/tests/gtest_column_variant.cpp new file mode 100644 index 00000000000..b701e2d3183 --- /dev/null +++ b/src/Columns/tests/gtest_column_variant.cpp @@ -0,0 +1,688 @@ +#include +#include +#include +#include +#include + +using namespace DB; + +TEST(ColumnVariant, CreateFromEmptyColumns) +{ + MutableColumns columns; + columns.push_back(ColumnUInt32::create()); + columns.push_back(ColumnString::create()); + auto column = ColumnVariant::create(std::move(columns)); + ASSERT_TRUE(column->empty() && column->getLocalDiscriminators().empty() && column->getOffsets().empty()); +} + +TEST(ColumnVariant, CreateFromEmptyColumnsWithLocalOrder) +{ + MutableColumns columns; + columns.push_back(ColumnUInt32::create()); + columns.push_back(ColumnString::create()); + std::vector local_to_global_discriminators; + local_to_global_discriminators.push_back(1); + local_to_global_discriminators.push_back(0); + auto column = ColumnVariant::create(std::move(columns), local_to_global_discriminators); + ASSERT_TRUE(column->empty() && column->getLocalDiscriminators().empty() && column->getOffsets().empty()); + ASSERT_EQ(column->localDiscriminatorByGlobal(0), 0); + ASSERT_EQ(column->localDiscriminatorByGlobal(1), 1); + ASSERT_EQ(column->globalDiscriminatorByLocal(0), 0); + ASSERT_EQ(column->globalDiscriminatorByLocal(1), 1); +} + +MutableColumns createColumns1() +{ + MutableColumns columns; + auto column1 = ColumnUInt64::create(); + column1->insertValue(42); + columns.push_back(std::move(column1)); + auto column2 = ColumnString::create(); + column2->insertData("Hello", 5); + column2->insertData("World", 5); + columns.push_back(std::move(column2)); + auto column3 = ColumnUInt32::create(); + columns.push_back(std::move(column3)); + return columns; +} + +MutableColumnPtr createDiscriminators1() +{ + auto discriminators_column = ColumnVariant::ColumnDiscriminators::create(); + discriminators_column->insertValue(0); + discriminators_column->insertValue(1); + discriminators_column->insertValue(ColumnVariant::NULL_DISCRIMINATOR); + discriminators_column->insertValue(1); + discriminators_column->insertValue(ColumnVariant::NULL_DISCRIMINATOR); + return discriminators_column; +} + +void reorderColumns(const std::vector & local_to_global_order, MutableColumns & columns) +{ + MutableColumns res; + for (auto global_discr : local_to_global_order) + res.push_back(std::move(columns[global_discr])); + columns = std::move(res); +} + +template +void reorderDiscriminators(const std::vector & local_to_global_order, Ptr & discriminators) +{ + std::vector global_to_local_order(local_to_global_order.size()); + for (size_t i = 0; i != local_to_global_order.size(); ++i) + global_to_local_order[local_to_global_order[i]] = i; + + auto & discriminators_data = assert_cast(discriminators.get())->getData(); + for (auto & discr : discriminators_data) + { + if (discr != ColumnVariant::NULL_DISCRIMINATOR) + discr = global_to_local_order[discr]; + } +} + +MutableColumnPtr createOffsets1() +{ + auto offsets = ColumnVariant::ColumnOffsets::create(); + offsets->insertValue(0); + offsets->insertValue(0); + offsets->insertValue(0); + offsets->insertValue(1); + offsets->insertValue(0); + return offsets; +} + +std::vector createLocalToGlobalOrder1() +{ + std::vector local_to_global_discriminators; + local_to_global_discriminators.push_back(1); + local_to_global_discriminators.push_back(2); + local_to_global_discriminators.push_back(0); + return local_to_global_discriminators; +} + +void checkColumnVariant1(ColumnVariant * column) +{ + const auto & offsets = column->getOffsets(); + ASSERT_EQ(column->size(), 5); + ASSERT_EQ(offsets[0], 0); + ASSERT_EQ(offsets[1], 0); + ASSERT_EQ(offsets[3], 1); + ASSERT_TRUE(column->isDefaultAt(2) && column->isDefaultAt(4)); + ASSERT_EQ((*column)[0].get(), 42); + ASSERT_EQ((*column)[1].get(), "Hello"); + ASSERT_TRUE((*column)[2].isNull()); + ASSERT_EQ((*column)[3].get(), "World"); + ASSERT_TRUE((*column)[4].isNull()); +} + +void checkColumnVariant1Order(ColumnVariant * column) +{ + ASSERT_EQ(column->localDiscriminatorByGlobal(0), 2); + ASSERT_EQ(column->localDiscriminatorByGlobal(1), 0); + ASSERT_EQ(column->localDiscriminatorByGlobal(2), 1); + ASSERT_EQ(column->globalDiscriminatorByLocal(0), 1); + ASSERT_EQ(column->globalDiscriminatorByLocal(1), 2); + ASSERT_EQ(column->globalDiscriminatorByLocal(2), 0); + ASSERT_EQ(column->localDiscriminatorAt(0), 2); + ASSERT_EQ(column->localDiscriminatorAt(1), 0); + ASSERT_EQ(column->localDiscriminatorAt(2), ColumnVariant::NULL_DISCRIMINATOR); + ASSERT_EQ(column->localDiscriminatorAt(3), 0); + ASSERT_EQ(column->localDiscriminatorAt(4), ColumnVariant::NULL_DISCRIMINATOR); + ASSERT_EQ(column->globalDiscriminatorAt(0), 0); + ASSERT_EQ(column->globalDiscriminatorAt(1), 1); + ASSERT_EQ(column->globalDiscriminatorAt(2), ColumnVariant::NULL_DISCRIMINATOR); + ASSERT_EQ(column->globalDiscriminatorAt(3), 1); + ASSERT_EQ(column->globalDiscriminatorAt(4), ColumnVariant::NULL_DISCRIMINATOR); +} + +TEST(ColumnVariant, CreateFromDiscriminatorsAndColumns) +{ + auto columns = createColumns1(); + auto discriminators = createDiscriminators1(); + auto column = ColumnVariant::create(std::move(discriminators), std::move(columns)); + checkColumnVariant1(column.get()); +} + +TEST(ColumnVariant, CreateFromDiscriminatorsAndColumnsWithLocalOrder) +{ + auto local_to_global_order = createLocalToGlobalOrder1(); + auto columns = createColumns1(); + reorderColumns(local_to_global_order, columns); + auto discriminators = createDiscriminators1(); + reorderDiscriminators(local_to_global_order, discriminators); + auto column = ColumnVariant::create(std::move(discriminators), std::move(columns), createLocalToGlobalOrder1()); + checkColumnVariant1(column.get()); + checkColumnVariant1Order(column.get()); +} + +TEST(ColumnVariant, CreateFromDiscriminatorsOffsetsAndColumns) +{ + auto columns = createColumns1(); + auto discriminators = createDiscriminators1(); + auto offsets = createOffsets1(); + auto column = ColumnVariant::create(std::move(discriminators), std::move(offsets), std::move(columns)); + checkColumnVariant1(column.get()); +} + +TEST(ColumnVariant, CreateFromDiscriminatorsOffsetsAndColumnsWithLocalOrder) +{ + auto local_to_global_order = createLocalToGlobalOrder1(); + auto columns = createColumns1(); + reorderColumns(local_to_global_order, columns); + auto discriminators = createDiscriminators1(); + reorderDiscriminators(local_to_global_order, discriminators); + auto offsets = createOffsets1(); + auto column = ColumnVariant::create(std::move(discriminators), std::move(offsets), std::move(columns), createLocalToGlobalOrder1()); + checkColumnVariant1(column.get()); + checkColumnVariant1Order(column.get()); +} + +ColumnVariant::MutablePtr createVariantWithOneFullColumNoNulls(size_t size, bool change_order) +{ + MutableColumns columns; + auto column1 = ColumnUInt64::create(); + for (size_t i = 0; i != size; ++i) + column1->insertValue(i); + columns.push_back(std::move(column1)); + auto column2 = ColumnString::create(); + columns.push_back(std::move(column2)); + auto column3 = ColumnUInt32::create(); + columns.push_back(std::move(column3)); + auto discriminators_column = ColumnVariant::ColumnDiscriminators::create(); + for (size_t i = 0; i != size; ++i) + discriminators_column->insertValue(0); + if (change_order) + { + auto local_to_global_order = createLocalToGlobalOrder1(); + reorderColumns(local_to_global_order, columns); + reorderDiscriminators(local_to_global_order, discriminators_column); + return ColumnVariant::create(std::move(discriminators_column), std::move(columns), createLocalToGlobalOrder1()); + } + return ColumnVariant::create(std::move(discriminators_column), std::move(columns)); +} + +TEST(ColumnVariant, CreateFromDiscriminatorsAndOneFullColumnNoNulls) +{ + auto column = createVariantWithOneFullColumNoNulls(3, false); + const auto & offsets = column->getOffsets(); + ASSERT_EQ(column->size(), 3); + ASSERT_EQ(offsets[0], 0); + ASSERT_EQ(offsets[1], 1); + ASSERT_EQ(offsets[2], 2); + ASSERT_EQ((*column)[0].get(), 0); + ASSERT_EQ((*column)[1].get(), 1); + ASSERT_EQ((*column)[2].get(), 2); +} + +TEST(ColumnVariant, CreateFromDiscriminatorsAndOneFullColumnNoNullsWithLocalOrder) +{ + auto column = createVariantWithOneFullColumNoNulls(3, true); + const auto & offsets = column->getOffsets(); + ASSERT_EQ(column->size(), 3); + ASSERT_EQ(offsets[0], 0); + ASSERT_EQ(offsets[1], 1); + ASSERT_EQ(offsets[2], 2); + ASSERT_EQ((*column)[0].get(), 0); + ASSERT_EQ((*column)[1].get(), 1); + ASSERT_EQ((*column)[2].get(), 2); + ASSERT_EQ(column->localDiscriminatorAt(0), 2); + ASSERT_EQ(column->localDiscriminatorAt(1), 2); + ASSERT_EQ(column->localDiscriminatorAt(2), 2); + ASSERT_EQ(column->globalDiscriminatorAt(0), 0); + ASSERT_EQ(column->globalDiscriminatorAt(0), 0); + ASSERT_EQ(column->globalDiscriminatorAt(0), 0); +} + +TEST(ColumnVariant, CloneResizedToEmpty) +{ + auto column = ColumnVariant::create(createDiscriminators1(), createOffsets1(), createColumns1()); + auto resized_column = column->cloneResized(0); + ASSERT_TRUE(resized_column->empty()); +} + +TEST(ColumnVariant, CloneResizedToLarge) +{ + auto column = ColumnVariant::create(createDiscriminators1(), createOffsets1(), createColumns1()); + auto resized_column = column->cloneResized(7); + const auto * resized_column_variant = assert_cast(resized_column.get()); + ASSERT_EQ(resized_column_variant->size(), 7); + const auto & offsets = resized_column_variant->getOffsets(); + for (size_t i = 0; i != 7; ++i) + { + if (i == 3) + ASSERT_EQ(offsets[i], 1); + else + ASSERT_EQ(offsets[i], 0); + } + + const auto & discriminators = resized_column_variant->getLocalDiscriminators(); + std::vector null_indexes = {2, 4, 5, 6}; + for (size_t i : null_indexes) + ASSERT_EQ(discriminators[i], ColumnVariant::NULL_DISCRIMINATOR); + ASSERT_EQ(resized_column_variant->getVariantByLocalDiscriminator(0).size(), 1); + ASSERT_EQ(resized_column_variant->getVariantByLocalDiscriminator(1).size(), 2); +} + +TEST(ColumnVariant, CloneResizedWithOneFullColumnNoNulls) +{ + auto column = createVariantWithOneFullColumNoNulls(5, false); + auto resized_column = column->cloneResized(3); + const auto * resized_column_variant = assert_cast(resized_column.get()); + ASSERT_EQ(resized_column_variant->size(), 3); + const auto & offsets = resized_column_variant->getOffsets(); + for (size_t i = 0; i != 3; ++i) + ASSERT_EQ(offsets[i], i); + const auto & discriminators = resized_column_variant->getLocalDiscriminators(); + for (size_t i = 0; i != 3; ++i) + ASSERT_EQ(discriminators[i], 0); + ASSERT_EQ(resized_column_variant->getVariantByLocalDiscriminator(0).size(), 3); + ASSERT_EQ(resized_column_variant->getVariantByLocalDiscriminator(1).size(), 0); + ASSERT_EQ(resized_column_variant->getVariantByLocalDiscriminator(2).size(), 0); +} + +MutableColumns createColumns2() +{ + MutableColumns columns; + auto column1 = ColumnUInt64::create(); + column1->insertValue(42); + column1->insertValue(43); + column1->insertValue(44); + columns.push_back(std::move(column1)); + auto column2 = ColumnString::create(); + column2->insertData("Hello", 5); + column2->insertData("World", 5); + columns.push_back(std::move(column2)); + auto column3 = ColumnUInt8::create(); + columns.push_back(std::move(column3)); + return columns; +} + +TEST(ColumnVariant, CloneResizedGeneral1) +{ + /// D c1 c2 c3 + /// 0 42 Hello + /// 1 43 World + /// NULL 44 + /// 0 + /// 1 + /// NULL + /// 0 + auto discriminators_column = ColumnVariant::ColumnDiscriminators::create(); + discriminators_column->insertValue(0); + discriminators_column->insertValue(1); + discriminators_column->insertValue(ColumnVariant::NULL_DISCRIMINATOR); + discriminators_column->insertValue(0); + discriminators_column->insertValue(1); + discriminators_column->insertValue(ColumnVariant::NULL_DISCRIMINATOR); + discriminators_column->insertValue(0); + auto column = ColumnVariant::create(std::move(discriminators_column), createColumns2()); + auto resized_column = column->cloneResized(4); + const auto * resized_column_variant = assert_cast(resized_column.get()); + ASSERT_EQ(resized_column_variant->size(), 4); + ASSERT_EQ(resized_column_variant->getVariantByLocalDiscriminator(0).size(), 2); + ASSERT_EQ(resized_column_variant->getVariantByLocalDiscriminator(1).size(), 1); + ASSERT_EQ(resized_column_variant->getVariantByLocalDiscriminator(2).size(), 0); + const auto & discriminators = resized_column_variant->getLocalDiscriminators(); + ASSERT_EQ(discriminators[0], 0); + ASSERT_EQ(discriminators[1], 1); + ASSERT_EQ(discriminators[2], ColumnVariant::NULL_DISCRIMINATOR); + ASSERT_EQ(discriminators[3], 0); + const auto & offsets = resized_column_variant->getOffsets(); + ASSERT_EQ(offsets[0], 0); + ASSERT_EQ(offsets[1], 0); + ASSERT_EQ(offsets[3], 1); + ASSERT_EQ((*resized_column_variant)[0].get(), 42); + ASSERT_EQ((*resized_column_variant)[1].get(), "Hello"); + ASSERT_EQ((*resized_column_variant)[3].get(), 43); +} + +TEST(ColumnVariant, CloneResizedGeneral2) +{ + /// D c1 c2 c3 + /// 0 42 Hello + /// NULL 43 World + /// NULL 44 + /// 0 + /// 1 + /// 1 + /// 0 + auto discriminators_column = ColumnVariant::ColumnDiscriminators::create(); + discriminators_column->insertValue(0); + discriminators_column->insertValue(ColumnVariant::NULL_DISCRIMINATOR); + discriminators_column->insertValue(ColumnVariant::NULL_DISCRIMINATOR); + discriminators_column->insertValue(0); + discriminators_column->insertValue(1); + discriminators_column->insertValue(1); + discriminators_column->insertValue(0); + auto column = ColumnVariant::create(std::move(discriminators_column), createColumns2()); + auto resized_column = column->cloneResized(3); + const auto * resized_column_variant = assert_cast(resized_column.get()); + ASSERT_EQ(resized_column_variant->size(), 3); + ASSERT_EQ(resized_column_variant->getVariantByLocalDiscriminator(0).size(), 1); + ASSERT_EQ(resized_column_variant->getVariantByLocalDiscriminator(1).size(), 0); + ASSERT_EQ(resized_column_variant->getVariantByLocalDiscriminator(2).size(), 0); + const auto & discriminators = resized_column_variant->getLocalDiscriminators(); + ASSERT_EQ(discriminators[0], 0); + ASSERT_EQ(discriminators[1], ColumnVariant::NULL_DISCRIMINATOR); + ASSERT_EQ(discriminators[2], ColumnVariant::NULL_DISCRIMINATOR); + const auto & offsets = resized_column_variant->getOffsets(); + ASSERT_EQ(offsets[0], 0); + ASSERT_EQ((*resized_column_variant)[0].get(), 42); +} + +TEST(ColumnVariant, CloneResizedGeneral3) +{ + /// D c1 c2 c3 + /// 0 42 Hello + /// 1 43 World + /// 1 44 + /// 0 + /// NULL + /// NULL + /// 0 + auto discriminators_column = ColumnVariant::ColumnDiscriminators::create(); + discriminators_column->insertValue(0); + discriminators_column->insertValue(1); + discriminators_column->insertValue(1); + discriminators_column->insertValue(0); + discriminators_column->insertValue(ColumnVariant::NULL_DISCRIMINATOR); + discriminators_column->insertValue(ColumnVariant::NULL_DISCRIMINATOR); + discriminators_column->insertValue(0); + auto column = ColumnVariant::create(std::move(discriminators_column), createColumns2()); + auto resized_column = column->cloneResized(5); + const auto * resized_column_variant = assert_cast(resized_column.get()); + ASSERT_EQ(resized_column_variant->size(), 5); + ASSERT_EQ(resized_column_variant->getVariantByLocalDiscriminator(0).size(), 2); + ASSERT_EQ(resized_column_variant->getVariantByLocalDiscriminator(1).size(), 2); + ASSERT_EQ(resized_column_variant->getVariantByLocalDiscriminator(2).size(), 0); + const auto & discriminators = resized_column_variant->getLocalDiscriminators(); + ASSERT_EQ(discriminators[0], 0); + ASSERT_EQ(discriminators[1], 1); + ASSERT_EQ(discriminators[2], 1); + ASSERT_EQ(discriminators[3], 0); + const auto & offsets = resized_column_variant->getOffsets(); + ASSERT_EQ(offsets[0], 0); + ASSERT_EQ(offsets[1], 0); + ASSERT_EQ(offsets[2], 1); + ASSERT_EQ(offsets[3], 1); + ASSERT_EQ((*resized_column_variant)[0].get(), 42); + ASSERT_EQ((*resized_column_variant)[1].get(), "Hello"); + ASSERT_EQ((*resized_column_variant)[2].get(), "World"); + ASSERT_EQ((*resized_column_variant)[3].get(), 43); +} + +MutableColumnPtr createDiscriminators2() +{ + auto discriminators_column = ColumnVariant::ColumnDiscriminators::create(); + discriminators_column->insertValue(0); + discriminators_column->insertValue(1); + discriminators_column->insertValue(ColumnVariant::NULL_DISCRIMINATOR); + discriminators_column->insertValue(0); + discriminators_column->insertValue(1); + discriminators_column->insertValue(ColumnVariant::NULL_DISCRIMINATOR); + discriminators_column->insertValue(0); + return discriminators_column; +} + +std::vector createLocalToGlobalOrder2() +{ + std::vector local_to_global_discriminators; + local_to_global_discriminators.push_back(2); + local_to_global_discriminators.push_back(0); + local_to_global_discriminators.push_back(1); + return local_to_global_discriminators; +} + +ColumnVariant::MutablePtr createVariantColumn1(bool reorder) +{ + auto columns = createColumns1(); + auto discriminators = createDiscriminators1(); + if (!reorder) + return ColumnVariant::create(std::move(discriminators), std::move(columns)); + auto local_to_global_order = createLocalToGlobalOrder1(); + reorderColumns(local_to_global_order, columns); + reorderDiscriminators(local_to_global_order, discriminators); + return ColumnVariant::create(std::move(discriminators), std::move(columns), local_to_global_order); +} + +ColumnVariant::MutablePtr createVariantColumn2(bool reorder) +{ + auto columns = createColumns2(); + auto discriminators = createDiscriminators2(); + if (!reorder) + return ColumnVariant::create(std::move(discriminators), std::move(columns)); + auto local_to_global_order = createLocalToGlobalOrder2(); + reorderColumns(local_to_global_order, columns); + reorderDiscriminators(local_to_global_order, discriminators); + return ColumnVariant::create(std::move(discriminators), std::move(columns), local_to_global_order); +} + +TEST(ColumnVariant, InsertFrom) +{ + for (bool change_order : {false, true}) + { + auto column_to = createVariantColumn1(change_order); + auto column_from = createVariantColumn2(change_order); + column_to->insertFrom(*column_from, 3); + ASSERT_EQ(column_to->globalDiscriminatorAt(5), 0); + ASSERT_EQ((*column_to)[5].get(), 43); + } +} + +TEST(ColumnVariant, InsertRangeFromOneColumnNoNulls) +{ + for (bool change_order : {false, true}) + { + auto column_to = createVariantColumn2(change_order); + auto column_from = createVariantWithOneFullColumNoNulls(5, change_order); + column_to->insertRangeFrom(*column_from, 2, 2); + ASSERT_EQ(column_to->globalDiscriminatorAt(7), 0); + ASSERT_EQ(column_to->globalDiscriminatorAt(8), 0); + ASSERT_EQ((*column_to)[7].get(), 2); + ASSERT_EQ((*column_to)[8].get(), 3); + } +} + +TEST(ColumnVariant, InsertRangeFromGeneral) +{ + for (bool change_order : {false, true}) + { + auto column_to = createVariantColumn1(change_order); + auto column_from = createVariantColumn2(change_order); + column_to->insertRangeFrom(*column_from, 1, 4); + ASSERT_EQ(column_to->globalDiscriminatorAt(5), 1); + ASSERT_EQ(column_to->globalDiscriminatorAt(6), ColumnVariant::NULL_DISCRIMINATOR); + ASSERT_EQ(column_to->globalDiscriminatorAt(7), 0); + ASSERT_EQ(column_to->globalDiscriminatorAt(8), 1); + ASSERT_EQ((*column_to)[5].get(), "Hello"); + ASSERT_EQ((*column_to)[7].get(), 43); + ASSERT_EQ((*column_to)[8].get(), "World"); + } +} + +TEST(ColumnVariant, InsertManyFrom) +{ + for (bool change_order : {false, true}) + { + auto column_to = createVariantColumn1(change_order); + auto column_from = createVariantColumn2(change_order); + column_to->insertManyFrom(*column_from, 3, 2); + ASSERT_EQ(column_to->globalDiscriminatorAt(5), 0); + ASSERT_EQ(column_to->globalDiscriminatorAt(6), 0); + ASSERT_EQ((*column_to)[5].get(), 43); + ASSERT_EQ((*column_to)[6].get(), 43); + } +} + +TEST(ColumnVariant, PopBackOneColumnNoNulls) +{ + auto column = createVariantWithOneFullColumNoNulls(5, false); + column->popBack(3); + ASSERT_EQ(column->size(), 2); + ASSERT_EQ(column->getVariantByLocalDiscriminator(0).size(), 2); + ASSERT_EQ((*column)[0].get(), 0); + ASSERT_EQ((*column)[1].get(), 1); +} + +TEST(ColumnVariant, PopBackGeneral) +{ + auto column = ColumnVariant::create(createDiscriminators2(), createColumns2()); + column->popBack(4); + ASSERT_EQ(column->size(), 3); + ASSERT_EQ(column->getVariantByLocalDiscriminator(0).size(), 1); + ASSERT_EQ(column->getVariantByLocalDiscriminator(1).size(), 1); + ASSERT_EQ((*column)[0].get(), 42); + ASSERT_EQ((*column)[1].get(), "Hello"); + ASSERT_TRUE((*column)[2].isNull()); +} + +TEST(ColumnVariant, FilterOneColumnNoNulls) +{ + auto column = createVariantWithOneFullColumNoNulls(3, false); + IColumn::Filter filter; + filter.push_back(1); + filter.push_back(0); + filter.push_back(1); + auto filtered_column = column->filter(filter, -1); + ASSERT_EQ(filtered_column->size(), 2); + ASSERT_EQ((*filtered_column)[0].get(), 0); + ASSERT_EQ((*filtered_column)[1].get(), 2); +} + +TEST(ColumnVariant, FilterGeneral) +{ + auto column = ColumnVariant::create(createDiscriminators2(), createColumns2()); + IColumn::Filter filter; + filter.push_back(0); + filter.push_back(1); + filter.push_back(1); + filter.push_back(0); + filter.push_back(0); + filter.push_back(1); + filter.push_back(0); + auto filtered_column = column->filter(filter, -1); + ASSERT_EQ(filtered_column->size(), 3); + ASSERT_EQ((*filtered_column)[0].get(), "Hello"); + ASSERT_TRUE((*filtered_column)[1].isNull()); + ASSERT_TRUE((*filtered_column)[2].isNull()); +} + +TEST(ColumnVariant, PermuteAndIndexOneColumnNoNulls) +{ + auto column = createVariantWithOneFullColumNoNulls(4, false); + IColumn::Permutation permutation; + permutation.push_back(1); + permutation.push_back(3); + permutation.push_back(2); + permutation.push_back(0); + auto permuted_column = column->permute(permutation, 3); + ASSERT_EQ(permuted_column->size(), 3); + ASSERT_EQ((*permuted_column)[0].get(), 1); + ASSERT_EQ((*permuted_column)[1].get(), 3); + ASSERT_EQ((*permuted_column)[2].get(), 2); + + auto index = ColumnUInt64::create(); + index->getData() = std::move(permutation); + auto indexed_column = column->index(*index, 3); + ASSERT_EQ(indexed_column->size(), 3); + ASSERT_EQ((*indexed_column)[0].get(), 1); + ASSERT_EQ((*indexed_column)[1].get(), 3); + ASSERT_EQ((*indexed_column)[2].get(), 2); +} + +TEST(ColumnVariant, PermuteGeneral) +{ + auto column = ColumnVariant::create(createDiscriminators2(), createColumns2()); + IColumn::Permutation permutation; + permutation.push_back(3); + permutation.push_back(4); + permutation.push_back(1); + permutation.push_back(5); + auto permuted_column = column->permute(permutation, 4); + ASSERT_EQ(permuted_column->size(), 4); + ASSERT_EQ((*permuted_column)[0].get(), 43); + ASSERT_EQ((*permuted_column)[1].get(), "World"); + ASSERT_EQ((*permuted_column)[2].get(), "Hello"); + ASSERT_TRUE((*permuted_column)[3].isNull()); +} + +TEST(ColumnVariant, ReplicateOneColumnNoNull) +{ + auto column = createVariantWithOneFullColumNoNulls(3, false); + IColumn::Offsets offsets; + offsets.push_back(0); + offsets.push_back(3); + offsets.push_back(6); + auto replicated_column = column->replicate(offsets); + ASSERT_EQ(replicated_column->size(), 6); + ASSERT_EQ((*replicated_column)[0].get(), 1); + ASSERT_EQ((*replicated_column)[1].get(), 1); + ASSERT_EQ((*replicated_column)[2].get(), 1); + ASSERT_EQ((*replicated_column)[3].get(), 2); + ASSERT_EQ((*replicated_column)[4].get(), 2); + ASSERT_EQ((*replicated_column)[5].get(), 2); +} + +TEST(ColumnVariant, ReplicateGeneral) +{ + auto column = ColumnVariant::create(createDiscriminators1(), createColumns1()); + IColumn::Offsets offsets; + offsets.push_back(1); + offsets.push_back(3); + offsets.push_back(5); + offsets.push_back(5); + offsets.push_back(7); + auto replicated_column = column->replicate(offsets); + ASSERT_EQ(replicated_column->size(), 7); + ASSERT_EQ((*replicated_column)[0].get(), 42); + ASSERT_EQ((*replicated_column)[1].get(), "Hello"); + ASSERT_EQ((*replicated_column)[2].get(), "Hello"); + ASSERT_TRUE((*replicated_column)[3].isNull()); + ASSERT_TRUE((*replicated_column)[4].isNull()); + ASSERT_TRUE((*replicated_column)[5].isNull()); + ASSERT_TRUE((*replicated_column)[6].isNull()); +} + +TEST(ColumnVariant, ScatterOneColumnNoNulls) +{ + auto column = createVariantWithOneFullColumNoNulls(5, false); + IColumn::Selector selector; + selector.push_back(0); + selector.push_back(1); + selector.push_back(2); + selector.push_back(0); + selector.push_back(1); + auto columns = column->scatter(3, selector); + ASSERT_EQ(columns[0]->size(), 2); + ASSERT_EQ((*columns[0])[0].get(), 0); + ASSERT_EQ((*columns[0])[1].get(), 3); + ASSERT_EQ(columns[1]->size(), 2); + ASSERT_EQ((*columns[1])[0].get(), 1); + ASSERT_EQ((*columns[1])[1].get(), 4); + ASSERT_EQ(columns[2]->size(), 1); + ASSERT_EQ((*columns[2])[0].get(), 2); +} + +TEST(ColumnVariant, ScatterGeneral) +{ + auto column = ColumnVariant::create(createDiscriminators2(), createColumns2()); + IColumn::Selector selector; + selector.push_back(0); + selector.push_back(0); + selector.push_back(2); + selector.push_back(0); + selector.push_back(1); + selector.push_back(2); + selector.push_back(1); + auto columns = column->scatter(3, selector); + ASSERT_EQ(columns[0]->size(), 3); + ASSERT_EQ((*columns[0])[0].get(), 42); + ASSERT_EQ((*columns[0])[1].get(), "Hello"); + ASSERT_EQ((*columns[0])[2].get(), 43); + ASSERT_EQ(columns[1]->size(), 2); + ASSERT_EQ((*columns[1])[0].get(), "World"); + ASSERT_EQ((*columns[1])[1].get(), 44); + ASSERT_EQ(columns[2]->size(), 2); + ASSERT_TRUE((*columns[2])[0].isNull()); + ASSERT_TRUE((*columns[2])[1].isNull()); +} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 7e50a81ada8..0151dcb982d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -816,6 +816,7 @@ class IColumn; M(Bool, function_json_value_return_type_allow_complex, false, "Allow function JSON_VALUE to return complex type, such as: struct, array, map.", 0) \ M(Bool, use_with_fill_by_sorting_prefix, true, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with different values in sorting prefix are filled independently", 0) \ M(Bool, optimize_uniq_to_count, true, "Rewrite uniq and its variants(except uniqUpTo) to count if subquery has distinct or group by clause.", 0) \ + M(Bool, use_variant_when_no_common_type_in_if, false, "Use Variant as a result type for if/multiIf in case when there is no common type for arguments", 0) \ \ /** Experimental functions */ \ M(Bool, allow_experimental_materialized_postgresql_table, false, "Allows to use the MaterializedPostgreSQL table engine. Disabled by default, because this feature is experimental", 0) \ @@ -823,6 +824,7 @@ class IColumn; M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \ M(Bool, allow_experimental_hash_functions, false, "Enable experimental hash functions", 0) \ M(Bool, allow_experimental_object_type, false, "Allow Object and JSON data types", 0) \ + M(Bool, allow_experimental_variant_type, false, "Allow Variant data type", 0) \ M(Bool, allow_experimental_annoy_index, false, "Allows to use Annoy index. Disabled by default because this feature is experimental", 0) \ M(Bool, allow_experimental_usearch_index, false, "Allows to use USearch index. Disabled by default because this feature is experimental", 0) \ M(UInt64, max_limit_for_ann_queries, 1'000'000, "SELECT queries with LIMIT bigger than this setting cannot use ANN indexes. Helps to prevent memory overflows in ANN search indexes.", 0) \ diff --git a/src/Core/TypeId.h b/src/Core/TypeId.h index 9c634d2321c..7003e880cd5 100644 --- a/src/Core/TypeId.h +++ b/src/Core/TypeId.h @@ -49,6 +49,7 @@ enum class TypeIndex IPv4, IPv6, JSONPaths, + Variant, }; /** diff --git a/src/DataTypes/DataTypeFactory.cpp b/src/DataTypes/DataTypeFactory.cpp index 415f24d8151..d154b386ace 100644 --- a/src/DataTypes/DataTypeFactory.cpp +++ b/src/DataTypes/DataTypeFactory.cpp @@ -290,6 +290,7 @@ DataTypeFactory::DataTypeFactory() registerDataTypeDomainGeo(*this); registerDataTypeMap(*this); registerDataTypeObject(*this); + registerDataTypeVariant(*this); } DataTypeFactory & DataTypeFactory::instance() diff --git a/src/DataTypes/DataTypeFactory.h b/src/DataTypes/DataTypeFactory.h index ba7c1a3d7fe..a2aeb6f3646 100644 --- a/src/DataTypes/DataTypeFactory.h +++ b/src/DataTypes/DataTypeFactory.h @@ -100,5 +100,6 @@ void registerDataTypeDomainBool(DataTypeFactory & factory); void registerDataTypeDomainSimpleAggregateFunction(DataTypeFactory & factory); void registerDataTypeDomainGeo(DataTypeFactory & factory); void registerDataTypeObject(DataTypeFactory & factory); +void registerDataTypeVariant(DataTypeFactory & factory); } diff --git a/src/DataTypes/DataTypeNullable.cpp b/src/DataTypes/DataTypeNullable.cpp index 41a9a1de543..484d779551f 100644 --- a/src/DataTypes/DataTypeNullable.cpp +++ b/src/DataTypes/DataTypeNullable.cpp @@ -114,5 +114,33 @@ DataTypePtr makeNullableOrLowCardinalityNullable(const DataTypePtr & type) return std::make_shared(type); } +DataTypePtr makeNullableOrLowCardinalityNullableSafe(const DataTypePtr & type) +{ + if (isNullableOrLowCardinalityNullable(type)) + return type; + + if (type->lowCardinality()) + { + const auto & dictionary_type = assert_cast(*type).getDictionaryType(); + return std::make_shared(makeNullable(dictionary_type)); + } + + return makeNullableSafe(type); +} + +DataTypePtr removeNullableOrLowCardinalityNullable(const DataTypePtr & type) +{ + if (type->isNullable()) + return static_cast(*type).getNestedType(); + + if (type->isLowCardinalityNullable()) + { + auto dict_type = removeNullable(static_cast(*type).getDictionaryType()); + return std::make_shared(dict_type); + } + + return type; + +} } diff --git a/src/DataTypes/DataTypeNullable.h b/src/DataTypes/DataTypeNullable.h index 06d46fb15ed..7ad0e1ba5f1 100644 --- a/src/DataTypes/DataTypeNullable.h +++ b/src/DataTypes/DataTypeNullable.h @@ -54,5 +54,8 @@ DataTypePtr makeNullable(const DataTypePtr & type); DataTypePtr makeNullableSafe(const DataTypePtr & type); DataTypePtr removeNullable(const DataTypePtr & type); DataTypePtr makeNullableOrLowCardinalityNullable(const DataTypePtr & type); +DataTypePtr makeNullableOrLowCardinalityNullableSafe(const DataTypePtr & type); +/// Nullable(T) -> T, LowCardinality(Nullable(T)) -> T +DataTypePtr removeNullableOrLowCardinalityNullable(const DataTypePtr & type); } diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index fd2e5e6a784..df9af203618 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -172,11 +173,15 @@ MutableColumnPtr DataTypeTuple::createColumn() const MutableColumnPtr DataTypeTuple::createColumn(const ISerialization & serialization) const { + /// If we read Tuple as Variant subcolumn, it may be wrapped to SerializationVariantElement. + /// Here we don't need it, so we drop this wrapper. + const auto * current_serialization = &serialization; + while (const auto * serialization_variant_element = typeid_cast(current_serialization)) + current_serialization = serialization_variant_element->getNested().get(); + /// If we read subcolumn of nested Tuple, it may be wrapped to SerializationNamed /// several times to allow to reconstruct the substream path name. /// Here we don't need substream path name, so we drop first several wrapper serializations. - - const auto * current_serialization = &serialization; while (const auto * serialization_named = typeid_cast(current_serialization)) current_serialization = serialization_named->getNested().get(); diff --git a/src/DataTypes/DataTypeVariant.cpp b/src/DataTypes/DataTypeVariant.cpp new file mode 100644 index 00000000000..77e1c504cf8 --- /dev/null +++ b/src/DataTypes/DataTypeVariant.cpp @@ -0,0 +1,197 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int EMPTY_DATA_PASSED; +} + + +DataTypeVariant::DataTypeVariant(const DataTypes & variants_) +{ + /// Sort nested types by their full names and squash identical types. + std::map name_to_type; + for (const auto & type : variants_) + { + /// Nullable(...), LowCardinality(Nullable(...)) and Variant(...) types are not allowed inside Variant type. + if (isNullableOrLowCardinalityNullable(type)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Nullable/LowCardinality(Nullable) types are not allowed inside Variant type"); + if (type->getTypeId() == TypeIndex::Variant) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Nested Variant types are not allowed"); + /// Don't use Nothing type as a variant. + if (!isNothing(type)) + name_to_type[type->getName()] = type; + } + + variants.reserve(name_to_type.size()); + for (const auto & [_, type] : name_to_type) + variants.push_back(type); + + if (variants.empty()) + throw Exception(ErrorCodes::EMPTY_DATA_PASSED, "Variant cannot be empty"); + + if (variants.size() > ColumnVariant::MAX_NESTED_COLUMNS) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Variant type with more than {} nested types is not allowed", ColumnVariant::MAX_NESTED_COLUMNS); +} + +std::string DataTypeVariant::doGetName() const +{ + size_t size = variants.size(); + WriteBufferFromOwnString s; + + s << "Variant("; + for (size_t i = 0; i < size; ++i) + { + if (i != 0) + s << ", "; + + s << variants[i]->getName(); + } + s << ")"; + + return s.str(); +} + +std::string DataTypeVariant::doGetPrettyName(size_t indent) const +{ + size_t size = variants.size(); + WriteBufferFromOwnString s; + s << "Variant(\n"; + + for (size_t i = 0; i != size; ++i) + { + if (i != 0) + s << ",\n"; + + s << fourSpaceIndent(indent + 1) << variants[i]->getPrettyName(indent + 1); + } + + s << '\n' << fourSpaceIndent(indent) << ')'; + return s.str(); +} + +MutableColumnPtr DataTypeVariant::createColumn() const +{ + size_t size = variants.size(); + MutableColumns nested_columns; + nested_columns.reserve(size); + for (size_t i = 0; i < size; ++i) + nested_columns.push_back(variants[i]->createColumn()); + + return ColumnVariant::create(std::move(nested_columns)); +} + + +Field DataTypeVariant::getDefault() const +{ + return Null(); +} + +bool DataTypeVariant::equals(const IDataType & rhs) const +{ + if (typeid(rhs) != typeid(*this)) + return false; + + const DataTypeVariant & rhs_variant = static_cast(rhs); + + size_t size = variants.size(); + if (size != rhs_variant.variants.size()) + return false; + + for (size_t i = 0; i < size; ++i) + if (!variants[i]->equals(*rhs_variant.variants[i])) + return false; + + return true; +} + +bool DataTypeVariant::textCanContainOnlyValidUTF8() const +{ + return std::all_of(variants.begin(), variants.end(), [](auto && elem) { return elem->textCanContainOnlyValidUTF8(); }); +} + +bool DataTypeVariant::haveMaximumSizeOfValue() const +{ + return std::all_of(variants.begin(), variants.end(), [](auto && elem) { return elem->haveMaximumSizeOfValue(); }); +} + +bool DataTypeVariant::hasDynamicSubcolumns() const +{ + return std::any_of(variants.begin(), variants.end(), [](auto && elem) { return elem->hasDynamicSubcolumns(); }); +} + +std::optional DataTypeVariant::tryGetVariantDiscriminator(const DataTypePtr & type) const +{ + String type_name = type->getName(); + for (size_t i = 0; i != variants.size(); ++i) + { + /// We don't use equals here, because it doesn't respect custom type names. + if (variants[i]->getName() == type_name) + return i; + } + + return std::nullopt; +} + +size_t DataTypeVariant::getMaximumSizeOfValueInMemory() const +{ + size_t max_size = 0; + for (const auto & elem : variants) + { + size_t elem_max_size = elem->getMaximumSizeOfValueInMemory(); + if (elem_max_size > max_size) + max_size = elem_max_size; + } + return max_size; +} + +SerializationPtr DataTypeVariant::doGetDefaultSerialization() const +{ + SerializationVariant::VariantSerializations serializations; + serializations.reserve(variants.size()); + Names variant_names; + variant_names.reserve(variants.size()); + + for (const auto & variant : variants) + { + serializations.push_back(variant->getDefaultSerialization()); + variant_names.push_back(variant->getName()); + } + + return std::make_shared(std::move(serializations), std::move(variant_names), SerializationVariant::getVariantsDeserializeTextOrder(variants), getName()); +} + +static DataTypePtr create(const ASTPtr & arguments) +{ + if (!arguments || arguments->children.empty()) + throw Exception(ErrorCodes::EMPTY_DATA_PASSED, "Variant cannot be empty"); + + DataTypes nested_types; + nested_types.reserve(arguments->children.size()); + + for (const ASTPtr & child : arguments->children) + nested_types.emplace_back(DataTypeFactory::instance().get(child)); + + return std::make_shared(nested_types); +} + + +void registerDataTypeVariant(DataTypeFactory & factory) +{ + factory.registerDataType("Variant", create); +} + +} diff --git a/src/DataTypes/DataTypeVariant.h b/src/DataTypes/DataTypeVariant.h new file mode 100644 index 00000000000..60113a188b0 --- /dev/null +++ b/src/DataTypes/DataTypeVariant.h @@ -0,0 +1,64 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +/** Variant data type. + * This type represents a union of other data types. + * For example, type Variant(T1, T2, ..., TN) means that each row of this type + * has a value of either type T1 or T2 or ... or TN or none of them (NULL value). + * Nullable(...), LowCardinality(Nullable(...)) and Variant(...) types are not allowed + * inside Variant type. + * The order of nested types doesn't matter: Variant(T1, T2) = Variant(T2, T1). + * To have global order of nested types we sort variants by type names on Variant creation. + * The index of a variant in a sorted list is called global variant discriminator. + */ +class DataTypeVariant final : public IDataType +{ +private: + DataTypes variants; + +public: + static constexpr bool is_parametric = true; + + explicit DataTypeVariant(const DataTypes & variants_); + + TypeIndex getTypeId() const override { return TypeIndex::Variant; } + const char * getFamilyName() const override { return "Variant"; } + + bool canBeInsideNullable() const override { return false; } + bool supportsSparseSerialization() const override { return false; } + bool canBeInsideSparseColumns() const override { return false; } + + MutableColumnPtr createColumn() const override; + + Field getDefault() const override; + + bool equals(const IDataType & rhs) const override; + + bool isParametric() const override { return true; } + bool haveSubtypes() const override { return true; } + bool textCanContainOnlyValidUTF8() const override; + bool haveMaximumSizeOfValue() const override; + bool hasDynamicSubcolumns() const override; + size_t getMaximumSizeOfValueInMemory() const override; + + const DataTypePtr & getVariant(size_t i) const { return variants[i]; } + const DataTypes & getVariants() const { return variants; } + + /// Check if Variant has provided type in the list of variants and return its discriminator. + std::optional tryGetVariantDiscriminator(const DataTypePtr & type) const; + +private: + std::string doGetName() const override; + std::string doGetPrettyName(size_t indent) const override; + SerializationPtr doGetDefaultSerialization() const override; +}; + +} + diff --git a/src/DataTypes/EnumValues.cpp b/src/DataTypes/EnumValues.cpp index 9df49e765a7..8a4b1304d5e 100644 --- a/src/DataTypes/EnumValues.cpp +++ b/src/DataTypes/EnumValues.cpp @@ -74,6 +74,27 @@ T EnumValues::getValue(StringRef field_name, bool try_treat_as_id) const return it->getMapped(); } +template +bool EnumValues::tryGetValue(T & x, StringRef field_name, bool try_treat_as_id) const +{ + const auto it = name_to_value_map.find(field_name); + if (!it) + { + /// It is used in CSV and TSV input formats. If we fail to find given string in + /// enum names, we will try to treat it as enum id. + if (try_treat_as_id) + { + ReadBufferFromMemory tmp_buf(field_name.data, field_name.size); + if (!tryReadText(x, tmp_buf) || !tmp_buf.eof() || !value_to_name_map.contains(x)) + return false; + return true; + } + return false; + } + x = it->getMapped(); + return true; +} + template Names EnumValues::getAllRegisteredNames() const { diff --git a/src/DataTypes/EnumValues.h b/src/DataTypes/EnumValues.h index 5189f7a56f5..889878bc60f 100644 --- a/src/DataTypes/EnumValues.h +++ b/src/DataTypes/EnumValues.h @@ -7,7 +7,7 @@ namespace DB { -namespace ErrorCodes +namespace ErrorCodesEnumValues { extern const int BAD_ARGUMENTS; } @@ -42,6 +42,11 @@ public: return it; } + bool hasValue(const T & value) const + { + return value_to_name_map.contains(value); + } + /// throws exception if value is not valid const StringRef & getNameForValue(const T & value) const { @@ -60,6 +65,7 @@ public: } T getValue(StringRef field_name, bool try_treat_as_id = false) const; + bool tryGetValue(T & x, StringRef field_name, bool try_treat_as_id = false) const; template bool containsAll(const TValues & rhs_values) const diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 2a7e0f246de..392c56343e3 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -109,11 +109,26 @@ Ptr IDataType::getForSubcolumn( bool throw_if_null) const { Ptr res; - forEachSubcolumn([&](const auto &, const auto & name, const auto & subdata) + + ISerialization::StreamCallback callback_with_data = [&](const auto & subpath) { - if (name == subcolumn_name) - res = subdata.*member; - }, data); + for (size_t i = 0; i < subpath.size(); ++i) + { + size_t prefix_len = i + 1; + if (!subpath[i].visited && ISerialization::hasSubcolumnForPath(subpath, prefix_len)) + { + auto name = ISerialization::getSubcolumnNameForStream(subpath, prefix_len); + /// Create data from path only if it's requested subcolumn. + if (name == subcolumn_name) + res = ISerialization::createFromPath(subpath, prefix_len).*member; + } + subpath[i].visited = true; + } + }; + + ISerialization::EnumerateStreamsSettings settings; + settings.position_independent_encoding = false; + data.serialization->enumerateStreams(settings, callback_with_data, data); if (!res && throw_if_null) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in type {}", subcolumn_name, getName()); diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index eabf066bc3d..ccdf54f57c3 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -412,6 +412,8 @@ struct WhichDataType constexpr bool isSimple() const { return isInt() || isUInt() || isFloat() || isString(); } constexpr bool isLowCardinality() const { return idx == TypeIndex::LowCardinality; } + + constexpr bool isVariant() const { return idx == TypeIndex::Variant; } }; /// IDataType helpers (alternative for IDataType virtual methods with single point of truth) @@ -464,6 +466,7 @@ template inline bool isTuple(const T & data_type) { return WhichDat template inline bool isMap(const T & data_type) {return WhichDataType(data_type).isMap(); } template inline bool isInterval(const T & data_type) {return WhichDataType(data_type).isInterval(); } template inline bool isObject(const T & data_type) { return WhichDataType(data_type).isObject(); } +template inline bool isVariant(const T & data_type) { return WhichDataType(data_type).isVariant(); } template inline bool isNothing(const T & data_type) { return WhichDataType(data_type).isNothing(); } diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index e70dc6a2380..86a37949dc8 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -55,6 +55,9 @@ String ISerialization::Substream::toString() const return fmt::format("TupleElement({}, escape_tuple_delimiter = {})", tuple_element_name, escape_tuple_delimiter ? "true" : "false"); + if (type == VariantElement) + return fmt::format("VariantElement({})", variant_element_name); + return String(magic_enum::enum_name(type)); } @@ -172,6 +175,10 @@ String getNameForSubstreamPath( else stream_name += "." + it->tuple_element_name; } + else if (it->type == Substream::VariantDiscriminators) + stream_name += ".discr"; + else if (it->type == Substream::VariantElement) + stream_name += "." + it->variant_element_name; } return stream_name; @@ -252,6 +259,45 @@ bool ISerialization::isSpecialCompressionAllowed(const SubstreamPath & path) return true; } +#define TRY_DESERIALIZE_TEXT(deserialize) \ + size_t prev_size = column.size(); \ + try \ + { \ + deserialize(column, istr, settings); \ + return true; \ + } \ + catch (...) \ + { \ + if (column.size() > prev_size) \ + column.popBack(column.size() - prev_size); \ + return false; \ + } \ + +bool ISerialization::tryDeserializeTextCSV(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings & settings) const +{ + TRY_DESERIALIZE_TEXT(deserializeTextCSV) +} + +bool ISerialization::tryDeserializeTextEscaped(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings & settings) const +{ + TRY_DESERIALIZE_TEXT(deserializeTextEscaped) +} + +bool ISerialization::tryDeserializeTextJSON(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings & settings) const +{ + TRY_DESERIALIZE_TEXT(deserializeTextJSON) +} + +bool ISerialization::tryDeserializeTextQuoted(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings & settings) const +{ + TRY_DESERIALIZE_TEXT(deserializeTextQuoted) +} + +bool ISerialization::tryDeserializeWholeText(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings & settings) const +{ + TRY_DESERIALIZE_TEXT(deserializeWholeText) +} + void ISerialization::deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { String field; @@ -261,6 +307,15 @@ void ISerialization::deserializeTextRaw(IColumn & column, ReadBuffer & istr, con deserializeWholeText(column, buf, settings); } +bool ISerialization::tryDeserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + String field; + /// Read until \t or \n. + readString(field, istr); + ReadBufferFromString buf(field); + return tryDeserializeWholeText(column, buf, settings); +} + void ISerialization::serializeTextMarkdown( const DB::IColumn & column, size_t row_num, DB::WriteBuffer & ostr, const DB::FormatSettings & settings) const { @@ -288,7 +343,9 @@ bool ISerialization::hasSubcolumnForPath(const SubstreamPath & path, size_t pref size_t last_elem = prefix_len - 1; return path[last_elem].type == Substream::NullMap || path[last_elem].type == Substream::TupleElement - || path[last_elem].type == Substream::ArraySizes; + || path[last_elem].type == Substream::ArraySizes + || path[last_elem].type == Substream::VariantDiscriminators + || path[last_elem].type == Substream::VariantElement; } ISerialization::SubstreamData ISerialization::createFromPath(const SubstreamPath & path, size_t prefix_len) @@ -317,6 +374,8 @@ void ISerialization::throwUnexpectedDataAfterParsedValue(IColumn & column, ReadB { WriteBufferFromOwnString ostr; serializeText(column, column.size() - 1, ostr, settings); + /// Restore correct column size. + column.popBack(1); throw Exception( ErrorCodes::UNEXPECTED_DATA_AFTER_PARSED_VALUE, "Unexpected data '{}' after parsed {} value '{}'", diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index 030c3c6d81e..f0273f59d1f 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -152,6 +152,10 @@ public: ObjectStructure, ObjectData, + VariantDiscriminators, + VariantElements, + VariantElement, + Regular, }; @@ -160,6 +164,9 @@ public: /// Index of tuple element, starting at 1 or name. String tuple_element_name; + /// The name of a variant element type. + String variant_element_name; + /// Do we need to escape a dot in filenames for tuple elements. bool escape_tuple_delimiter = true; @@ -320,17 +327,20 @@ public: virtual void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0; virtual void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0; + virtual bool tryDeserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const; /** Text serialization as a literal that may be inserted into a query. */ virtual void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0; virtual void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0; + virtual bool tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const; /** Text serialization for the CSV format. */ virtual void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0; virtual void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0; + virtual bool tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const; /** Text serialization for displaying on a terminal or saving into a text file, and the like. * Without escaping or quoting. @@ -340,11 +350,13 @@ public: /** Text deserialization in case when buffer contains only one value, without any escaping and delimiters. */ virtual void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0; + virtual bool tryDeserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const; /** Text serialization intended for using in JSON format. */ virtual void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0; virtual void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0; + virtual bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const; virtual void serializeTextJSONPretty(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t /*indent*/) const { serializeTextJSON(column, row_num, ostr, settings); @@ -364,6 +376,7 @@ public: * additional code in data types serialization and ReadHelpers. */ virtual void deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const; + virtual bool tryDeserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const; virtual void serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const; virtual void serializeTextMarkdown(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const; diff --git a/src/DataTypes/Serializations/SerializationArray.cpp b/src/DataTypes/Serializations/SerializationArray.cpp index c804f58c567..be23278ef25 100644 --- a/src/DataTypes/Serializations/SerializationArray.cpp +++ b/src/DataTypes/Serializations/SerializationArray.cpp @@ -417,9 +417,11 @@ static void serializeTextImpl(const IColumn & column, size_t row_num, WriteBuffe } -template -static void deserializeTextImpl(IColumn & column, ReadBuffer & istr, Reader && read_nested, bool allow_unenclosed) +template +static ReturnType deserializeTextImpl(IColumn & column, ReadBuffer & istr, Reader && read_nested, bool allow_unenclosed) { + static constexpr bool throw_exception = std::is_same_v; + ColumnArray & column_array = assert_cast(column); ColumnArray::Offsets & offsets = column_array.getOffsets(); @@ -431,7 +433,18 @@ static void deserializeTextImpl(IColumn & column, ReadBuffer & istr, Reader && r if (checkChar('[', istr)) has_braces = true; else if (!allow_unenclosed) - throw Exception(ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT, "Array does not start with '[' character"); + { + if constexpr (throw_exception) + throw Exception(ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT, "Array does not start with '[' character"); + return ReturnType(false); + } + + auto on_error_no_throw = [&]() + { + if (size) + nested_column.popBack(size); + return ReturnType(false); + }; try { @@ -441,11 +454,17 @@ static void deserializeTextImpl(IColumn & column, ReadBuffer & istr, Reader && r if (!first) { if (*istr.position() == ',') + { ++istr.position(); + } else - throw ParsingException(ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT, - "Cannot read array from text, expected comma or end of array, found '{}'", - *istr.position()); + { + if constexpr (throw_exception) + throw ParsingException(ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT, + "Cannot read array from text, expected comma or end of array, found '{}'", + *istr.position()); + return on_error_no_throw(); + } } first = false; @@ -455,25 +474,42 @@ static void deserializeTextImpl(IColumn & column, ReadBuffer & istr, Reader && r if (*istr.position() == ']') break; - read_nested(nested_column); + if constexpr (throw_exception) + read_nested(nested_column); + else if (!read_nested(nested_column)) + return on_error_no_throw(); + ++size; skipWhitespaceIfAny(istr); } if (has_braces) - assertChar(']', istr); + { + if constexpr (throw_exception) + assertChar(']', istr); + else if (!checkChar(']', istr)) + return on_error_no_throw(); + } else /// If array is not enclosed in braces, we read until EOF. - assertEOF(istr); + { + if constexpr (throw_exception) + assertEOF(istr); + else if (!istr.eof()) + return on_error_no_throw(); + } } catch (...) { if (size) nested_column.popBack(size); - throw; + if constexpr (throw_exception) + throw; + return ReturnType(false); } offsets.push_back(offsets.back() + size); + return ReturnType(true); } @@ -492,8 +528,8 @@ void SerializationArray::deserializeText(IColumn & column, ReadBuffer & istr, co deserializeTextImpl(column, istr, [&](IColumn & nested_column) { - if (settings.null_as_default) - SerializationNullable::deserializeTextQuotedImpl(nested_column, istr, settings, nested); + if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(nested_column)) + SerializationNullable::deserializeNullAsDefaultOrNestedTextQuoted(nested_column, istr, settings, nested); else nested->deserializeTextQuoted(nested_column, istr, settings); }, false); @@ -502,6 +538,29 @@ void SerializationArray::deserializeText(IColumn & column, ReadBuffer & istr, co throwUnexpectedDataAfterParsedValue(column, istr, settings, "Array"); } +bool SerializationArray::tryDeserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const +{ + auto read_nested = [&](IColumn & nested_column) + { + if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(nested_column)) + return SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextQuoted(nested_column, istr, settings, nested); + return nested->tryDeserializeTextQuoted(nested_column, istr, settings); + }; + + bool ok = deserializeTextImpl(column, istr, std::move(read_nested), false); + + if (!ok) + return false; + + if (whole && !istr.eof()) + { + column.popBack(1); + return false; + } + + return true; +} + void SerializationArray::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { const ColumnArray & column_array = assert_cast(column); @@ -557,13 +616,25 @@ void SerializationArray::deserializeTextJSON(IColumn & column, ReadBuffer & istr deserializeTextImpl(column, istr, [&](IColumn & nested_column) { - if (settings.null_as_default) - SerializationNullable::deserializeTextJSONImpl(nested_column, istr, settings, nested); + if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(nested_column)) + SerializationNullable::deserializeNullAsDefaultOrNestedTextJSON(nested_column, istr, settings, nested); else nested->deserializeTextJSON(nested_column, istr, settings); }, false); } +bool SerializationArray::tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + auto read_nested = [&](IColumn & nested_column) + { + if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(nested_column)) + return SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextJSON(nested_column, istr, settings, nested); + return nested->tryDeserializeTextJSON(nested_column, istr, settings); + }; + + return deserializeTextImpl(column, istr, std::move(read_nested), false); +} + void SerializationArray::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { @@ -606,8 +677,8 @@ void SerializationArray::deserializeTextCSV(IColumn & column, ReadBuffer & istr, deserializeTextImpl(column, rb, [&](IColumn & nested_column) { - if (settings.null_as_default) - SerializationNullable::deserializeTextCSVImpl(nested_column, rb, settings, nested); + if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(nested_column)) + SerializationNullable::deserializeNullAsDefaultOrNestedTextCSV(nested_column, rb, settings, nested); else nested->deserializeTextCSV(nested_column, rb, settings); }, true); @@ -617,12 +688,43 @@ void SerializationArray::deserializeTextCSV(IColumn & column, ReadBuffer & istr, deserializeTextImpl(column, rb, [&](IColumn & nested_column) { - if (settings.null_as_default) - SerializationNullable::deserializeTextQuotedImpl(nested_column, rb, settings, nested); + if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(nested_column)) + SerializationNullable::deserializeNullAsDefaultOrNestedTextQuoted(nested_column, rb, settings, nested); else nested->deserializeTextQuoted(nested_column, rb, settings); }, true); } } +bool SerializationArray::tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + String s; + if (!tryReadCSV(s, istr, settings.csv)) + return false; + ReadBufferFromString rb(s); + + if (settings.csv.arrays_as_nested_csv) + { + auto read_nested = [&](IColumn & nested_column) + { + if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(nested_column)) + return SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextCSV(nested_column, rb, settings, nested); + return nested->tryDeserializeTextCSV(nested_column, rb, settings); + }; + + return deserializeTextImpl(column, rb, read_nested, true); + } + else + { + auto read_nested = [&](IColumn & nested_column) + { + if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(nested_column)) + return SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextQuoted(nested_column, rb, settings, nested); + return nested->tryDeserializeTextQuoted(nested_column, rb, settings); + }; + + return deserializeTextImpl(column, rb, read_nested, true); + } +} + } diff --git a/src/DataTypes/Serializations/SerializationArray.h b/src/DataTypes/Serializations/SerializationArray.h index de331169db5..82f5e8bce45 100644 --- a/src/DataTypes/Serializations/SerializationArray.h +++ b/src/DataTypes/Serializations/SerializationArray.h @@ -20,15 +20,18 @@ public: void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const override; + bool tryDeserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const override; void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextJSONPretty(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t indent) const override; void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; /** Streaming serialization of arrays is arranged in a special way: * - elements placed in a row are written/read without array sizes; diff --git a/src/DataTypes/Serializations/SerializationBool.cpp b/src/DataTypes/Serializations/SerializationBool.cpp index 41b5bf806e5..f745fac4d30 100644 --- a/src/DataTypes/Serializations/SerializationBool.cpp +++ b/src/DataTypes/Serializations/SerializationBool.cpp @@ -150,30 +150,42 @@ bool tryDeserializeAllVariants(ColumnUInt8 * column, ReadBuffer & istr) return true; } -void deserializeImpl( +template +ReturnType deserializeImpl( IColumn & column, ReadBuffer & istr, const FormatSettings & settings, std::function check_end_of_value) { + static constexpr bool throw_exception = std::is_same_v; + ColumnUInt8 * col = checkAndGetDeserializeColumnType(column); + auto restore_column_if_needed = [&, prev_size = col->size()]() + { + if (col->size() > prev_size) + col->popBack(1); + }; PeekableReadBuffer buf(istr); buf.setCheckpoint(); if (checkString(settings.bool_true_representation, buf) && check_end_of_value(buf)) { col->insert(true); - return; + return ReturnType(true); } buf.rollbackToCheckpoint(); if (checkString(settings.bool_false_representation, buf) && check_end_of_value(buf)) { - col->insert(false); buf.dropCheckpoint(); if (buf.hasUnreadData()) - throw Exception( - ErrorCodes::CANNOT_PARSE_BOOL, - "Cannot continue parsing after parsed bool value because it will result in the loss of some data. It may happen if " - "bool_true_representation or bool_false_representation contains some delimiters of input format"); - return; + { + if constexpr (throw_exception) + throw Exception( + ErrorCodes::CANNOT_PARSE_BOOL, + "Cannot continue parsing after parsed bool value because it will result in the loss of some data. It may happen if " + "bool_true_representation or bool_false_representation contains some delimiters of input format"); + return ReturnType(false); + } + col->insert(false); + return ReturnType(true); } buf.rollbackToCheckpoint(); @@ -181,22 +193,31 @@ void deserializeImpl( { buf.dropCheckpoint(); if (buf.hasUnreadData()) - throw Exception( - ErrorCodes::CANNOT_PARSE_BOOL, - "Cannot continue parsing after parsed bool value because it will result in the loss of some data. It may happen if " - "bool_true_representation or bool_false_representation contains some delimiters of input format"); - return; + { + if constexpr (throw_exception) + throw Exception( + ErrorCodes::CANNOT_PARSE_BOOL, + "Cannot continue parsing after parsed bool value because it will result in the loss of some data. It may happen if " + "bool_true_representation or bool_false_representation contains some delimiters of input format"); + restore_column_if_needed(); + return ReturnType(false); + } + return ReturnType(true); } buf.makeContinuousMemoryFromCheckpointToPos(); buf.rollbackToCheckpoint(); - throw Exception( - ErrorCodes::CANNOT_PARSE_BOOL, - "Cannot parse boolean value here: '{}', should be '{}' or '{}' controlled by setting bool_true_representation and " - "bool_false_representation or one of " - "True/False/T/F/Y/N/Yes/No/On/Off/Enable/Disable/Enabled/Disabled/1/0", - String(buf.position(), std::min(10lu, buf.available())), - settings.bool_true_representation, settings.bool_false_representation); + restore_column_if_needed(); + if constexpr (throw_exception) + throw Exception( + ErrorCodes::CANNOT_PARSE_BOOL, + "Cannot parse boolean value here: '{}', should be '{}' or '{}' controlled by setting bool_true_representation and " + "bool_false_representation or one of " + "True/False/T/F/Y/N/Yes/No/On/Off/Enable/Disable/Enabled/Disabled/1/0", + String(buf.position(), std::min(10lu, buf.available())), + settings.bool_true_representation, settings.bool_false_representation); + + return ReturnType(false); } } @@ -225,6 +246,14 @@ void SerializationBool::deserializeTextEscaped(IColumn & column, ReadBuffer & is deserializeImpl(column, istr, settings, [](ReadBuffer & buf){ return buf.eof() || *buf.position() == '\t' || *buf.position() == '\n'; }); } +bool SerializationBool::tryDeserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + if (istr.eof()) + return false; + + return deserializeImpl(column, istr, settings, [](ReadBuffer & buf){ return buf.eof() || *buf.position() == '\t' || *buf.position() == '\n'; }); +} + void SerializationBool::serializeTextJSON(const IColumn &column, size_t row_num, WriteBuffer &ostr, const FormatSettings &settings) const { serializeSimple(column, row_num, ostr, settings); @@ -250,6 +279,33 @@ void SerializationBool::deserializeTextJSON(IColumn &column, ReadBuffer &istr, c col->insert(value); } +bool SerializationBool::tryDeserializeTextJSON(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings &) const +{ + if (istr.eof()) + return false; + + ColumnUInt8 * col = checkAndGetDeserializeColumnType(column); + bool value = false; + char first_char = *istr.position(); + if (first_char == 't' || first_char == 'f') + { + if (!readBoolTextWord(value, istr)) + return false; + } + else if (first_char == '1' || first_char == '0') + { + /// Doesn't throw. + readBoolText(value, istr); + } + else + { + return false; + } + + col->insert(value); + return true; +} + void SerializationBool::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { serializeCustom(column, row_num, ostr, settings); @@ -263,6 +319,14 @@ void SerializationBool::deserializeTextCSV(IColumn & column, ReadBuffer & istr, deserializeImpl(column, istr, settings, [&](ReadBuffer & buf){ return buf.eof() || *buf.position() == settings.csv.delimiter || *buf.position() == '\n' || *buf.position() == '\r'; }); } +bool SerializationBool::tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + if (istr.eof()) + return false; + + return deserializeImpl(column, istr, settings, [&](ReadBuffer & buf){ return buf.eof() || *buf.position() == settings.csv.delimiter || *buf.position() == '\n' || *buf.position() == '\r'; }); +} + void SerializationBool::serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { serializeCustom(column, row_num, ostr, settings); @@ -276,15 +340,30 @@ void SerializationBool::deserializeTextRaw(IColumn & column, ReadBuffer & istr, deserializeImpl(column, istr, settings, [&](ReadBuffer & buf){ return buf.eof() || *buf.position() == '\t' || *buf.position() == '\n'; }); } +bool SerializationBool::tryDeserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + if (istr.eof()) + return false; + + return deserializeImpl(column, istr, settings, [&](ReadBuffer & buf){ return buf.eof() || *buf.position() == '\t' || *buf.position() == '\n'; }); +} + void SerializationBool::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { serializeSimple(column, row_num, ostr, settings); } -void SerializationBool::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +template +ReturnType deserializeTextQuotedImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) { + static constexpr bool throw_exception = std::is_same_v; + if (istr.eof()) - throw Exception(ErrorCodes::CANNOT_PARSE_BOOL, "Expected boolean value but get EOF."); + { + if constexpr (throw_exception) + throw Exception(ErrorCodes::CANNOT_PARSE_BOOL, "Expected boolean value but get EOF."); + return ReturnType(false); + } auto * col = checkAndGetDeserializeColumnType(column); @@ -292,11 +371,17 @@ void SerializationBool::deserializeTextQuoted(IColumn & column, ReadBuffer & ist switch (symbol) { case 't': - assertStringCaseInsensitive("true", istr); + if constexpr (throw_exception) + assertStringCaseInsensitive("true", istr); + else if (!checkStringCaseInsensitive("true", istr)) + return ReturnType(false); col->insert(true); break; case 'f': - assertStringCaseInsensitive("false", istr); + if constexpr (throw_exception) + assertStringCaseInsensitive("false", istr); + else if (!checkStringCaseInsensitive("false", istr)) + return ReturnType(false); col->insert(false); break; case '1': @@ -307,16 +392,40 @@ void SerializationBool::deserializeTextQuoted(IColumn & column, ReadBuffer & ist break; case '\'': ++istr.position(); - deserializeImpl(column, istr, settings, [](ReadBuffer & buf){ return !buf.eof() && *buf.position() == '\''; }); - assertChar('\'', istr); + if constexpr (throw_exception) + { + deserializeImpl(column, istr, settings, [](ReadBuffer & buf){ return !buf.eof() && *buf.position() == '\''; }); + assertChar('\'', istr); + } + else + { + if (!deserializeImpl(column, istr, settings, [](ReadBuffer & buf) { return !buf.eof() && *buf.position() == '\''; }) || !checkChar('\'', istr)) + return ReturnType(false); + } break; default: - throw Exception( - ErrorCodes::CANNOT_PARSE_BOOL, - "Cannot parse boolean value here: '{}', should be true/false, 1/0 or on of " - "True/False/T/F/Y/N/Yes/No/On/Off/Enable/Disable/Enabled/Disabled/1/0 in quotes", - String(istr.position(), std::min(10ul, istr.available()))); + { + if constexpr (throw_exception) + throw Exception( + ErrorCodes::CANNOT_PARSE_BOOL, + "Cannot parse boolean value here: '{}', should be true/false, 1/0 or on of " + "True/False/T/F/Y/N/Yes/No/On/Off/Enable/Disable/Enabled/Disabled/1/0 in quotes", + String(istr.position(), std::min(10ul, istr.available()))); + return ReturnType(false); + } } + + return ReturnType(true); +} + +void SerializationBool::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + deserializeTextQuotedImpl(column, istr, settings); +} + +bool SerializationBool::tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + return deserializeTextQuotedImpl(column, istr, settings); } void SerializationBool::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const @@ -327,6 +436,14 @@ void SerializationBool::deserializeWholeText(IColumn & column, ReadBuffer & istr deserializeImpl(column, istr, settings, [&](ReadBuffer & buf){ return buf.eof(); }); } +bool SerializationBool::tryDeserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + if (istr.eof()) + return false; + + return deserializeImpl(column, istr, settings, [&](ReadBuffer & buf){ return buf.eof(); }); +} + void SerializationBool::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { serializeSimple(column, row_num, ostr, settings); diff --git a/src/DataTypes/Serializations/SerializationBool.h b/src/DataTypes/Serializations/SerializationBool.h index a5aa0ca80a2..3e511b7249e 100644 --- a/src/DataTypes/Serializations/SerializationBool.h +++ b/src/DataTypes/Serializations/SerializationBool.h @@ -15,21 +15,27 @@ public: void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; - void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; void deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; }; diff --git a/src/DataTypes/Serializations/SerializationCustomSimpleText.cpp b/src/DataTypes/Serializations/SerializationCustomSimpleText.cpp index 03564bac64b..abe443cab1b 100644 --- a/src/DataTypes/Serializations/SerializationCustomSimpleText.cpp +++ b/src/DataTypes/Serializations/SerializationCustomSimpleText.cpp @@ -24,6 +24,12 @@ void deserializeFromString(const SerializationCustomSimpleText & domain, IColumn domain.deserializeText(column, istr, settings, true); } +bool tryDeserializeFromString(const SerializationCustomSimpleText & domain, IColumn & column, const String & s, const FormatSettings & settings) +{ + ReadBufferFromString istr(s); + return domain.tryDeserializeText(column, istr, settings, true); +} + } namespace DB @@ -34,6 +40,19 @@ SerializationCustomSimpleText::SerializationCustomSimpleText(const Serialization { } +bool SerializationCustomSimpleText::tryDeserializeText(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings & settings, bool whole) const +{ + try + { + deserializeText(column, istr, settings, whole); + return true; + } + catch (...) + { + return false; + } +} + void SerializationCustomSimpleText::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { String str; @@ -41,6 +60,13 @@ void SerializationCustomSimpleText::deserializeWholeText(IColumn & column, ReadB deserializeFromString(*this, column, str, settings); } +bool SerializationCustomSimpleText::tryDeserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + String str; + readStringUntilEOF(str, istr); + return tryDeserializeFromString(*this, column, str, settings); +} + void SerializationCustomSimpleText::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeEscapedString(serializeToString(*this, column, row_num, settings), ostr); @@ -53,6 +79,13 @@ void SerializationCustomSimpleText::deserializeTextEscaped(IColumn & column, Rea deserializeFromString(*this, column, str, settings); } +bool SerializationCustomSimpleText::tryDeserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + String str; + readEscapedString(str, istr); + return tryDeserializeFromString(*this, column, str, settings); +} + void SerializationCustomSimpleText::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeQuotedString(serializeToString(*this, column, row_num, settings), ostr); @@ -65,6 +98,14 @@ void SerializationCustomSimpleText::deserializeTextQuoted(IColumn & column, Read deserializeFromString(*this, column, str, settings); } +bool SerializationCustomSimpleText::tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + String str; + if (!tryReadQuotedString(str, istr)) + return false; + return tryDeserializeFromString(*this, column, str, settings); +} + void SerializationCustomSimpleText::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeCSVString(serializeToString(*this, column, row_num, settings), ostr); @@ -77,6 +118,13 @@ void SerializationCustomSimpleText::deserializeTextCSV(IColumn & column, ReadBuf deserializeFromString(*this, column, str, settings); } +bool SerializationCustomSimpleText::tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + String str; + readCSVStringInto(str, istr, settings.csv); + return tryDeserializeFromString(*this, column, str, settings); +} + void SerializationCustomSimpleText::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeJSONString(serializeToString(*this, column, row_num, settings), ostr, settings); @@ -89,6 +137,14 @@ void SerializationCustomSimpleText::deserializeTextJSON(IColumn & column, ReadBu deserializeFromString(*this, column, str, settings); } +bool SerializationCustomSimpleText::tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + String str; + if (!tryReadJSONStringInto(str, istr)) + return false; + return tryDeserializeFromString(*this, column, str, settings); +} + void SerializationCustomSimpleText::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeXMLStringForTextElement(serializeToString(*this, column, row_num, settings), ostr); diff --git a/src/DataTypes/Serializations/SerializationCustomSimpleText.h b/src/DataTypes/Serializations/SerializationCustomSimpleText.h index 0c909350002..c80a57e234c 100644 --- a/src/DataTypes/Serializations/SerializationCustomSimpleText.h +++ b/src/DataTypes/Serializations/SerializationCustomSimpleText.h @@ -22,20 +22,24 @@ public: /// whole = true means that buffer contains only one value, so we should read until EOF. /// It's needed to check if there is garbage after parsed field. virtual void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const = 0; + virtual bool tryDeserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const; /** Text deserialization in case when buffer contains only one value, without any escaping and delimiters. */ void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; /** Text serialization with escaping but without quoting. */ void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; /** Text serialization as a literal that may be inserted into a query. */ void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; /** Text serialization for the CSV format. */ @@ -44,12 +48,14 @@ public: * (the delimiter is not consumed). */ void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; /** Text serialization intended for using in JSON format. * force_quoting_64bit_integers parameter forces to brace UInt64 and Int64 types into quotes. */ void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; /** Text serialization for putting into the XML format. */ diff --git a/src/DataTypes/Serializations/SerializationDate.cpp b/src/DataTypes/Serializations/SerializationDate.cpp index 534f599a072..38e1bb87b6d 100644 --- a/src/DataTypes/Serializations/SerializationDate.cpp +++ b/src/DataTypes/Serializations/SerializationDate.cpp @@ -22,6 +22,15 @@ void SerializationDate::deserializeWholeText(IColumn & column, ReadBuffer & istr throwUnexpectedDataAfterParsedValue(column, istr, settings, "Date"); } +bool SerializationDate::tryDeserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +{ + DayNum x; + if (!tryReadDateText(x, istr, time_zone) || !istr.eof()) + return false; + assert_cast(column).getData().push_back(x); + return true; +} + void SerializationDate::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { DayNum x; @@ -29,6 +38,15 @@ void SerializationDate::deserializeTextEscaped(IColumn & column, ReadBuffer & is assert_cast(column).getData().push_back(x); } +bool SerializationDate::tryDeserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +{ + DayNum x; + if (!tryReadDateText(x, istr, time_zone)) + return false; + assert_cast(column).getData().push_back(x); + return true; +} + void SerializationDate::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { serializeText(column, row_num, ostr, settings); @@ -50,6 +68,16 @@ void SerializationDate::deserializeTextQuoted(IColumn & column, ReadBuffer & ist assert_cast(column).getData().push_back(x); /// It's important to do this at the end - for exception safety. } +bool SerializationDate::tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +{ + DayNum x; + if (!checkChar('\'', istr) || !tryReadDateText(x, istr, time_zone) || !checkChar('\'', istr)) + return false; + + assert_cast(column).getData().push_back(x); + return true; +} + void SerializationDate::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeChar('"', ostr); @@ -66,6 +94,15 @@ void SerializationDate::deserializeTextJSON(IColumn & column, ReadBuffer & istr, assert_cast(column).getData().push_back(x); } +bool SerializationDate::tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +{ + DayNum x; + if (!checkChar('"', istr) || !tryReadDateText(x, istr, time_zone) || !checkChar('"', istr)) + return false; + assert_cast(column).getData().push_back(x); + return true; +} + void SerializationDate::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeChar('"', ostr); @@ -80,6 +117,15 @@ void SerializationDate::deserializeTextCSV(IColumn & column, ReadBuffer & istr, assert_cast(column).getData().push_back(value); } +bool SerializationDate::tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +{ + DayNum value; + if (!tryReadCSV(value, istr, time_zone)) + return false; + assert_cast(column).getData().push_back(value); + return true; +} + SerializationDate::SerializationDate(const DateLUTImpl & time_zone_) : time_zone(time_zone_) { } diff --git a/src/DataTypes/Serializations/SerializationDate.h b/src/DataTypes/Serializations/SerializationDate.h index f751b06fba6..dcf79eb49da 100644 --- a/src/DataTypes/Serializations/SerializationDate.h +++ b/src/DataTypes/Serializations/SerializationDate.h @@ -13,14 +13,19 @@ public: void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; protected: const DateLUTImpl & time_zone; diff --git a/src/DataTypes/Serializations/SerializationDate32.cpp b/src/DataTypes/Serializations/SerializationDate32.cpp index 851710de839..70a22d59e42 100644 --- a/src/DataTypes/Serializations/SerializationDate32.cpp +++ b/src/DataTypes/Serializations/SerializationDate32.cpp @@ -21,6 +21,15 @@ void SerializationDate32::deserializeWholeText(IColumn & column, ReadBuffer & is throwUnexpectedDataAfterParsedValue(column, istr, settings, "Date32"); } +bool SerializationDate32::tryDeserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +{ + ExtendedDayNum x; + if (!tryReadDateText(x, istr, time_zone) || !istr.eof()) + return false; + assert_cast(column).getData().push_back(x); + return true; +} + void SerializationDate32::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { ExtendedDayNum x; @@ -28,6 +37,15 @@ void SerializationDate32::deserializeTextEscaped(IColumn & column, ReadBuffer & assert_cast(column).getData().push_back(x); } +bool SerializationDate32::tryDeserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +{ + ExtendedDayNum x; + if (!tryReadDateText(x, istr, time_zone)) + return false; + assert_cast(column).getData().push_back(x); + return true; +} + void SerializationDate32::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { serializeText(column, row_num, ostr, settings); @@ -49,6 +67,15 @@ void SerializationDate32::deserializeTextQuoted(IColumn & column, ReadBuffer & i assert_cast(column).getData().push_back(x); /// It's important to do this at the end - for exception safety. } +bool SerializationDate32::tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +{ + ExtendedDayNum x; + if (!checkChar('\'', istr) || !tryReadDateText(x, istr, time_zone) || !checkChar('\'', istr)) + return false; + assert_cast(column).getData().push_back(x); /// It's important to do this at the end - for exception safety. + return true; +} + void SerializationDate32::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeChar('"', ostr); @@ -65,6 +92,15 @@ void SerializationDate32::deserializeTextJSON(IColumn & column, ReadBuffer & ist assert_cast(column).getData().push_back(x); } +bool SerializationDate32::tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +{ + ExtendedDayNum x; + if (!checkChar('"', istr) || !tryReadDateText(x, istr, time_zone) || !checkChar('"', istr)) + return false; + assert_cast(column).getData().push_back(x); + return true; +} + void SerializationDate32::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeChar('"', ostr); @@ -79,6 +115,15 @@ void SerializationDate32::deserializeTextCSV(IColumn & column, ReadBuffer & istr assert_cast(column).getData().push_back(value.getExtenedDayNum()); } +bool SerializationDate32::tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +{ + LocalDate value; + if (!tryReadCSV(value, istr)) + return false; + assert_cast(column).getData().push_back(value.getExtenedDayNum()); + return true; +} + SerializationDate32::SerializationDate32(const DateLUTImpl & time_zone_) : time_zone(time_zone_) { } diff --git a/src/DataTypes/Serializations/SerializationDate32.h b/src/DataTypes/Serializations/SerializationDate32.h index 49560fb6c7d..be2e2b76c1d 100644 --- a/src/DataTypes/Serializations/SerializationDate32.h +++ b/src/DataTypes/Serializations/SerializationDate32.h @@ -12,14 +12,19 @@ public: void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; protected: const DateLUTImpl & time_zone; diff --git a/src/DataTypes/Serializations/SerializationDateTime.cpp b/src/DataTypes/Serializations/SerializationDateTime.cpp index 77beb0d9b75..17465d85e9d 100644 --- a/src/DataTypes/Serializations/SerializationDateTime.cpp +++ b/src/DataTypes/Serializations/SerializationDateTime.cpp @@ -21,15 +21,56 @@ inline void readText(time_t & x, ReadBuffer & istr, const FormatSettings & setti switch (settings.date_time_input_format) { case FormatSettings::DateTimeInputFormat::Basic: - readDateTimeText(x, istr, time_zone); - return; + readDateTimeTextImpl<>(x, istr, time_zone); + break; case FormatSettings::DateTimeInputFormat::BestEffort: parseDateTimeBestEffort(x, istr, time_zone, utc_time_zone); - return; + break; case FormatSettings::DateTimeInputFormat::BestEffortUS: parseDateTimeBestEffortUS(x, istr, time_zone, utc_time_zone); - return; + break; } + + if (x < 0) + x = 0; +} + +inline void readAsIntText(time_t & x, ReadBuffer & istr) +{ + readIntText(x, istr); + if (x < 0) + x = 0; +} + +inline bool tryReadText(time_t & x, ReadBuffer & istr, const FormatSettings & settings, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone) +{ + bool res; + switch (settings.date_time_input_format) + { + case FormatSettings::DateTimeInputFormat::Basic: + res = tryReadDateTimeText(x, istr, time_zone); + break; + case FormatSettings::DateTimeInputFormat::BestEffort: + res = tryParseDateTimeBestEffort(x, istr, time_zone, utc_time_zone); + break; + case FormatSettings::DateTimeInputFormat::BestEffortUS: + res = tryParseDateTimeBestEffortUS(x, istr, time_zone, utc_time_zone); + break; + } + + if (x < 0) + x = 0; + + return res; +} + +inline bool tryReadAsIntText(time_t & x, ReadBuffer & istr) +{ + if (!tryReadIntText(x, istr)) + return false; + if (x < 0) + x = 0; + return true; } } @@ -68,15 +109,32 @@ void SerializationDateTime::deserializeWholeText(IColumn & column, ReadBuffer & throwUnexpectedDataAfterParsedValue(column, istr, settings, "DateTime"); } +bool SerializationDateTime::tryDeserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + time_t x = 0; + if (!tryReadText(x, istr, settings, time_zone, utc_time_zone) || !istr.eof()) + return false; + + assert_cast(column).getData().push_back(static_cast(x)); + return true; +} + void SerializationDateTime::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { time_t x = 0; readText(x, istr, settings, time_zone, utc_time_zone); - if (x < 0) - x = 0; assert_cast(column).getData().push_back(static_cast(x)); } +bool SerializationDateTime::tryDeserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + time_t x = 0; + if (!tryReadText(x, istr, settings, time_zone, utc_time_zone)) + return false; + assert_cast(column).getData().push_back(static_cast(x)); + return true; +} + void SerializationDateTime::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeChar('\'', ostr); @@ -94,15 +152,32 @@ void SerializationDateTime::deserializeTextQuoted(IColumn & column, ReadBuffer & } else /// Just 1504193808 or 01504193808 { - readIntText(x, istr); + readAsIntText(x, istr); } - if (x < 0) - x = 0; /// It's important to do this at the end - for exception safety. assert_cast(column).getData().push_back(static_cast(x)); } +bool SerializationDateTime::tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + time_t x = 0; + if (checkChar('\'', istr)) /// Cases: '2017-08-31 18:36:48' or '1504193808' + { + if (!tryReadText(x, istr, settings, time_zone, utc_time_zone) || !checkChar('\'', istr)) + return false; + } + else /// Just 1504193808 or 01504193808 + { + if (!tryReadAsIntText(x, istr)) + return false; + } + + /// It's important to do this at the end - for exception safety. + assert_cast(column).getData().push_back(static_cast(x)); + return true; +} + void SerializationDateTime::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeChar('"', ostr); @@ -120,13 +195,30 @@ void SerializationDateTime::deserializeTextJSON(IColumn & column, ReadBuffer & i } else { - readIntText(x, istr); + readAsIntText(x, istr); } - if (x < 0) - x = 0; + assert_cast(column).getData().push_back(static_cast(x)); } +bool SerializationDateTime::tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + time_t x = 0; + if (checkChar('"', istr)) + { + if (!tryReadText(x, istr, settings, time_zone, utc_time_zone) || !checkChar('"', istr)) + return false; + } + else + { + if (!tryReadIntText(x, istr)) + return false; + } + + assert_cast(column).getData().push_back(static_cast(x)); + return true; +} + void SerializationDateTime::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeChar('"', ostr); @@ -165,13 +257,48 @@ void SerializationDateTime::deserializeTextCSV(IColumn & column, ReadBuffer & is readCSVString(datetime_str, istr, settings.csv); ReadBufferFromString buf(datetime_str); readText(x, buf, settings, time_zone, utc_time_zone); + if (!buf.eof()) + throwUnexpectedDataAfterParsedValue(column, istr, settings, "DateTime"); } } - if (x < 0) - x = 0; - assert_cast(column).getData().push_back(static_cast(x)); } +bool SerializationDateTime::tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + time_t x = 0; + + if (istr.eof()) + return false; + + char maybe_quote = *istr.position(); + + if (maybe_quote == '\'' || maybe_quote == '\"') + { + ++istr.position(); + if (!tryReadText(x, istr, settings, time_zone, utc_time_zone) || !checkChar(maybe_quote, istr)) + return false; + } + else + { + if (settings.csv.delimiter != ',' || settings.date_time_input_format == FormatSettings::DateTimeInputFormat::Basic) + { + if (!tryReadText(x, istr, settings, time_zone, utc_time_zone)) + return false; + } + else + { + String datetime_str; + readCSVString(datetime_str, istr, settings.csv); + ReadBufferFromString buf(datetime_str); + if (!tryReadText(x, buf, settings, time_zone, utc_time_zone) || !buf.eof()) + return false; + } + } + + assert_cast(column).getData().push_back(static_cast(x)); + return true; +} + } diff --git a/src/DataTypes/Serializations/SerializationDateTime.h b/src/DataTypes/Serializations/SerializationDateTime.h index f4a142483e5..584b0c4116b 100644 --- a/src/DataTypes/Serializations/SerializationDateTime.h +++ b/src/DataTypes/Serializations/SerializationDateTime.h @@ -15,14 +15,19 @@ public: void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; }; } diff --git a/src/DataTypes/Serializations/SerializationDateTime64.cpp b/src/DataTypes/Serializations/SerializationDateTime64.cpp index 93891886000..a19619bf8d3 100644 --- a/src/DataTypes/Serializations/SerializationDateTime64.cpp +++ b/src/DataTypes/Serializations/SerializationDateTime64.cpp @@ -47,6 +47,16 @@ void SerializationDateTime64::deserializeText(IColumn & column, ReadBuffer & ist throwUnexpectedDataAfterParsedValue(column, istr, settings, "DateTime64"); } +bool SerializationDateTime64::tryDeserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const +{ + DateTime64 result = 0; + if (tryReadDateTime64Text(result, scale, istr, time_zone) || (whole && istr.eof())) + return false; + + assert_cast(column).getData().push_back(result); + return true; +} + void SerializationDateTime64::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { deserializeTextEscaped(column, istr, settings); @@ -75,6 +85,29 @@ static inline void readText(DateTime64 & x, UInt32 scale, ReadBuffer & istr, con } } +static inline bool tryReadText(DateTime64 & x, UInt32 scale, ReadBuffer & istr, const FormatSettings & settings, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone) +{ + switch (settings.date_time_input_format) + { + case FormatSettings::DateTimeInputFormat::Basic: + return tryReadDateTime64Text(x, scale, istr, time_zone); + case FormatSettings::DateTimeInputFormat::BestEffort: + return tryParseDateTime64BestEffort(x, scale, istr, time_zone, utc_time_zone); + case FormatSettings::DateTimeInputFormat::BestEffortUS: + return tryParseDateTime64BestEffortUS(x, scale, istr, time_zone, utc_time_zone); + } +} + + +bool SerializationDateTime64::tryDeserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + DateTime64 x = 0; + if (!tryReadText(x, scale, istr, settings, time_zone, utc_time_zone) || !istr.eof()) + return false; + assert_cast(column).getData().push_back(x); + return true; +} + void SerializationDateTime64::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { DateTime64 x = 0; @@ -82,6 +115,15 @@ void SerializationDateTime64::deserializeTextEscaped(IColumn & column, ReadBuffe assert_cast(column).getData().push_back(x); } +bool SerializationDateTime64::tryDeserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + DateTime64 x = 0; + if (!tryReadText(x, scale, istr, settings, time_zone, utc_time_zone)) + return false; + assert_cast(column).getData().push_back(x); + return true; +} + void SerializationDateTime64::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeChar('\'', ostr); @@ -104,6 +146,23 @@ void SerializationDateTime64::deserializeTextQuoted(IColumn & column, ReadBuffer assert_cast(column).getData().push_back(x); /// It's important to do this at the end - for exception safety. } +bool SerializationDateTime64::tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + DateTime64 x = 0; + if (checkChar('\'', istr)) /// Cases: '2017-08-31 18:36:48' or '1504193808' + { + if (tryReadText(x, scale, istr, settings, time_zone, utc_time_zone) || !checkChar('\'', istr)) + return false; + } + else /// Just 1504193808 or 01504193808 + { + if (!tryReadIntText(x, istr)) + return false; + } + assert_cast(column).getData().push_back(x); /// It's important to do this at the end - for exception safety. + return true; +} + void SerializationDateTime64::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeChar('"', ostr); @@ -126,6 +185,23 @@ void SerializationDateTime64::deserializeTextJSON(IColumn & column, ReadBuffer & assert_cast(column).getData().push_back(x); } +bool SerializationDateTime64::tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + DateTime64 x = 0; + if (checkChar('"', istr)) + { + if (!tryReadText(x, scale, istr, settings, time_zone, utc_time_zone) || !checkChar('"', istr)) + return false; + } + else + { + if (!tryReadIntText(x, istr)) + return false; + } + assert_cast(column).getData().push_back(x); + return true; +} + void SerializationDateTime64::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeChar('"', ostr); @@ -170,4 +246,40 @@ void SerializationDateTime64::deserializeTextCSV(IColumn & column, ReadBuffer & assert_cast(column).getData().push_back(x); } +bool SerializationDateTime64::tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + DateTime64 x = 0; + + if (istr.eof()) + return false; + + char maybe_quote = *istr.position(); + + if (maybe_quote == '\'' || maybe_quote == '\"') + { + ++istr.position(); + if (!tryReadText(x, scale, istr, settings, time_zone, utc_time_zone) || !checkChar(maybe_quote, istr)) + return false; + } + else + { + if (settings.csv.delimiter != ',' || settings.date_time_input_format == FormatSettings::DateTimeInputFormat::Basic) + { + if (tryReadText(x, scale, istr, settings, time_zone, utc_time_zone)) + return false; + } + else + { + String datetime_str; + readCSVString(datetime_str, istr, settings.csv); + ReadBufferFromString buf(datetime_str); + if (!tryReadText(x, scale, buf, settings, time_zone, utc_time_zone) || !buf.eof()) + return false; + } + } + + assert_cast(column).getData().push_back(x); + return true; +} + } diff --git a/src/DataTypes/Serializations/SerializationDateTime64.h b/src/DataTypes/Serializations/SerializationDateTime64.h index f817edbf0dd..b49bd1e9098 100644 --- a/src/DataTypes/Serializations/SerializationDateTime64.h +++ b/src/DataTypes/Serializations/SerializationDateTime64.h @@ -15,15 +15,21 @@ public: void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const override; + bool tryDeserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const override; void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; }; } diff --git a/src/DataTypes/Serializations/SerializationDecimal.cpp b/src/DataTypes/Serializations/SerializationDecimal.cpp index b576b7a048c..d632c224783 100644 --- a/src/DataTypes/Serializations/SerializationDecimal.cpp +++ b/src/DataTypes/Serializations/SerializationDecimal.cpp @@ -16,11 +16,19 @@ namespace ErrorCodes } template -bool SerializationDecimal::tryReadText(T & x, ReadBuffer & istr, UInt32 precision, UInt32 scale) +bool SerializationDecimal::tryReadText(T & x, ReadBuffer & istr, UInt32 precision, UInt32 scale, bool csv) { UInt32 unread_scale = scale; - if (!tryReadDecimalText(istr, x, precision, unread_scale)) - return false; + if (csv) + { + if (!tryReadCSVDecimalText(istr, x, precision, unread_scale)) + return false; + } + else + { + if (!tryReadDecimalText(istr, x, precision, unread_scale)) + return false; + } if (common::mulOverflow(x.value, DecimalUtils::scaleMultiplier(unread_scale), x.value)) return false; @@ -59,6 +67,16 @@ void SerializationDecimal::deserializeText(IColumn & column, ReadBuffer & ist ISerialization::throwUnexpectedDataAfterParsedValue(column, istr, settings, "Decimal"); } +template +bool SerializationDecimal::tryDeserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const +{ + T x; + if (!tryReadText(x, istr) || (whole && !istr.eof())) + return false; + assert_cast(column).getData().push_back(x); + return true; +} + template void SerializationDecimal::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { @@ -67,6 +85,16 @@ void SerializationDecimal::deserializeTextCSV(IColumn & column, ReadBuffer & assert_cast(column).getData().push_back(x); } +template +bool SerializationDecimal::tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +{ + T x; + if (!tryReadText(x, istr, true)) + return false; + assert_cast(column).getData().push_back(x); + return true; +} + template void SerializationDecimal::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { @@ -88,6 +116,18 @@ void SerializationDecimal::deserializeTextJSON(IColumn & column, ReadBuffer & assertChar('"', istr); } +template +bool SerializationDecimal::tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +{ + bool have_quotes = checkChar('"', istr); + T x; + if (!tryReadText(x, istr) || (have_quotes && !checkChar('"', istr))) + return false; + + assert_cast(column).getData().push_back(x); + return true; +} + template class SerializationDecimal; template class SerializationDecimal; diff --git a/src/DataTypes/Serializations/SerializationDecimal.h b/src/DataTypes/Serializations/SerializationDecimal.h index 57decdd0973..22a8eb1a47c 100644 --- a/src/DataTypes/Serializations/SerializationDecimal.h +++ b/src/DataTypes/Serializations/SerializationDecimal.h @@ -16,15 +16,19 @@ public: void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const override; + bool tryDeserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void readText(T & x, ReadBuffer & istr, bool csv = false) const { readText(x, istr, this->precision, this->scale, csv); } + bool tryReadText(T & x, ReadBuffer & istr, bool csv = false) const { return tryReadText(x, istr, this->precision, this->scale, csv); } static void readText(T & x, ReadBuffer & istr, UInt32 precision_, UInt32 scale_, bool csv = false); - static bool tryReadText(T & x, ReadBuffer & istr, UInt32 precision_, UInt32 scale_); + static bool tryReadText(T & x, ReadBuffer & istr, UInt32 precision_, UInt32 scale_, bool csv = false); }; } diff --git a/src/DataTypes/Serializations/SerializationEnum.cpp b/src/DataTypes/Serializations/SerializationEnum.cpp index 9b3a437e9cf..6ad55913738 100644 --- a/src/DataTypes/Serializations/SerializationEnum.cpp +++ b/src/DataTypes/Serializations/SerializationEnum.cpp @@ -34,6 +34,27 @@ void SerializationEnum::deserializeTextEscaped(IColumn & column, ReadBuffe } } +template +bool SerializationEnum::tryDeserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + FieldType x; + if (settings.tsv.enum_as_number) + { + if (!tryReadValue(istr, x)) + return false; + } + else + { + std::string field_name; + readEscapedString(field_name, istr); + if (!this->tryGetValue(x, StringRef(field_name), true)) + return false; + } + + assert_cast(column).getData().push_back(x); + return true; +} + template void SerializationEnum::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { @@ -48,6 +69,18 @@ void SerializationEnum::deserializeTextQuoted(IColumn & column, ReadBuffer assert_cast(column).getData().push_back(ref_enum_values.getValue(StringRef(field_name))); } +template +bool SerializationEnum::tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +{ + std::string field_name; + readQuotedStringWithSQLStyle(field_name, istr); + FieldType x; + if (!this->tryGetValue(x, StringRef(field_name))) + return false; + assert_cast(column).getData().push_back(x); + return true; +} + template void SerializationEnum::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { @@ -65,6 +98,27 @@ void SerializationEnum::deserializeWholeText(IColumn & column, ReadBuffer } } +template +bool SerializationEnum::tryDeserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + FieldType x; + if (settings.tsv.enum_as_number) + { + if (!tryReadValue(istr, x) || !istr.eof()) + return false; + } + else + { + std::string field_name; + readStringUntilEOF(field_name, istr); + if (!this->tryGetValue(x, StringRef(field_name), true)) + return false; + } + + assert_cast(column).getData().push_back(x); + return true; +} + template void SerializationEnum::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { @@ -90,6 +144,27 @@ void SerializationEnum::deserializeTextJSON(IColumn & column, ReadBuffer & } } +template +bool SerializationEnum::tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +{ + FieldType x; + if (!istr.eof() && *istr.position() != '"') + { + if (!tryReadValue(istr, x)) + return false; + } + else + { + std::string field_name; + readJSONString(field_name, istr); + if (!this->tryGetValue(x, StringRef(field_name))) + return false; + } + + assert_cast(column).getData().push_back(x); + return true; +} + template void SerializationEnum::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { @@ -109,6 +184,28 @@ void SerializationEnum::deserializeTextCSV(IColumn & column, ReadBuffer & } } +template +bool SerializationEnum::tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + FieldType x; + + if (settings.csv.enum_as_number) + { + if (!tryReadValue(istr, x)) + return false; + } + else + { + std::string field_name; + readCSVString(field_name, istr, settings.csv); + if (!this->tryGetValue(x, StringRef(field_name), true)) + return false; + } + + assert_cast(column).getData().push_back(x); + return true; +} + template void SerializationEnum::serializeTextMarkdown( const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const diff --git a/src/DataTypes/Serializations/SerializationEnum.h b/src/DataTypes/Serializations/SerializationEnum.h index 03b134e59a6..708161dc5fd 100644 --- a/src/DataTypes/Serializations/SerializationEnum.h +++ b/src/DataTypes/Serializations/SerializationEnum.h @@ -34,15 +34,20 @@ public: void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeTextMarkdown(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; @@ -53,6 +58,14 @@ public: return ref_enum_values.findByValue(x)->first; } + bool tryReadValue(ReadBuffer & istr, FieldType & x) const + { + if (!tryReadText(x, istr) || !this->hasValue(x)) + return false; + + return true; + } + std::optional> own_enum_values; std::shared_ptr> own_enum_type; const EnumValues & ref_enum_values; diff --git a/src/DataTypes/Serializations/SerializationFixedString.cpp b/src/DataTypes/Serializations/SerializationFixedString.cpp index fa50af52f2f..23e959d80c9 100644 --- a/src/DataTypes/Serializations/SerializationFixedString.cpp +++ b/src/DataTypes/Serializations/SerializationFixedString.cpp @@ -150,12 +150,49 @@ static inline void read(const SerializationFixedString & self, IColumn & column, } } +bool SerializationFixedString::tryAlignStringLength(size_t n, PaddedPODArray & data, size_t string_start) +{ + size_t length = data.size() - string_start; + if (length < n) + { + data.resize_fill(string_start + n); + } + else if (length > n) + { + data.resize_assume_reserved(string_start); + return false; + } + + return true; +} + +template +static inline bool tryRead(const SerializationFixedString & self, IColumn & column, Reader && reader) +{ + ColumnFixedString::Chars & data = typeid_cast(column).getChars(); + size_t prev_size = data.size(); + try + { + return reader(data) && SerializationFixedString::tryAlignStringLength(self.getN(), data, prev_size); + } + catch (...) + { + data.resize_assume_reserved(prev_size); + return false; + } +} + void SerializationFixedString::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { read(*this, column, [&istr](ColumnFixedString::Chars & data) { readEscapedStringInto(data, istr); }); } +bool SerializationFixedString::tryDeserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +{ + return tryRead(*this, column, [&istr](ColumnFixedString::Chars & data) { readEscapedStringInto(data, istr); return true; }); +} + void SerializationFixedString::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { @@ -169,12 +206,22 @@ void SerializationFixedString::deserializeTextQuoted(IColumn & column, ReadBuffe read(*this, column, [&istr](ColumnFixedString::Chars & data) { readQuotedStringInto(data, istr); }); } +bool SerializationFixedString::tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +{ + return tryRead(*this, column, [&istr](ColumnFixedString::Chars & data) { return tryReadQuotedStringInto(data, istr); }); +} + void SerializationFixedString::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { read(*this, column, [&istr](ColumnFixedString::Chars & data) { readStringUntilEOFInto(data, istr); }); } +bool SerializationFixedString::tryDeserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +{ + return tryRead(*this, column, [&istr](ColumnFixedString::Chars & data) { readStringUntilEOFInto(data, istr); return true; }); +} + void SerializationFixedString::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { @@ -188,6 +235,10 @@ void SerializationFixedString::deserializeTextJSON(IColumn & column, ReadBuffer read(*this, column, [&istr](ColumnFixedString::Chars & data) { readJSONStringInto(data, istr); }); } +bool SerializationFixedString::tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +{ + return tryRead(*this, column, [&istr](ColumnFixedString::Chars & data) { return tryReadJSONStringInto(data, istr); }); +} void SerializationFixedString::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { @@ -208,6 +259,11 @@ void SerializationFixedString::deserializeTextCSV(IColumn & column, ReadBuffer & read(*this, column, [&istr, &csv = settings.csv](ColumnFixedString::Chars & data) { readCSVStringInto(data, istr, csv); }); } +bool SerializationFixedString::tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + return tryRead(*this, column, [&istr, &csv = settings.csv](ColumnFixedString::Chars & data) { readCSVStringInto(data, istr, csv); return true; }); +} + void SerializationFixedString::serializeTextMarkdown( const DB::IColumn & column, size_t row_num, DB::WriteBuffer & ostr, const DB::FormatSettings & settings) const { diff --git a/src/DataTypes/Serializations/SerializationFixedString.h b/src/DataTypes/Serializations/SerializationFixedString.h index c27b10ad158..8eb4eacdbff 100644 --- a/src/DataTypes/Serializations/SerializationFixedString.h +++ b/src/DataTypes/Serializations/SerializationFixedString.h @@ -26,20 +26,25 @@ public: void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextMarkdown(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; @@ -47,6 +52,7 @@ public: /// If the length is less than getN() the function will add zero characters up to getN(). /// If the length is greater than getN() the function will throw an exception. static void alignStringLength(size_t n, PaddedPODArray & data, size_t string_start); + static bool tryAlignStringLength(size_t n, PaddedPODArray & data, size_t string_start); }; } diff --git a/src/DataTypes/Serializations/SerializationIPv4andIPv6.cpp b/src/DataTypes/Serializations/SerializationIPv4andIPv6.cpp new file mode 100644 index 00000000000..81c4af97401 --- /dev/null +++ b/src/DataTypes/Serializations/SerializationIPv4andIPv6.cpp @@ -0,0 +1,188 @@ +#include + +namespace DB +{ + +template +void SerializationIP::serializeText(const DB::IColumn & column, size_t row_num, DB::WriteBuffer & ostr, const DB::FormatSettings &) const +{ + writeText(assert_cast &>(column).getData()[row_num], ostr); +} + +template +void SerializationIP::deserializeText(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings & settings, bool whole) const +{ + IPv x; + readText(x, istr); + + if (whole && !istr.eof()) + throwUnexpectedDataAfterParsedValue(column, istr, settings, TypeName.data()); + + assert_cast &>(column).getData().push_back(x); +} + +template +bool SerializationIP::tryDeserializeText(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings &, bool whole) const +{ + IPv x; + if (!tryReadText(x, istr) || (whole && !istr.eof())) + return false; + + assert_cast &>(column).getData().push_back(x); + return true; +} + +template +void SerializationIP::serializeTextQuoted(const DB::IColumn & column, size_t row_num, DB::WriteBuffer & ostr, const DB::FormatSettings & settings) const +{ + writeChar('\'', ostr); + serializeText(column, row_num, ostr, settings); + writeChar('\'', ostr); +} + +template +void SerializationIP::deserializeTextQuoted(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings &) const +{ + IPv x; + assertChar('\'', istr); + readText(x, istr); + assertChar('\'', istr); + assert_cast &>(column).getData().push_back(x); /// It's important to do this at the end - for exception safety. +} + +template +bool SerializationIP::tryDeserializeTextQuoted(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings &) const +{ + IPv x; + if (!checkChar('\'', istr) || !tryReadText(x, istr) || !checkChar('\'', istr)) + return false; + assert_cast &>(column).getData().push_back(x); + return true; +} + +template +void SerializationIP::serializeTextJSON(const DB::IColumn & column, size_t row_num, DB::WriteBuffer & ostr, const DB::FormatSettings & settings) const +{ + writeChar('"', ostr); + serializeText(column, row_num, ostr, settings); + writeChar('"', ostr); +} + +template +void SerializationIP::deserializeTextJSON(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings & settings) const +{ + IPv x; + assertChar('"', istr); + readText(x, istr); + /// this code looks weird, but we want to throw specific exception to match original behavior... + if (istr.eof()) + assertChar('"', istr); + if (*istr.position() != '"') + throwUnexpectedDataAfterParsedValue(column, istr, settings, TypeName.data()); + istr.ignore(); + + assert_cast &>(column).getData().push_back(x); +} + +template +bool SerializationIP::tryDeserializeTextJSON(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings &) const +{ + IPv x; + if (!checkChar('"', istr) || !tryReadText(x, istr) || !checkChar('"', istr)) + return false; + + assert_cast &>(column).getData().push_back(x); + return true; +} + +template +void SerializationIP::serializeTextCSV(const DB::IColumn & column, size_t row_num, DB::WriteBuffer & ostr, const DB::FormatSettings & settings) const +{ + writeChar('"', ostr); + serializeText(column, row_num, ostr, settings); + writeChar('"', ostr); +} + +template +void SerializationIP::deserializeTextCSV(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings &) const +{ + IPv value; + readCSV(value, istr); + + assert_cast &>(column).getData().push_back(value); +} + +template +bool SerializationIP::tryDeserializeTextCSV(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings &) const +{ + IPv value; + if (!tryReadCSV(value, istr)) + return false; + + assert_cast &>(column).getData().push_back(value); + return true; +} + +template +void SerializationIP::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const +{ + IPv x = field.get(); + if constexpr (std::is_same_v) + writeBinary(x, ostr); + else + writeBinaryLittleEndian(x, ostr); +} + +template +void SerializationIP::deserializeBinary(DB::Field & field, DB::ReadBuffer & istr, const DB::FormatSettings &) const +{ + IPv x; + if constexpr (std::is_same_v) + readBinary(x, istr); + else + readBinaryLittleEndian(x, istr); + field = NearestFieldType(x); +} + +template +void SerializationIP::serializeBinary(const DB::IColumn & column, size_t row_num, DB::WriteBuffer & ostr, const DB::FormatSettings &) const +{ + writeBinary(assert_cast &>(column).getData()[row_num], ostr); +} + +template +void SerializationIP::deserializeBinary(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings &) const +{ + IPv x; + readBinary(x.toUnderType(), istr); + assert_cast &>(column).getData().push_back(x); +} + +template +void SerializationIP::serializeBinaryBulk(const DB::IColumn & column, DB::WriteBuffer & ostr, size_t offset, size_t limit) const +{ + const typename ColumnVector::Container & x = typeid_cast &>(column).getData(); + + size_t size = x.size(); + + if (limit == 0 || offset + limit > size) + limit = size - offset; + + if (limit) + ostr.write(reinterpret_cast(&x[offset]), sizeof(IPv) * limit); +} + +template +void SerializationIP::deserializeBinaryBulk(DB::IColumn & column, DB::ReadBuffer & istr, size_t limit, double) const +{ + typename ColumnVector::Container & x = typeid_cast &>(column).getData(); + size_t initial_size = x.size(); + x.resize(initial_size + limit); + size_t size = istr.readBig(reinterpret_cast(&x[initial_size]), sizeof(IPv) * limit); + x.resize(initial_size + size / sizeof(IPv)); +} + +template class SerializationIP; +template class SerializationIP; + +} diff --git a/src/DataTypes/Serializations/SerializationIPv4andIPv6.h b/src/DataTypes/Serializations/SerializationIPv4andIPv6.h index 7d8669fd444..a53f257646b 100644 --- a/src/DataTypes/Serializations/SerializationIPv4andIPv6.h +++ b/src/DataTypes/Serializations/SerializationIPv4andIPv6.h @@ -13,123 +13,30 @@ template class SerializationIP : public SimpleTextSerialization { public: - void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override - { - writeText(assert_cast &>(column).getData()[row_num], ostr); - } - void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const override - { - IPv x; - readText(x, istr); + void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const override; + bool tryDeserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const override; - if (whole && !istr.eof()) - throwUnexpectedDataAfterParsedValue(column, istr, settings, TypeName.data()); + void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; - assert_cast &>(column).getData().push_back(x); - } - void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override - { - serializeText(column, row_num, ostr, settings); - } - void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override - { - deserializeText(column, istr, settings, false); - } - void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override - { - writeChar('\'', ostr); - serializeText(column, row_num, ostr, settings); - writeChar('\'', ostr); - } - void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override - { - IPv x; - assertChar('\'', istr); - readText(x, istr); - assertChar('\'', istr); - assert_cast &>(column).getData().push_back(x); /// It's important to do this at the end - for exception safety. - } - void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override - { - writeChar('"', ostr); - serializeText(column, row_num, ostr, settings); - writeChar('"', ostr); - } - void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override - { - IPv x; - assertChar('"', istr); - readText(x, istr); - /// this code looks weird, but we want to throw specific exception to match original behavior... - if (istr.eof()) - assertChar('"', istr); - if (*istr.position() != '"') - throwUnexpectedDataAfterParsedValue(column, istr, settings, TypeName.data()); - istr.ignore(); + void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; - assert_cast &>(column).getData().push_back(x); - } - void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override - { - writeChar('"', ostr); - serializeText(column, row_num, ostr, settings); - writeChar('"', ostr); - } - void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &/* settings*/) const override - { - IPv value; - readCSV(value, istr); + void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &/* settings*/) const override; + bool tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &/* settings*/) const override; - assert_cast &>(column).getData().push_back(value); - } + void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const override; - void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const override - { - IPv x = field.get(); - if constexpr (std::is_same_v) - writeBinary(x, ostr); - else - writeBinaryLittleEndian(x, ostr); - } - void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const override - { - IPv x; - if constexpr (std::is_same_v) - readBinary(x, istr); - else - readBinaryLittleEndian(x, istr); - field = NearestFieldType(x); - } - void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override - { - writeBinary(assert_cast &>(column).getData()[row_num], ostr); - } - void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override - { - IPv x; - readBinary(x.toUnderType(), istr); - assert_cast &>(column).getData().push_back(x); - } - void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override - { - const typename ColumnVector::Container & x = typeid_cast &>(column).getData(); + void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; - size_t size = x.size(); - - if (limit == 0 || offset + limit > size) - limit = size - offset; - - if (limit) - ostr.write(reinterpret_cast(&x[offset]), sizeof(IPv) * limit); - } - void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double /*avg_value_size_hint*/) const override - { - typename ColumnVector::Container & x = typeid_cast &>(column).getData(); - size_t initial_size = x.size(); - x.resize(initial_size + limit); - size_t size = istr.readBig(reinterpret_cast(&x[initial_size]), sizeof(IPv) * limit); - x.resize(initial_size + size / sizeof(IPv)); - } + void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override; + void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double /*avg_value_size_hint*/) const override; }; using SerializationIPv4 = SerializationIP; diff --git a/src/DataTypes/Serializations/SerializationLowCardinality.cpp b/src/DataTypes/Serializations/SerializationLowCardinality.cpp index 3e1cbdb00f5..9efe05042ed 100644 --- a/src/DataTypes/Serializations/SerializationLowCardinality.cpp +++ b/src/DataTypes/Serializations/SerializationLowCardinality.cpp @@ -700,6 +700,11 @@ void SerializationLowCardinality::deserializeTextEscaped(IColumn & column, ReadB deserializeImpl(column, &ISerialization::deserializeTextEscaped, istr, settings); } +bool SerializationLowCardinality::tryDeserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + return tryDeserializeImpl(column, &ISerialization::tryDeserializeTextEscaped, istr, settings); +} + void SerializationLowCardinality::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { serializeImpl(column, row_num, &ISerialization::serializeTextQuoted, ostr, settings); @@ -710,11 +715,21 @@ void SerializationLowCardinality::deserializeTextQuoted(IColumn & column, ReadBu deserializeImpl(column, &ISerialization::deserializeTextQuoted, istr, settings); } +bool SerializationLowCardinality::tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + return tryDeserializeImpl(column, &ISerialization::tryDeserializeTextQuoted, istr, settings); +} + void SerializationLowCardinality::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { deserializeImpl(column, &ISerialization::deserializeWholeText, istr, settings); } +bool SerializationLowCardinality::tryDeserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + return tryDeserializeImpl(column, &ISerialization::tryDeserializeWholeText, istr, settings); +} + void SerializationLowCardinality::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { serializeImpl(column, row_num, &ISerialization::serializeTextCSV, ostr, settings); @@ -725,6 +740,11 @@ void SerializationLowCardinality::deserializeTextCSV(IColumn & column, ReadBuffe deserializeImpl(column, &ISerialization::deserializeTextCSV, istr, settings); } +bool SerializationLowCardinality::tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + return tryDeserializeImpl(column, &ISerialization::tryDeserializeTextCSV, istr, settings); +} + void SerializationLowCardinality::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { serializeImpl(column, row_num, &ISerialization::serializeText, ostr, settings); @@ -740,6 +760,11 @@ void SerializationLowCardinality::deserializeTextJSON(IColumn & column, ReadBuff deserializeImpl(column, &ISerialization::deserializeTextJSON, istr, settings); } +bool SerializationLowCardinality::tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + return tryDeserializeImpl(column, &ISerialization::tryDeserializeTextJSON, istr, settings); +} + void SerializationLowCardinality::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { serializeImpl(column, row_num, &ISerialization::serializeTextXML, ostr, settings); @@ -750,6 +775,11 @@ void SerializationLowCardinality::deserializeTextRaw(IColumn & column, ReadBuffe deserializeImpl(column, &ISerialization::deserializeTextRaw, istr, settings); } +bool SerializationLowCardinality::tryDeserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + return tryDeserializeImpl(column, &ISerialization::tryDeserializeTextRaw, istr, settings); +} + void SerializationLowCardinality::serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { serializeImpl(column, row_num, &ISerialization::serializeTextRaw, ostr, settings); @@ -769,7 +799,7 @@ template void SerializationLowCardinality::deserializeImpl( IColumn & column, SerializationLowCardinality::DeserializeFunctionPtr func, Args &&... args) const { - auto & low_cardinality_column= getColumnLowCardinality(column); + auto & low_cardinality_column = getColumnLowCardinality(column); auto temp_column = low_cardinality_column.getDictionary().getNestedColumn()->cloneEmpty(); auto serialization = dictionary_type->getDefaultSerialization(); @@ -778,4 +808,19 @@ void SerializationLowCardinality::deserializeImpl( low_cardinality_column.insertFromFullColumn(*temp_column, 0); } +template +bool SerializationLowCardinality::tryDeserializeImpl( + IColumn & column, SerializationLowCardinality::TryDeserializeFunctionPtr func, Args &&... args) const +{ + auto & low_cardinality_column = getColumnLowCardinality(column); + auto temp_column = low_cardinality_column.getDictionary().getNestedColumn()->cloneEmpty(); + + auto serialization = dictionary_type->getDefaultSerialization(); + if (!(serialization.get()->*func)(*temp_column, std::forward(args)...)) + return false; + + low_cardinality_column.insertFromFullColumn(*temp_column, 0); + return true; +} + } diff --git a/src/DataTypes/Serializations/SerializationLowCardinality.h b/src/DataTypes/Serializations/SerializationLowCardinality.h index 5f56bcf8108..d2c3a95c702 100644 --- a/src/DataTypes/Serializations/SerializationLowCardinality.h +++ b/src/DataTypes/Serializations/SerializationLowCardinality.h @@ -55,16 +55,22 @@ public: void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; void deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; private: @@ -79,6 +85,12 @@ private: template void deserializeImpl(IColumn & column, DeserializeFunctionPtr func, Args &&... args) const; + + template + using TryDeserializeFunctionPtr = bool (ISerialization::*)(IColumn &, Params ...) const; + + template + bool tryDeserializeImpl(IColumn & column, TryDeserializeFunctionPtr func, Args &&... args) const; }; } diff --git a/src/DataTypes/Serializations/SerializationMap.cpp b/src/DataTypes/Serializations/SerializationMap.cpp index 7588e630689..7b6f87baf2e 100644 --- a/src/DataTypes/Serializations/SerializationMap.cpp +++ b/src/DataTypes/Serializations/SerializationMap.cpp @@ -115,9 +115,11 @@ void SerializationMap::serializeTextImpl( writeChar('}', ostr); } -template -void SerializationMap::deserializeTextImpl(IColumn & column, ReadBuffer & istr, Reader && reader) const +template +ReturnType SerializationMap::deserializeTextImpl(IColumn & column, ReadBuffer & istr, Reader && reader) const { + static constexpr bool throw_exception = std::is_same_v; + auto & column_map = assert_cast(column); auto & nested_array = column_map.getNestedColumn(); @@ -128,7 +130,21 @@ void SerializationMap::deserializeTextImpl(IColumn & column, ReadBuffer & istr, auto & value_column = nested_tuple.getColumn(1); size_t size = 0; - assertChar('{', istr); + if constexpr (throw_exception) + assertChar('{', istr); + else if (!checkChar('{', istr)) + return ReturnType(false); + + auto on_error_no_throw = [&]() + { + if (size) + { + nested_tuple.getColumnPtr(0) = key_column.cut(0, offsets.back()); + nested_tuple.getColumnPtr(1) = value_column.cut(0, offsets.back()); + } + + return ReturnType(false); + }; try { @@ -138,9 +154,15 @@ void SerializationMap::deserializeTextImpl(IColumn & column, ReadBuffer & istr, if (!first) { if (*istr.position() == ',') + { ++istr.position(); + } else - throw Exception(ErrorCodes::CANNOT_READ_MAP_FROM_TEXT, "Cannot read Map from text"); + { + if constexpr (throw_exception) + throw Exception(ErrorCodes::CANNOT_READ_MAP_FROM_TEXT, "Cannot read Map from text"); + return on_error_no_throw(); + } } first = false; @@ -150,19 +172,32 @@ void SerializationMap::deserializeTextImpl(IColumn & column, ReadBuffer & istr, if (*istr.position() == '}') break; - reader(istr, key, key_column); + if constexpr (throw_exception) + reader(istr, key, key_column); + else if (!reader(istr, key, key_column)) + return on_error_no_throw(); + ++size; skipWhitespaceIfAny(istr); - assertChar(':', istr); + if constexpr (throw_exception) + assertChar(':', istr); + else if (!checkChar(':', istr)) + return on_error_no_throw(); skipWhitespaceIfAny(istr); - reader(istr, value, value_column); + if constexpr (throw_exception) + reader(istr, value, value_column); + else if (!reader(istr, value, value_column)) + return on_error_no_throw(); skipWhitespaceIfAny(istr); } - assertChar('}', istr); + if constexpr (throw_exception) + assertChar('}', istr); + else if (!checkChar('}', istr)) + return on_error_no_throw(); } catch (...) { @@ -171,10 +206,14 @@ void SerializationMap::deserializeTextImpl(IColumn & column, ReadBuffer & istr, nested_tuple.getColumnPtr(0) = key_column.cut(0, offsets.back()); nested_tuple.getColumnPtr(1) = value_column.cut(0, offsets.back()); } - throw; + + if constexpr (throw_exception) + throw; + return ReturnType(false); } offsets.push_back(offsets.back() + size); + return ReturnType(true); } void SerializationMap::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -192,8 +231,8 @@ void SerializationMap::deserializeText(IColumn & column, ReadBuffer & istr, cons deserializeTextImpl(column, istr, [&settings](ReadBuffer & buf, const SerializationPtr & subcolumn_serialization, IColumn & subcolumn) { - if (settings.null_as_default) - SerializationNullable::deserializeTextQuotedImpl(subcolumn, buf, settings, subcolumn_serialization); + if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(subcolumn)) + SerializationNullable::deserializeNullAsDefaultOrNestedTextQuoted(subcolumn, buf, settings, subcolumn_serialization); else subcolumn_serialization->deserializeTextQuoted(subcolumn, buf, settings); }); @@ -202,6 +241,28 @@ void SerializationMap::deserializeText(IColumn & column, ReadBuffer & istr, cons throwUnexpectedDataAfterParsedValue(column, istr, settings, "Map"); } +bool SerializationMap::tryDeserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const +{ + auto reader = [&settings](ReadBuffer & buf, const SerializationPtr & subcolumn_serialization, IColumn & subcolumn) + { + if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(subcolumn)) + return SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextQuoted(subcolumn, buf, settings, subcolumn_serialization); + return subcolumn_serialization->tryDeserializeTextQuoted(subcolumn, buf, settings); + }; + + auto ok = deserializeTextImpl(column, istr, reader); + if (!ok) + return false; + + if (whole && !istr.eof()) + { + column.popBack(1); + return false; + } + + return true; +} + void SerializationMap::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { serializeTextImpl(column, row_num, ostr, @@ -260,13 +321,25 @@ void SerializationMap::deserializeTextJSON(IColumn & column, ReadBuffer & istr, deserializeTextImpl(column, istr, [&settings](ReadBuffer & buf, const SerializationPtr & subcolumn_serialization, IColumn & subcolumn) { - if (settings.null_as_default) - SerializationNullable::deserializeTextJSONImpl(subcolumn, buf, settings, subcolumn_serialization); + if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(subcolumn)) + SerializationNullable::deserializeNullAsDefaultOrNestedTextJSON(subcolumn, buf, settings, subcolumn_serialization); else subcolumn_serialization->deserializeTextJSON(subcolumn, buf, settings); }); } +bool SerializationMap::tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + auto reader = [&settings](ReadBuffer & buf, const SerializationPtr & subcolumn_serialization, IColumn & subcolumn) + { + if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(subcolumn)) + return SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextJSON(subcolumn, buf, settings, subcolumn_serialization); + return subcolumn_serialization->tryDeserializeTextJSON(subcolumn, buf, settings); + }; + + return deserializeTextImpl(column, istr, reader); +} + void SerializationMap::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { const auto & column_map = assert_cast(column); @@ -308,6 +381,15 @@ void SerializationMap::deserializeTextCSV(IColumn & column, ReadBuffer & istr, c deserializeText(column, rb, settings, true); } +bool SerializationMap::tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + String s; + if (!tryReadCSV(s, istr, settings.csv)) + return false; + ReadBufferFromString rb(s); + return tryDeserializeText(column, rb, settings, true); +} + void SerializationMap::enumerateStreams( EnumerateStreamsSettings & settings, const StreamCallback & callback, diff --git a/src/DataTypes/Serializations/SerializationMap.h b/src/DataTypes/Serializations/SerializationMap.h index f32c656757d..3e27ef1b04a 100644 --- a/src/DataTypes/Serializations/SerializationMap.h +++ b/src/DataTypes/Serializations/SerializationMap.h @@ -24,13 +24,16 @@ public: void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const override; + bool tryDeserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const override; void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextJSONPretty(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t indent) const override; void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void enumerateStreams( EnumerateStreamsSettings & settings, @@ -68,8 +71,8 @@ private: template void serializeTextImpl(const IColumn & column, size_t row_num, WriteBuffer & ostr, KeyWriter && key_writer, ValueWriter && value_writer) const; - template - void deserializeTextImpl(IColumn & column, ReadBuffer & istr, Reader && reader) const; + template + ReturnType deserializeTextImpl(IColumn & column, ReadBuffer & istr, Reader && reader) const; }; } diff --git a/src/DataTypes/Serializations/SerializationNamed.cpp b/src/DataTypes/Serializations/SerializationNamed.cpp index ca60948ce68..1a9cbe9a37d 100644 --- a/src/DataTypes/Serializations/SerializationNamed.cpp +++ b/src/DataTypes/Serializations/SerializationNamed.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB { diff --git a/src/DataTypes/Serializations/SerializationNothing.h b/src/DataTypes/Serializations/SerializationNothing.h index 02974d1ca76..7d1fff55b01 100644 --- a/src/DataTypes/Serializations/SerializationNothing.h +++ b/src/DataTypes/Serializations/SerializationNothing.h @@ -25,6 +25,7 @@ public: void deserializeBinary(IColumn &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); } void serializeText(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); } void deserializeText(IColumn &, ReadBuffer &, const FormatSettings &, bool) const override { throwNoSerialization(); } + bool tryDeserializeText(IColumn &, ReadBuffer &, const FormatSettings &, bool) const override { throwNoSerialization(); } /// These methods read and write zero bytes just to allow to figure out size of column. void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override; diff --git a/src/DataTypes/Serializations/SerializationNullable.cpp b/src/DataTypes/Serializations/SerializationNullable.cpp index 15203bdc9fa..e7f0e61f2a5 100644 --- a/src/DataTypes/Serializations/SerializationNullable.cpp +++ b/src/DataTypes/Serializations/SerializationNullable.cpp @@ -187,55 +187,59 @@ void SerializationNullable::serializeBinary(const IColumn & column, size_t row_n nested->serializeBinary(col.getNestedColumn(), row_num, ostr, settings); } -/// Deserialize value into ColumnNullable. -/// We need to insert both to nested column and to null byte map, or, in case of exception, to not insert at all. -template -requires std::same_as -static ReturnType -safeDeserialize(IColumn & column, const ISerialization &, CheckForNull && check_for_null, DeserializeNested && deserialize_nested) +template +ReturnType safeAppendToNullMap(ColumnNullable & column, bool is_null) { - ColumnNullable & col = assert_cast(column); - - if (check_for_null()) + try { - col.insertDefault(); + column.getNullMapData().push_back(is_null); } - else + catch (...) { - deserialize_nested(col.getNestedColumn()); - - try - { - col.getNullMapData().push_back(0); - } - catch (...) - { - col.getNestedColumn().popBack(1); + column.getNestedColumn().popBack(1); + if constexpr (std::is_same_v) throw; - } + return ReturnType(false); } + + return ReturnType(true); } -/// Deserialize value into non-nullable column. In case of NULL, insert default value and return false. +/// Deserialize value into non-nullable column. In case of NULL, insert default and set is_null to true. +/// If ReturnType is bool, return true if parsing was succesfull and false in case of any error. template -requires std::same_as -static ReturnType -safeDeserialize(IColumn & column, const ISerialization &, CheckForNull && check_for_null, DeserializeNested && deserialize_nested) +static ReturnType deserializeImpl(IColumn & column, ReadBuffer & buf, CheckForNull && check_for_null, DeserializeNested && deserialize_nested, bool & is_null) { - bool insert_default = check_for_null(); - if (insert_default) + is_null = check_for_null(buf); + if (is_null) + { column.insertDefault(); + } else - deserialize_nested(column); - return !insert_default; + { + if constexpr (std::is_same_v) + deserialize_nested(column, buf); + else if (!deserialize_nested(column, buf)) + return ReturnType(false); + } + + return ReturnType(true); } void SerializationNullable::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - safeDeserialize(column, *nested, - [&istr] { bool is_null = false; readBinary(is_null, istr); return is_null; }, - [this, &istr, settings] (IColumn & nested_column) { nested->deserializeBinary(nested_column, istr, settings); }); + ColumnNullable & col = assert_cast(column); + bool is_null; + auto check_for_null = [](ReadBuffer & buf) + { + bool is_null_ = false; + readBinary(is_null_, buf); + return is_null_; + }; + auto deserialize_nested = [this, &settings] (IColumn & nested_column, ReadBuffer & buf) { nested->deserializeBinary(nested_column, buf, settings); }; + deserializeImpl(col.getNestedColumn(), istr, check_for_null, deserialize_nested, is_null); + safeAppendToNullMap(col, is_null); } @@ -244,20 +248,19 @@ void SerializationNullable::serializeTextEscaped(const IColumn & column, size_t const ColumnNullable & col = assert_cast(column); if (col.isNullAt(row_num)) - writeString(settings.tsv.null_representation, ostr); + serializeNullEscaped(ostr, settings); else nested->serializeTextEscaped(col.getNestedColumn(), row_num, ostr, settings); } - -void SerializationNullable::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +void SerializationNullable::serializeNullEscaped(DB::WriteBuffer & ostr, const DB::FormatSettings & settings) { - deserializeTextEscapedImpl(column, istr, settings, nested); + writeString(settings.tsv.null_representation, ostr); } -void SerializationNullable::deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +bool SerializationNullable::tryDeserializeNullEscaped(DB::ReadBuffer & istr, const DB::FormatSettings & settings) { - deserializeTextRawImpl(column, istr, settings, nested); + return checkString(settings.tsv.null_representation, istr); } void SerializationNullable::serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -265,72 +268,73 @@ void SerializationNullable::serializeTextRaw(const IColumn & column, size_t row_ const ColumnNullable & col = assert_cast(column); if (col.isNullAt(row_num)) - writeString(settings.tsv.null_representation, ostr); + serializeNullRaw(ostr, settings); else nested->serializeTextRaw(col.getNestedColumn(), row_num, ostr, settings); } -template -ReturnType SerializationNullable::deserializeTextRawImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested) +void SerializationNullable::serializeNullRaw(DB::WriteBuffer & ostr, const DB::FormatSettings & settings) { - return deserializeTextEscapedAndRawImpl(column, istr, settings, nested); + writeString(settings.tsv.null_representation, ostr); } -template -ReturnType SerializationNullable::deserializeTextEscapedImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, - const SerializationPtr & nested) +bool SerializationNullable::tryDeserializeNullRaw(DB::ReadBuffer & istr, const DB::FormatSettings & settings) { - return deserializeTextEscapedAndRawImpl(column, istr, settings, nested); + return checkString(settings.tsv.null_representation, istr); } template -ReturnType SerializationNullable::deserializeTextEscapedAndRawImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, - const SerializationPtr & nested_serialization) +ReturnType deserializeTextEscapedAndRawImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested_serialization, bool & is_null) { + static constexpr bool throw_exception = std::is_same_v; + const String & null_representation = settings.tsv.null_representation; + auto deserialize_nested = [&nested_serialization, &settings] (IColumn & nested_column, ReadBuffer & buf_) + { + if constexpr (throw_exception) + { + if constexpr (escaped) + nested_serialization->deserializeTextEscaped(nested_column, buf_, settings); + else + nested_serialization->deserializeTextRaw(nested_column, buf_, settings); + } + else + { + if constexpr (escaped) + return nested_serialization->tryDeserializeTextEscaped(nested_column, buf_, settings); + else + return nested_serialization->tryDeserializeTextRaw(nested_column, buf_, settings); + } + }; /// Some data types can deserialize absence of data (e.g. empty string), so eof is ok. if (istr.eof() || (!null_representation.empty() && *istr.position() != null_representation[0])) { /// This is not null, surely. - return safeDeserialize(column, *nested_serialization, - [] { return false; }, - [&nested_serialization, &istr, &settings] (IColumn & nested_column) - { - if constexpr (escaped) - nested_serialization->deserializeTextEscaped(nested_column, istr, settings); - else - nested_serialization->deserializeTextRaw(nested_column, istr, settings); - }); + return deserializeImpl(column, istr, [](ReadBuffer &){ return false; }, deserialize_nested, is_null); } /// Check if we have enough data in buffer to check if it's a null. if (istr.available() > null_representation.size()) { - auto check_for_null = [&istr, &null_representation]() + auto check_for_null = [&null_representation](ReadBuffer & buf) { - auto * pos = istr.position(); - if (checkString(null_representation, istr) && (*istr.position() == '\t' || *istr.position() == '\n')) + auto * pos = buf.position(); + if (checkString(null_representation, buf) && (*buf.position() == '\t' || *buf.position() == '\n')) return true; - istr.position() = pos; + buf.position() = pos; return false; }; - auto deserialize_nested = [&nested_serialization, &settings, &istr] (IColumn & nested_column) - { - if constexpr (escaped) - nested_serialization->deserializeTextEscaped(nested_column, istr, settings); - else - nested_serialization->deserializeTextRaw(nested_column, istr, settings); - }; - return safeDeserialize(column, *nested_serialization, check_for_null, deserialize_nested); + return deserializeImpl(column, istr, check_for_null, deserialize_nested, is_null); } /// We don't have enough data in buffer to check if it's a null. /// Use PeekableReadBuffer to make a checkpoint before checking null /// representation and rollback if check was failed. - PeekableReadBuffer buf(istr, true); - auto check_for_null = [&buf, &null_representation]() + PeekableReadBuffer peekable_buf(istr, true); + auto check_for_null = [&null_representation](ReadBuffer & buf_) { + auto & buf = assert_cast(buf_); buf.setCheckpoint(); SCOPE_EXIT(buf.dropCheckpoint()); if (checkString(null_representation, buf) && (buf.eof() || *buf.position() == '\t' || *buf.position() == '\n')) @@ -340,16 +344,18 @@ ReturnType SerializationNullable::deserializeTextEscapedAndRawImpl(IColumn & col return false; }; - auto deserialize_nested = [&nested_serialization, &settings, &buf, &null_representation, &istr] (IColumn & nested_column) + auto deserialize_nested_with_check = [&deserialize_nested, &nested_serialization, &settings, &null_representation, &istr] (IColumn & nested_column, ReadBuffer & buf_) { + auto & buf = assert_cast(buf_); auto * pos = buf.position(); - if constexpr (escaped) - nested_serialization->deserializeTextEscaped(nested_column, buf, settings); - else - nested_serialization->deserializeTextRaw(nested_column, buf, settings); + if constexpr (throw_exception) + deserialize_nested(nested_column, buf); + else if (!deserialize_nested(nested_column, buf)) + return ReturnType(false); + /// Check that we don't have any unread data in PeekableReadBuffer own memory. if (likely(!buf.hasUnreadData())) - return; + return ReturnType(true); /// We have some unread data in PeekableReadBuffer own memory. /// It can happen only if there is a string instead of a number @@ -358,6 +364,9 @@ ReturnType SerializationNullable::deserializeTextEscapedAndRawImpl(IColumn & col /// We also should delete incorrectly deserialized value from nested column. nested_column.popBack(1); + if constexpr (!throw_exception) + return ReturnType(false); + if (null_representation.find('\t') != std::string::npos || null_representation.find('\n') != std::string::npos) throw DB::ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "TSV custom null representation " "containing '\\t' or '\\n' may not work correctly for large input."); @@ -375,7 +384,63 @@ ReturnType SerializationNullable::deserializeTextEscapedAndRawImpl(IColumn & col istr.count(), std::string(pos, buf.position() - pos), parsed_value.str()); }; - return safeDeserialize(column, *nested_serialization, check_for_null, deserialize_nested); + return deserializeImpl(column, peekable_buf, check_for_null, deserialize_nested_with_check, is_null); +} + +void SerializationNullable::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + ColumnNullable & col = assert_cast(column); + bool is_null; + deserializeTextEscapedAndRawImpl(col.getNestedColumn(), istr, settings, nested, is_null); + safeAppendToNullMap(col, is_null); +} + +bool SerializationNullable::tryDeserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + ColumnNullable & col = assert_cast(column); + bool is_null; + return deserializeTextEscapedAndRawImpl(col.getNestedColumn(), istr, settings, nested, is_null) && safeAppendToNullMap(col, is_null); +} + +bool SerializationNullable::deserializeNullAsDefaultOrNestedTextEscaped(IColumn & nested_column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested_serialization) +{ + bool is_null; + deserializeTextEscapedAndRawImpl(nested_column, istr, settings, nested_serialization, is_null); + return !is_null; +} + +bool SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextEscaped(IColumn & nested_column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested_serialization) +{ + bool is_null; + return deserializeTextEscapedAndRawImpl(nested_column, istr, settings, nested_serialization, is_null); +} + +void SerializationNullable::deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + ColumnNullable & col = assert_cast(column); + bool is_null; + deserializeTextEscapedAndRawImpl(col.getNestedColumn(), istr, settings, nested, is_null); + safeAppendToNullMap(col, is_null); +} + +bool SerializationNullable::tryDeserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + ColumnNullable & col = assert_cast(column); + bool is_null; + return deserializeTextEscapedAndRawImpl(col.getNestedColumn(), istr, settings, nested, is_null) && safeAppendToNullMap(col, is_null); +} + +bool SerializationNullable::deserializeNullAsDefaultOrNestedTextRaw(IColumn & nested_column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested_serialization) +{ + bool is_null; + deserializeTextEscapedAndRawImpl(nested_column, istr, settings, nested_serialization, is_null); + return !is_null; +} + +bool SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextRaw(IColumn & nested_column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested_serialization) +{ + bool is_null; + return deserializeTextEscapedAndRawImpl(nested_column, istr, settings, nested_serialization, is_null); } void SerializationNullable::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -383,45 +448,51 @@ void SerializationNullable::serializeTextQuoted(const IColumn & column, size_t r const ColumnNullable & col = assert_cast(column); if (col.isNullAt(row_num)) - writeCString("NULL", ostr); + serializeNullQuoted(ostr); else nested->serializeTextQuoted(col.getNestedColumn(), row_num, ostr, settings); } - -void SerializationNullable::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +void SerializationNullable::serializeNullQuoted(DB::WriteBuffer & ostr) { - deserializeTextQuotedImpl(column, istr, settings, nested); + writeCString("NULL", ostr); +} + +bool SerializationNullable::tryDeserializeNullQuoted(DB::ReadBuffer & istr) +{ + return checkStringCaseInsensitive("NULL", istr); } template -ReturnType SerializationNullable::deserializeTextQuotedImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, - const SerializationPtr & nested) +ReturnType deserializeTextQuotedImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested, bool & is_null) { + static constexpr bool throw_exception = std::is_same_v; + + auto deserialize_nested = [&nested, &settings] (IColumn & nested_column, ReadBuffer & buf) + { + if constexpr (!throw_exception) + return nested->tryDeserializeTextQuoted(nested_column, buf, settings); + nested->deserializeTextQuoted(nested_column, buf, settings); + }; + if (istr.eof() || (*istr.position() != 'N' && *istr.position() != 'n')) { /// This is not null, surely. - return safeDeserialize(column, *nested, - [] { return false; }, - [&nested, &istr, &settings] (IColumn & nested_column) { nested->deserializeTextQuoted(nested_column, istr, settings); }); + return deserializeImpl(column, istr, [](ReadBuffer &){ return false; }, deserialize_nested, is_null); } /// Check if we have enough data in buffer to check if it's a null. if (istr.available() >= 4) { - auto check_for_null = [&istr]() + auto check_for_null = [](ReadBuffer & buf) { - auto * pos = istr.position(); - if (checkStringCaseInsensitive("NULL", istr)) + auto * pos = buf.position(); + if (checkStringCaseInsensitive("NULL", buf)) return true; - istr.position() = pos; + buf.position() = pos; return false; }; - auto deserialize_nested = [&nested, &settings, &istr] (IColumn & nested_column) - { - nested->deserializeTextQuoted(nested_column, istr, settings); - }; - return safeDeserialize(column, *nested, check_for_null, deserialize_nested); + return deserializeImpl(column, istr, check_for_null, deserialize_nested, is_null); } /// We don't have enough data in buffer to check if it's a NULL @@ -429,9 +500,10 @@ ReturnType SerializationNullable::deserializeTextQuotedImpl(IColumn & column, Re /// to differentiate for example NULL and NaN for float) /// Use PeekableReadBuffer to make a checkpoint before checking /// null and rollback if the check was failed. - PeekableReadBuffer buf(istr, true); - auto check_for_null = [&buf]() + PeekableReadBuffer peekable_buf(istr, true); + auto check_for_null = [](ReadBuffer & buf_) { + auto & buf = assert_cast(buf_); buf.setCheckpoint(); SCOPE_EXIT(buf.dropCheckpoint()); if (checkStringCaseInsensitive("NULL", buf)) @@ -441,39 +513,74 @@ ReturnType SerializationNullable::deserializeTextQuotedImpl(IColumn & column, Re return false; }; - auto deserialize_nested = [&nested, &settings, &buf] (IColumn & nested_column) + auto deserialize_nested_with_check = [&deserialize_nested] (IColumn & nested_column, ReadBuffer & buf_) { - nested->deserializeTextQuoted(nested_column, buf, settings); + auto & buf = assert_cast(buf_); + + if constexpr (throw_exception) + deserialize_nested(nested_column, buf); + else if (!deserialize_nested(nested_column, buf)) + return false; + /// Check that we don't have any unread data in PeekableReadBuffer own memory. if (likely(!buf.hasUnreadData())) - return; + return ReturnType(true); /// We have some unread data in PeekableReadBuffer own memory. /// It can happen only if there is an unquoted string instead of a number. /// We also should delete incorrectly deserialized value from nested column. nested_column.popBack(1); + + if constexpr (!throw_exception) + return ReturnType(false); + throw DB::ParsingException( ErrorCodes::CANNOT_READ_ALL_DATA, "Error while parsing Nullable: got an unquoted string {} instead of a number", String(buf.position(), std::min(10ul, buf.available()))); }; - return safeDeserialize(column, *nested, check_for_null, deserialize_nested); + return deserializeImpl(column, peekable_buf, check_for_null, deserialize_nested_with_check, is_null); } -void SerializationNullable::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +void SerializationNullable::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - deserializeWholeTextImpl(column, istr, settings, nested); + ColumnNullable & col = assert_cast(column); + bool is_null; + deserializeTextQuotedImpl(col.getNestedColumn(), istr, settings, nested, is_null); + safeAppendToNullMap(col, is_null); +} + +bool SerializationNullable::tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + ColumnNullable & col = assert_cast(column); + bool is_null; + return deserializeTextQuotedImpl(col.getNestedColumn(), istr, settings, nested, is_null) && safeAppendToNullMap(col, is_null); +} + +bool SerializationNullable::deserializeNullAsDefaultOrNestedTextQuoted(DB::IColumn & nested_column, DB::ReadBuffer & istr, const DB::FormatSettings & settings, const DB::SerializationPtr & nested_serialization) +{ + bool is_null; + deserializeTextQuotedImpl(nested_column, istr, settings, nested_serialization, is_null); + return !is_null; +} + +bool SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextQuoted(DB::IColumn & nested_column, DB::ReadBuffer & istr, const DB::FormatSettings & settings, const DB::SerializationPtr & nested_serialization) +{ + bool is_null; + return deserializeTextQuotedImpl(nested_column, istr, settings, nested_serialization, is_null); } template -ReturnType SerializationNullable::deserializeWholeTextImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, - const SerializationPtr & nested) +ReturnType deserializeWholeTextImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested, bool & is_null) { - PeekableReadBuffer buf(istr, true); - auto check_for_null = [&buf]() + static constexpr bool throw_exception = std::is_same_v; + + PeekableReadBuffer peekable_buf(istr, true); + auto check_for_null = [](ReadBuffer & buf_) { + auto & buf = assert_cast(buf_); buf.setCheckpoint(); SCOPE_EXIT(buf.dropCheckpoint()); @@ -488,15 +595,46 @@ ReturnType SerializationNullable::deserializeWholeTextImpl(IColumn & column, Rea return false; }; - auto deserialize_nested = [&nested, &settings, &buf] (IColumn & nested_column) + auto deserialize_nested = [&nested, &settings] (IColumn & nested_column, ReadBuffer & buf_) { + auto & buf = assert_cast(buf_); + if constexpr (!throw_exception) + return nested->tryDeserializeWholeText(nested_column, buf, settings); + nested->deserializeWholeText(nested_column, buf, settings); assert(!buf.hasUnreadData()); }; - return safeDeserialize(column, *nested, check_for_null, deserialize_nested); + return deserializeImpl(column, peekable_buf, check_for_null, deserialize_nested, is_null); } +void SerializationNullable::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + ColumnNullable & col = assert_cast(column); + bool is_null; + deserializeWholeTextImpl(col.getNestedColumn(), istr, settings, nested, is_null); + safeAppendToNullMap(col, is_null); +} + +bool SerializationNullable::tryDeserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + ColumnNullable & col = assert_cast(column); + bool is_null; + return deserializeWholeTextImpl(col.getNestedColumn(), istr, settings, nested, is_null) && safeAppendToNullMap(col, is_null); +} + +bool SerializationNullable::deserializeNullAsDefaultOrNestedWholeText(DB::IColumn & nested_column, DB::ReadBuffer & istr, const DB::FormatSettings & settings, const DB::SerializationPtr & nested_serialization) +{ + bool is_null; + deserializeWholeTextImpl(nested_column, istr, settings, nested_serialization, is_null); + return !is_null; +} + +bool SerializationNullable::tryDeserializeNullAsDefaultOrNestedWholeText(DB::IColumn & nested_column, DB::ReadBuffer & istr, const DB::FormatSettings & settings, const DB::SerializationPtr & nested_serialization) +{ + bool is_null; + return deserializeWholeTextImpl(nested_column, istr, settings, nested_serialization, is_null); +} void SerializationNullable::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { @@ -508,48 +646,56 @@ void SerializationNullable::serializeTextCSV(const IColumn & column, size_t row_ nested->serializeTextCSV(col.getNestedColumn(), row_num, ostr, settings); } -void SerializationNullable::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +void SerializationNullable::serializeNullCSV(DB::WriteBuffer & ostr, const DB::FormatSettings & settings) { - deserializeTextCSVImpl(column, istr, settings, nested); + writeString(settings.csv.null_representation, ostr); +} + +bool SerializationNullable::tryDeserializeNullCSV(DB::ReadBuffer & istr, const DB::FormatSettings & settings) +{ + return checkString(settings.csv.null_representation, istr); } template -ReturnType SerializationNullable::deserializeTextCSVImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, - const SerializationPtr & nested_serialization) +ReturnType deserializeTextCSVImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested_serialization, bool & is_null) { + static constexpr bool throw_exception = std::is_same_v; + + auto deserialize_nested = [&nested_serialization, &settings] (IColumn & nested_column, ReadBuffer & buf) + { + if constexpr (!throw_exception) + return nested_serialization->tryDeserializeTextCSV(nested_column, buf, settings); + nested_serialization->deserializeTextCSV(nested_column, buf, settings); + }; + const String & null_representation = settings.csv.null_representation; if (istr.eof() || (!null_representation.empty() && *istr.position() != null_representation[0])) { /// This is not null, surely. - return safeDeserialize(column, *nested_serialization, - [] { return false; }, - [&nested_serialization, &istr, &settings] (IColumn & nested_column) { nested_serialization->deserializeTextCSV(nested_column, istr, settings); }); + return deserializeImpl(column, istr, [](ReadBuffer &){ return false; }, deserialize_nested, is_null); } /// Check if we have enough data in buffer to check if it's a null. if (settings.csv.custom_delimiter.empty() && istr.available() > null_representation.size()) { - auto check_for_null = [&istr, &null_representation, &settings]() + auto check_for_null = [&null_representation, &settings](ReadBuffer & buf) { - auto * pos = istr.position(); - if (checkString(null_representation, istr) && (*istr.position() == settings.csv.delimiter || *istr.position() == '\r' || *istr.position() == '\n')) + auto * pos = buf.position(); + if (checkString(null_representation, buf) && (*buf.position() == settings.csv.delimiter || *buf.position() == '\r' || *buf.position() == '\n')) return true; - istr.position() = pos; + buf.position() = pos; return false; }; - auto deserialize_nested = [&nested_serialization, &settings, &istr] (IColumn & nested_column) - { - nested_serialization->deserializeTextCSV(nested_column, istr, settings); - }; - return safeDeserialize(column, *nested_serialization, check_for_null, deserialize_nested); + return deserializeImpl(column, istr, check_for_null, deserialize_nested, is_null); } /// We don't have enough data in buffer to check if it's a null. /// Use PeekableReadBuffer to make a checkpoint before checking null /// representation and rollback if the check was failed. - PeekableReadBuffer buf(istr, true); - auto check_for_null = [&buf, &null_representation, &settings]() + PeekableReadBuffer peekable_buf(istr, true); + auto check_for_null = [&null_representation, &settings](ReadBuffer & buf_) { + auto & buf = assert_cast(buf_); buf.setCheckpoint(); SCOPE_EXIT(buf.dropCheckpoint()); if (checkString(null_representation, buf)) @@ -572,13 +718,18 @@ ReturnType SerializationNullable::deserializeTextCSVImpl(IColumn & column, ReadB return false; }; - auto deserialize_nested = [&nested_serialization, &settings, &buf, &null_representation, &istr] (IColumn & nested_column) + auto deserialize_nested_with_check = [&deserialize_nested, &nested_serialization, &settings, &null_representation, &istr] (IColumn & nested_column, ReadBuffer & buf_) { + auto & buf = assert_cast(buf_); auto * pos = buf.position(); - nested_serialization->deserializeTextCSV(nested_column, buf, settings); + if constexpr (throw_exception) + deserialize_nested(nested_column, buf); + else if (!deserialize_nested(nested_column, buf)) + return ReturnType(false); + /// Check that we don't have any unread data in PeekableReadBuffer own memory. if (likely(!buf.hasUnreadData())) - return; + return ReturnType(true); /// We have some unread data in PeekableReadBuffer own memory. /// It can happen only if there is an unquoted string instead of a number @@ -587,6 +738,9 @@ ReturnType SerializationNullable::deserializeTextCSVImpl(IColumn & column, ReadB /// We also should delete incorrectly deserialized value from nested column. nested_column.popBack(1); + if constexpr (!throw_exception) + return ReturnType(false); + if (null_representation.find(settings.csv.delimiter) != std::string::npos || null_representation.find('\r') != std::string::npos || null_representation.find('\n') != std::string::npos) throw DB::ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "CSV custom null representation containing " @@ -602,7 +756,35 @@ ReturnType SerializationNullable::deserializeTextCSVImpl(IColumn & column, ReadB istr.count(), std::string(pos, buf.position() - pos), parsed_value.str()); }; - return safeDeserialize(column, *nested_serialization, check_for_null, deserialize_nested); + return deserializeImpl(column, peekable_buf, check_for_null, deserialize_nested_with_check, is_null); +} + +void SerializationNullable::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + ColumnNullable & col = assert_cast(column); + bool is_null; + deserializeTextCSVImpl(col.getNestedColumn(), istr, settings, nested, is_null); + safeAppendToNullMap(col, is_null); +} + +bool SerializationNullable::tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + ColumnNullable & col = assert_cast(column); + bool is_null; + return deserializeTextCSVImpl(col.getNestedColumn(), istr, settings, nested, is_null) && safeAppendToNullMap(col, is_null); +} + +bool SerializationNullable::deserializeNullAsDefaultOrNestedTextCSV(DB::IColumn & nested_column, DB::ReadBuffer & istr, const DB::FormatSettings & settings, const DB::SerializationPtr & nested_serialization) +{ + bool is_null; + deserializeTextCSVImpl(nested_column, istr, settings, nested_serialization, is_null); + return !is_null; +} + +bool SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextCSV(DB::IColumn & nested_column, DB::ReadBuffer & istr, const DB::FormatSettings & settings, const DB::SerializationPtr & nested_serialization) +{ + bool is_null; + return deserializeTextCSVImpl(nested_column, istr, settings, nested_serialization, is_null); } void SerializationNullable::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -616,38 +798,86 @@ void SerializationNullable::serializeText(const IColumn & column, size_t row_num /// This assumes UTF-8 and proper font support. This is Ok, because Pretty formats are "presentational", not for data exchange. if (col.isNullAt(row_num)) - { - if (settings.pretty.charset == FormatSettings::Pretty::Charset::UTF8) - writeCString("ᴺᵁᴸᴸ", ostr); - else - writeCString("NULL", ostr); - } + serializeNullText(ostr, settings); else nested->serializeText(col.getNestedColumn(), row_num, ostr, settings); } +void SerializationNullable::serializeNullText(DB::WriteBuffer & ostr, const DB::FormatSettings & settings) +{ + if (settings.pretty.charset == FormatSettings::Pretty::Charset::UTF8) + writeCString("ᴺᵁᴸᴸ", ostr); + else + writeCString("NULL", ostr); +} + +bool SerializationNullable::tryDeserializeNullText(DB::ReadBuffer & istr) +{ + if (checkCharCaseInsensitive('N', istr)) + return checkStringCaseInsensitive("ULL", istr); + return checkStringCaseInsensitive("ᴺᵁᴸᴸ", istr); +} + void SerializationNullable::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { const ColumnNullable & col = assert_cast(column); if (col.isNullAt(row_num)) - writeCString("null", ostr); + serializeNullJSON(ostr); else nested->serializeTextJSON(col.getNestedColumn(), row_num, ostr, settings); } -void SerializationNullable::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +void SerializationNullable::serializeNullJSON(DB::WriteBuffer & ostr) { - deserializeTextJSONImpl(column, istr, settings, nested); + writeCString("null", ostr); +} + +bool SerializationNullable::tryDeserializeNullJSON(DB::ReadBuffer & istr) +{ + return checkString("null", istr); } template -ReturnType SerializationNullable::deserializeTextJSONImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, - const SerializationPtr & nested) +ReturnType deserializeTextJSONImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested, bool & is_null) { - return safeDeserialize(column, *nested, - [&istr] { return checkStringByFirstCharacterAndAssertTheRest("null", istr); }, - [&nested, &istr, &settings] (IColumn & nested_column) { nested->deserializeTextJSON(nested_column, istr, settings); }); + auto check_for_null = [](ReadBuffer & buf){ return checkStringByFirstCharacterAndAssertTheRest("null", buf); }; + auto deserialize_nested = [&nested, &settings](IColumn & nested_column, ReadBuffer & buf) + { + if constexpr (std::is_same_v) + return nested->tryDeserializeTextJSON(nested_column, buf, settings); + nested->deserializeTextJSON(nested_column, buf, settings); + }; + + return deserializeImpl(column, istr, check_for_null, deserialize_nested, is_null); +} + +void SerializationNullable::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + ColumnNullable & col = assert_cast(column); + bool is_null; + deserializeTextJSONImpl(col.getNestedColumn(), istr, settings, nested, is_null); + safeAppendToNullMap(col, is_null); +} + +bool SerializationNullable::tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + ColumnNullable & col = assert_cast(column); + bool is_null; + return deserializeTextJSONImpl(col.getNestedColumn(), istr, settings, nested, is_null) && safeAppendToNullMap(col, is_null); +} + +bool SerializationNullable::deserializeNullAsDefaultOrNestedTextJSON(DB::IColumn & nested_column, DB::ReadBuffer & istr, const DB::FormatSettings & settings, const DB::SerializationPtr & nested_serialization) +{ + bool is_null; + deserializeTextJSONImpl(nested_column, istr, settings, nested_serialization, is_null); + return !is_null; +} + +bool SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextJSON(DB::IColumn & nested_column, DB::ReadBuffer & istr, const DB::FormatSettings & settings, const DB::SerializationPtr & nested_serialization) +{ + bool is_null; + return deserializeTextJSONImpl(nested_column, istr, settings, nested_serialization, is_null); } void SerializationNullable::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -660,11 +890,9 @@ void SerializationNullable::serializeTextXML(const IColumn & column, size_t row_ nested->serializeTextXML(col.getNestedColumn(), row_num, ostr, settings); } -template bool SerializationNullable::deserializeWholeTextImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested); -template bool SerializationNullable::deserializeTextEscapedImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested); -template bool SerializationNullable::deserializeTextQuotedImpl(IColumn & column, ReadBuffer & istr, const FormatSettings &, const SerializationPtr & nested); -template bool SerializationNullable::deserializeTextCSVImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested); -template bool SerializationNullable::deserializeTextJSONImpl(IColumn & column, ReadBuffer & istr, const FormatSettings &, const SerializationPtr & nested); -template bool SerializationNullable::deserializeTextRawImpl(IColumn & column, ReadBuffer & istr, const FormatSettings &, const SerializationPtr & nested); +void SerializationNullable::serializeNullXML(DB::WriteBuffer & ostr) +{ + writeCString("\\N", ostr); +} } diff --git a/src/DataTypes/Serializations/SerializationNullable.h b/src/DataTypes/Serializations/SerializationNullable.h index 3ec01b46de5..37858ccdefd 100644 --- a/src/DataTypes/Serializations/SerializationNullable.h +++ b/src/DataTypes/Serializations/SerializationNullable.h @@ -51,9 +51,12 @@ public: void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; @@ -66,31 +69,49 @@ public: * In CSV, non-NULL string value, starting with \N characters, must be placed in quotes, to avoid ambiguity. */ void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; - /// If ReturnType is bool, check for NULL and deserialize value into non-nullable column (and return true) or insert default value of nested type (and return false) - /// If ReturnType is void, deserialize Nullable(T) - template - static ReturnType deserializeWholeTextImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested); - template - static ReturnType deserializeTextEscapedImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested); - template - static ReturnType deserializeTextQuotedImpl(IColumn & column, ReadBuffer & istr, const FormatSettings &, const SerializationPtr & nested); - template - static ReturnType deserializeTextCSVImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested); - template - static ReturnType deserializeTextJSONImpl(IColumn & column, ReadBuffer & istr, const FormatSettings &, const SerializationPtr & nested); - template - static ReturnType deserializeTextRawImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested); - template - static ReturnType deserializeTextEscapedAndRawImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested); + /// If Check for NULL and deserialize value into non-nullable column (and return true) or insert default value of nested type (and return false) + static bool deserializeNullAsDefaultOrNestedWholeText(IColumn & nested_column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested_serialization); + static bool deserializeNullAsDefaultOrNestedTextEscaped(IColumn & nested_column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested_serialization); + static bool deserializeNullAsDefaultOrNestedTextQuoted(IColumn & nested_column, ReadBuffer & istr, const FormatSettings &, const SerializationPtr & nested_serialization); + static bool deserializeNullAsDefaultOrNestedTextCSV(IColumn & nested_column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested_serialization); + static bool deserializeNullAsDefaultOrNestedTextJSON(IColumn & nested_column, ReadBuffer & istr, const FormatSettings &, const SerializationPtr & nested_serialization); + static bool deserializeNullAsDefaultOrNestedTextRaw(IColumn & nested_column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested_serialization); + + /// If Check for NULL and deserialize value into non-nullable column or insert default value of nested type. + /// Return true if parsing was successful and false in case of any error. + static bool tryDeserializeNullAsDefaultOrNestedWholeText(IColumn & nested_column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested_serialization); + static bool tryDeserializeNullAsDefaultOrNestedTextEscaped(IColumn & nested_column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested_serialization); + static bool tryDeserializeNullAsDefaultOrNestedTextQuoted(IColumn & nested_column, ReadBuffer & istr, const FormatSettings &, const SerializationPtr & nested_serialization); + static bool tryDeserializeNullAsDefaultOrNestedTextCSV(IColumn & nested_column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested_serialization); + static bool tryDeserializeNullAsDefaultOrNestedTextJSON(IColumn & nested_column, ReadBuffer & istr, const FormatSettings &, const SerializationPtr & nested_serialization); + static bool tryDeserializeNullAsDefaultOrNestedTextRaw(IColumn & nested_column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested_serialization); + + + static void serializeNullEscaped(WriteBuffer & ostr, const FormatSettings & settings); + static bool tryDeserializeNullEscaped(ReadBuffer & istr, const FormatSettings & settings); + static void serializeNullQuoted(WriteBuffer & ostr); + static bool tryDeserializeNullQuoted(ReadBuffer & istr); + static void serializeNullCSV(WriteBuffer & ostr, const FormatSettings & settings); + static bool tryDeserializeNullCSV(ReadBuffer & istr, const FormatSettings & settings); + static void serializeNullJSON(WriteBuffer & ostr); + static bool tryDeserializeNullJSON(ReadBuffer & istr); + static void serializeNullRaw(WriteBuffer & ostr, const FormatSettings & settings); + static bool tryDeserializeNullRaw(ReadBuffer & istr, const FormatSettings & settings); + static void serializeNullText(WriteBuffer & ostr, const FormatSettings & settings); + static bool tryDeserializeNullText(ReadBuffer & istr); + static void serializeNullXML(WriteBuffer & ostr); private: struct SubcolumnCreator : public ISubcolumnCreator diff --git a/src/DataTypes/Serializations/SerializationNumber.cpp b/src/DataTypes/Serializations/SerializationNumber.cpp index b6c7e4618b8..bdb4dfc6735 100644 --- a/src/DataTypes/Serializations/SerializationNumber.cpp +++ b/src/DataTypes/Serializations/SerializationNumber.cpp @@ -37,6 +37,18 @@ void SerializationNumber::deserializeText(IColumn & column, ReadBuffer & istr throwUnexpectedDataAfterParsedValue(column, istr, settings, "Number"); } +template +bool SerializationNumber::tryDeserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const +{ + T x; + + if (!tryReadText(x, istr) || (whole && !istr.eof())) + return false; + + assert_cast &>(column).getData().push_back(x); + return true; +} + template void SerializationNumber::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { @@ -44,9 +56,10 @@ void SerializationNumber::serializeTextJSON(const IColumn & column, size_t ro writeJSONNumber(x, ostr, settings); } -template -void SerializationNumber::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +template +ReturnType deserializeTextJSONImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) { + static constexpr bool throw_exception = std::is_same_v; bool has_quote = false; if (!istr.eof() && *istr.position() == '"') /// We understand the number both in quotes and without. { @@ -54,13 +67,16 @@ void SerializationNumber::deserializeTextJSON(IColumn & column, ReadBuffer & ++istr.position(); } - FieldType x; + T x; /// null if (!has_quote && !istr.eof() && *istr.position() == 'n') { ++istr.position(); - assertString("ull", istr); + if constexpr (throw_exception) + assertString("ull", istr); + else if (!checkString("ull", istr)) + return ReturnType(false); x = NaNOrZero(); } @@ -73,26 +89,62 @@ void SerializationNumber::deserializeTextJSON(IColumn & column, ReadBuffer & { // extra conditions to parse true/false strings into 1/0 if (istr.eof()) - throwReadAfterEOF(); + { + if constexpr (throw_exception) + throwReadAfterEOF(); + else + return false; + } + if (*istr.position() == 't' || *istr.position() == 'f') { bool tmp = false; - readBoolTextWord(tmp, istr); + if constexpr (throw_exception) + readBoolTextWord(tmp, istr); + else if (!readBoolTextWord(tmp, istr)) + return ReturnType(false); + x = tmp; } else - readText(x, istr); + { + if constexpr (throw_exception) + readText(x, istr); + else if (!tryReadText(x, istr)) + return ReturnType(false); + } } else { - readText(x, istr); + if constexpr (throw_exception) + readText(x, istr); + else if (!tryReadText(x, istr)) + return ReturnType(false); } if (has_quote) - assertChar('"', istr); + { + if constexpr (throw_exception) + assertChar('"', istr); + else if (!checkChar('"', istr)) + return ReturnType(false); + } } assert_cast &>(column).getData().push_back(x); + return ReturnType(true); +} + +template +void SerializationNumber::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + deserializeTextJSONImpl(column, istr, settings); +} + +template +bool SerializationNumber::tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + return deserializeTextJSONImpl(column, istr, settings); } template @@ -103,6 +155,16 @@ void SerializationNumber::deserializeTextCSV(IColumn & column, ReadBuffer & i assert_cast &>(column).getData().push_back(x); } +template +bool SerializationNumber::tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & /*settings*/) const +{ + FieldType x; + if (!tryReadCSV(x, istr)) + return false; + assert_cast &>(column).getData().push_back(x); + return true; +} + template void SerializationNumber::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const { diff --git a/src/DataTypes/Serializations/SerializationNumber.h b/src/DataTypes/Serializations/SerializationNumber.h index 972c6c9a30f..9d53dc9c494 100644 --- a/src/DataTypes/Serializations/SerializationNumber.h +++ b/src/DataTypes/Serializations/SerializationNumber.h @@ -20,9 +20,12 @@ public: void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const override; + bool tryDeserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const override; void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; /** Format is platform-dependent. */ void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const override; diff --git a/src/DataTypes/Serializations/SerializationString.cpp b/src/DataTypes/Serializations/SerializationString.cpp index 788ff429088..1680ec8a333 100644 --- a/src/DataTypes/Serializations/SerializationString.cpp +++ b/src/DataTypes/Serializations/SerializationString.cpp @@ -272,40 +272,67 @@ void SerializationString::serializeTextEscaped(const IColumn & column, size_t ro } -template -static inline void read(IColumn & column, Reader && reader) +template +static inline ReturnType read(IColumn & column, Reader && reader) { + static constexpr bool throw_exception = std::is_same_v; ColumnString & column_string = assert_cast(column); ColumnString::Chars & data = column_string.getChars(); ColumnString::Offsets & offsets = column_string.getOffsets(); size_t old_chars_size = data.size(); size_t old_offsets_size = offsets.size(); - try - { - reader(data); - data.push_back(0); - offsets.push_back(data.size()); - } - catch (...) + auto restore_column = [&]() { offsets.resize_assume_reserved(old_offsets_size); data.resize_assume_reserved(old_chars_size); - throw; + }; + + try + { + if constexpr (throw_exception) + { + reader(data); + } + else if (!reader(data)) + { + restore_column(); + return false; + } + + data.push_back(0); + offsets.push_back(data.size()); + return ReturnType(true); + } + catch (...) + { + restore_column(); + if constexpr (throw_exception) + throw; + else + return false; } } void SerializationString::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { - read(column, [&](ColumnString::Chars & data) { readStringUntilEOFInto(data, istr); }); + read(column, [&](ColumnString::Chars & data) { readStringUntilEOFInto(data, istr); }); } +bool SerializationString::tryDeserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +{ + return read(column, [&](ColumnString::Chars & data) { readStringUntilEOFInto(data, istr); return true; }); +} void SerializationString::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { - read(column, [&](ColumnString::Chars & data) { readEscapedStringInto(data, istr); }); + read(column, [&](ColumnString::Chars & data) { readEscapedStringInto(data, istr); }); } +bool SerializationString::tryDeserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +{ + return read(column, [&](ColumnString::Chars & data) { readEscapedStringInto(data, istr); return true; }); +} void SerializationString::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { @@ -315,7 +342,12 @@ void SerializationString::serializeTextQuoted(const IColumn & column, size_t row void SerializationString::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { - read(column, [&](ColumnString::Chars & data) { readQuotedStringInto(data, istr); }); + read(column, [&](ColumnString::Chars & data) { readQuotedStringInto(data, istr); }); +} + +bool SerializationString::tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +{ + return read(column, [&](ColumnString::Chars & data) { return tryReadQuotedStringInto(data, istr); }); } @@ -329,11 +361,11 @@ void SerializationString::deserializeTextJSON(IColumn & column, ReadBuffer & ist { if (settings.json.read_objects_as_strings && !istr.eof() && *istr.position() == '{') { - read(column, [&](ColumnString::Chars & data) { readJSONObjectPossiblyInvalid(data, istr); }); + read(column, [&](ColumnString::Chars & data) { readJSONObjectPossiblyInvalid(data, istr); }); } else if (settings.json.read_arrays_as_strings && !istr.eof() && *istr.position() == '[') { - read(column, [&](ColumnString::Chars & data) { readJSONArrayInto(data, istr); }); + read(column, [&](ColumnString::Chars & data) { readJSONArrayInto(data, istr); }); } else if (settings.json.read_numbers_as_strings && !istr.eof() && *istr.position() != '"') { @@ -342,12 +374,40 @@ void SerializationString::deserializeTextJSON(IColumn & column, ReadBuffer & ist Float64 tmp; ReadBufferFromString buf(field); if (tryReadFloatText(tmp, buf) && buf.eof()) - read(column, [&](ColumnString::Chars & data) { data.insert(field.begin(), field.end()); }); + read(column, [&](ColumnString::Chars & data) { data.insert(field.begin(), field.end()); }); else throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot parse JSON String value here: {}", field); } else - read(column, [&](ColumnString::Chars & data) { readJSONStringInto(data, istr); }); + read(column, [&](ColumnString::Chars & data) { readJSONStringInto(data, istr); }); +} + +bool SerializationString::tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + if (settings.json.read_objects_as_strings && !istr.eof() && *istr.position() == '{') + return read(column, [&](ColumnString::Chars & data) { return readJSONObjectPossiblyInvalid(data, istr); }); + + if (settings.json.read_arrays_as_strings && !istr.eof() && *istr.position() == '[') + return read(column, [&](ColumnString::Chars & data) { return readJSONArrayInto(data, istr); }); + + if (settings.json.read_numbers_as_strings && !istr.eof() && *istr.position() != '"') + { + String field; + if (!tryReadJSONField(field, istr)) + return false; + + Float64 tmp; + ReadBufferFromString buf(field); + if (tryReadFloatText(tmp, buf) && buf.eof()) + { + read(column, [&](ColumnString::Chars & data) { data.insert(field.begin(), field.end()); }); + return true; + } + + return false; + } + + return read(column, [&](ColumnString::Chars & data) { return tryReadJSONStringInto(data, istr); }); } @@ -365,7 +425,12 @@ void SerializationString::serializeTextCSV(const IColumn & column, size_t row_nu void SerializationString::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - read(column, [&](ColumnString::Chars & data) { readCSVStringInto(data, istr, settings.csv); }); + read(column, [&](ColumnString::Chars & data) { readCSVStringInto(data, istr, settings.csv); }); +} + +bool SerializationString::tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + return read(column, [&](ColumnString::Chars & data) { readCSVStringInto(data, istr, settings.csv); return true; }); } void SerializationString::serializeTextMarkdown( diff --git a/src/DataTypes/Serializations/SerializationString.h b/src/DataTypes/Serializations/SerializationString.h index cd4cdf79c11..89ab84f0d22 100644 --- a/src/DataTypes/Serializations/SerializationString.h +++ b/src/DataTypes/Serializations/SerializationString.h @@ -18,20 +18,25 @@ public: void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextMarkdown(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; }; diff --git a/src/DataTypes/Serializations/SerializationTuple.cpp b/src/DataTypes/Serializations/SerializationTuple.cpp index cbbe97eb05c..c0b0658e6b4 100644 --- a/src/DataTypes/Serializations/SerializationTuple.cpp +++ b/src/DataTypes/Serializations/SerializationTuple.cpp @@ -62,15 +62,35 @@ void SerializationTuple::serializeBinary(const IColumn & column, size_t row_num, } -template -static void addElementSafe(size_t num_elems, IColumn & column, F && impl) +template +static ReturnType addElementSafe(size_t num_elems, IColumn & column, F && impl) { + static constexpr bool throw_exception = std::is_same_v; + /// We use the assumption that tuples of zero size do not exist. size_t old_size = column.size(); + auto restore_elements = [&]() + { + for (size_t i = 0; i < num_elems; ++i) + { + auto & element_column = extractElementColumn(column, i); + if (element_column.size() > old_size) + element_column.popBack(1); + } + }; + try { - impl(); + if constexpr (throw_exception) + { + impl(); + } + else if (!impl()) + { + restore_elements(); + return ReturnType(false); + } // Check that all columns now have the same size. size_t new_size = column.size(); @@ -81,22 +101,23 @@ static void addElementSafe(size_t num_elems, IColumn & column, F && impl) { // This is not a logical error because it may work with // user-supplied data. - throw Exception(ErrorCodes::SIZES_OF_COLUMNS_IN_TUPLE_DOESNT_MATCH, - "Cannot read a tuple because not all elements are present"); + if constexpr (throw_exception) + throw Exception(ErrorCodes::SIZES_OF_COLUMNS_IN_TUPLE_DOESNT_MATCH, + "Cannot read a tuple because not all elements are present"); + restore_elements(); + return ReturnType(false); } } } catch (...) { - for (size_t i = 0; i < num_elems; ++i) - { - auto & element_column = extractElementColumn(column, i); - if (element_column.size() > old_size) - element_column.popBack(1); - } - - throw; + restore_elements(); + if constexpr (throw_exception) + throw; + return ReturnType(false); } + + return ReturnType(true); } void SerializationTuple::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const @@ -120,25 +141,51 @@ void SerializationTuple::serializeText(const IColumn & column, size_t row_num, W writeChar(')', ostr); } -void SerializationTuple::deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const +template +ReturnType SerializationTuple::deserializeTextImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const { - const size_t size = elems.size(); - assertChar('(', istr); + static constexpr bool throw_exception = std::is_same_v; - addElementSafe(elems.size(), column, [&] + const size_t size = elems.size(); + if constexpr (throw_exception) + assertChar('(', istr); + else if (!checkChar('(', istr)) + return ReturnType(false); + + auto impl = [&]() { for (size_t i = 0; i < size; ++i) { skipWhitespaceIfAny(istr); if (i != 0) { - assertChar(',', istr); + if constexpr (throw_exception) + assertChar(',', istr); + else if (!checkChar(',', istr)) + return ReturnType(false); + skipWhitespaceIfAny(istr); } - if (settings.null_as_default) - SerializationNullable::deserializeTextQuotedImpl(extractElementColumn(column, i), istr, settings, elems[i]); + + auto & element_column = extractElementColumn(column, i); + if constexpr (throw_exception) + { + if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(element_column)) + SerializationNullable::deserializeNullAsDefaultOrNestedTextQuoted(element_column, istr, settings, elems[i]); + else + elems[i]->deserializeTextQuoted(element_column, istr, settings); + } else - elems[i]->deserializeTextQuoted(extractElementColumn(column, i), istr, settings); + { + bool ok; + if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(element_column)) + ok = SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextQuoted(element_column, istr, settings, elems[i]); + else + ok = elems[i]->tryDeserializeTextQuoted(element_column, istr, settings); + + if (!ok) + return false; + } } // Special format for one element tuple (1,) @@ -150,11 +197,35 @@ void SerializationTuple::deserializeText(IColumn & column, ReadBuffer & istr, co } skipWhitespaceIfAny(istr); - assertChar(')', istr); + if constexpr (throw_exception) + assertChar(')', istr); + else if (!checkChar(')', istr)) + return ReturnType(false); if (whole && !istr.eof()) - throwUnexpectedDataAfterParsedValue(column, istr, settings, "Tuple"); - }); + { + if constexpr (throw_exception) + throwUnexpectedDataAfterParsedValue(column, istr, settings, "Tuple"); + return ReturnType(false); + } + + return ReturnType(true); + }; + + if constexpr (throw_exception) + addElementSafe(elems.size(), column, impl); + else + return addElementSafe(elems.size(), column, impl); +} + +void SerializationTuple::deserializeText(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings & settings, bool whole) const +{ + deserializeTextImpl(column, istr, settings, whole); +} + +bool SerializationTuple::tryDeserializeText(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings & settings, bool whole) const +{ + return deserializeTextImpl(column, istr, settings, whole); } void SerializationTuple::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -239,16 +310,39 @@ void SerializationTuple::serializeTextJSONPretty(const IColumn & column, size_t } } -void SerializationTuple::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +template +ReturnType SerializationTuple::deserializeTextJSONImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { + static constexpr bool throw_exception = std::is_same_v; + + auto deserialize_element = [&](IColumn & element_column, size_t element_pos) + { + if constexpr (throw_exception) + { + if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(element_column)) + SerializationNullable::deserializeNullAsDefaultOrNestedTextJSON(element_column, istr, settings, elems[element_pos]); + else + elems[element_pos]->deserializeTextJSON(element_column, istr, settings); + } + else + { + if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(element_column)) + return SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextJSON(element_column, istr, settings, elems[element_pos]); + return elems[element_pos]->tryDeserializeTextJSON(element_column, istr, settings); + } + }; + if (settings.json.read_named_tuples_as_objects && have_explicit_names) { skipWhitespaceIfAny(istr); - assertChar('{', istr); + if constexpr (throw_exception) + assertChar('{', istr); + else if (!checkChar('{', istr)) + return ReturnType(false); skipWhitespaceIfAny(istr); - addElementSafe(elems.size(), column, [&] + auto impl = [&]() { std::vector seen_elements(elems.size(), 0); size_t processed = 0; @@ -256,18 +350,32 @@ void SerializationTuple::deserializeTextJSON(IColumn & column, ReadBuffer & istr while (!istr.eof() && *istr.position() != '}') { if (!settings.json.ignore_unknown_keys_in_named_tuple && processed == elems.size()) - throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected number of elements in named tuple. Expected no more than {} (consider enabling input_format_json_ignore_unknown_keys_in_named_tuple setting)", elems.size()); + { + if constexpr (throw_exception) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected number of elements in named tuple. Expected no more than {} (consider enabling input_format_json_ignore_unknown_keys_in_named_tuple setting)", elems.size()); + return ReturnType(false); + } if (processed + skipped > 0) { - assertChar(',', istr); + if constexpr (throw_exception) + assertChar(',', istr); + else if (!checkChar(',', istr)) + return ReturnType(false); skipWhitespaceIfAny(istr); } std::string name; - readDoubleQuotedString(name, istr); + if constexpr (throw_exception) + readDoubleQuotedString(name, istr); + else if (!tryReadDoubleQuotedString(name, istr)) + return ReturnType(false); + skipWhitespaceIfAny(istr); - assertChar(':', istr); + if constexpr (throw_exception) + assertChar(':', istr); + else if (!checkChar(':', istr)) + return ReturnType(false); skipWhitespaceIfAny(istr); const size_t element_pos = getPositionByName(name); @@ -275,36 +383,52 @@ void SerializationTuple::deserializeTextJSON(IColumn & column, ReadBuffer & istr { if (settings.json.ignore_unknown_keys_in_named_tuple) { - skipJSONField(istr, name); + if constexpr (throw_exception) + skipJSONField(istr, name); + else if (!trySkipJSONField(istr, name)) + return ReturnType(false); + skipWhitespaceIfAny(istr); ++skipped; continue; } else - throw Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK, "Tuple doesn't have element with name '{}', enable setting input_format_json_ignore_unknown_keys_in_named_tuple", name); + { + if constexpr (throw_exception) + throw Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK, "Tuple doesn't have element with name '{}', enable setting input_format_json_ignore_unknown_keys_in_named_tuple", name); + return ReturnType(false); + } } seen_elements[element_pos] = 1; auto & element_column = extractElementColumn(column, element_pos); - try + if constexpr (throw_exception) { - if (settings.null_as_default) - SerializationNullable::deserializeTextJSONImpl(element_column, istr, settings, elems[element_pos]); - else - elems[element_pos]->deserializeTextJSON(element_column, istr, settings); + try + { + deserialize_element(element_column, element_pos); + } + catch (Exception & e) + { + e.addMessage("(while reading the value of nested key " + name + ")"); + throw; + } } - catch (Exception & e) + else { - e.addMessage("(while reading the value of nested key " + name + ")"); - throw; + if (!deserialize_element(element_column, element_pos)) + return ReturnType(false); } skipWhitespaceIfAny(istr); ++processed; } - assertChar('}', istr); + if constexpr (throw_exception) + assertChar('}', istr); + else if (!checkChar('}', istr)) + return ReturnType(false); /// Check if we have missing elements. if (processed != elems.size()) @@ -315,41 +439,87 @@ void SerializationTuple::deserializeTextJSON(IColumn & column, ReadBuffer & istr continue; if (!settings.json.defaults_for_missing_elements_in_named_tuple) - throw Exception( - ErrorCodes::INCORRECT_DATA, - "JSON object doesn't contain tuple element {}. If you want to insert defaults in case of missing elements, " - "enable setting input_format_json_defaults_for_missing_elements_in_named_tuple", - elems[element_pos]->getElementName()); + { + if constexpr (throw_exception) + throw Exception( + ErrorCodes::INCORRECT_DATA, + "JSON object doesn't contain tuple element {}. If you want to insert defaults in case of missing elements, " + "enable setting input_format_json_defaults_for_missing_elements_in_named_tuple", + elems[element_pos]->getElementName()); + return ReturnType(false); + } auto & element_column = extractElementColumn(column, element_pos); element_column.insertDefault(); } } - }); + + return ReturnType(true); + }; + + if constexpr (throw_exception) + addElementSafe(elems.size(), column, impl); + else + return addElementSafe(elems.size(), column, impl); } else { - assertChar('[', istr); + skipWhitespaceIfAny(istr); + if constexpr (throw_exception) + assertChar('[', istr); + else if (!checkChar('[', istr)) + return ReturnType(false); + skipWhitespaceIfAny(istr); - addElementSafe(elems.size(), column, [&] + auto impl = [&]() { for (size_t i = 0; i < elems.size(); ++i) { skipWhitespaceIfAny(istr); if (i != 0) { - assertChar(',', istr); + if constexpr (throw_exception) + assertChar(',', istr); + else if (!checkChar(',', istr)) + return ReturnType(false); skipWhitespaceIfAny(istr); } - elems[i]->deserializeTextJSON(extractElementColumn(column, i), istr, settings); + + auto & element_column = extractElementColumn(column, i); + + if constexpr (throw_exception) + deserialize_element(element_column, i); + else if (!deserialize_element(element_column, i)) + return ReturnType(false); } skipWhitespaceIfAny(istr); - assertChar(']', istr); - }); + if constexpr (throw_exception) + assertChar(']', istr); + else if (!checkChar(']', istr)) + return ReturnType(false); + + return ReturnType(true); + }; + + if constexpr (throw_exception) + addElementSafe(elems.size(), column, impl); + else + return addElementSafe(elems.size(), column, impl); } } +void SerializationTuple::deserializeTextJSON(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings & settings) const +{ + deserializeTextJSONImpl(column, istr, settings); +} + +bool SerializationTuple::tryDeserializeTextJSON(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings & settings) const +{ + return deserializeTextJSONImpl(column, istr, settings); +} + + void SerializationTuple::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeCString("", ostr); @@ -385,14 +555,48 @@ void SerializationTuple::deserializeTextCSV(IColumn & column, ReadBuffer & istr, assertChar(settings.csv.tuple_delimiter, istr); skipWhitespaceIfAny(istr); } - if (settings.null_as_default) - SerializationNullable::deserializeTextCSVImpl(extractElementColumn(column, i), istr, settings, elems[i]); + + auto & element_column = extractElementColumn(column, i); + if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(element_column)) + SerializationNullable::deserializeNullAsDefaultOrNestedTextCSV(element_column, istr, settings, elems[i]); else - elems[i]->deserializeTextCSV(extractElementColumn(column, i), istr, settings); + elems[i]->deserializeTextCSV(element_column, istr, settings); } }); } +bool SerializationTuple::tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + return addElementSafe(elems.size(), column, [&] + { + const size_t size = elems.size(); + for (size_t i = 0; i < size; ++i) + { + if (i != 0) + { + skipWhitespaceIfAny(istr); + if (!checkChar(settings.csv.tuple_delimiter, istr)) + return false; + skipWhitespaceIfAny(istr); + } + + auto & element_column = extractElementColumn(column, i); + if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(element_column)) + { + if (!SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextCSV(element_column, istr, settings, elems[i])) + return false; + } + else + { + if (!elems[i]->tryDeserializeTextCSV(element_column, istr, settings)) + return false; + } + } + + return true; + }); +} + void SerializationTuple::enumerateStreams( EnumerateStreamsSettings & settings, const StreamCallback & callback, diff --git a/src/DataTypes/Serializations/SerializationTuple.h b/src/DataTypes/Serializations/SerializationTuple.h index 7325259f440..d9c63a05217 100644 --- a/src/DataTypes/Serializations/SerializationTuple.h +++ b/src/DataTypes/Serializations/SerializationTuple.h @@ -23,14 +23,17 @@ public: void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const override; + bool tryDeserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const override; void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextJSONPretty(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t indent) const override; void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; /// Tuples in CSV format will be serialized as separate columns (that is, losing their nesting in the tuple). void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; /** Each sub-column in a tuple is serialized in separate stream. */ @@ -73,6 +76,15 @@ private: bool have_explicit_names; size_t getPositionByName(const String & name) const; + + template + ReturnType deserializeTextImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const; + + template + ReturnType deserializeTextJSONImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const; + + template + ReturnType deserializeTextCSVImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const; }; } diff --git a/src/DataTypes/Serializations/SerializationUUID.cpp b/src/DataTypes/Serializations/SerializationUUID.cpp index 5cf17b4c0c8..5a7aeca67a0 100644 --- a/src/DataTypes/Serializations/SerializationUUID.cpp +++ b/src/DataTypes/Serializations/SerializationUUID.cpp @@ -25,15 +25,16 @@ void SerializationUUID::deserializeText(IColumn & column, ReadBuffer & istr, con throwUnexpectedDataAfterParsedValue(column, istr, settings, "UUID"); } -void SerializationUUID::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +bool SerializationUUID::tryDeserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const { - deserializeText(column, istr, settings, false); + UUID x; + if (!tryReadText(x, istr) || (whole && !istr.eof())) + return false; + + assert_cast(column).getData().push_back(x); + return true; } -void SerializationUUID::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const -{ - serializeText(column, row_num, ostr, settings); -} void SerializationUUID::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { @@ -76,6 +77,17 @@ void SerializationUUID::deserializeTextQuoted(IColumn & column, ReadBuffer & ist assert_cast(column).getData().push_back(std::move(uuid)); /// It's important to do this at the end - for exception safety. } +bool SerializationUUID::tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +{ + UUID uuid; + String field; + if (!checkChar('\'', istr) || !tryReadText(uuid, istr) || !checkChar('\'', istr)) + return false; + + assert_cast(column).getData().push_back(std::move(uuid)); + return true; +} + void SerializationUUID::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeChar('"', ostr); @@ -92,6 +104,15 @@ void SerializationUUID::deserializeTextJSON(IColumn & column, ReadBuffer & istr, assert_cast(column).getData().push_back(x); } +bool SerializationUUID::tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +{ + UUID x; + if (!checkChar('"', istr) || !tryReadText(x, istr) || !checkChar('"', istr)) + return false; + assert_cast(column).getData().push_back(x); + return true; +} + void SerializationUUID::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeChar('"', ostr); @@ -106,6 +127,14 @@ void SerializationUUID::deserializeTextCSV(IColumn & column, ReadBuffer & istr, assert_cast(column).getData().push_back(value); } +bool SerializationUUID::tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +{ + UUID value; + if (!tryReadCSV(value, istr)) + return false; + assert_cast(column).getData().push_back(value); + return true; +} void SerializationUUID::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const { diff --git a/src/DataTypes/Serializations/SerializationUUID.h b/src/DataTypes/Serializations/SerializationUUID.h index da8c15f7279..458504f8f42 100644 --- a/src/DataTypes/Serializations/SerializationUUID.h +++ b/src/DataTypes/Serializations/SerializationUUID.h @@ -10,14 +10,16 @@ class SerializationUUID : public SimpleTextSerialization public: void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const override; - void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; - void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const override; void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const override; diff --git a/src/DataTypes/Serializations/SerializationVariant.cpp b/src/DataTypes/Serializations/SerializationVariant.cpp new file mode 100644 index 00000000000..ebd44fd6955 --- /dev/null +++ b/src/DataTypes/Serializations/SerializationVariant.cpp @@ -0,0 +1,828 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int NOT_IMPLEMENTED; + extern const int INCORRECT_DATA; +} + +void SerializationVariant::enumerateStreams( + EnumerateStreamsSettings & settings, + const StreamCallback & callback, + const SubstreamData & data) const +{ + const auto * type_variant = data.type ? &assert_cast(*data.type) : nullptr; + const auto * column_variant = data.column ? &assert_cast(*data.column) : nullptr; + + auto discriminators_serialization = std::make_shared(std::make_shared>(), "discr", false); + auto local_discriminators = column_variant ? column_variant->getLocalDiscriminatorsPtr() : nullptr; + + settings.path.push_back(Substream::VariantDiscriminators); + auto discriminators_data = SubstreamData(discriminators_serialization) + .withType(type_variant ? std::make_shared>() : nullptr) + .withColumn(column_variant ? column_variant->getLocalDiscriminatorsPtr() : nullptr) + .withSerializationInfo(data.serialization_info); + + settings.path.back().data = discriminators_data; + callback(settings.path); + settings.path.pop_back(); + + settings.path.push_back(Substream::VariantElements); + settings.path.back().data = data; + + for (size_t i = 0; i < variants.size(); ++i) + { + settings.path.back().creator = std::make_shared(local_discriminators, variant_names[i], i, column_variant ? column_variant->localDiscriminatorByGlobal(i) : i); + + auto variant_data = SubstreamData(variants[i]) + .withType(type_variant ? type_variant->getVariant(i) : nullptr) + .withColumn(column_variant ? column_variant->getVariantPtrByGlobalDiscriminator(i) : nullptr) + .withSerializationInfo(data.serialization_info); + + addVariantElementToPath(settings.path, i); + settings.path.back().data = variant_data; + variants[i]->enumerateStreams(settings, callback, variant_data); + settings.path.pop_back(); + } + + settings.path.pop_back(); +} + +struct SerializeBinaryBulkStateVariant : public ISerialization::SerializeBinaryBulkState +{ + std::vector states; +}; + +struct DeserializeBinaryBulkStateVariant : public ISerialization::DeserializeBinaryBulkState +{ + std::vector states; +}; + +void SerializationVariant::serializeBinaryBulkStatePrefix( + const IColumn & column, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const +{ + const ColumnVariant & col = assert_cast(column); + + auto variant_state = std::make_shared(); + variant_state->states.resize(variants.size()); + + settings.path.push_back(Substream::VariantElements); + + for (size_t i = 0; i < variants.size(); ++i) + { + addVariantElementToPath(settings.path, i); + variants[i]->serializeBinaryBulkStatePrefix(col.getVariantByGlobalDiscriminator(i), settings, variant_state->states[i]); + settings.path.pop_back(); + } + + settings.path.pop_back(); + state = std::move(variant_state); +} + + +void SerializationVariant::serializeBinaryBulkStateSuffix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const +{ + auto * variant_state = checkAndGetState(state); + + settings.path.push_back(Substream::VariantElements); + for (size_t i = 0; i < variants.size(); ++i) + { + addVariantElementToPath(settings.path, i); + variants[i]->serializeBinaryBulkStateSuffix(settings, variant_state->states[i]); + settings.path.pop_back(); + } + settings.path.pop_back(); +} + + +void SerializationVariant::deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state) const +{ + auto variant_state = std::make_shared(); + variant_state->states.resize(variants.size()); + + settings.path.push_back(Substream::VariantElements); + for (size_t i = 0; i < variants.size(); ++i) + { + addVariantElementToPath(settings.path, i); + variants[i]->deserializeBinaryBulkStatePrefix(settings, variant_state->states[i]); + settings.path.pop_back(); + } + + settings.path.pop_back(); + state = std::move(variant_state); +} + + +void SerializationVariant::serializeBinaryBulkWithMultipleStreams( + const IColumn & column, + size_t offset, + size_t limit, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const +{ + const ColumnVariant & col = assert_cast(column); + if (const size_t size = col.size(); limit == 0 || offset + limit > size) + limit = size - offset; + + settings.path.push_back(Substream::VariantDiscriminators); + auto * discriminators_stream = settings.getter(settings.path); + settings.path.pop_back(); + + if (!discriminators_stream) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty stream for VariantDiscriminators in SerializationVariant::serializeBinaryBulkWithMultipleStreams"); + + auto * variant_state = checkAndGetState(state); + + /// If offset = 0 and limit == col.size() or we have only NULLs, we don't need to calculate + /// offsets and limits for variants and need to just serialize whole columns. + if ((offset == 0 && limit == col.size()) || col.hasOnlyNulls()) + { + /// First, serialize discriminators. + /// If we have only NULLs or local and global discriminators are the same, just serialize the column as is. + if (col.hasOnlyNulls() || col.hasGlobalVariantsOrder()) + { + SerializationNumber().serializeBinaryBulk(col.getLocalDiscriminatorsColumn(), *discriminators_stream, offset, limit); + } + /// If local and global discriminators are different, we should convert local to global before serializing (because we don't serialize the mapping). + else + { + const auto & local_discriminators = col.getLocalDiscriminators(); + for (size_t i = offset; i != offset + limit; ++i) + writeBinaryLittleEndian(col.globalDiscriminatorByLocal(local_discriminators[i]), *discriminators_stream); + } + + /// Second, serialize variants in global order. + settings.path.push_back(Substream::VariantElements); + for (size_t i = 0; i != variants.size(); ++i) + { + addVariantElementToPath(settings.path, i); + variants[i]->serializeBinaryBulkWithMultipleStreams(col.getVariantByGlobalDiscriminator(i), 0, 0, settings, variant_state->states[i]); + settings.path.pop_back(); + } + settings.path.pop_back(); + return; + } + + /// If we have only one non empty variant and no NULLs, we can use the same limit offset for this variant. + if (auto non_empty_local_discr = col.getLocalDiscriminatorOfOneNoneEmptyVariantNoNulls()) + { + /// First, serialize discriminators. + /// We know that all discriminators are the same, so we just need to serialize this discriminator limit times. + auto non_empty_global_discr = col.globalDiscriminatorByLocal(*non_empty_local_discr); + for (size_t i = 0; i != limit; ++i) + writeBinaryLittleEndian(non_empty_global_discr, *discriminators_stream); + + /// Second, serialize variants in global order. + settings.path.push_back(Substream::VariantElements); + for (size_t i = 0; i != variants.size(); ++i) + { + addVariantElementToPath(settings.path, i); + /// For non empty variant use the same offset/limit as for whole Variant column + if (i == non_empty_global_discr) + variants[i]->serializeBinaryBulkWithMultipleStreams(col.getVariantByGlobalDiscriminator(i), offset, limit, settings, variant_state->states[i]); + /// For empty variants, use just 0/0, they won't serialize anything. + else + variants[i]->serializeBinaryBulkWithMultipleStreams(col.getVariantByGlobalDiscriminator(i), 0, 0, settings, variant_state->states[i]); + settings.path.pop_back(); + } + settings.path.pop_back(); + return; + } + + /// In general case we should iterate through local discriminators in range [offset, offset + limit] to serialize global discriminators and calculate offset/limit pair for each variant. + const auto & local_discriminators = col.getLocalDiscriminators(); + const auto & offsets = col.getOffsets(); + std::vector> variant_offsets_and_limits(variants.size(), {0, 0}); + size_t end = offset + limit; + for (size_t i = offset; i < end; ++i) + { + auto global_discr = col.globalDiscriminatorByLocal(local_discriminators[i]); + writeBinaryLittleEndian(global_discr, *discriminators_stream); + + if (global_discr != ColumnVariant::NULL_DISCRIMINATOR) + { + /// If we see this discriminator for the first time, update offset + if (!variant_offsets_and_limits[global_discr].second) + variant_offsets_and_limits[global_discr].first = offsets[i]; + /// Update limit for this discriminator. + ++variant_offsets_and_limits[global_discr].second; + } + } + + /// If limit for some variant is 0, it means that we don't have its discriminator in the range. + /// Set offset to the size of column for such variants, so we won't serialize values from them. + for (size_t i = 0; i != variant_offsets_and_limits.size(); ++i) + { + if (!variant_offsets_and_limits[i].second) + variant_offsets_and_limits[i].first = col.getVariantByGlobalDiscriminator(i).size(); + } + + /// Serialize variants in global order. + settings.path.push_back(Substream::VariantElements); + for (size_t i = 0; i != variants.size(); ++i) + { + addVariantElementToPath(settings.path, i); + variants[i]->serializeBinaryBulkWithMultipleStreams( + col.getVariantByGlobalDiscriminator(i), + variant_offsets_and_limits[i].first, + variant_offsets_and_limits[i].second, + settings, + variant_state->states[i]); + settings.path.pop_back(); + } + settings.path.pop_back(); +} + + +void SerializationVariant::deserializeBinaryBulkWithMultipleStreams( + ColumnPtr & column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const +{ + auto mutable_column = column->assumeMutable(); + ColumnVariant & col = assert_cast(*mutable_column); + /// We always serialize Variant column with global variants order, + /// so while deserialization column should be always with global variants order. + if (!col.hasGlobalVariantsOrder()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to deserialize data into Variant column with not global variants order"); + + /// First, deserialize new discriminators. + /// We deserialize them into a separate column to be able to use substream cache, + /// so if we also need to deserialize some of sub columns, we will read discriminators only once. + settings.path.push_back(Substream::VariantDiscriminators); + ColumnPtr discriminators; + if (auto cached_discriminators = getFromSubstreamsCache(cache, settings.path)) + { + discriminators = cached_discriminators; + } + else + { + auto * discriminators_stream = settings.getter(settings.path); + if (!discriminators_stream) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty stream for VariantDiscriminators in SerializationVariant::deserializeBinaryBulkWithMultipleStreams"); + + discriminators = ColumnVariant::ColumnDiscriminators::create(); + SerializationNumber().deserializeBinaryBulk(*discriminators->assumeMutable(), *discriminators_stream, limit, 0); + addToSubstreamsCache(cache, settings.path, discriminators); + } + + settings.path.pop_back(); + + /// Iterate through new discriminators, append them to column and calculate the limit for each variant. + /// While calculating limits we can also fill offsets column (we store offsets only in memory). + const auto & discriminators_data = assert_cast(*discriminators).getData(); + auto & local_discriminators = col.getLocalDiscriminators(); + local_discriminators.reserve(local_discriminators.size() + limit); + auto & offsets = col.getOffsets(); + offsets.reserve(offsets.size() + limit); + std::vector variant_limits(variants.size(), 0); + for (size_t i = 0; i != limit; ++i) + { + ColumnVariant::Discriminator discr = discriminators_data[i]; + local_discriminators.push_back(discr); + if (discr == ColumnVariant::NULL_DISCRIMINATOR) + offsets.emplace_back(); + else + offsets.push_back(col.getVariantByLocalDiscriminator(discr).size() + variant_limits[discr]++); + } + + /// Now we can deserialize variants according to their limits. + auto * variant_state = checkAndGetState(state); + settings.path.push_back(Substream::VariantElements); + for (size_t i = 0; i != variants.size(); ++i) + { + addVariantElementToPath(settings.path, i); + variants[i]->deserializeBinaryBulkWithMultipleStreams(col.getVariantPtrByLocalDiscriminator(i), variant_limits[i], settings, variant_state->states[i], cache); + settings.path.pop_back(); + } + settings.path.pop_back(); +} + +void SerializationVariant::addVariantElementToPath(DB::ISerialization::SubstreamPath & path, size_t i) const +{ + path.push_back(Substream::VariantElement); + path.back().variant_element_name = variant_names[i]; +} + +void SerializationVariant::serializeBinary(const Field & /*field*/, WriteBuffer & /*ostr*/, const FormatSettings & /*settings*/) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method serializeBinary from a field is not implemented for SerializationVariant"); +} + +void SerializationVariant::deserializeBinary(Field & /*field*/, ReadBuffer & /*istr*/, const FormatSettings & /*settings*/) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method deserializeBinary to a field is not implemented for SerializationVariant"); +} + +void SerializationVariant::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + const ColumnVariant & col = assert_cast(column); + auto global_discr = col.globalDiscriminatorAt(row_num); + writeBinaryLittleEndian(global_discr, ostr); + if (global_discr != ColumnVariant::NULL_DISCRIMINATOR) + variants[global_discr]->serializeBinary(col.getVariantByGlobalDiscriminator(global_discr), col.offsetAt(row_num), ostr, settings); +} + +void SerializationVariant::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + ColumnVariant & col = assert_cast(column); + ColumnVariant::Discriminator global_discr; + readBinaryLittleEndian(global_discr, istr); + if (global_discr == ColumnVariant::NULL_DISCRIMINATOR) + { + col.insertDefault(); + } + else + { + auto & variant_column = col.getVariantByGlobalDiscriminator(global_discr); + variants[global_discr]->deserializeBinary(variant_column, istr, settings); + col.getLocalDiscriminators().push_back(col.localDiscriminatorByGlobal(global_discr)); + col.getOffsets().push_back(variant_column.size() - 1); + } +} + +namespace +{ + +std::unordered_map getTypesTextDeserializePriorityMap() +{ + static const std::vector priorities = { + /// Complex types have highest priority. + TypeIndex::Array, + TypeIndex::Tuple, + TypeIndex::Map, + TypeIndex::AggregateFunction, + + /// Enums can be parsed both from strings and numbers. + /// So they have high enough priority. + TypeIndex::Enum8, + TypeIndex::Enum16, + + /// Types that can be parsed from strings. + TypeIndex::UUID, + TypeIndex::IPv4, + TypeIndex::IPv6, + + /// Types that can be parsed from numbers. + /// The order: + /// 1) Integers + /// 2) Big Integers + /// 3) Decimals + /// 4) Floats + /// In each group small types have higher priority. + TypeIndex::Int8, + TypeIndex::UInt8, + TypeIndex::Int16, + TypeIndex::UInt16, + TypeIndex::Int32, + TypeIndex::UInt32, + TypeIndex::Int64, + TypeIndex::UInt64, + TypeIndex::Int128, + TypeIndex::UInt128, + TypeIndex::Int256, + TypeIndex::UInt256, + TypeIndex::Decimal32, + TypeIndex::Decimal64, + TypeIndex::Decimal128, + TypeIndex::Decimal256, + TypeIndex::Float32, + TypeIndex::Float64, + + /// Dates and DateTimes. More simple Date types have higher priority. + /// They have lower priority as numbers as some DateTimes sometimes can + /// be also parsed from numbers, but we don't want it usually. + TypeIndex::Date, + TypeIndex::Date32, + TypeIndex::DateTime, + TypeIndex::DateTime64, + + /// String types have almost the lowest priority, + /// as in text formats almost all data can + /// be deserialized into String type. + TypeIndex::FixedString, + TypeIndex::String, + }; + + std::unordered_map priority_map; + priority_map.reserve(priorities.size()); + for (size_t i = 0; i != priorities.size(); ++i) + priority_map[priorities[i]] = priorities.size() - i; + return priority_map; +} + +/// We want to create more or less optimal order of types in which we will try text deserializations. +/// To do it, for each type we calculate a priority and then sort them by this priority. +/// Above we defined priority of each data type, but types can be nested and also we can have LowCardinality and Nullable. +/// To sort any nested types we create a priority that is a tuple of 3 elements: +/// 1) The maximum depth of nested types like Array/Map/Tuple. +/// 2) The combination of simple and complex types priorities. +/// 3) The depth of nested types LowCardinality/Nullable. +/// So, when we will sort types, first we will sort by the maximum depth of nested types, so more nested types are deserialized first, +/// then for types with the same depth we sort by the types priority, and last we sort by the depth of LowCardinality/Nullable types, +/// so if we have types with the same level of nesting and the same priority, we will first try to deserialize LowCardinality/Nullable types +/// (for example if we have types Array(Array(String)) and Array(Array(Nullable(String))). +/// This is just a batch of heuristics, +std::tuple getTypeTextDeserializePriority(const DataTypePtr & type, size_t nested_depth, size_t simple_nested_depth, std::unordered_map & priority_map) +{ + if (const auto * nullable_type = typeid_cast(type.get())) + return getTypeTextDeserializePriority(nullable_type->getNestedType(), nested_depth, simple_nested_depth + 1, priority_map); + + if (const auto * lc_type = typeid_cast(type.get())) + return getTypeTextDeserializePriority(lc_type->getDictionaryType(), nested_depth, simple_nested_depth + 1, priority_map); + + if (const auto * array_type = typeid_cast(type.get())) + { + auto [elements_nested_depth, elements_priority, elements_simple_nested_depth] = getTypeTextDeserializePriority(array_type->getNestedType(), nested_depth + 1, simple_nested_depth, priority_map); + return {elements_nested_depth, elements_priority + priority_map[TypeIndex::Array], elements_simple_nested_depth}; + } + + if (const auto * tuple_type = typeid_cast(type.get())) + { + size_t max_nested_depth = 0; + size_t sum_priority = 0; + size_t max_simple_nested_depth = 0; + for (const auto & elem : tuple_type->getElements()) + { + auto [elem_nested_depth, elem_priority, elem_simple_nested_depth] = getTypeTextDeserializePriority(elem, nested_depth + 1, simple_nested_depth, priority_map); + sum_priority += elem_priority; + if (elem_nested_depth > max_nested_depth) + max_nested_depth = elem_nested_depth; + if (elem_simple_nested_depth > max_simple_nested_depth) + max_simple_nested_depth = elem_simple_nested_depth; + } + + return {max_nested_depth, sum_priority + priority_map[TypeIndex::Tuple], max_simple_nested_depth}; + } + + if (const auto * map_type = typeid_cast(type.get())) + { + auto [key_max_depth, key_priority, key_simple_nested_depth] = getTypeTextDeserializePriority(map_type->getKeyType(), nested_depth + 1, simple_nested_depth, priority_map); + auto [value_max_depth, value_priority, value_simple_nested_depth] = getTypeTextDeserializePriority(map_type->getValueType(), nested_depth + 1, simple_nested_depth, priority_map); + return {std::max(key_max_depth, value_max_depth), key_priority + value_priority + priority_map[TypeIndex::Map], std::max(key_simple_nested_depth, value_simple_nested_depth)}; + } + + if (const auto * variant_type = typeid_cast(type.get())) + { + size_t max_priority = 0; + size_t max_depth = 0; + size_t max_simple_nested_depth = 0; + for (const auto & variant : variant_type->getVariants()) + { + auto [variant_max_depth, variant_priority, variant_simple_nested_depth] = getTypeTextDeserializePriority(variant, nested_depth, simple_nested_depth, priority_map); + if (variant_priority > max_priority) + max_priority = variant_priority; + if (variant_max_depth > max_depth) + max_depth = variant_max_depth; + if (variant_simple_nested_depth > max_simple_nested_depth) + max_simple_nested_depth = variant_simple_nested_depth; + } + + return {max_depth, max_priority, max_simple_nested_depth}; + } + + return {nested_depth, priority_map[type->getTypeId()], simple_nested_depth}; +} + +} + +std::vector SerializationVariant::getVariantsDeserializeTextOrder(const DB::DataTypes & variant_types) +{ + std::vector> priorities; + priorities.reserve(variant_types.size()); + std::vector order; + order.reserve(variant_types.size()); + auto priority_map = getTypesTextDeserializePriorityMap(); + for (size_t i = 0; i != variant_types.size(); ++i) + { + priorities.push_back(getTypeTextDeserializePriority(variant_types[i], 0, 0, priority_map)); + order.push_back(i); + } + + std::sort(order.begin(), order.end(), [&](size_t left, size_t right) { return priorities[left] > priorities[right]; }); + String types_order; + for (auto i : order) + types_order += " " + variant_types[i]->getName(); + return order; +} + + +bool SerializationVariant::tryDeserializeImpl( + IColumn & column, + const String & field, + std::function check_for_null, + std::function try_deserialize_variant) const +{ + auto & column_variant = assert_cast(column); + ReadBufferFromString null_buf(field); + if (check_for_null(null_buf) && null_buf.eof()) + { + column_variant.insertDefault(); + return true; + } + + for (size_t global_discr : deserialize_text_order) + { + ReadBufferFromString variant_buf(field); + /// Usually try_deserialize_variant should not throw an exception, but let's use try/catch just in case. + try + { + auto & variant_column = column_variant.getVariantByGlobalDiscriminator(global_discr); + size_t prev_size = variant_column.size(); + if (try_deserialize_variant(variant_column, variants[global_discr], variant_buf) && variant_buf.eof()) + { + column_variant.getLocalDiscriminators().push_back(column_variant.localDiscriminatorByGlobal(global_discr)); + column_variant.getOffsets().push_back(prev_size); + return true; + } + else if (variant_column.size() > prev_size) + { + variant_column.popBack(1); + } + } + catch (...) + { + /// Try next variant. + } + } + + return false; +} + +void SerializationVariant::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + const ColumnVariant & col = assert_cast(column); + auto global_discr = col.globalDiscriminatorAt(row_num); + if (global_discr == ColumnVariant::NULL_DISCRIMINATOR) + SerializationNullable::serializeNullEscaped(ostr, settings); + else + variants[global_discr]->serializeTextEscaped(col.getVariantByGlobalDiscriminator(global_discr), col.offsetAt(row_num), ostr, settings); +} + +bool SerializationVariant::tryDeserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + String field; + readEscapedString(field, istr); + return tryDeserializeTextEscapedImpl(column, field, settings); +} + +void SerializationVariant::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + String field; + readEscapedString(field, istr); + if (!tryDeserializeTextEscapedImpl(column, field, settings)) + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot parse escaped value of type {} here: {}", variant_name, field); +} + +bool SerializationVariant::tryDeserializeTextEscapedImpl(DB::IColumn & column, const String & field, const DB::FormatSettings & settings) const +{ + auto check_for_null = [&](ReadBuffer & buf) + { + return SerializationNullable::tryDeserializeNullEscaped(buf, settings); + }; + auto try_deserialize_variant =[&](IColumn & variant_column, const SerializationPtr & variant_serialization, ReadBuffer & buf) + { + return variant_serialization->tryDeserializeTextEscaped(variant_column, buf, settings); + }; + + return tryDeserializeImpl(column, field, check_for_null, try_deserialize_variant); +} + +void SerializationVariant::serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + const ColumnVariant & col = assert_cast(column); + auto global_discr = col.globalDiscriminatorAt(row_num); + if (global_discr == ColumnVariant::NULL_DISCRIMINATOR) + SerializationNullable::serializeNullRaw(ostr, settings); + else + variants[global_discr]->serializeTextRaw(col.getVariantByGlobalDiscriminator(global_discr), col.offsetAt(row_num), ostr, settings); +} + +bool SerializationVariant::tryDeserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + String field; + readString(field, istr); + return tryDeserializeTextRawImpl(column, field, settings); +} + +void SerializationVariant::deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + String field; + readString(field, istr); + if (!tryDeserializeTextRawImpl(column, field, settings)) + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot parse raw value of type {} here: {}", variant_name, field); +} + +bool SerializationVariant::tryDeserializeTextRawImpl(DB::IColumn & column, const String & field, const DB::FormatSettings & settings) const +{ + auto check_for_null = [&](ReadBuffer & buf) + { + return SerializationNullable::tryDeserializeNullRaw(buf, settings); + }; + auto try_deserialize_variant =[&](IColumn & variant_column, const SerializationPtr & variant_serialization, ReadBuffer & buf) + { + return variant_serialization->tryDeserializeTextRaw(variant_column, buf, settings); + }; + + return tryDeserializeImpl(column, field, check_for_null, try_deserialize_variant); +} + +void SerializationVariant::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + const ColumnVariant & col = assert_cast(column); + auto global_discr = col.globalDiscriminatorAt(row_num); + if (global_discr == ColumnVariant::NULL_DISCRIMINATOR) + SerializationNullable::serializeNullQuoted(ostr); + else + variants[global_discr]->serializeTextQuoted(col.getVariantByGlobalDiscriminator(global_discr), col.offsetAt(row_num), ostr, settings); +} + +bool SerializationVariant::tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + String field; + if (!tryReadQuotedField(field, istr)) + return false; + return tryDeserializeTextQuotedImpl(column, field, settings); +} + +void SerializationVariant::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + String field; + readQuotedField(field, istr); + if (!tryDeserializeTextQuotedImpl(column, field, settings)) + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot parse quoted value of type {} here: {}", variant_name, field); +} + +bool SerializationVariant::tryDeserializeTextQuotedImpl(DB::IColumn & column, const String & field, const DB::FormatSettings & settings) const +{ + auto check_for_null = [&](ReadBuffer & buf) + { + return SerializationNullable::tryDeserializeNullQuoted(buf); + }; + auto try_deserialize_variant =[&](IColumn & variant_column, const SerializationPtr & variant_serialization, ReadBuffer & buf) + { + return variant_serialization->tryDeserializeTextQuoted(variant_column, buf, settings); + }; + + return tryDeserializeImpl(column, field, check_for_null, try_deserialize_variant); +} + +void SerializationVariant::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + const ColumnVariant & col = assert_cast(column); + auto global_discr = col.globalDiscriminatorAt(row_num); + if (global_discr == ColumnVariant::NULL_DISCRIMINATOR) + SerializationNullable::serializeNullCSV(ostr, settings); + else + variants[global_discr]->serializeTextCSV(col.getVariantByGlobalDiscriminator(global_discr), col.offsetAt(row_num), ostr, settings); +} + +bool SerializationVariant::tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + String field; + readCSVStringInto(field, istr, settings.csv); + return tryDeserializeTextCSVImpl(column, field, settings); +} + +void SerializationVariant::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + String field; + readCSVField(field, istr, settings.csv); + if (!tryDeserializeTextCSVImpl(column, field, settings)) + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot parse CSV value of type {} here: {}", variant_name, field); +} + +bool SerializationVariant::tryDeserializeTextCSVImpl(DB::IColumn & column, const String & field, const DB::FormatSettings & settings) const +{ + auto check_for_null = [&](ReadBuffer & buf) + { + return SerializationNullable::tryDeserializeNullCSV(buf, settings); + }; + auto try_deserialize_variant =[&](IColumn & variant_column, const SerializationPtr & variant_serialization, ReadBuffer & buf) + { + return variant_serialization->tryDeserializeTextCSV(variant_column, buf, settings); + }; + + return tryDeserializeImpl(column, field, check_for_null, try_deserialize_variant); +} + +void SerializationVariant::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + const ColumnVariant & col = assert_cast(column); + auto global_discr = col.globalDiscriminatorAt(row_num); + if (global_discr == ColumnVariant::NULL_DISCRIMINATOR) + SerializationNullable::serializeNullText(ostr, settings); + else + variants[global_discr]->serializeText(col.getVariantByGlobalDiscriminator(global_discr), col.offsetAt(row_num), ostr, settings); +} + +bool SerializationVariant::tryDeserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + String field; + readStringUntilEOF(field, istr); + return tryDeserializeWholeTextImpl(column, field, settings); +} + +void SerializationVariant::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + String field; + readStringUntilEOF(field, istr); + if (!tryDeserializeWholeTextImpl(column, field, settings)) + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot parse text value of type {} here: {}", variant_name, field); +} + +bool SerializationVariant::tryDeserializeWholeTextImpl(DB::IColumn & column, const String & field, const DB::FormatSettings & settings) const +{ + auto check_for_null = [&](ReadBuffer & buf) + { + return SerializationNullable::tryDeserializeNullText(buf); + }; + auto try_deserialize_variant =[&](IColumn & variant_column, const SerializationPtr & variant_serialization, ReadBuffer & buf) + { + return variant_serialization->tryDeserializeWholeText(variant_column, buf, settings); + }; + + return tryDeserializeImpl(column, field, check_for_null, try_deserialize_variant); +} + +void SerializationVariant::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + const ColumnVariant & col = assert_cast(column); + auto global_discr = col.globalDiscriminatorAt(row_num); + if (global_discr == ColumnVariant::NULL_DISCRIMINATOR) + SerializationNullable::serializeNullJSON(ostr); + else + variants[global_discr]->serializeTextJSON(col.getVariantByGlobalDiscriminator(global_discr), col.offsetAt(row_num), ostr, settings); +} + +bool SerializationVariant::tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + String field; + if (!tryReadJSONField(field, istr)) + return false; + return tryDeserializeTextJSONImpl(column, field, settings); +} + +void SerializationVariant::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + String field; + readJSONField(field, istr); + if (!tryDeserializeTextJSONImpl(column, field, settings)) + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot parse JSON value of type {} here: {}", variant_name, field); +} + +bool SerializationVariant::tryDeserializeTextJSONImpl(DB::IColumn & column, const String & field, const DB::FormatSettings & settings) const +{ + auto check_for_null = [&](ReadBuffer & buf) + { + return SerializationNullable::tryDeserializeNullJSON(buf); + }; + auto try_deserialize_variant =[&](IColumn & variant_column, const SerializationPtr & variant_serialization, ReadBuffer & buf) + { + return variant_serialization->tryDeserializeTextJSON(variant_column, buf, settings); + }; + + return tryDeserializeImpl(column, field, check_for_null, try_deserialize_variant); +} + +void SerializationVariant::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + const ColumnVariant & col = assert_cast(column); + auto global_discr = col.globalDiscriminatorAt(row_num); + if (global_discr == ColumnVariant::NULL_DISCRIMINATOR) + SerializationNullable::serializeNullXML(ostr); + else + variants[global_discr]->serializeTextXML(col.getVariantByGlobalDiscriminator(global_discr), col.offsetAt(row_num), ostr, settings); +} + +} diff --git a/src/DataTypes/Serializations/SerializationVariant.h b/src/DataTypes/Serializations/SerializationVariant.h new file mode 100644 index 00000000000..b6bee94c65f --- /dev/null +++ b/src/DataTypes/Serializations/SerializationVariant.h @@ -0,0 +1,116 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class SerializationVariant : public ISerialization +{ +public: + using VariantSerializations = std::vector; + + explicit SerializationVariant( + const VariantSerializations & variants_, + const std::vector & variant_names_, + const std::vector & deserialize_text_order_, + const String & variant_name_) + : variants(variants_), variant_names(variant_names_), deserialize_text_order(deserialize_text_order_), variant_name(variant_name_) + { + } + + void enumerateStreams( + EnumerateStreamsSettings & settings, + const StreamCallback & callback, + const SubstreamData & data) const override; + + void serializeBinaryBulkStatePrefix( + const IColumn & column, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void serializeBinaryBulkStateSuffix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state) const override; + + void serializeBinaryBulkWithMultipleStreams( + const IColumn & column, + size_t offset, + size_t limit, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void deserializeBinaryBulkWithMultipleStreams( + ColumnPtr & column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const override; + + void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const override; + + void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + + void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + + void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + + void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + + void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + + void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + + void serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + bool tryDeserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + + void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + + /// Determine the order in which we should try to deserialize variants. + /// In some cases the text representation of a value can be deserialized + /// into several types (for example, almost all text values can be deserialized + /// into String type), so we uses some heuristics to determine the more optimal order. + static std::vector getVariantsDeserializeTextOrder(const DataTypes & variant_types); + +private: + void addVariantElementToPath(SubstreamPath & path, size_t i) const; + + bool tryDeserializeTextEscapedImpl(IColumn & column, const String & field, const FormatSettings & settings) const; + bool tryDeserializeTextQuotedImpl(IColumn & column, const String & field, const FormatSettings & settings) const; + bool tryDeserializeWholeTextImpl(IColumn & column, const String & field, const FormatSettings & settings) const; + bool tryDeserializeTextCSVImpl(IColumn & column, const String & field, const FormatSettings & settings) const; + bool tryDeserializeTextJSONImpl(IColumn & column, const String & field, const FormatSettings & settings) const; + bool tryDeserializeTextRawImpl(IColumn & column, const String & field, const FormatSettings & settings) const; + + bool tryDeserializeImpl( + IColumn & column, + const String & field, + std::function check_for_null, + std::function try_deserialize_nested) const; + + VariantSerializations variants; + std::vector variant_names; + std::vector deserialize_text_order; + /// Name of Variant data type for better exception messages. + String variant_name; +}; + +} diff --git a/src/DataTypes/Serializations/SerializationVariantElement.cpp b/src/DataTypes/Serializations/SerializationVariantElement.cpp new file mode 100644 index 00000000000..4b24ee5754e --- /dev/null +++ b/src/DataTypes/Serializations/SerializationVariantElement.cpp @@ -0,0 +1,241 @@ +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; + extern const int LOGICAL_ERROR; +} + +void SerializationVariantElement::enumerateStreams( + DB::ISerialization::EnumerateStreamsSettings & settings, + const DB::ISerialization::StreamCallback & callback, + const DB::ISerialization::SubstreamData & data) const +{ + /// We will need stream for discriminators during deserialization. + settings.path.push_back(Substream::VariantDiscriminators); + callback(settings.path); + settings.path.pop_back(); + + addVariantToPath(settings.path); + settings.path.back().data = data; + nested_serialization->enumerateStreams(settings, callback, data); + removeVariantFromPath(settings.path); +} + +void SerializationVariantElement::serializeBinaryBulkStatePrefix(const IColumn &, SerializeBinaryBulkSettings &, SerializeBinaryBulkStatePtr &) const +{ + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, "Method serializeBinaryBulkStatePrefix is not implemented for SerializationVariantElement"); +} + +void SerializationVariantElement::serializeBinaryBulkStateSuffix(SerializeBinaryBulkSettings &, SerializeBinaryBulkStatePtr &) const +{ + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, "Method serializeBinaryBulkStateSuffix is not implemented for SerializationVariantElement"); +} + +void SerializationVariantElement::deserializeBinaryBulkStatePrefix(DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const +{ + addVariantToPath(settings.path); + nested_serialization->deserializeBinaryBulkStatePrefix(settings, state); + removeVariantFromPath(settings.path); +} + +void SerializationVariantElement::serializeBinaryBulkWithMultipleStreams(const IColumn &, size_t, size_t, SerializeBinaryBulkSettings &, SerializeBinaryBulkStatePtr &) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method serializeBinaryBulkWithMultipleStreams is not implemented for SerializationVariantElement"); +} + +void SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams( + ColumnPtr & column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const +{ + auto mutable_column = column->assumeMutable(); + ColumnNullable * nullable_col = typeid_cast(mutable_column.get()); + NullMap * null_map = nullable_col ? &nullable_col->getNullMapData() : nullptr; + + /// First, deserialize discriminators from Variant column. + settings.path.push_back(Substream::VariantDiscriminators); + ColumnPtr discriminators; + if (auto cached_discriminators = getFromSubstreamsCache(cache, settings.path)) + { + discriminators = cached_discriminators; + } + else + { + auto * discriminators_stream = settings.getter(settings.path); + if (!discriminators_stream) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty stream for VariantDiscriminators in SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams"); + + discriminators = ColumnVariant::ColumnDiscriminators::create(); + SerializationNumber().deserializeBinaryBulk(*discriminators->assumeMutable(), *discriminators_stream, limit, 0); + addToSubstreamsCache(cache, settings.path, discriminators); + } + settings.path.pop_back(); + + /// Iterate through discriminators to calculate the size of the variant. + const auto & discriminators_data = assert_cast(*discriminators).getData(); + size_t variant_size = 0; + for (auto discr : discriminators_data) + variant_size += discr == variant_discriminator; + + /// Now we know the size of the variant and can deserialize it. + + /// If the size of variant column is the same as the size of discriminators, + /// we can deserialize new values directly into our column. + if (variant_size == discriminators_data.size()) + { + addVariantToPath(settings.path); + /// Special case when our result column is LowCardinality(Nullable(T)). + /// In this case the variant type is LowCardinality(T), and we cannot just + /// deserialize its values directly into LowCardinality(Nullable(T)) column. + /// We create a separate column with type LowCardinality(T), deserialize + /// values into it and then insert into result column using insertRangeFrom. + if (isColumnLowCardinalityNullable(*column)) + { + ColumnPtr variant_col = mutable_column->cloneEmpty(); + /// LowCardinality(Nullable(T)) -> LowCardinality(T) + assert_cast(*variant_col->assumeMutable()).nestedRemoveNullable(); + nested_serialization->deserializeBinaryBulkWithMultipleStreams(variant_col, limit, settings, state, cache); + mutable_column->insertRangeFrom(*variant_col, 0, variant_col->size()); + } + else + { + nested_serialization->deserializeBinaryBulkWithMultipleStreams(nullable_col ? nullable_col->getNestedColumnPtr() : column, limit, settings, state, cache); + } + if (nullable_col) + null_map->resize_fill(null_map->size() + limit, 0); + removeVariantFromPath(settings.path); + return; + } + + /// If variant size is 0, just fill column with default values. + if (variant_size == 0) + { + mutable_column->insertManyDefaults(limit); + return; + } + + /// In general case we should deserialize variant into a separate column, + /// iterate through discriminators and insert values from variant only when + /// row contains its discriminator and default value otherwise. + mutable_column->reserve(mutable_column->size() + limit); + mutable_column = nullable_col ? nullable_col->getNestedColumnPtr()->assumeMutable() : std::move(mutable_column); + ColumnPtr variant_col = mutable_column->cloneEmpty(); + + /// Special case when our result column is LowCardinality(Nullable(T)). + /// We should remove Nullable from variant column before deserialization. + if (isColumnLowCardinalityNullable(*column)) + assert_cast(*variant_col->assumeMutable()).nestedRemoveNullable(); + + addVariantToPath(settings.path); + nested_serialization->deserializeBinaryBulkWithMultipleStreams(variant_col, variant_size, settings, state, cache); + removeVariantFromPath(settings.path); + + size_t variant_index = 0; + for (auto discr : discriminators_data) + { + if (discr == variant_discriminator) + { + if (null_map) + null_map->push_back(0); + mutable_column->insertFrom(*variant_col, variant_index++); + } + else + { + if (null_map) + null_map->push_back(1); + mutable_column->insertDefault(); + } + } +} + +void SerializationVariantElement::addVariantToPath(DB::ISerialization::SubstreamPath & path) const +{ + path.push_back(Substream::VariantElements); + path.push_back(Substream::VariantElement); + path.back().variant_element_name = variant_element_name; +} + +void SerializationVariantElement::removeVariantFromPath(DB::ISerialization::SubstreamPath & path) const +{ + path.pop_back(); + path.pop_back(); +} + +SerializationVariantElement::VariantSubcolumnCreator::VariantSubcolumnCreator( + const DB::ColumnPtr & local_discriminators_, + const DB::String & variant_element_name_, + const ColumnVariant::Discriminator global_variant_discriminator_, + const ColumnVariant::Discriminator local_variant_discriminator_) + : local_discriminators(local_discriminators_) + , variant_element_name(variant_element_name_) + , global_variant_discriminator(global_variant_discriminator_) + , local_variant_discriminator(local_variant_discriminator_) +{ +} + +DataTypePtr SerializationVariantElement::VariantSubcolumnCreator::create(const DB::DataTypePtr & prev) const +{ + return makeNullableOrLowCardinalityNullableSafe(prev); +} + +SerializationPtr SerializationVariantElement::VariantSubcolumnCreator::create(const DB::SerializationPtr & prev) const +{ + return std::make_shared(prev, variant_element_name, global_variant_discriminator); +} + +ColumnPtr SerializationVariantElement::VariantSubcolumnCreator::create(const DB::ColumnPtr & prev) const +{ + /// Case when original Variant column contained only one non-empty variant and no NULLs. + /// In this case just use this variant. + if (prev->size() == local_discriminators->size()) + return makeNullableOrLowCardinalityNullableSafe(prev); + + /// If this variant is empty, fill result column with default values. + if (prev->empty()) + { + auto res = IColumn::mutate(makeNullableOrLowCardinalityNullableSafe(prev)); + res->insertManyDefaults(local_discriminators->size()); + return res; + } + + /// In general case we should iterate through discriminators and create null-map for our variant. + NullMap null_map; + null_map.reserve(local_discriminators->size()); + const auto & local_discriminators_data = assert_cast(*local_discriminators).getData(); + for (auto local_discr : local_discriminators_data) + null_map.push_back(local_discr != local_variant_discriminator); + + /// Now we can create new column from null-map and variant column using IColumn::expand. + auto res_column = IColumn::mutate(prev); + + /// Special case for LowCardinality. We want the result to be LowCardinality(Nullable), + /// but we don't have a good way to apply null-mask for LowCardinality(), so, we first + /// convert our column to LowCardinality(Nullable()) and then use expand which will + /// fill rows with 0 in mask with default value (that is NULL). + if (prev->lowCardinality()) + res_column = assert_cast(*res_column).cloneNullable(); + + res_column->expand(null_map, /*inverted = */ true); + + if (res_column->canBeInsideNullable()) + { + auto null_map_col = ColumnUInt8::create(); + null_map_col->getData() = std::move(null_map); + return ColumnNullable::create(std::move(res_column), std::move(null_map_col)); + } + + return res_column; +} + +} diff --git a/src/DataTypes/Serializations/SerializationVariantElement.h b/src/DataTypes/Serializations/SerializationVariantElement.h new file mode 100644 index 00000000000..c343c219cf3 --- /dev/null +++ b/src/DataTypes/Serializations/SerializationVariantElement.h @@ -0,0 +1,87 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class SerializationVariant; + +/// Serialization for Variant element when we read it as a subcolumn. +class SerializationVariantElement final : public SerializationWrapper +{ +private: + /// To be able to deserialize Variant element as a subcolumn + /// we need its type name and global discriminator. + String variant_element_name; + ColumnVariant::Discriminator variant_discriminator; + +public: + SerializationVariantElement(const SerializationPtr & nested_, const String & variant_element_name_, ColumnVariant::Discriminator variant_discriminator_) + : SerializationWrapper(nested_) + , variant_element_name(variant_element_name_) + , variant_discriminator(variant_discriminator_) + { + } + + void enumerateStreams( + EnumerateStreamsSettings & settings, + const StreamCallback & callback, + const SubstreamData & data) const override; + + void serializeBinaryBulkStatePrefix( + const IColumn & column, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void serializeBinaryBulkStateSuffix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state) const override; + + void serializeBinaryBulkWithMultipleStreams( + const IColumn & column, + size_t offset, + size_t limit, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void deserializeBinaryBulkWithMultipleStreams( + ColumnPtr & column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const override; + +private: + friend SerializationVariant; + + void addVariantToPath(SubstreamPath & path) const; + void removeVariantFromPath(SubstreamPath & path) const; + + struct VariantSubcolumnCreator : public ISubcolumnCreator + { + const ColumnPtr local_discriminators; + const String variant_element_name; + const ColumnVariant::Discriminator global_variant_discriminator; + const ColumnVariant::Discriminator local_variant_discriminator; + + VariantSubcolumnCreator( + const ColumnPtr & local_discriminators_, + const String & variant_element_name_, + const ColumnVariant::Discriminator global_variant_discriminator_, + const ColumnVariant::Discriminator local_variant_discriminator_); + + DataTypePtr create(const DataTypePtr & prev) const override; + ColumnPtr create(const ColumnPtr & prev) const override; + SerializationPtr create(const SerializationPtr & prev) const override; + }; +}; + +} diff --git a/src/DataTypes/Serializations/SerializationWrapper.cpp b/src/DataTypes/Serializations/SerializationWrapper.cpp index 18e4891ee65..bde52bb8096 100644 --- a/src/DataTypes/Serializations/SerializationWrapper.cpp +++ b/src/DataTypes/Serializations/SerializationWrapper.cpp @@ -96,6 +96,11 @@ void SerializationWrapper::deserializeTextEscaped(IColumn & column, ReadBuffer & nested_serialization->deserializeTextEscaped(column, istr, settings); } +bool SerializationWrapper::tryDeserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + return nested_serialization->tryDeserializeTextEscaped(column, istr, settings); +} + void SerializationWrapper::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { nested_serialization->serializeTextQuoted(column, row_num, ostr, settings); @@ -106,6 +111,11 @@ void SerializationWrapper::deserializeTextQuoted(IColumn & column, ReadBuffer & nested_serialization->deserializeTextQuoted(column, istr, settings); } +bool SerializationWrapper::tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + return nested_serialization->tryDeserializeTextQuoted(column, istr, settings); +} + void SerializationWrapper::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { nested_serialization->serializeTextCSV(column, row_num, ostr, settings); @@ -116,6 +126,11 @@ void SerializationWrapper::deserializeTextCSV(IColumn & column, ReadBuffer & ist nested_serialization->deserializeTextCSV(column, istr, settings); } +bool SerializationWrapper::tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + return nested_serialization->tryDeserializeTextCSV(column, istr, settings); +} + void SerializationWrapper::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { nested_serialization->serializeText(column, row_num, ostr, settings); @@ -126,6 +141,11 @@ void SerializationWrapper::deserializeWholeText(IColumn & column, ReadBuffer & i nested_serialization->deserializeWholeText(column, istr, settings); } +bool SerializationWrapper::tryDeserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + return nested_serialization->tryDeserializeWholeText(column, istr, settings); +} + void SerializationWrapper::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { nested_serialization->serializeTextJSON(column, row_num, ostr, settings); @@ -136,6 +156,11 @@ void SerializationWrapper::deserializeTextJSON(IColumn & column, ReadBuffer & is nested_serialization->deserializeTextJSON(column, istr, settings); } +bool SerializationWrapper::tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + return nested_serialization->tryDeserializeTextJSON(column, istr, settings); +} + void SerializationWrapper::serializeTextJSONPretty(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t indent) const { nested_serialization->serializeTextJSONPretty(column, row_num, ostr, settings, indent); diff --git a/src/DataTypes/Serializations/SerializationWrapper.h b/src/DataTypes/Serializations/SerializationWrapper.h index 31900f93148..6c5e2046062 100644 --- a/src/DataTypes/Serializations/SerializationWrapper.h +++ b/src/DataTypes/Serializations/SerializationWrapper.h @@ -63,18 +63,23 @@ public: void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextJSONPretty(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t indent) const override; void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; diff --git a/src/DataTypes/Serializations/SimpleTextSerialization.h b/src/DataTypes/Serializations/SimpleTextSerialization.h index 0247f30b30a..11f56de73d1 100644 --- a/src/DataTypes/Serializations/SimpleTextSerialization.h +++ b/src/DataTypes/Serializations/SimpleTextSerialization.h @@ -36,29 +36,67 @@ protected: deserializeText(column, istr, settings, true); } + bool tryDeserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override + { + return tryDeserializeText(column, istr, settings, true); + } + void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override { deserializeText(column, istr, settings, false); } + bool tryDeserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override + { + return tryDeserializeText(column, istr, settings, false); + } + void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override { deserializeText(column, istr, settings, false); } + bool tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override + { + return tryDeserializeText(column, istr, settings, false); + } + void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override { deserializeText(column, istr, settings, false); } + bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override + { + return tryDeserializeText(column, istr, settings, false); + } + void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override { deserializeText(column, istr, settings, false); } + bool tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override + { + return tryDeserializeText(column, istr, settings, false); + } + /// whole = true means that buffer contains only one value, so we should read until EOF. /// It's needed to check if there is garbage after parsed field. virtual void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const = 0; + + virtual bool tryDeserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const + { + try + { + deserializeText(column, istr, settings, whole); + return true; + } + catch (...) + { + return false; + } + } }; } diff --git a/src/DataTypes/Utils.cpp b/src/DataTypes/Utils.cpp index e58331a8bcb..2f29d57d454 100644 --- a/src/DataTypes/Utils.cpp +++ b/src/DataTypes/Utils.cpp @@ -223,6 +223,7 @@ bool canBeSafelyCasted(const DataTypePtr & from_type, const DataTypePtr & to_typ case TypeIndex::AggregateFunction: case TypeIndex::Nothing: case TypeIndex::JSONPaths: + case TypeIndex::Variant: return false; } diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 36dd858dcf7..76f6dc25aae 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -907,6 +907,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep query_context->setSetting("allow_experimental_nlp_functions", 1); query_context->setSetting("allow_experimental_hash_functions", 1); query_context->setSetting("allow_experimental_object_type", 1); + query_context->setSetting("allow_experimental_variant_type", 1); query_context->setSetting("allow_experimental_annoy_index", 1); query_context->setSetting("allow_experimental_usearch_index", 1); query_context->setSetting("allow_experimental_bigint_types", 1); diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index 9cc7cb3b89e..a2528f9f948 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -109,31 +109,31 @@ bool deserializeFieldByEscapingRule( { case FormatSettings::EscapingRule::Escaped: if (parse_as_nullable) - read = SerializationNullable::deserializeTextEscapedImpl(column, buf, format_settings, serialization); + read = SerializationNullable::deserializeNullAsDefaultOrNestedTextEscaped(column, buf, format_settings, serialization); else serialization->deserializeTextEscaped(column, buf, format_settings); break; case FormatSettings::EscapingRule::Quoted: if (parse_as_nullable) - read = SerializationNullable::deserializeTextQuotedImpl(column, buf, format_settings, serialization); + read = SerializationNullable::deserializeNullAsDefaultOrNestedTextQuoted(column, buf, format_settings, serialization); else serialization->deserializeTextQuoted(column, buf, format_settings); break; case FormatSettings::EscapingRule::CSV: if (parse_as_nullable) - read = SerializationNullable::deserializeTextCSVImpl(column, buf, format_settings, serialization); + read = SerializationNullable::deserializeNullAsDefaultOrNestedTextCSV(column, buf, format_settings, serialization); else serialization->deserializeTextCSV(column, buf, format_settings); break; case FormatSettings::EscapingRule::JSON: if (parse_as_nullable) - read = SerializationNullable::deserializeTextJSONImpl(column, buf, format_settings, serialization); + read = SerializationNullable::deserializeNullAsDefaultOrNestedTextJSON(column, buf, format_settings, serialization); else serialization->deserializeTextJSON(column, buf, format_settings); break; case FormatSettings::EscapingRule::Raw: if (parse_as_nullable) - read = SerializationNullable::deserializeTextRawImpl(column, buf, format_settings, serialization); + read = SerializationNullable::deserializeNullAsDefaultOrNestedTextRaw(column, buf, format_settings, serialization); else serialization->deserializeTextRaw(column, buf, format_settings); break; diff --git a/src/Formats/JSONUtils.cpp b/src/Formats/JSONUtils.cpp index 7ddfdb6b572..4e7795f61bd 100644 --- a/src/Formats/JSONUtils.cpp +++ b/src/Formats/JSONUtils.cpp @@ -288,14 +288,14 @@ namespace JSONUtils ReadBufferFromString buf(str); if (as_nullable) - return SerializationNullable::deserializeWholeTextImpl(column, buf, format_settings, serialization); + return SerializationNullable::deserializeNullAsDefaultOrNestedWholeText(column, buf, format_settings, serialization); serialization->deserializeWholeText(column, buf, format_settings); return true; } if (as_nullable) - return SerializationNullable::deserializeTextJSONImpl(column, in, format_settings, serialization); + return SerializationNullable::deserializeNullAsDefaultOrNestedTextJSON(column, in, format_settings, serialization); serialization->deserializeTextJSON(column, in, format_settings); return true; diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index e2ba188d015..6890e412f75 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -946,7 +946,7 @@ namespace if constexpr (is_json) ok = tryReadJSONStringInto(field, buf); else - ok = tryReadQuotedStringInto(field, buf); + ok = tryReadQuotedString(field, buf); if (!ok) return nullptr; diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index bef1e7b420a..f9f61ceed0d 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -28,6 +28,7 @@ #include #include #include +#include #include #include #include @@ -40,6 +41,7 @@ #include #include #include +#include #include #include #include @@ -4067,6 +4069,259 @@ arguments, result_type, input_rows_count); \ "Cast to Object can be performed only from flatten named Tuple, Map or String. Got: {}", from_type->getName()); } + WrapperType createVariantToVariantWrapper(const DataTypeVariant & from_variant, const DataTypeVariant & to_variant) const + { + /// We support only extension of variant type, so, only new types can be added. + /// For example: Variant(T1, T2) -> Variant(T1, T2, T3) is supported, but Variant(T1, T2) -> Variant(T1, T3) is not supported. + /// We want to extend Variant type for free without rewriting the data, but we sort data types inside Variant during type creation + /// (we do it because we want Variant(T1, T2) to be the same as Variant(T2, T1)), but after extension the order of variant types + /// (and so their discriminators) can be different. For example: Variant(T1, T3) -> Variant(T1, T2, T3). + /// To avoid full rewrite of discriminators column, ColumnVariant supports it's local order of variant columns (and so local + /// discriminators) and stores mapping global order -> local order. + /// So, to extend Variant with new types for free, we should keep old local order for old variants, append new variants and change + /// mapping global order -> local order according to the new global order. + + /// Create map (new variant type) -> (it's global discriminator in new order). + const auto & new_variants = to_variant.getVariants(); + std::unordered_map new_variant_types_to_new_global_discriminator; + new_variant_types_to_new_global_discriminator.reserve(new_variants.size()); + for (size_t i = 0; i != new_variants.size(); ++i) + new_variant_types_to_new_global_discriminator[new_variants[i]->getName()] = i; + + /// Create set of old variant types. + const auto & old_variants = from_variant.getVariants(); + std::unordered_map old_variant_types_to_old_global_discriminator; + old_variant_types_to_old_global_discriminator.reserve(old_variants.size()); + for (size_t i = 0; i != old_variants.size(); ++i) + old_variant_types_to_old_global_discriminator[old_variants[i]->getName()] = i; + + /// Check that the set of old variants types is a subset of new variant types and collect new global discriminator for each old global discriminator. + std::unordered_map old_global_discriminator_to_new; + old_global_discriminator_to_new.reserve(old_variants.size()); + for (const auto & [old_variant_type, old_discriminator] : old_variant_types_to_old_global_discriminator) + { + auto it = new_variant_types_to_new_global_discriminator.find(old_variant_type); + if (it == new_variant_types_to_new_global_discriminator.end()) + throw Exception( + ErrorCodes::CANNOT_CONVERT_TYPE, + "Cannot convert type {} to {}. Conversion between Variant types is allowed only when new Variant type is an extension " + "of an initial one", from_variant.getName(), to_variant.getName()); + old_global_discriminator_to_new[old_discriminator] = it->second; + } + + /// Collect variant types and their global discriminators that should be added to the old Variant to get the new Variant. + std::vector> variant_types_and_discriminators_to_add; + variant_types_and_discriminators_to_add.reserve(new_variants.size() - old_variants.size()); + for (size_t i = 0; i != new_variants.size(); ++i) + { + if (!old_variant_types_to_old_global_discriminator.contains(new_variants[i]->getName())) + variant_types_and_discriminators_to_add.emplace_back(new_variants[i], i); + } + + return [old_global_discriminator_to_new, variant_types_and_discriminators_to_add] + (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable *, size_t) -> ColumnPtr + { + const auto & column_variant = assert_cast(*arguments.front().column.get()); + size_t num_old_variants = column_variant.getNumVariants(); + Columns new_variant_columns; + new_variant_columns.reserve(num_old_variants + variant_types_and_discriminators_to_add.size()); + std::vector new_local_to_global_discriminators; + new_local_to_global_discriminators.reserve(num_old_variants + variant_types_and_discriminators_to_add.size()); + for (size_t i = 0; i != num_old_variants; ++i) + { + new_variant_columns.push_back(column_variant.getVariantPtrByLocalDiscriminator(i)); + new_local_to_global_discriminators.push_back(old_global_discriminator_to_new.at(column_variant.globalDiscriminatorByLocal(i))); + } + + for (const auto & [new_variant_type, new_global_discriminator] : variant_types_and_discriminators_to_add) + { + new_variant_columns.push_back(new_variant_type->createColumn()); + new_local_to_global_discriminators.push_back(new_global_discriminator); + } + + return ColumnVariant::create(column_variant.getLocalDiscriminatorsPtr(), column_variant.getOffsetsPtr(), new_variant_columns, new_local_to_global_discriminators); + }; + } + + WrapperType createVariantToColumnWrapper(const DataTypeVariant & from_variant, const DataTypePtr & to_type) const + { + const auto & variant_types = from_variant.getVariants(); + std::vector variant_wrappers; + variant_wrappers.reserve(variant_types.size()); + + /// Create conversion wrapper for each variant. + for (const auto & variant_type : variant_types) + variant_wrappers.push_back(prepareUnpackDictionaries(variant_type, to_type)); + + return [variant_wrappers, variant_types, to_type] + (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) -> ColumnPtr + { + const auto & column_variant = assert_cast(*arguments.front().column.get()); + + /// First, cast each variant to the result type. + std::vector casted_variant_columns; + casted_variant_columns.reserve(variant_types.size()); + for (size_t i = 0; i != variant_types.size(); ++i) + { + auto variant_col = column_variant.getVariantPtrByLocalDiscriminator(i); + ColumnsWithTypeAndName variant = {{variant_col, variant_types[i], "" }}; + const auto & variant_wrapper = variant_wrappers[column_variant.globalDiscriminatorByLocal(i)]; + casted_variant_columns.push_back(variant_wrapper(variant, result_type, nullptr, variant_col->size())); + } + + /// Second, construct resulting column from casted variant columns according to discriminators. + const auto & local_discriminators = column_variant.getLocalDiscriminators(); + auto res = result_type->createColumn(); + res->reserve(input_rows_count); + for (size_t i = 0; i != input_rows_count; ++i) + { + auto local_discr = local_discriminators[i]; + if (local_discr == ColumnVariant::NULL_DISCRIMINATOR) + res->insertDefault(); + else + res->insertFrom(*casted_variant_columns[local_discr], column_variant.offsetAt(i)); + } + + return res; + }; + } + + static ColumnPtr createVariantFromDescriptorsAndOneNonEmptyVariant(const DataTypes & variant_types, const ColumnPtr & discriminators, const ColumnPtr & variant, ColumnVariant::Discriminator variant_discr) + { + Columns variants; + variants.reserve(variant_types.size()); + for (size_t i = 0; i != variant_types.size(); ++i) + { + if (i == variant_discr) + variants.emplace_back(variant); + else + variants.push_back(variant_types[i]->createColumn()); + } + + return ColumnVariant::create(discriminators, variants); + } + + WrapperType createColumnToVariantWrapper(const DataTypePtr & from_type, const DataTypeVariant & to_variant) const + { + /// We allow converting NULL to Variant(...) as Variant can store NULLs. + if (from_type->onlyNull()) + { + return [](ColumnsWithTypeAndName &, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) -> ColumnPtr + { + auto result_column = result_type->createColumn(); + result_column->insertManyDefaults(input_rows_count); + return result_column; + }; + } + + auto variant_discr_opt = to_variant.tryGetVariantDiscriminator(removeNullableOrLowCardinalityNullable(from_type)); + if (!variant_discr_opt) + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Cannot convert type {} to {}. Conversion to Variant allowed only for types from this Variant", from_type->getName(), to_variant.getName()); + + return [variant_discr = *variant_discr_opt] + (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t) -> ColumnPtr + { + const auto & result_variant_type = assert_cast(*result_type); + const auto & variant_types = result_variant_type.getVariants(); + if (const ColumnNullable * col_nullable = typeid_cast(arguments.front().column.get())) + { + const auto & column = col_nullable->getNestedColumnPtr(); + const auto & null_map = col_nullable->getNullMapData(); + IColumn::Filter filter; + filter.reserve(column->size()); + auto discriminators = ColumnVariant::ColumnDiscriminators::create(); + auto & discriminators_data = discriminators->getData(); + discriminators_data.reserve(column->size()); + size_t variant_size_hint = 0; + for (size_t i = 0; i != column->size(); ++i) + { + if (null_map[i]) + { + discriminators_data.push_back(ColumnVariant::NULL_DISCRIMINATOR); + filter.push_back(0); + } + else + { + discriminators_data.push_back(variant_discr); + filter.push_back(1); + ++variant_size_hint; + } + } + + ColumnPtr variant_column; + /// If there were no NULLs, just use the column. + if (variant_size_hint == column->size()) + variant_column = column; + /// Otherwise we should use filtered column. + else + variant_column = column->filter(filter, variant_size_hint); + return createVariantFromDescriptorsAndOneNonEmptyVariant(variant_types, std::move(discriminators), variant_column, variant_discr); + } + else if (isColumnLowCardinalityNullable(*arguments.front().column)) + { + const auto & column = arguments.front().column; + + /// Variant column cannot have LowCardinality(Nullable(...)) variant, as Variant column stores NULLs itself. + /// We should create a null-map, insert NULL_DISCRIMINATOR on NULL values and filter initial column. + const auto & col_lc = assert_cast(*column); + const auto & indexes = col_lc.getIndexes(); + auto null_index = col_lc.getDictionary().getNullValueIndex(); + IColumn::Filter filter; + filter.reserve(col_lc.size()); + auto discriminators = ColumnVariant::ColumnDiscriminators::create(); + auto & discriminators_data = discriminators->getData(); + discriminators_data.reserve(col_lc.size()); + size_t variant_size_hint = 0; + for (size_t i = 0; i != col_lc.size(); ++i) + { + if (indexes.getUInt(i) == null_index) + { + discriminators_data.push_back(ColumnVariant::NULL_DISCRIMINATOR); + filter.push_back(0); + } + else + { + discriminators_data.push_back(variant_discr); + filter.push_back(1); + ++variant_size_hint; + } + } + + MutableColumnPtr variant_column; + /// If there were no NULLs, we can just clone the column. + if (variant_size_hint == col_lc.size()) + variant_column = IColumn::mutate(column); + /// Otherwise we should filter column. + else + variant_column = column->filter(filter, variant_size_hint)->assumeMutable(); + + assert_cast(*variant_column).nestedRemoveNullable(); + return createVariantFromDescriptorsAndOneNonEmptyVariant(variant_types, std::move(discriminators), std::move(variant_column), variant_discr); + } + else + { + const auto & column = arguments.front().column; + auto discriminators = ColumnVariant::ColumnDiscriminators::create(); + discriminators->getData().resize_fill(column->size(), variant_discr); + return createVariantFromDescriptorsAndOneNonEmptyVariant(variant_types, std::move(discriminators), column, variant_discr); + } + }; + } + + /// Wrapper for conversion to/from Variant type + WrapperType createVariantWrapper(const DataTypePtr & from_type, const DataTypePtr & to_type) const + { + if (const auto * from_variant = checkAndGetDataType(from_type.get())) + { + if (const auto * to_variant = checkAndGetDataType(to_type.get())) + return createVariantToVariantWrapper(*from_variant, *to_variant); + + return createVariantToColumnWrapper(*from_variant, to_type); + } + + return createColumnToVariantWrapper(from_type, assert_cast(*to_type)); + } + template WrapperType createEnumWrapper(const DataTypePtr & from_type, const DataTypeEnum * to_type) const { @@ -4246,6 +4501,11 @@ arguments, result_type, input_rows_count); \ WrapperType prepareUnpackDictionaries(const DataTypePtr & from_type, const DataTypePtr & to_type) const { + /// Conversion from/to Variant data type is processed in a special way. + /// We don't need to remove LowCardinality/Nullable. + if (isVariant(to_type) || isVariant(from_type)) + return createVariantWrapper(from_type, to_type); + const auto * from_low_cardinality = typeid_cast(from_type.get()); const auto * to_low_cardinality = typeid_cast(to_type.get()); const auto & from_nested = from_low_cardinality ? from_low_cardinality->getDictionaryType() : from_type; @@ -4253,7 +4513,7 @@ arguments, result_type, input_rows_count); \ if (from_type->onlyNull()) { - if (!to_nested->isNullable()) + if (!to_nested->isNullable() && !isVariant(to_type)) { if (cast_type == CastType::accurateOrNull) { diff --git a/src/Functions/if.cpp b/src/Functions/if.cpp index eba1733c683..b15bc5938be 100644 --- a/src/Functions/if.cpp +++ b/src/Functions/if.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -14,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -22,6 +24,8 @@ #include #include #include +#include + #include @@ -215,9 +219,16 @@ class FunctionIf : public FunctionIfBase { public: static constexpr auto name = "if"; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr context) + { + return std::make_shared(context->getSettingsRef().allow_experimental_variant_type && context->getSettingsRef().use_variant_when_no_common_type_in_if); + } + + FunctionIf(bool use_variant_when_no_common_type_ = false) : FunctionIfBase(), use_variant_when_no_common_type(use_variant_when_no_common_type_) {} private: + bool use_variant_when_no_common_type = false; + template static UInt32 decimalScale(const ColumnsWithTypeAndName & arguments [[maybe_unused]]) { @@ -626,13 +637,23 @@ private: } static ColumnPtr executeGeneric( - const ColumnUInt8 * cond_col, const ColumnsWithTypeAndName & arguments, size_t input_rows_count) + const ColumnUInt8 * cond_col, const ColumnsWithTypeAndName & arguments, size_t input_rows_count, bool use_variant_when_no_common_type) { /// Convert both columns to the common type (if needed). const ColumnWithTypeAndName & arg1 = arguments[1]; const ColumnWithTypeAndName & arg2 = arguments[2]; - DataTypePtr common_type = getLeastSupertype(DataTypes{arg1.type, arg2.type}); + DataTypePtr common_type; + if (use_variant_when_no_common_type) + { + common_type = tryGetLeastSupertype(DataTypes{arg1.type, arg2.type}); + if (!common_type) + common_type = std::make_shared(DataTypes{removeNullableOrLowCardinalityNullable(arg1.type), removeNullableOrLowCardinalityNullable(arg2.type)}); + } + else + { + common_type = getLeastSupertype(DataTypes{arg1.type, arg2.type}); + } ColumnPtr col_then = castColumn(arg1, common_type); ColumnPtr col_else = castColumn(arg2, common_type); @@ -807,6 +828,10 @@ private: ColumnPtr executeForNullableThenElse(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const { + /// If result type is Variant, we don't need to remove Nullable. + if (isVariant(result_type)) + return nullptr; + const ColumnWithTypeAndName & arg_cond = arguments[0]; const ColumnWithTypeAndName & arg_then = arguments[1]; const ColumnWithTypeAndName & arg_else = arguments[2]; @@ -912,6 +937,11 @@ private: assert_cast(*result_column).applyNullMap(assert_cast(*arg_cond.column)); return result_column; } + else if (auto * variant_column = typeid_cast(result_column.get())) + { + variant_column->applyNullMap(assert_cast(*arg_cond.column).getData()); + return result_column; + } else return ColumnNullable::create(materializeColumnIfConst(result_column), arg_cond.column); } @@ -950,6 +980,11 @@ private: assert_cast(*result_column).applyNegatedNullMap(assert_cast(*arg_cond.column)); return result_column; } + else if (auto * variant_column = typeid_cast(result_column.get())) + { + variant_column->applyNegatedNullMap(assert_cast(*arg_cond.column).getData()); + return result_column; + } else { size_t size = input_rows_count; @@ -1039,6 +1074,13 @@ public: throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument (condition) of function if. " "Must be UInt8.", arguments[0]->getName()); + if (use_variant_when_no_common_type) + { + if (auto res = tryGetLeastSupertype(DataTypes{arguments[1], arguments[2]})) + return res; + return std::make_shared(DataTypes{removeNullableOrLowCardinalityNullable(arguments[1]), removeNullableOrLowCardinalityNullable(arguments[2])}); + } + return getLeastSupertype(DataTypes{arguments[1], arguments[2]}); } @@ -1122,7 +1164,7 @@ public: || (res = executeGenericArray(cond_col, arguments, result_type)) || (res = executeTuple(arguments, result_type, input_rows_count)))) { - return executeGeneric(cond_col, arguments, input_rows_count); + return executeGeneric(cond_col, arguments, input_rows_count, use_variant_when_no_common_type); } return res; diff --git a/src/Functions/isNotNull.cpp b/src/Functions/isNotNull.cpp index cbdc08c2fab..360c2fc7f9f 100644 --- a/src/Functions/isNotNull.cpp +++ b/src/Functions/isNotNull.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include @@ -45,6 +46,18 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { const ColumnWithTypeAndName & elem = arguments[0]; + + if (isVariant(elem.type)) + { + const auto & discriminators = checkAndGetColumn(*elem.column)->getLocalDiscriminators(); + auto res = DataTypeUInt8().createColumn(); + auto & data = typeid_cast(*res).getData(); + data.reserve(discriminators.size()); + for (auto discr : discriminators) + data.push_back(discr != ColumnVariant::NULL_DISCRIMINATOR); + return res; + } + if (elem.type->isLowCardinalityNullable()) { const auto * low_cardinality_column = checkAndGetColumn(*elem.column); diff --git a/src/Functions/isNull.cpp b/src/Functions/isNull.cpp index cdce037088d..4bf4e44f866 100644 --- a/src/Functions/isNull.cpp +++ b/src/Functions/isNull.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -44,6 +45,18 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override { const ColumnWithTypeAndName & elem = arguments[0]; + + if (isVariant(elem.type)) + { + const auto & discriminators = checkAndGetColumn(*elem.column)->getLocalDiscriminators(); + auto res = DataTypeUInt8().createColumn(); + auto & data = typeid_cast(*res).getData(); + data.reserve(discriminators.size()); + for (auto discr : discriminators) + data.push_back(discr == ColumnVariant::NULL_DISCRIMINATOR); + return res; + } + if (elem.type->isLowCardinalityNullable()) { const auto * low_cardinality_column = checkAndGetColumn(*elem.column); diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index cdb9ca061c3..7a2e9444b2c 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include @@ -117,6 +118,15 @@ public: types_of_branches.emplace_back(arg); }); + if (context->getSettingsRef().allow_experimental_variant_type && context->getSettingsRef().use_variant_when_no_common_type_in_if) + { + if (auto res = tryGetLeastSupertype(types_of_branches)) + return res; + for (auto & type : types_of_branches) + type = removeNullableOrLowCardinalityNullable(type); + return std::make_shared(types_of_branches); + } + return getLeastSupertype(types_of_branches); } diff --git a/src/Functions/variantElement.cpp b/src/Functions/variantElement.cpp new file mode 100644 index 00000000000..7c63e1266e6 --- /dev/null +++ b/src/Functions/variantElement.cpp @@ -0,0 +1,238 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +namespace +{ + +/** Extract element of Variant by variant type name. + * Also the function looks through Arrays: you can get Array of Variant elements from Array of Variants. + */ +class FunctionVariantElement : public IFunction +{ +public: + static constexpr auto name = "variantElement"; + + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + String getName() const override { return name; } + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + const size_t number_of_arguments = arguments.size(); + + if (number_of_arguments < 2 || number_of_arguments > 3) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be 2 or 3", + getName(), number_of_arguments); + + size_t count_arrays = 0; + const IDataType * input_type = arguments[0].type.get(); + while (const DataTypeArray * array = checkAndGetDataType(input_type)) + { + input_type = array->getNestedType().get(); + ++count_arrays; + } + + const DataTypeVariant * variant_type = checkAndGetDataType(input_type); + if (!variant_type) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "First argument for function {} must be Variant or Array of Variant. Actual {}", + getName(), + arguments[0].type->getName()); + + std::optional variant_global_discr = getVariantGlobalDiscriminator(arguments[1].column, *variant_type, number_of_arguments); + if (variant_global_discr.has_value()) + { + DataTypePtr return_type = makeNullableOrLowCardinalityNullableSafe(variant_type->getVariant(variant_global_discr.value())); + + for (; count_arrays; --count_arrays) + return_type = std::make_shared(return_type); + + return return_type; + } + else + return arguments[2].type; + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + const auto & input_arg = arguments[0]; + const IDataType * input_type = input_arg.type.get(); + const IColumn * input_col = input_arg.column.get(); + + bool input_arg_is_const = false; + if (typeid_cast(input_col)) + { + input_col = assert_cast(input_col)->getDataColumnPtr().get(); + input_arg_is_const = true; + } + + Columns array_offsets; + while (const DataTypeArray * array_type = checkAndGetDataType(input_type)) + { + const ColumnArray * array_col = assert_cast(input_col); + + input_type = array_type->getNestedType().get(); + input_col = &array_col->getData(); + array_offsets.push_back(array_col->getOffsetsPtr()); + } + + const DataTypeVariant * input_type_as_variant = checkAndGetDataType(input_type); + const ColumnVariant * input_col_as_variant = checkAndGetColumn(input_col); + if (!input_type_as_variant || !input_col_as_variant) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "First argument for function {} must be Variant or array of Variants. Actual {}", getName(), input_arg.type->getName()); + + std::optional variant_global_discr = getVariantGlobalDiscriminator(arguments[1].column, *input_type_as_variant, arguments.size()); + + if (!variant_global_discr.has_value()) + return arguments[2].column; + + const auto & variant_type = input_type_as_variant->getVariant(*variant_global_discr); + const auto & variant_column = input_col_as_variant->getVariantPtrByGlobalDiscriminator(*variant_global_discr); + + /// If Variant has only NULLs or our variant doesn't have any real values, + /// just create column with default values and create null mask with 1. + if (input_col_as_variant->hasOnlyNulls() || variant_column->empty()) + { + auto res = variant_type->createColumn(); + + if (variant_type->lowCardinality()) + assert_cast(*res).nestedToNullable(); + + res->insertManyDefaults(input_col_as_variant->size()); + if (!variant_type->canBeInsideNullable()) + return wrapInArraysAndConstIfNeeded(std::move(res), array_offsets, input_arg_is_const, input_rows_count); + + auto null_map = ColumnUInt8::create(); + auto & null_map_data = null_map->getData(); + null_map_data.resize_fill(input_col_as_variant->size(), 1); + return wrapInArraysAndConstIfNeeded(ColumnNullable::create(std::move(res), std::move(null_map)), array_offsets, input_arg_is_const, input_rows_count); + } + + /// If we extract single non-empty column and have no NULLs, then just return this variant. + if (auto non_empty_local_discr = input_col_as_variant->getLocalDiscriminatorOfOneNoneEmptyVariantNoNulls()) + { + /// If we were trying to extract some other variant, + /// it would be empty and we would already processed this case above. + chassert(input_col_as_variant->globalDiscriminatorByLocal(*non_empty_local_discr) == variant_global_discr); + return wrapInArraysAndConstIfNeeded(makeNullableOrLowCardinalityNullableSafe(variant_column), array_offsets, input_arg_is_const, input_rows_count); + } + + /// In general case we should calculate null-mask for variant + /// according to the discriminators column and expand + /// variant column by this mask to get a full column (with default values on NULLs) + const auto & local_discriminators = input_col_as_variant->getLocalDiscriminators(); + auto null_map = ColumnUInt8::create(); + auto & null_map_data = null_map->getData(); + null_map_data.reserve(local_discriminators.size()); + auto variant_local_discr = input_col_as_variant->localDiscriminatorByGlobal(*variant_global_discr); + for (auto local_discr : local_discriminators) + null_map_data.push_back(local_discr != variant_local_discr); + + auto expanded_variant_column = IColumn::mutate(variant_column); + if (variant_type->lowCardinality()) + expanded_variant_column = assert_cast(*expanded_variant_column).cloneNullable(); + expanded_variant_column->expand(null_map_data, /*inverted = */ true); + if (variant_type->canBeInsideNullable()) + return wrapInArraysAndConstIfNeeded(ColumnNullable::create(std::move(expanded_variant_column), std::move(null_map)), array_offsets, input_arg_is_const, input_rows_count); + return wrapInArraysAndConstIfNeeded(std::move(expanded_variant_column), array_offsets, input_arg_is_const, input_rows_count); + } +private: + std::optional getVariantGlobalDiscriminator(const ColumnPtr & index_column, const DataTypeVariant & variant_type, size_t argument_size) const + { + const auto * name_col = checkAndGetColumnConst(index_column.get()); + if (!name_col) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Second argument to {} with Variant argument must be a constant String", + getName()); + + String variant_element_name = name_col->getValue(); + auto variant_element_type = DataTypeFactory::instance().tryGet(variant_element_name); + if (variant_element_type) + { + const auto & variants = variant_type.getVariants(); + for (size_t i = 0; i != variants.size(); ++i) + { + if (variants[i]->getName() == variant_element_type->getName()) + return i; + } + } + + if (argument_size == 2) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "{} doesn't contain variant with type {}", variant_type.getName(), variant_element_name); + return std::nullopt; + } + + ColumnPtr wrapInArraysAndConstIfNeeded(ColumnPtr res, const Columns & array_offsets, bool input_arg_is_const, size_t input_rows_count) const + { + for (auto it = array_offsets.rbegin(); it != array_offsets.rend(); ++it) + res = ColumnArray::create(res, *it); + + if (input_arg_is_const) + res = ColumnConst::create(res, input_rows_count); + + return res; + } +}; + +} + +REGISTER_FUNCTION(VariantElement) +{ + factory.registerFunction(FunctionDocumentation{ + .description = R"( +Extracts a column with specified type from a `Variant` column. +)", + .syntax{"tupleElement(variant, type_name, [, default_value])"}, + .arguments{{ + {"variant", "Variant column"}, + {"type_name", "The name of the variant type to extract"}, + {"default_value", "The default value that will be used if variant doesn't have variant with specified type. Can be any type. Optional"}}}, + .examples{{{ + "Example", + R"( +CREATE TABLE test (v Variant(UInt64, String, Array(UInt64))) ENGINE = Memory; +INSERT INTO test VALUES (NULL), (42), ('Hello, World!'), ([1, 2, 3]); +SELECT v, variantElement(v, 'String'), variantElement(v, 'UInt64'), variantElement(v, 'Array(UInt64)') FROM test;)", + R"( +┌─v─────────────┬─variantElement(v, 'String')─┬─variantElement(v, 'UInt64')─┬─variantElement(v, 'Array(UInt64)')─┐ +│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ [] │ +│ 42 │ ᴺᵁᴸᴸ │ 42 │ [] │ +│ Hello, World! │ Hello, World! │ ᴺᵁᴸᴸ │ [] │ +│ [1,2,3] │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ [1,2,3] │ +└───────────────┴─────────────────────────────┴─────────────────────────────┴────────────────────────────────────┘ +)"}}}, + .categories{"Variant"}, + }); +} + +} diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index ff5743a63af..2534f248d83 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -619,13 +619,16 @@ void readQuotedStringInto(Vector & s, ReadBuffer & buf) readAnyQuotedStringInto<'\'', enable_sql_style_quoting>(s, buf); } -template +template bool tryReadQuotedStringInto(Vector & s, ReadBuffer & buf) { - return readAnyQuotedStringInto<'\'', false, Vector, bool>(s, buf); + return readAnyQuotedStringInto<'\'', enable_sql_style_quoting, Vector, bool>(s, buf); } -template bool tryReadQuotedStringInto(String & s, ReadBuffer & buf); +template bool tryReadQuotedStringInto(String & s, ReadBuffer & buf); +template bool tryReadQuotedStringInto(String & s, ReadBuffer & buf); +template bool tryReadQuotedStringInto>(PaddedPODArray & s, ReadBuffer & buf); +template bool tryReadQuotedStringInto>(PaddedPODArray & s, ReadBuffer & buf); template void readDoubleQuotedStringInto(Vector & s, ReadBuffer & buf) @@ -633,6 +636,16 @@ void readDoubleQuotedStringInto(Vector & s, ReadBuffer & buf) readAnyQuotedStringInto<'"', enable_sql_style_quoting>(s, buf); } +template +bool tryReadDoubleQuotedStringInto(Vector & s, ReadBuffer & buf) +{ + return readAnyQuotedStringInto<'"', enable_sql_style_quoting, Vector, bool>(s, buf); +} + +template bool tryReadDoubleQuotedStringInto(String & s, ReadBuffer & buf); +template bool tryReadDoubleQuotedStringInto(String & s, ReadBuffer & buf); + + template void readBackQuotedStringInto(Vector & s, ReadBuffer & buf) { @@ -652,6 +665,18 @@ void readQuotedStringWithSQLStyle(String & s, ReadBuffer & buf) readQuotedStringInto(s, buf); } +bool tryReadQuotedString(String & s, ReadBuffer & buf) +{ + s.clear(); + return tryReadQuotedStringInto(s, buf); +} + +bool tryReadQuotedStringWithSQLStyle(String & s, ReadBuffer & buf) +{ + s.clear(); + return tryReadQuotedStringInto(s, buf); +} + template void readQuotedStringInto(PaddedPODArray & s, ReadBuffer & buf); template void readQuotedStringInto(String & s, ReadBuffer & buf); @@ -672,6 +697,18 @@ void readDoubleQuotedStringWithSQLStyle(String & s, ReadBuffer & buf) readDoubleQuotedStringInto(s, buf); } +bool tryReadDoubleQuotedString(String & s, ReadBuffer & buf) +{ + s.clear(); + return tryReadDoubleQuotedStringInto(s, buf); +} + +bool tryReadDoubleQuotedStringWithSQLStyle(String & s, ReadBuffer & buf) +{ + s.clear(); + return tryReadDoubleQuotedStringInto(s, buf); +} + void readBackQuotedString(String & s, ReadBuffer & buf) { s.clear(); @@ -691,7 +728,7 @@ concept WithResize = requires (T value) { value.size() } -> std::integral<>; }; -template +template void readCSVStringInto(Vector & s, ReadBuffer & buf, const FormatSettings::CSV & settings) { /// Empty string @@ -754,12 +791,20 @@ void readCSVStringInto(Vector & s, ReadBuffer & buf, const FormatSettings::CSV & { PeekableReadBuffer * peekable_buf = dynamic_cast(&buf); if (!peekable_buf) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Reading CSV string with custom delimiter is allowed only when using PeekableReadBuffer"); + { + if constexpr (allow_throw) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Reading CSV string with custom delimiter is allowed only when using PeekableReadBuffer"); + return; + } while (true) { if (peekable_buf->eof()) - throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected EOF while reading CSV string, expected custom delimiter \"{}\"", custom_delimiter); + { + if constexpr (allow_throw) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected EOF while reading CSV string, expected custom delimiter \"{}\"", custom_delimiter); + return; + } char * next_pos = reinterpret_cast(memchr(peekable_buf->position(), custom_delimiter[0], peekable_buf->available())); if (!next_pos) @@ -948,6 +993,9 @@ String readCSVFieldWithTwoPossibleDelimiters(PeekableReadBuffer & buf, const For template void readCSVStringInto>(PaddedPODArray & s, ReadBuffer & buf, const FormatSettings::CSV & settings); template void readCSVStringInto(NullOutput & s, ReadBuffer & buf, const FormatSettings::CSV & settings); +template void readCSVStringInto(String & s, ReadBuffer & buf, const FormatSettings::CSV & settings); +template void readCSVStringInto(String & s, ReadBuffer & buf, const FormatSettings::CSV & settings); +template void readCSVStringInto, false, false>(PaddedPODArray & s, ReadBuffer & buf, const FormatSettings::CSV & settings); template @@ -1069,15 +1117,18 @@ ReturnType readJSONObjectPossiblyInvalid(Vector & s, ReadBuffer & buf) } template void readJSONObjectPossiblyInvalid(String & s, ReadBuffer & buf); +template bool readJSONObjectPossiblyInvalid(String & s, ReadBuffer & buf); template void readJSONObjectPossiblyInvalid>(PaddedPODArray & s, ReadBuffer & buf); +template bool readJSONObjectPossiblyInvalid, bool>(PaddedPODArray & s, ReadBuffer & buf); -template -void readJSONArrayInto(Vector & s, ReadBuffer & buf) +template +ReturnType readJSONArrayInto(Vector & s, ReadBuffer & buf) { - readJSONObjectOrArrayPossiblyInvalid(s, buf); + return readJSONObjectOrArrayPossiblyInvalid(s, buf); } -template void readJSONArrayInto>(PaddedPODArray & s, ReadBuffer & buf); +template void readJSONArrayInto, void>(PaddedPODArray & s, ReadBuffer & buf); +template bool readJSONArrayInto, bool>(PaddedPODArray & s, ReadBuffer & buf); template ReturnType readDateTextFallback(LocalDate & date, ReadBuffer & buf) @@ -1217,6 +1268,13 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D return false; } + if constexpr (!throw_exception) + { + if (!isNumericASCII(s[0]) || !isNumericASCII(s[1]) || !isNumericASCII(s[2]) || !isNumericASCII(s[3]) + || !isNumericASCII(s[5]) || !isNumericASCII(s[6]) || !isNumericASCII(s[8]) || !isNumericASCII(s[9])) + return false; + } + UInt16 year = (s[0] - '0') * 1000 + (s[1] - '0') * 100 + (s[2] - '0') * 10 + (s[3] - '0'); UInt8 month = (s[5] - '0') * 10 + (s[6] - '0'); UInt8 day = (s[8] - '0') * 10 + (s[9] - '0'); @@ -1240,6 +1298,13 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D return false; } + if constexpr (!throw_exception) + { + if (!isNumericASCII(s[0]) || !isNumericASCII(s[1]) || !isNumericASCII(s[3]) || !isNumericASCII(s[4]) + || !isNumericASCII(s[6]) || !isNumericASCII(s[7])) + return false; + } + hour = (s[0] - '0') * 10 + (s[1] - '0'); minute = (s[3] - '0') * 10 + (s[4] - '0'); second = (s[6] - '0') * 10 + (s[7] - '0'); @@ -1259,7 +1324,14 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D { /// Not very efficient. for (const char * digit_pos = s; digit_pos < s_pos; ++digit_pos) + { + if constexpr (!throw_exception) + { + if (!isNumericASCII(*digit_pos)) + return false; + } datetime = datetime * 10 + *digit_pos - '0'; + } } datetime *= negative_multiplier; @@ -1282,14 +1354,24 @@ template bool readDateTimeTextFallback(time_t &, ReadBuffer &, cons template bool readDateTimeTextFallback(time_t &, ReadBuffer &, const DateLUTImpl &); -void skipJSONField(ReadBuffer & buf, StringRef name_of_field) +template +ReturnType skipJSONFieldImpl(ReadBuffer & buf, StringRef name_of_field) { + static constexpr bool throw_exception = std::is_same_v; + if (buf.eof()) - throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected EOF for key '{}'", name_of_field.toString()); + { + if constexpr (throw_exception) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected EOF for key '{}'", name_of_field.toString()); + return ReturnType(false); + } else if (*buf.position() == '"') /// skip double-quoted string { NullOutput sink; - readJSONStringInto(sink, buf); + if constexpr (throw_exception) + readJSONStringInto(sink, buf); + else if (!tryReadJSONStringInto(sink, buf)) + return ReturnType(false); } else if (isNumericASCII(*buf.position()) || *buf.position() == '-' || *buf.position() == '+' || *buf.position() == '.') /// skip number { @@ -1298,19 +1380,32 @@ void skipJSONField(ReadBuffer & buf, StringRef name_of_field) double v; if (!tryReadFloatText(v, buf)) - throw Exception(ErrorCodes::INCORRECT_DATA, "Expected a number field for key '{}'", name_of_field.toString()); + { + if constexpr (throw_exception) + throw Exception(ErrorCodes::INCORRECT_DATA, "Expected a number field for key '{}'", name_of_field.toString()); + return ReturnType(false); + } } else if (*buf.position() == 'n') /// skip null { - assertString("null", buf); + if constexpr (throw_exception) + assertString("null", buf); + else if (!checkString("null", buf)) + return ReturnType(false); } else if (*buf.position() == 't') /// skip true { - assertString("true", buf); + if constexpr (throw_exception) + assertString("true", buf); + else if (!checkString("true", buf)) + return ReturnType(false); } else if (*buf.position() == 'f') /// skip false { - assertString("false", buf); + if constexpr (throw_exception) + assertString("false", buf); + else if (!checkString("false", buf)) + return ReturnType(false); } else if (*buf.position() == '[') { @@ -1320,12 +1415,16 @@ void skipJSONField(ReadBuffer & buf, StringRef name_of_field) if (!buf.eof() && *buf.position() == ']') /// skip empty array { ++buf.position(); - return; + return ReturnType(true); } while (true) { - skipJSONField(buf, name_of_field); + if constexpr (throw_exception) + skipJSONFieldImpl(buf, name_of_field); + else if (!skipJSONFieldImpl(buf, name_of_field)) + return ReturnType(false); + skipWhitespaceIfAny(buf); if (!buf.eof() && *buf.position() == ',') @@ -1339,7 +1438,11 @@ void skipJSONField(ReadBuffer & buf, StringRef name_of_field) break; } else - throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected symbol for key '{}'", name_of_field.toString()); + { + if constexpr (throw_exception) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected symbol for key '{}'", name_of_field.toString()); + return ReturnType(false); + } } } else if (*buf.position() == '{') /// skip whole object @@ -1353,19 +1456,34 @@ void skipJSONField(ReadBuffer & buf, StringRef name_of_field) if (*buf.position() == '"') { NullOutput sink; - readJSONStringInto(sink, buf); + if constexpr (throw_exception) + readJSONStringInto(sink, buf); + else if (!tryReadJSONStringInto(sink, buf)) + return ReturnType(false); } else - throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected symbol for key '{}'", name_of_field.toString()); + { + if constexpr (throw_exception) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected symbol for key '{}'", name_of_field.toString()); + return ReturnType(false); + } // ':' skipWhitespaceIfAny(buf); if (buf.eof() || !(*buf.position() == ':')) - throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected symbol for key '{}'", name_of_field.toString()); + { + if constexpr (throw_exception) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected symbol for key '{}'", name_of_field.toString()); + return ReturnType(false); + } ++buf.position(); skipWhitespaceIfAny(buf); - skipJSONField(buf, name_of_field); + if constexpr (throw_exception) + skipJSONFieldImpl(buf, name_of_field); + else if (!skipJSONFieldImpl(buf, name_of_field)) + return ReturnType(false); + skipWhitespaceIfAny(buf); // optional ',' @@ -1377,14 +1495,32 @@ void skipJSONField(ReadBuffer & buf, StringRef name_of_field) } if (buf.eof()) - throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected EOF for key '{}'", name_of_field.toString()); + { + if constexpr (throw_exception) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected EOF for key '{}'", name_of_field.toString()); + return ReturnType(false); + } ++buf.position(); } else { - throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected symbol '{}' for key '{}'", - std::string(*buf.position(), 1), name_of_field.toString()); + if constexpr (throw_exception) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected symbol '{}' for key '{}'", + std::string(*buf.position(), 1), name_of_field.toString()); + return ReturnType(false); } + + return ReturnType(true); +} + +void skipJSONField(ReadBuffer & buf, StringRef name_of_field) +{ + skipJSONFieldImpl(buf, name_of_field); +} + +bool trySkipJSONField(ReadBuffer & buf, StringRef name_of_field) +{ + return skipJSONFieldImpl(buf, name_of_field); } @@ -1597,23 +1733,31 @@ void skipToNextRowOrEof(PeekableReadBuffer & buf, const String & row_after_delim } // Use PeekableReadBuffer to copy field to string after parsing. -template -static void readParsedValueInto(Vector & s, ReadBuffer & buf, ParseFunc parse_func) +template +static ReturnType readParsedValueInto(Vector & s, ReadBuffer & buf, ParseFunc parse_func) { PeekableReadBuffer peekable_buf(buf); peekable_buf.setCheckpoint(); - parse_func(peekable_buf); + if constexpr (std::is_same_v) + parse_func(peekable_buf); + else if (!parse_func(peekable_buf)) + return ReturnType(false); peekable_buf.makeContinuousMemoryFromCheckpointToPos(); auto * end = peekable_buf.position(); peekable_buf.rollbackToCheckpoint(); s.append(peekable_buf.position(), end); peekable_buf.position() = end; + return ReturnType(true); } -template -static void readQuotedStringFieldInto(Vector & s, ReadBuffer & buf) +template +static ReturnType readQuotedStringFieldInto(Vector & s, ReadBuffer & buf) { - assertChar('\'', buf); + if constexpr (std::is_same_v) + assertChar('\'', buf); + else if (!checkChar('\'', buf)) + return ReturnType(false); + s.push_back('\''); while (!buf.eof()) { @@ -1641,16 +1785,23 @@ static void readQuotedStringFieldInto(Vector & s, ReadBuffer & buf) } if (buf.eof()) - return; + return ReturnType(false); ++buf.position(); s.push_back('\''); + return ReturnType(true); } -template -static void readQuotedFieldInBracketsInto(Vector & s, ReadBuffer & buf) +template +static ReturnType readQuotedFieldInBracketsInto(Vector & s, ReadBuffer & buf) { - assertChar(opening_bracket, buf); + static constexpr bool throw_exception = std::is_same_v; + + if constexpr (throw_exception) + assertChar(opening_bracket, buf); + else if (!checkChar(opening_bracket, buf)) + return ReturnType(false); + s.push_back(opening_bracket); size_t balance = 1; @@ -1666,7 +1817,10 @@ static void readQuotedFieldInBracketsInto(Vector & s, ReadBuffer & buf) if (*buf.position() == '\'') { - readQuotedStringFieldInto(s, buf); + if constexpr (throw_exception) + readQuotedStringFieldInto(s, buf); + else if (!readQuotedStringFieldInto(s, buf)) + return ReturnType(false); } else if (*buf.position() == opening_bracket) { @@ -1681,13 +1835,20 @@ static void readQuotedFieldInBracketsInto(Vector & s, ReadBuffer & buf) ++buf.position(); } } + + if (balance) + return ReturnType(false); + + return ReturnType(true); } -template -void readQuotedFieldInto(Vector & s, ReadBuffer & buf) +template +ReturnType readQuotedFieldInto(Vector & s, ReadBuffer & buf) { + static constexpr bool throw_exception = std::is_same_v; + if (buf.eof()) - return; + return ReturnType(false); /// Possible values in 'Quoted' field: /// - Strings: '...' @@ -1699,35 +1860,47 @@ void readQuotedFieldInto(Vector & s, ReadBuffer & buf) /// - Number: integer, float, decimal. if (*buf.position() == '\'') - readQuotedStringFieldInto(s, buf); + return readQuotedStringFieldInto(s, buf); else if (*buf.position() == '[') - readQuotedFieldInBracketsInto<'[', ']'>(s, buf); + return readQuotedFieldInBracketsInto(s, buf); else if (*buf.position() == '(') - readQuotedFieldInBracketsInto<'(', ')'>(s, buf); + return readQuotedFieldInBracketsInto(s, buf); else if (*buf.position() == '{') - readQuotedFieldInBracketsInto<'{', '}'>(s, buf); + return readQuotedFieldInBracketsInto(s, buf); else if (checkCharCaseInsensitive('n', buf)) { /// NULL or NaN if (checkCharCaseInsensitive('u', buf)) { - assertStringCaseInsensitive("ll", buf); + if constexpr (throw_exception) + assertStringCaseInsensitive("ll", buf); + else if (!checkStringCaseInsensitive("ll", buf)) + return ReturnType(false); s.append("NULL"); } else { - assertStringCaseInsensitive("an", buf); + if constexpr (throw_exception) + assertStringCaseInsensitive("an", buf); + else if (!checkStringCaseInsensitive("an", buf)) + return ReturnType(false); s.append("NaN"); } } else if (checkCharCaseInsensitive('t', buf)) { - assertStringCaseInsensitive("rue", buf); + if constexpr (throw_exception) + assertStringCaseInsensitive("rue", buf); + else if (!checkStringCaseInsensitive("rue", buf)) + return ReturnType(false); s.append("true"); } else if (checkCharCaseInsensitive('f', buf)) { - assertStringCaseInsensitive("alse", buf); + if constexpr (throw_exception) + assertStringCaseInsensitive("alse", buf); + else if (!checkStringCaseInsensitive("alse", buf)) + return ReturnType(false); s.append("false"); } else @@ -1736,13 +1909,19 @@ void readQuotedFieldInto(Vector & s, ReadBuffer & buf) auto parse_func = [](ReadBuffer & in) { Float64 tmp; - readFloatText(tmp, in); + if constexpr (throw_exception) + readFloatText(tmp, in); + else + return tryReadFloatText(tmp, in); }; - readParsedValueInto(s, buf, parse_func); + + return readParsedValueInto(s, buf, parse_func); } + + return ReturnType(true); } -template void readQuotedFieldInto(NullOutput & s, ReadBuffer & buf); +template void readQuotedFieldInto(NullOutput & s, ReadBuffer & buf); void readQuotedField(String & s, ReadBuffer & buf) { @@ -1750,11 +1929,24 @@ void readQuotedField(String & s, ReadBuffer & buf) readQuotedFieldInto(s, buf); } +bool tryReadQuotedField(String & s, ReadBuffer & buf) +{ + s.clear(); + return readQuotedFieldInto(s, buf); +} + void readJSONField(String & s, ReadBuffer & buf) { s.clear(); auto parse_func = [](ReadBuffer & in) { skipJSONField(in, "json_field"); }; - readParsedValueInto(s, buf, parse_func); + readParsedValueInto(s, buf, parse_func); +} + +bool tryReadJSONField(String & s, ReadBuffer & buf) +{ + s.clear(); + auto parse_func = [](ReadBuffer & in) { return trySkipJSONField(in, "json_field"); }; + return readParsedValueInto(s, buf, parse_func); } void readTSVField(String & s, ReadBuffer & buf) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 17f3d3d4151..ad62a3deaca 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -257,26 +257,43 @@ inline void readBoolText(bool & x, ReadBuffer & buf) x = tmp != '0'; } -inline void readBoolTextWord(bool & x, ReadBuffer & buf, bool support_upper_case = false) +template +inline ReturnType readBoolTextWord(bool & x, ReadBuffer & buf, bool support_upper_case = false) { + static constexpr bool throw_exception = std::is_same_v; + if (buf.eof()) [[unlikely]] - throwReadAfterEOF(); + { + if constexpr (throw_exception) + throwReadAfterEOF(); + else + return ReturnType(false); + } switch (*buf.position()) { case 't': - assertString("true", buf); + if constexpr (throw_exception) + assertString("true", buf); + else if (!checkString("true", buf)) + return ReturnType(false); x = true; break; case 'f': - assertString("false", buf); + if constexpr (throw_exception) + assertString("false", buf); + else if (!checkString("false", buf)) + return ReturnType(false); x = false; break; case 'T': { if (support_upper_case) { - assertString("TRUE", buf); + if constexpr (throw_exception) + assertString("TRUE", buf); + else if (!checkString("TRUE", buf)) + return ReturnType(false); x = true; break; } @@ -287,7 +304,10 @@ inline void readBoolTextWord(bool & x, ReadBuffer & buf, bool support_upper_case { if (support_upper_case) { - assertString("FALSE", buf); + if constexpr (throw_exception) + assertString("FALSE", buf); + else if (!checkString("FALSE", buf)) + return ReturnType(false); x = false; break; } @@ -295,8 +315,15 @@ inline void readBoolTextWord(bool & x, ReadBuffer & buf, bool support_upper_case [[fallthrough]]; } default: - throw ParsingException(ErrorCodes::CANNOT_PARSE_BOOL, "Unexpected Bool value"); + { + if constexpr (throw_exception) + throw ParsingException(ErrorCodes::CANNOT_PARSE_BOOL, "Unexpected Bool value"); + else + return ReturnType(false); + } } + + return ReturnType(true); } enum class ReadIntTextCheckOverflow @@ -468,7 +495,10 @@ void readIntText(T & x, ReadBuffer & buf) template bool tryReadIntText(T & x, ReadBuffer & buf) { - return readIntTextImpl(x, buf); + if constexpr (is_decimal) + return tryReadIntText(x.value, buf); + else + return readIntTextImpl(x, buf); } @@ -477,16 +507,18 @@ bool tryReadIntText(T & x, ReadBuffer & buf) * - for numbers starting with zero, parsed only zero; * - symbol '+' before number is not supported; */ -template -void readIntTextUnsafe(T & x, ReadBuffer & buf) +template +ReturnType readIntTextUnsafe(T & x, ReadBuffer & buf) { + static constexpr bool throw_exception = std::is_same_v; bool negative = false; make_unsigned_t res = 0; auto on_error = [] { - if (throw_on_error) + if constexpr (throw_exception) throwReadAfterEOF(); + return ReturnType(false); }; if (buf.eof()) [[unlikely]] @@ -504,7 +536,7 @@ void readIntTextUnsafe(T & x, ReadBuffer & buf) { ++buf.position(); x = 0; - return; + return ReturnType(true); } while (!buf.eof()) @@ -523,12 +555,13 @@ void readIntTextUnsafe(T & x, ReadBuffer & buf) /// See note about undefined behaviour above. x = is_signed_v && negative ? -res : res; + return ReturnType(true); } template -void tryReadIntTextUnsafe(T & x, ReadBuffer & buf) +bool tryReadIntTextUnsafe(T & x, ReadBuffer & buf) { - return readIntTextUnsafe(x, buf); + return readIntTextUnsafe(x, buf); } @@ -550,9 +583,15 @@ void readEscapedString(String & s, ReadBuffer & buf); void readQuotedString(String & s, ReadBuffer & buf); void readQuotedStringWithSQLStyle(String & s, ReadBuffer & buf); +bool tryReadQuotedString(String & s, ReadBuffer & buf); +bool tryReadQuotedStringWithSQLStyle(String & s, ReadBuffer & buf); + void readDoubleQuotedString(String & s, ReadBuffer & buf); void readDoubleQuotedStringWithSQLStyle(String & s, ReadBuffer & buf); +bool tryReadDoubleQuotedString(String & s, ReadBuffer & buf); +bool tryReadDoubleQuotedStringWithSQLStyle(String & s, ReadBuffer & buf); + void readJSONString(String & s, ReadBuffer & buf); void readBackQuotedString(String & s, ReadBuffer & buf); @@ -615,7 +654,7 @@ void readBackQuotedStringInto(Vector & s, ReadBuffer & buf); template void readStringUntilEOFInto(Vector & s, ReadBuffer & buf); -template +template void readCSVStringInto(Vector & s, ReadBuffer & buf, const FormatSettings::CSV & settings); /// ReturnType is either bool or void. If bool, the function will return false instead of throwing an exception. @@ -628,7 +667,7 @@ bool tryReadJSONStringInto(Vector & s, ReadBuffer & buf) return readJSONStringInto(s, buf); } -template +template bool tryReadQuotedStringInto(Vector & s, ReadBuffer & buf); /// Reads chunk of data between {} in that way, @@ -637,8 +676,8 @@ bool tryReadQuotedStringInto(Vector & s, ReadBuffer & buf); template ReturnType readJSONObjectPossiblyInvalid(Vector & s, ReadBuffer & buf); -template -void readJSONArrayInto(Vector & s, ReadBuffer & buf); +template +ReturnType readJSONArrayInto(Vector & s, ReadBuffer & buf); template void readStringUntilWhitespaceInto(Vector & s, ReadBuffer & buf); @@ -962,6 +1001,13 @@ inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, cons { if (s[4] < '0' || s[4] > '9') { + if constexpr (!throw_exception) + { + if (!isNumericASCII(s[0]) || !isNumericASCII(s[1]) || !isNumericASCII(s[2]) || !isNumericASCII(s[3]) + || !isNumericASCII(s[5]) || !isNumericASCII(s[6]) || !isNumericASCII(s[8]) || !isNumericASCII(s[9])) + return ReturnType(false); + } + UInt16 year = (s[0] - '0') * 1000 + (s[1] - '0') * 100 + (s[2] - '0') * 10 + (s[3] - '0'); UInt8 month = (s[5] - '0') * 10 + (s[6] - '0'); UInt8 day = (s[8] - '0') * 10 + (s[9] - '0'); @@ -974,6 +1020,13 @@ inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, cons bool dt_long = (s[10] == ' ' || s[10] == 'T'); if (dt_long) { + if constexpr (!throw_exception) + { + if (!isNumericASCII(s[11]) || !isNumericASCII(s[12]) || !isNumericASCII(s[14]) || !isNumericASCII(s[15]) + || !isNumericASCII(s[17]) || !isNumericASCII(s[18])) + return ReturnType(false); + } + hour = (s[11] - '0') * 10 + (s[12] - '0'); minute = (s[14] - '0') * 10 + (s[15] - '0'); second = (s[17] - '0') * 10 + (s[18] - '0'); @@ -1311,6 +1364,11 @@ inline bool tryReadText(is_integer auto & x, ReadBuffer & buf) return tryReadIntText(x, buf); } +inline bool tryReadText(is_floating_point auto & x, ReadBuffer & buf) +{ + return tryReadFloatText(x, buf); +} + inline bool tryReadText(UUID & x, ReadBuffer & buf) { return tryReadUUIDText(x, buf); } inline bool tryReadText(IPv4 & x, ReadBuffer & buf) { return tryReadIPv4Text(x, buf); } inline bool tryReadText(IPv6 & x, ReadBuffer & buf) { return tryReadIPv6Text(x, buf); } @@ -1320,9 +1378,20 @@ inline void readText(is_floating_point auto & x, ReadBuffer & buf) { readFloatTe inline void readText(String & x, ReadBuffer & buf) { readEscapedString(x, buf); } inline void readText(DayNum & x, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { readDateText(x, buf, time_zone); } +inline bool tryReadText(DayNum & x, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { return tryReadDateText(x, buf, time_zone); } inline void readText(LocalDate & x, ReadBuffer & buf) { readDateText(x, buf); } +inline bool tryReadText(LocalDate & x, ReadBuffer & buf) { return tryReadDateText(x, buf); } inline void readText(LocalDateTime & x, ReadBuffer & buf) { readDateTimeText(x, buf); } +inline bool tryReadText(LocalDateTime & x, ReadBuffer & buf) +{ + time_t time; + if (!tryReadDateTimeText(time, buf)) + return false; + x = LocalDateTime(time, DateLUT::instance()); + return true; +} + inline void readText(UUID & x, ReadBuffer & buf) { readUUIDText(x, buf); } inline void readText(IPv4 & x, ReadBuffer & buf) { readIPv4Text(x, buf); } inline void readText(IPv6 & x, ReadBuffer & buf) { readIPv6Text(x, buf); } @@ -1400,39 +1469,71 @@ inline void readDoubleQuoted(LocalDateTime & x, ReadBuffer & buf) } /// CSV for numbers: quotes are optional, no special escaping rules. -template -inline void readCSVSimple(T & x, ReadBuffer & buf) +template +inline ReturnType readCSVSimple(T & x, ReadBuffer & buf) { + static constexpr bool throw_exception = std::is_same_v; + if (buf.eof()) [[unlikely]] - throwReadAfterEOF(); + { + if constexpr (throw_exception) + throwReadAfterEOF(); + return ReturnType(false); + } char maybe_quote = *buf.position(); if (maybe_quote == '\'' || maybe_quote == '\"') ++buf.position(); - readText(x, buf); + if constexpr (throw_exception) + readText(x, buf); + else if (!tryReadText(x, buf)) + return ReturnType(false); if (maybe_quote == '\'' || maybe_quote == '\"') - assertChar(maybe_quote, buf); + { + if constexpr (throw_exception) + assertChar(maybe_quote, buf); + else if (!checkChar(maybe_quote, buf)) + return ReturnType(false); + } + + return ReturnType(true); } // standalone overload for dates: to avoid instantiating DateLUTs while parsing other types -template -inline void readCSVSimple(T & x, ReadBuffer & buf, const DateLUTImpl & time_zone) +template +inline ReturnType readCSVSimple(T & x, ReadBuffer & buf, const DateLUTImpl & time_zone) { + static constexpr bool throw_exception = std::is_same_v; + if (buf.eof()) [[unlikely]] - throwReadAfterEOF(); + { + if constexpr (throw_exception) + throwReadAfterEOF(); + return ReturnType(false); + } char maybe_quote = *buf.position(); if (maybe_quote == '\'' || maybe_quote == '\"') ++buf.position(); - readText(x, buf, time_zone); + if constexpr (throw_exception) + readText(x, buf, time_zone); + else if (!tryReadText(x, buf, time_zone)) + return ReturnType(false); if (maybe_quote == '\'' || maybe_quote == '\"') - assertChar(maybe_quote, buf); + { + if constexpr (throw_exception) + assertChar(maybe_quote, buf); + else if (!checkChar(maybe_quote, buf)) + return ReturnType(false); + } + + return ReturnType(true); } template @@ -1442,18 +1543,52 @@ inline void readCSV(T & x, ReadBuffer & buf) readCSVSimple(x, buf); } +template +requires is_arithmetic_v +inline bool tryReadCSV(T & x, ReadBuffer & buf) +{ + return readCSVSimple(x, buf); +} + inline void readCSV(String & x, ReadBuffer & buf, const FormatSettings::CSV & settings) { readCSVString(x, buf, settings); } +inline bool tryReadCSV(String & x, ReadBuffer & buf, const FormatSettings::CSV & settings) +{ + x.clear(); + readCSVStringInto(x, buf, settings); + return true; +} + inline void readCSV(LocalDate & x, ReadBuffer & buf) { readCSVSimple(x, buf); } +inline bool tryReadCSV(LocalDate & x, ReadBuffer & buf) { return readCSVSimple(x, buf); } + inline void readCSV(DayNum & x, ReadBuffer & buf) { readCSVSimple(x, buf); } +inline bool tryReadCSV(DayNum & x, ReadBuffer & buf) { return readCSVSimple(x, buf); } inline void readCSV(DayNum & x, ReadBuffer & buf, const DateLUTImpl & time_zone) { readCSVSimple(x, buf, time_zone); } +inline bool tryReadCSV(DayNum & x, ReadBuffer & buf, const DateLUTImpl & time_zone) { return readCSVSimple(x, buf, time_zone); } + inline void readCSV(LocalDateTime & x, ReadBuffer & buf) { readCSVSimple(x, buf); } +inline bool tryReadCSV(LocalDateTime & x, ReadBuffer & buf) { return readCSVSimple(x, buf); } + inline void readCSV(UUID & x, ReadBuffer & buf) { readCSVSimple(x, buf); } +inline bool tryReadCSV(UUID & x, ReadBuffer & buf) { return readCSVSimple(x, buf); } + inline void readCSV(IPv4 & x, ReadBuffer & buf) { readCSVSimple(x, buf); } +inline bool tryReadCSV(IPv4 & x, ReadBuffer & buf) { return readCSVSimple(x, buf); } + inline void readCSV(IPv6 & x, ReadBuffer & buf) { readCSVSimple(x, buf); } +inline bool tryReadCSV(IPv6 & x, ReadBuffer & buf) { return readCSVSimple(x, buf); } + inline void readCSV(UInt128 & x, ReadBuffer & buf) { readCSVSimple(x, buf); } +inline bool tryReadCSV(UInt128 & x, ReadBuffer & buf) { return readCSVSimple(x, buf); } + inline void readCSV(Int128 & x, ReadBuffer & buf) { readCSVSimple(x, buf); } +inline bool tryReadCSV(Int128 & x, ReadBuffer & buf) { return readCSVSimple(x, buf); } + inline void readCSV(UInt256 & x, ReadBuffer & buf) { readCSVSimple(x, buf); } +inline bool tryReadCSV(UInt256 & x, ReadBuffer & buf) { return readCSVSimple(x, buf); } + inline void readCSV(Int256 & x, ReadBuffer & buf) { readCSVSimple(x, buf); } +inline bool tryReadCSV(Int256 & x, ReadBuffer & buf) { return readCSVSimple(x, buf); } template void readBinary(std::vector & x, ReadBuffer & buf) @@ -1535,6 +1670,7 @@ inline void skipWhitespaceIfAny(ReadBuffer & buf, bool one_line = false) /// Skips json value. void skipJSONField(ReadBuffer & buf, StringRef name_of_field); +bool trySkipJSONField(ReadBuffer & buf, StringRef name_of_field); /** Read serialized exception. @@ -1749,12 +1885,14 @@ struct PcgDeserializer } }; -template -void readQuotedFieldInto(Vector & s, ReadBuffer & buf); +template +ReturnType readQuotedFieldInto(Vector & s, ReadBuffer & buf); void readQuotedField(String & s, ReadBuffer & buf); +bool tryReadQuotedField(String & s, ReadBuffer & buf); void readJSONField(String & s, ReadBuffer & buf); +bool tryReadJSONField(String & s, ReadBuffer & buf); void readTSVField(String & s, ReadBuffer & buf); diff --git a/src/IO/readDecimalText.h b/src/IO/readDecimalText.h index 9fd9c439b87..81bde87f1f1 100644 --- a/src/IO/readDecimalText.h +++ b/src/IO/readDecimalText.h @@ -224,4 +224,24 @@ inline void readCSVDecimalText(ReadBuffer & buf, T & x, uint32_t precision, uint assertChar(maybe_quote, buf); } +template +inline bool tryReadCSVDecimalText(ReadBuffer & buf, T & x, uint32_t precision, uint32_t & scale) +{ + if (buf.eof()) + return false; + + char maybe_quote = *buf.position(); + + if (maybe_quote == '\'' || maybe_quote == '\"') + ++buf.position(); + + if (!tryReadDecimalText(buf, x, precision, scale)) + return false; + + if ((maybe_quote == '\'' || maybe_quote == '\"') && !checkChar(maybe_quote, buf)) + return false; + + return true; +} + } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index bf07f4ed3ee..51f767afc04 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -939,6 +939,20 @@ void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & creat } } } + if (!create.attach && !settings.allow_experimental_variant_type) + { + for (const auto & [name, type] : properties.columns.getAllPhysical()) + { + if (isVariant(type)) + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Cannot create table with column '{}' which type is '{}' " + "because experimental Variant type is not allowed. " + "Set setting allow_experimental_variant_type = 1 in order to allow it", + name, type->getName()); + } + } + } } namespace diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 283289f0dfc..32b24cba940 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -532,7 +532,7 @@ BlockIO InterpreterInsertQuery::execute() { /// Change query sample block columns to Nullable to allow inserting nullable columns, where NULL values will be substituted with /// default column values (in AddingDefaultsTransform), so all values will be cast correctly. - if (isNullableOrLowCardinalityNullable(input_columns[col_idx].type) && !isNullableOrLowCardinalityNullable(query_columns[col_idx].type) && output_columns.has(query_columns[col_idx].name)) + if (isNullableOrLowCardinalityNullable(input_columns[col_idx].type) && !isNullableOrLowCardinalityNullable(query_columns[col_idx].type) && !isVariant(query_columns[col_idx].type) && output_columns.has(query_columns[col_idx].name)) query_sample_block.setColumn(col_idx, ColumnWithTypeAndName(makeNullableOrLowCardinalityNullable(query_columns[col_idx].column), makeNullableOrLowCardinalityNullable(query_columns[col_idx].type), query_columns[col_idx].name)); } } diff --git a/src/Interpreters/inplaceBlockConversions.cpp b/src/Interpreters/inplaceBlockConversions.cpp index c7a1cab8bac..fd8f5b154c4 100644 --- a/src/Interpreters/inplaceBlockConversions.cpp +++ b/src/Interpreters/inplaceBlockConversions.cpp @@ -237,17 +237,36 @@ static std::unordered_map collectOffsetsColumns( { auto & offsets_column = offsets_columns[stream_name]; if (!offsets_column) + { offsets_column = current_offsets_column; + } + else + { + /// If we are inside Variant element, it may happen that + /// offsets are different, because when we read Variant + /// element as a subcolumn, we expand this column according + /// to the discriminators, so, offsets column can be changed. + /// In this case we should select the original offsets column + /// of this stream, which is the smallest one. + bool inside_variant_element = false; + for (const auto & elem : subpath) + inside_variant_element |= elem.type == ISerialization::Substream::VariantElement; - #ifndef NDEBUG - const auto & offsets_data = assert_cast(*offsets_column).getData(); - const auto & current_offsets_data = assert_cast(*current_offsets_column).getData(); + if (offsets_column->size() != current_offsets_column->size() && inside_variant_element) + offsets_column = offsets_column->size() < current_offsets_column->size() ? offsets_column : current_offsets_column; +#ifndef NDEBUG + else + { + const auto & offsets_data = assert_cast(*offsets_column).getData(); + const auto & current_offsets_data = assert_cast(*current_offsets_column).getData(); - if (offsets_data != current_offsets_data) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Found non-equal columns with offsets (sizes: {} and {}) for stream {}", - offsets_data.size(), current_offsets_data.size(), stream_name); - #endif + if (offsets_data != current_offsets_data) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Found non-equal columns with offsets (sizes: {} and {}) for stream {}", + offsets_data.size(), current_offsets_data.size(), stream_name); + } +#endif + } } }, available_column->type, res_columns[i]); } diff --git a/src/Interpreters/parseColumnsListForTableFunction.cpp b/src/Interpreters/parseColumnsListForTableFunction.cpp index 87f76f7f824..551a883d093 100644 --- a/src/Interpreters/parseColumnsListForTableFunction.cpp +++ b/src/Interpreters/parseColumnsListForTableFunction.cpp @@ -60,6 +60,17 @@ void validateDataType(const DataTypePtr & type, const DataTypeValidationSettings MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS); } } + + if (!settings.allow_experimental_variant_type) + { + if (isVariant(type)) + { + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Cannot create column with type '{}' because experimental Variant type is not allowed. " + "Set setting allow_experimental_variant_type = 1 in order to allow it", type->getName()); + } + } } ColumnsDescription parseColumnsListFromString(const std::string & structure, const ContextPtr & context) diff --git a/src/Interpreters/parseColumnsListForTableFunction.h b/src/Interpreters/parseColumnsListForTableFunction.h index ef1bbe5498e..1fbbfa4b12f 100644 --- a/src/Interpreters/parseColumnsListForTableFunction.h +++ b/src/Interpreters/parseColumnsListForTableFunction.h @@ -18,12 +18,14 @@ struct DataTypeValidationSettings : allow_suspicious_low_cardinality_types(settings.allow_suspicious_low_cardinality_types) , allow_experimental_object_type(settings.allow_experimental_object_type) , allow_suspicious_fixed_string_types(settings.allow_suspicious_fixed_string_types) + , allow_experimental_variant_type(settings.allow_experimental_variant_type) { } bool allow_suspicious_low_cardinality_types = true; bool allow_experimental_object_type = true; bool allow_suspicious_fixed_string_types = true; + bool allow_experimental_variant_type = true; }; void validateDataType(const DataTypePtr & type, const DataTypeValidationSettings & settings); diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 594221fe050..5dc791f7003 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -250,7 +250,7 @@ bool ParserTableAsStringLiteralIdentifier::parseImpl(Pos & pos, ASTPtr & node, E ReadBufferFromMemory in(pos->begin, pos->size()); String s; - if (!tryReadQuotedStringInto(s, in)) + if (!tryReadQuotedString(s, in)) { expected.add(pos, "string literal"); return false; diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 5e12ec18d27..cab0f7523f1 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -391,7 +391,7 @@ bool CSVFormatReader::readFieldImpl(ReadBuffer & istr, DB::IColumn & column, con if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) { /// If value is null but type is not nullable then use default value instead. - return SerializationNullable::deserializeTextCSVImpl(column, istr, format_settings, serialization); + return SerializationNullable::deserializeNullAsDefaultOrNestedTextCSV(column, istr, format_settings, serialization); } /// Read the column normally. diff --git a/src/Processors/Formats/Impl/MySQLDumpRowInputFormat.cpp b/src/Processors/Formats/Impl/MySQLDumpRowInputFormat.cpp index 7e8b4accf4d..9c7f095e661 100644 --- a/src/Processors/Formats/Impl/MySQLDumpRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/MySQLDumpRowInputFormat.cpp @@ -409,7 +409,7 @@ bool MySQLDumpRowInputFormat::readField(IColumn & column, size_t column_idx) const auto & type = types[column_idx]; const auto & serialization = serializations[column_idx]; if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) - return SerializationNullable::deserializeTextQuotedImpl(column, *in, format_settings, serialization); + return SerializationNullable::deserializeNullAsDefaultOrNestedTextQuoted(column, *in, format_settings, serialization); serialization->deserializeTextQuoted(column, *in, format_settings); return true; diff --git a/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp b/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp index f4f92583473..0f68c28ab1f 100644 --- a/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp @@ -147,7 +147,7 @@ bool TSKVRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ex const auto & type = getPort().getHeader().getByPosition(index).type; const auto & serialization = serializations[index]; if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) - read_columns[index] = SerializationNullable::deserializeTextEscapedImpl(*columns[index], *in, format_settings, serialization); + read_columns[index] = SerializationNullable::deserializeNullAsDefaultOrNestedTextEscaped(*columns[index], *in, format_settings, serialization); else serialization->deserializeTextEscaped(*columns[index], *in, format_settings); } diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index 3205adc2a48..88eb11d130d 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -167,7 +167,7 @@ bool TabSeparatedFormatReader::readField(IColumn & column, const DataTypePtr & t if (is_raw) { if (as_nullable) - return SerializationNullable::deserializeTextRawImpl(column, *buf, format_settings, serialization); + return SerializationNullable::deserializeNullAsDefaultOrNestedTextRaw(column, *buf, format_settings, serialization); serialization->deserializeTextRaw(column, *buf, format_settings); return true; @@ -175,7 +175,7 @@ bool TabSeparatedFormatReader::readField(IColumn & column, const DataTypePtr & t if (as_nullable) - return SerializationNullable::deserializeTextEscapedImpl(column, *buf, format_settings, serialization); + return SerializationNullable::deserializeNullAsDefaultOrNestedTextEscaped(column, *buf, format_settings, serialization); serialization->deserializeTextEscaped(column, *buf, format_settings); return true; diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index d55ccce8879..a7b5795b89e 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -293,7 +293,7 @@ bool ValuesBlockInputFormat::tryReadValue(IColumn & column, size_t column_idx) const auto & type = types[column_idx]; const auto & serialization = serializations[column_idx]; if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) - read = SerializationNullable::deserializeTextQuotedImpl(column, *buf, format_settings, serialization); + read = SerializationNullable::deserializeNullAsDefaultOrNestedTextQuoted(column, *buf, format_settings, serialization); else serialization->deserializeTextQuoted(column, *buf, format_settings); } diff --git a/tests/queries/0_stateless/02940_variant_text_deserialization.reference b/tests/queries/0_stateless/02940_variant_text_deserialization.reference new file mode 100644 index 00000000000..98725917567 --- /dev/null +++ b/tests/queries/0_stateless/02940_variant_text_deserialization.reference @@ -0,0 +1,516 @@ +JSON +String +{"v":null,"variantElement(v, 'String')":null} +{"v":"string","variantElement(v, 'String')":"string"} +{"v":"42","variantElement(v, 'String')":null} +FixedString +{"v":null,"variantElement(v, 'FixedString(4)')":null} +{"v":"string","variantElement(v, 'FixedString(4)')":null} +{"v":"abcd","variantElement(v, 'FixedString(4)')":"abcd"} +Bool +{"v":null,"variantElement(v, 'Bool')":null} +{"v":"string","variantElement(v, 'Bool')":null} +{"v":true,"variantElement(v, 'Bool')":true} +Integers +{"v":null,"variantElement(v, 'Int8')":null} +{"v":"string","variantElement(v, 'Int8')":null} +{"v":-1,"variantElement(v, 'Int8')":-1} +{"v":0,"variantElement(v, 'Int8')":0} +{"v":"10000000000","variantElement(v, 'Int8')":null} +{"v":null,"variantElement(v, 'UInt8')":null} +{"v":"string","variantElement(v, 'UInt8')":null} +{"v":"-1","variantElement(v, 'UInt8')":null} +{"v":0,"variantElement(v, 'UInt8')":0} +{"v":"10000000000","variantElement(v, 'UInt8')":null} +{"v":null,"variantElement(v, 'Int16')":null} +{"v":"string","variantElement(v, 'Int16')":null} +{"v":-1,"variantElement(v, 'Int16')":-1} +{"v":0,"variantElement(v, 'Int16')":0} +{"v":"10000000000","variantElement(v, 'Int16')":null} +{"v":null,"variantElement(v, 'UInt16')":null} +{"v":"string","variantElement(v, 'UInt16')":null} +{"v":"-1","variantElement(v, 'UInt16')":null} +{"v":0,"variantElement(v, 'UInt16')":0} +{"v":"10000000000","variantElement(v, 'UInt16')":null} +{"v":null,"variantElement(v, 'Int32')":null} +{"v":"string","variantElement(v, 'Int32')":null} +{"v":-1,"variantElement(v, 'Int32')":-1} +{"v":0,"variantElement(v, 'Int32')":0} +{"v":"10000000000","variantElement(v, 'Int32')":null} +{"v":null,"variantElement(v, 'UInt32')":null} +{"v":"string","variantElement(v, 'UInt32')":null} +{"v":"-1","variantElement(v, 'UInt32')":null} +{"v":0,"variantElement(v, 'UInt32')":0} +{"v":"10000000000","variantElement(v, 'UInt32')":null} +{"v":null,"variantElement(v, 'Int64')":null} +{"v":"string","variantElement(v, 'Int64')":null} +{"v":"-1","variantElement(v, 'Int64')":"-1"} +{"v":"0","variantElement(v, 'Int64')":"0"} +{"v":"10000000000000000000000","variantElement(v, 'Int64')":null} +{"v":null,"variantElement(v, 'UInt64')":null} +{"v":"string","variantElement(v, 'UInt64')":null} +{"v":"-1","variantElement(v, 'UInt64')":null} +{"v":"0","variantElement(v, 'UInt64')":"0"} +{"v":"10000000000000000000000","variantElement(v, 'UInt64')":null} +{"v":null,"variantElement(v, 'Int128')":null} +{"v":"string","variantElement(v, 'Int128')":null} +{"v":"-1","variantElement(v, 'Int128')":"-1"} +{"v":"0","variantElement(v, 'Int128')":"0"} +{"v":null,"variantElement(v, 'UInt128')":null} +{"v":"string","variantElement(v, 'UInt128')":null} +{"v":"-1","variantElement(v, 'UInt128')":null} +{"v":"0","variantElement(v, 'UInt128')":"0"} +Floats +{"v":null,"variantElement(v, 'Float32')":null} +{"v":"string","variantElement(v, 'Float32')":null} +{"v":42.42,"variantElement(v, 'Float32')":42.42} +{"v":null,"variantElement(v, 'Float64')":null} +{"v":"string","variantElement(v, 'Float64')":null} +{"v":42.42,"variantElement(v, 'Float64')":42.42} +Decimals +{"v":null,"variantElement(v, 'Decimal32(6)')":null} +{"v":"string","variantElement(v, 'Decimal32(6)')":null} +{"v":42.42,"variantElement(v, 'Decimal32(6)')":42.42} +{"v":"4242424242424242424242424242424242424242424242424242424242424242424242424242424242424242424242.424242424242424242","variantElement(v, 'Decimal32(6)')":null} +{"v":null,"variantElement(v, 'Decimal64(6)')":null} +{"v":"string","variantElement(v, 'Decimal64(6)')":null} +{"v":42.42,"variantElement(v, 'Decimal64(6)')":42.42} +{"v":"4242424242424242424242424242424242424242424242424242424242424242424242424242424242424242424242.424242424242424242","variantElement(v, 'Decimal64(6)')":null} +{"v":null,"variantElement(v, 'Decimal128(6)')":null} +{"v":"string","variantElement(v, 'Decimal128(6)')":null} +{"v":42.42,"variantElement(v, 'Decimal128(6)')":42.42} +{"v":"4242424242424242424242424242424242424242424242424242424242424242424242424242424242424242424242.424242424242424242","variantElement(v, 'Decimal128(6)')":null} +{"v":null,"variantElement(v, 'Decimal256(6)')":null} +{"v":"string","variantElement(v, 'Decimal256(6)')":null} +{"v":42.42,"variantElement(v, 'Decimal256(6)')":42.42} +{"v":"4242424242424242424242424242424242424242424242424242424242424242424242424242424242424242424242.424242424242424242","variantElement(v, 'Decimal256(6)')":null} +Dates and DateTimes +{"v":null,"variantElement(v, 'Date')":null} +{"v":"string","variantElement(v, 'Date')":null} +{"v":"2020-01-01","variantElement(v, 'Date')":"2020-01-01"} +{"v":"2020-01-01 00:00:00.999","variantElement(v, 'Date')":null} +{"v":null,"variantElement(v, 'Date32')":null} +{"v":"string","variantElement(v, 'Date32')":null} +{"v":"1900-01-01","variantElement(v, 'Date32')":"1900-01-01"} +{"v":"2020-01-01 00:00:00.999","variantElement(v, 'Date32')":null} +{"v":null,"variantElement(v, 'DateTime')":null} +{"v":"string","variantElement(v, 'DateTime')":null} +{"v":"2020-01-01 00:00:00","variantElement(v, 'DateTime')":"2020-01-01 00:00:00"} +{"v":"2020-01-01 00:00:00.999","variantElement(v, 'DateTime')":null} +{"v":null,"variantElement(v, 'DateTime64')":null} +{"v":"string","variantElement(v, 'DateTime64')":null} +{"v":"2020-01-01 00:00:00.999","variantElement(v, 'DateTime64')":"2020-01-01 00:00:00.999"} +{"v":"2020-01-01 00:00:00.999999999 ABC","variantElement(v, 'DateTime64')":null} +UUID +{"v":null,"variantElement(v, 'UUID')":null} +{"v":"string","variantElement(v, 'UUID')":null} +{"v":"c8619cca-0caa-445e-ae76-1d4f6e0b3927","variantElement(v, 'UUID')":"c8619cca-0caa-445e-ae76-1d4f6e0b3927"} +IPv4 +{"v":null,"variantElement(v, 'IPv4')":null} +{"v":"string","variantElement(v, 'IPv4')":null} +{"v":"127.0.0.1","variantElement(v, 'IPv4')":"127.0.0.1"} +IPv6 +{"v":null,"variantElement(v, 'IPv6')":null} +{"v":"string","variantElement(v, 'IPv6')":null} +{"v":"2001:db8:85a3::8a2e:370:7334","variantElement(v, 'IPv6')":"2001:db8:85a3::8a2e:370:7334"} +Enum +{"v":null,"variantElement(v, 'Enum(\\'a\\' = 1)')":null} +{"v":"string","variantElement(v, 'Enum(\\'a\\' = 1)')":null} +{"v":"a","variantElement(v, 'Enum(\\'a\\' = 1)')":"a"} +{"v":"a","variantElement(v, 'Enum(\\'a\\' = 1)')":"a"} +{"v":2,"variantElement(v, 'Enum(\\'a\\' = 1)')":null} +Map +{"v":null,"variantElement(v, 'Map(String, UInt64)')":{}} +{"v":"string","variantElement(v, 'Map(String, UInt64)')":{}} +{"v":{"a":"42","b":"43","c":"0"},"variantElement(v, 'Map(String, UInt64)')":{"a":"42","b":"43","c":"0"}} +{"v":"{\"c\" : 44, \"d\" : [1,2,3]}","variantElement(v, 'Map(String, UInt64)')":{}} +Tuple +{"v":null,"variantElement(v, 'Tuple(a UInt64, b UInt64)')":{"a":"0","b":"0"}} +{"v":"string","variantElement(v, 'Tuple(a UInt64, b UInt64)')":{"a":"0","b":"0"}} +{"v":{"a":"42","b":"0"},"variantElement(v, 'Tuple(a UInt64, b UInt64)')":{"a":"42","b":"0"}} +{"v":{"a":"44","b":"0"},"variantElement(v, 'Tuple(a UInt64, b UInt64)')":{"a":"44","b":"0"}} +\N (0,0) +string (0,0) +(42,0) (42,0) +{"a" : 44, "d" : 32} (0,0) +Array +{"v":null,"variantElement(v, 'Array(UInt64)')":[]} +{"v":"string","variantElement(v, 'Array(UInt64)')":[]} +{"v":["1","2","3"],"variantElement(v, 'Array(UInt64)')":["1","2","3"]} +{"v":["0","0","0"],"variantElement(v, 'Array(UInt64)')":["0","0","0"]} +{"v":"[1, 2, \"hello\"]","variantElement(v, 'Array(UInt64)')":[]} +LowCardinality +{"v":null,"variantElement(v, 'LowCardinality(String)')":null} +{"v":"string","variantElement(v, 'LowCardinality(String)')":"string"} +{"v":"42","variantElement(v, 'LowCardinality(String)')":null} +{"v":null,"variantElement(v, 'Array(LowCardinality(Nullable(String)))')":[]} +{"v":["string",null],"variantElement(v, 'Array(LowCardinality(Nullable(String)))')":["string",null]} +{"v":"42","variantElement(v, 'Array(LowCardinality(Nullable(String)))')":[]} +Nullable +{"v":null,"variantElement(v, 'Array(Nullable(String))')":[]} +{"v":"string","variantElement(v, 'Array(Nullable(String))')":[]} +{"v":["hello",null,"world"],"variantElement(v, 'Array(Nullable(String))')":["hello",null,"world"]} +{"repeat('-', 80)":"--------------------------------------------------------------------------------"} +CSV +String +\N,\N +"string","string" +"string","string" +42,\N +FixedString +\N,\N +"string",\N +"string",\N +"abcd","abcd" +Bool +\N,\N +"Truee",\N +true,true +Integers +\N,\N +"string",\N +-1,-1 +0,0 +10000000000,\N +"42d42",\N +\N,\N +"string",\N +-1,\N +0,0 +10000000000,\N +"42d42",\N +\N,\N +"string",\N +-1,-1 +0,0 +10000000000,\N +"42d42",\N +\N,\N +"string",\N +-1,\N +0,0 +10000000000,\N +"42d42",\N +\N,\N +"string",\N +-1,-1 +0,0 +10000000000,\N +"42d42",\N +\N,\N +"string",\N +-1,\N +0,0 +10000000000,\N +"42d42",\N +\N,\N +"string",\N +-1,-1 +0,0 +10000000000000000000000,\N +"42d42",\N +\N,\N +"string",\N +-1,\N +0,0 +10000000000000000000000,\N +"42d42",\N +\N,\N +"string",\N +-1,-1 +0,0 +"42d42",\N +\N,\N +"string",\N +-1,\N +0,0 +"42d42",\N +Floats +\N,\N +"string",\N +42.42,42.42 +"42.d42",\N +\N,\N +"string",\N +42.42,42.42 +"42.d42",\N +Decimals +\N,\N +"string",\N +42.42,42.42 +"42d42",\N +"4242424242424242424242424242424242424242424242424242424242424242424242424242424242424242424242.424242424242424242",\N +\N,\N +"string",\N +42.42,42.42 +"42d42",\N +"4242424242424242424242424242424242424242424242424242424242424242424242424242424242424242424242.424242424242424242",\N +\N,\N +"string",\N +42.42,42.42 +"42d42",\N +"4242424242424242424242424242424242424242424242424242424242424242424242424242424242424242424242.424242424242424242",\N +\N,\N +"string",\N +42.42,42.42 +"42d42",\N +"4242424242424242424242424242424242424242424242424242424242424242424242424242424242424242424242.424242424242424242",\N +Dates and DateTimes +\N,\N +"string",\N +"2020-01-d1",\N +"2020-01-01","2020-01-01" +"2020-01-01 00:00:00.999",\N +\N,\N +"string",\N +"2020-01-d1",\N +"1900-01-01","1900-01-01" +"2020-01-01 00:00:00.999",\N +\N,\N +"string",\N +"2020-01-d1",\N +"2020-01-01 00:00:00","2020-01-01 00:00:00" +"2020-01-01 00:00:00.999",\N +\N,\N +"string",\N +"2020-01-d1",\N +"2020-01-01 00:00:00.999","2020-01-01 00:00:00.999" +"2020-01-01 00:00:00.999999999 ABC",\N +UUID +\N,\N +"string",\N +"c8619cca-0caa-445e-ae76-1d4f6e0b3927","c8619cca-0caa-445e-ae76-1d4f6e0b3927" +"c8619cca-0caa-445e-ae76-1d4f6e0b3927AAA",\N +IPv4 +\N,\N +"string",\N +"127.0.0.1","127.0.0.1" +"127.0.0.1AAA",\N +IPv6 +\N,\N +"string",\N +"2001:db8:85a3::8a2e:370:7334","2001:db8:85a3::8a2e:370:7334" +"2001:0db8:85a3:0000:0000:8a2e:0370:7334AAA",\N +Enum +\N,\N +"string",\N +"a","a" +"a","a" +2,\N +"aa",\N +Map +\N,"{}" +"string","{}" +"{'a':42,'b':43,'c':0}","{'a':42,'b':43,'c':0}" +"{'c' : 44, 'd' : [1,2,3]}","{}" +"{'c' : 44","{}" +Array +\N,"[]" +"string","[]" +"[1,2,3]","[1,2,3]" +"[0,0,0]","[0,0,0]" +"[1, 2, 'hello']","[]" +"[1, 2","[]" +LowCardinality +\N,\N +"string","string" +42,\N +\N,"[]" +"['string',NULL]","['string',NULL]" +"['string', nul]","[]" +42,"[]" +Nullable +\N,"[]" +"string","[]" +"['hello',NULL,'world']","['hello',NULL,'world']" +"['hello', nul]","[]" +{"repeat('-', 80)":"--------------------------------------------------------------------------------"} +TSV +String +\N \N +string string +42 \N +FixedString +\N \N +string \N +abcd abcd +Bool +\N \N +Truee \N +true true +Integers +\N \N +string \N +-1 -1 +0 0 +10000000000 \N +42d42 \N +\N \N +string \N +-1 \N +0 0 +10000000000 \N +42d42 \N +\N \N +string \N +-1 -1 +0 0 +10000000000 \N +42d42 \N +\N \N +string \N +-1 \N +0 0 +10000000000 \N +42d42 \N +\N \N +string \N +-1 -1 +0 0 +10000000000 \N +42d42 \N +\N \N +string \N +-1 \N +0 0 +10000000000 \N +42d42 \N +\N \N +string \N +-1 -1 +0 0 +10000000000000000000000 \N +42d42 \N +\N \N +string \N +-1 \N +0 0 +10000000000000000000000 \N +42d42 \N +\N \N +string \N +-1 -1 +0 0 +42d42 \N +\N \N +string \N +-1 \N +0 0 +42d42 \N +Floats +\N \N +string \N +42.42 42.42 +42.d42 \N +\N \N +string \N +42.42 42.42 +42.d42 \N +Decimals +\N \N +string \N +42.42 42.42 +42d42 \N +4242424242424242424242424242424242424242424242424242424242424242424242424242424242424242424242.424242424242424242 \N +\N \N +string \N +42.42 42.42 +42d42 \N +4242424242424242424242424242424242424242424242424242424242424242424242424242424242424242424242.424242424242424242 \N +\N \N +string \N +42.42 42.42 +42d42 \N +4242424242424242424242424242424242424242424242424242424242424242424242424242424242424242424242.424242424242424242 \N +\N \N +string \N +42.42 42.42 +42d42 \N +4242424242424242424242424242424242424242424242424242424242424242424242424242424242424242424242.424242424242424242 \N +Dates and DateTimes +\N \N +string \N +2020-01-d1 \N +2020-01-01 2020-01-01 +2020-01-01 00:00:00.999 \N +\N \N +string \N +2020-01-d1 \N +1900-01-01 1900-01-01 +2020-01-01 00:00:00.999 \N +\N \N +string \N +2020-01-d1 \N +2020-01-01 00:00:00 2020-01-01 00:00:00 +2020-01-01 00:00:00.999 \N +\N \N +string \N +2020-01-d1 \N +2020-01-01 00:00:00.999 2020-01-01 00:00:00.999 +2020-01-01 00:00:00.999999999 ABC \N +UUID +\N \N +string \N +c8619cca-0caa-445e-ae76-1d4f6e0b3927 c8619cca-0caa-445e-ae76-1d4f6e0b3927 +c8619cca-0caa-445e-ae76-1d4f6e0b3927AAA \N +IPv4 +\N \N +string \N +127.0.0.1 127.0.0.1 +127.0.0.1AAA \N +IPv6 +\N \N +string \N +2001:db8:85a3::8a2e:370:7334 2001:db8:85a3::8a2e:370:7334 +2001:0db8:85a3:0000:0000:8a2e:0370:7334AAA \N +Enum +\N \N +string \N +a a +a a +2 \N +aa \N +Map +\N {} +string {} +{'a':42,'b':43,'c':0} {'a':42,'b':43,'c':0} +{\'c\' : 44, \'d\' : [1,2,3]} {} +{\'c\' : 44 {} +Array +\N [] +string [] +[1,2,3] [1,2,3] +[0,0,0] [0,0,0] +[1, 2, \'hello\'] [] +[1, 2 [] +LowCardinality +\N \N +string string +42 \N +\N [] +['string',NULL] ['string',NULL] +[\'string\', nul] [] +42 [] +Nullable +\N [] +string [] +['hello',NULL,'world'] ['hello',NULL,'world'] +[\'hello\', nul] [] +{"repeat('-', 80)":"--------------------------------------------------------------------------------"} +Values +String +(NULL,NULL),('string','string'),(42,NULL)FixedString +(NULL,NULL),('string',NULL),('abcd','abcd')Bool +(NULL,NULL),(true,true)Integers +(NULL,NULL),('string',NULL),(-1,-1),(0,0),(10000000000,NULL)(NULL,NULL),('string',NULL),(-1,NULL),(0,0),(10000000000,NULL)(NULL,NULL),('string',NULL),(-1,-1),(0,0),(10000000000,NULL)(NULL,NULL),('string',NULL),(-1,NULL),(0,0),(10000000000,NULL)(NULL,NULL),('string',NULL),(-1,-1),(0,0),(10000000000,NULL)(NULL,NULL),('string',NULL),(-1,NULL),(0,0),(10000000000,NULL)(NULL,NULL),('string',NULL),(-1,-1),(0,0),(10000000000000000000000,NULL)(NULL,NULL),('string',NULL),(-1,NULL),(0,0),(10000000000000000000000,NULL)(NULL,NULL),('string',NULL),(-1,-1),(0,0)(NULL,NULL),('string',NULL),(-1,NULL),(0,0)Floats +(NULL,NULL),('string',NULL),(42.42,42.42)(NULL,NULL),('string',NULL),(42.42,42.42)Decimals +(NULL,NULL),('string',NULL),(42.42,42.42)(NULL,NULL),('string',NULL),(42.42,42.42)(NULL,NULL),('string',NULL),(42.42,42.42)(NULL,NULL),('string',NULL),(42.42,42.42)Dates and DateTimes +(NULL,NULL),('string',NULL),('1970-01-01 00:00:00.000',NULL),('2020-01-01','2020-01-01'),('2020-01-01 00:00:00.999',NULL)(NULL,NULL),('string',NULL),('1970-01-01 00:00:00.000',NULL),('1900-01-01','1900-01-01'),('2020-01-01 00:00:00.999',NULL)(NULL,NULL),('string',NULL),('1970-01-01 00:00:00.000',NULL),('2020-01-01 00:00:00','2020-01-01 00:00:00'),('2020-01-01 00:00:00.999',NULL)(NULL,NULL),('string',NULL),('1970-01-01 00:00:00.000','1970-01-01 00:00:00.000'),('2020-01-01 00:00:00.999',NULL),('2020-01-01 00:00:00.999999999 ABC',NULL)UUID +(NULL,NULL),('string',NULL),('c8619cca-0caa-445e-ae76-1d4f6e0b3927','c8619cca-0caa-445e-ae76-1d4f6e0b3927'),('c8619cca-0caa-445e-ae76-1d4f6e0b3927AAA',NULL)IPv4 +(NULL,NULL),('string',NULL),('127.0.0.1','127.0.0.1'),('127.0.0.1AAA',NULL)IPv6 +(NULL,NULL),('string',NULL),('2001:db8:85a3::8a2e:370:7334','2001:db8:85a3::8a2e:370:7334'),('2001:0db8:85a3:0000:0000:8a2e:0370:7334AAA',NULL)Enum +(NULL,NULL),('string',NULL),('a','a'),(1,NULL),(2,NULL),('aa',NULL)Map +(NULL,{}),('string',{}),({'a':42,'b':43,'c':0},{'a':42,'b':43,'c':0})Array +(NULL,[]),('string',[]),([1,2,3],[1,2,3]),([0,0,0],[0,0,0])LowCardinality +(NULL,NULL),('string','string'),(42,NULL)(NULL,[]),(['string',NULL],['string',NULL]),(42,[])Nullable +(NULL,[]),('string',[]),(['hello',NULL,'world'],['hello',NULL,'world']) diff --git a/tests/queries/0_stateless/02940_variant_text_deserialization.sql b/tests/queries/0_stateless/02940_variant_text_deserialization.sql new file mode 100644 index 00000000000..041d02088ef --- /dev/null +++ b/tests/queries/0_stateless/02940_variant_text_deserialization.sql @@ -0,0 +1,266 @@ +set allow_experimental_variant_type = 1; +set session_timezone = 'UTC'; + +select 'JSON'; +select 'String'; +select v, variantElement(v, 'String') from format(JSONEachRow, 'v Variant(String, UInt64)', '{"v" : null}, {"v" : "string"}, {"v" : 42}') format JSONEachRow; + +select 'FixedString'; +select v, variantElement(v, 'FixedString(4)') from format(JSONEachRow, 'v Variant(String, FixedString(4))', '{"v" : null}, {"v" : "string"}, {"v" : "abcd"}') format JSONEachRow; + +select 'Bool'; +select v, variantElement(v, 'Bool') from format(JSONEachRow, 'v Variant(String, Bool)', '{"v" : null}, {"v" : "string"}, {"v" : true}') format JSONEachRow; + +select 'Integers'; +select v, variantElement(v, 'Int8') from format(JSONEachRow, 'v Variant(String, Int8, UInt64)', '{"v" : null}, {"v" : "string"}, {"v" : -1}, {"v" : 0}, {"v" : 10000000000}') format JSONEachRow; +select v, variantElement(v, 'UInt8') from format(JSONEachRow, 'v Variant(String, UInt8, Int64)', '{"v" : null}, {"v" : "string"}, {"v" : -1}, {"v" : 0}, {"v" : 10000000000}') format JSONEachRow; +select v, variantElement(v, 'Int16') from format(JSONEachRow, 'v Variant(String, Int16, Int64)', '{"v" : null}, {"v" : "string"}, {"v" : -1}, {"v" : 0}, {"v" : 10000000000}') format JSONEachRow; +select v, variantElement(v, 'UInt16') from format(JSONEachRow, 'v Variant(String, UInt16, Int64)', '{"v" : null}, {"v" : "string"}, {"v" : -1}, {"v" : 0}, {"v" : 10000000000}') format JSONEachRow; +select v, variantElement(v, 'Int32') from format(JSONEachRow, 'v Variant(String, Int32, Int64)', '{"v" : null}, {"v" : "string"}, {"v" : -1}, {"v" : 0}, {"v" : 10000000000}') format JSONEachRow; +select v, variantElement(v, 'UInt32') from format(JSONEachRow, 'v Variant(String, UInt32, Int64)', '{"v" : null}, {"v" : "string"}, {"v" : -1}, {"v" : 0}, {"v" : 10000000000}') format JSONEachRow; +select v, variantElement(v, 'Int64') from format(JSONEachRow, 'v Variant(String, Int64, Int128)', '{"v" : null}, {"v" : "string"}, {"v" : -1}, {"v" : 0}, {"v" : 10000000000000000000000}') format JSONEachRow; +select v, variantElement(v, 'UInt64') from format(JSONEachRow, 'v Variant(String, UInt64, Int128)', '{"v" : null}, {"v" : "string"}, {"v" : -1}, {"v" : 0}, {"v" : 10000000000000000000000}') format JSONEachRow; +select v, variantElement(v, 'Int128') from format(JSONEachRow, 'v Variant(String, Int128, Int256)', '{"v" : null}, {"v" : "string"}, {"v" : -1}, {"v" : 0}') format JSONEachRow; +select v, variantElement(v, 'UInt128') from format(JSONEachRow, 'v Variant(String, UInt128, Int256)', '{"v" : null}, {"v" : "string"}, {"v" : -1}, {"v" : 0}') format JSONEachRow; + +select 'Floats'; +select v, variantElement(v, 'Float32') from format(JSONEachRow, 'v Variant(String, Float32)', '{"v" : null}, {"v" : "string"}, {"v" : 42.42}') format JSONEachRow; +select v, variantElement(v, 'Float64') from format(JSONEachRow, 'v Variant(String, Float64)', '{"v" : null}, {"v" : "string"}, {"v" : 42.42}') format JSONEachRow; + +select 'Decimals'; +select v, variantElement(v, 'Decimal32(6)') from format(JSONEachRow, 'v Variant(String, Decimal32(6))', '{"v" : null}, {"v" : "string"}, {"v" : 42.42}, {"v" : 4242424242424242424242424242424242424242424242424242424242424242424242424242424242424242424242.424242424242424242}') format JSONEachRow; +select v, variantElement(v, 'Decimal64(6)') from format(JSONEachRow, 'v Variant(String, Decimal64(6))', '{"v" : null}, {"v" : "string"}, {"v" : 42.42}, {"v" : 4242424242424242424242424242424242424242424242424242424242424242424242424242424242424242424242.424242424242424242}') format JSONEachRow; +select v, variantElement(v, 'Decimal128(6)') from format(JSONEachRow, 'v Variant(String, Decimal128(6))', '{"v" : null}, {"v" : "string"}, {"v" : 42.42}, {"v" : 4242424242424242424242424242424242424242424242424242424242424242424242424242424242424242424242.424242424242424242}') format JSONEachRow; +select v, variantElement(v, 'Decimal256(6)') from format(JSONEachRow, 'v Variant(String, Decimal256(6))', '{"v" : null}, {"v" : "string"}, {"v" : 42.42}, {"v" : 4242424242424242424242424242424242424242424242424242424242424242424242424242424242424242424242.424242424242424242}') format JSONEachRow; + +select 'Dates and DateTimes'; +select v, variantElement(v, 'Date') from format(JSONEachRow, 'v Variant(String, Date, DateTime64)', '{"v" : null}, {"v" : "string"}, {"v" : "2020-01-01"}, {"v" : "2020-01-01 00:00:00.999"}') format JSONEachRow; +select v, variantElement(v, 'Date32') from format(JSONEachRow, 'v Variant(String, Date32, DateTime64)', '{"v" : null}, {"v" : "string"}, {"v" : "1900-01-01"}, {"v" : "2020-01-01 00:00:00.999"}') format JSONEachRow; +select v, variantElement(v, 'DateTime') from format(JSONEachRow, 'v Variant(String, DateTime, DateTime64)', '{"v" : null}, {"v" : "string"}, {"v" : "2020-01-01 00:00:00"}, {"v" : "2020-01-01 00:00:00.999"}') format JSONEachRow; +select v, variantElement(v, 'DateTime64') from format(JSONEachRow, 'v Variant(String, DateTime64)', '{"v" : null}, {"v" : "string"}, {"v" : "2020-01-01 00:00:00.999"}, {"v" : "2020-01-01 00:00:00.999999999 ABC"}') format JSONEachRow; + +select 'UUID'; +select v, variantElement(v, 'UUID') from format(JSONEachRow, 'v Variant(String, UUID)', '{"v" : null}, {"v" : "string"}, {"v" : "c8619cca-0caa-445e-ae76-1d4f6e0b3927"}') format JSONEachRow; + +select 'IPv4'; +select v, variantElement(v, 'IPv4') from format(JSONEachRow, 'v Variant(String, IPv4)', '{"v" : null}, {"v" : "string"}, {"v" : "127.0.0.1"}') format JSONEachRow; + +select 'IPv6'; +select v, variantElement(v, 'IPv6') from format(JSONEachRow, 'v Variant(String, IPv6)', '{"v" : null}, {"v" : "string"}, {"v" : "2001:0db8:85a3:0000:0000:8a2e:0370:7334"}') format JSONEachRow; + +select 'Enum'; +select v, variantElement(v, 'Enum(''a'' = 1)') from format(JSONEachRow, 'v Variant(String, UInt32, Enum(''a'' = 1))', '{"v" : null}, {"v" : "string"}, {"v" : "a"}, {"v" : 1}, {"v" : 2}') format JSONEachRow; + +select 'Map'; +select v, variantElement(v, 'Map(String, UInt64)') from format(JSONEachRow, 'v Variant(String, Map(String, UInt64))', '{"v" : null}, {"v" : "string"}, {"v" : {"a" : 42, "b" : 43, "c" : null}}, {"v" : {"c" : 44, "d" : [1,2,3]}}') format JSONEachRow; + +select 'Tuple'; +select v, variantElement(v, 'Tuple(a UInt64, b UInt64)') from format(JSONEachRow, 'v Variant(String, Tuple(a UInt64, b UInt64))', '{"v" : null}, {"v" : "string"}, {"v" : {"a" : 42, "b" : null}}, {"v" : {"a" : 44, "d" : 32}}') format JSONEachRow; +select v, variantElement(v, 'Tuple(a UInt64, b UInt64)') from format(JSONEachRow, 'v Variant(String, Tuple(a UInt64, b UInt64))', '{"v" : null}, {"v" : "string"}, {"v" : {"a" : 42, "b" : null}}, {"v" : {"a" : 44, "d" : 32}}') settings input_format_json_defaults_for_missing_elements_in_named_tuple=0; + +select 'Array'; +select v, variantElement(v, 'Array(UInt64)') from format(JSONEachRow, 'v Variant(String, Array(UInt64))', '{"v" : null}, {"v" : "string"}, {"v" : [1, 2, 3]}, {"v" : [null, null, null]} {"v" : [1, 2, "hello"]}') format JSONEachRow; + +select 'LowCardinality'; +select v, variantElement(v, 'LowCardinality(String)') from format(JSONEachRow, 'v Variant(LowCardinality(String), UInt64)', '{"v" : null}, {"v" : "string"}, {"v" : 42}') format JSONEachRow; +select v, variantElement(v, 'Array(LowCardinality(Nullable(String)))') from format(JSONEachRow, 'v Variant(Array(LowCardinality(Nullable(String))), UInt64)', '{"v" : null}, {"v" : ["string", null]}, {"v" : 42}') format JSONEachRow; + +select 'Nullable'; +select v, variantElement(v, 'Array(Nullable(String))') from format(JSONEachRow, 'v Variant(String, Array(Nullable(String)))', '{"v" : null}, {"v" : "string"}, {"v" : ["hello", null, "world"]}') format JSONEachRow; + +select repeat('-', 80) format JSONEachRow; + +select 'CSV'; +select 'String'; +select v, variantElement(v, 'String') from format(CSV, 'v Variant(String, UInt64)', '\\N\n"string"\nstring\n42') format CSV; + +select 'FixedString'; +select v, variantElement(v, 'FixedString(4)') from format(CSV, 'v Variant(String, FixedString(4))', '\\N\n"string"\nstring\n"abcd"') format CSV; + +select 'Bool'; +select v, variantElement(v, 'Bool') from format(CSV, 'v Variant(String, Bool)', '\\N\nTruee\nTrue') format CSV; + +select 'Integers'; +select v, variantElement(v, 'Int8') from format(CSV, 'v Variant(String, Int8, UInt64)', '\n"string"\n-1\n0\n10000000000\n42d42') format CSV; +select v, variantElement(v, 'UInt8') from format(CSV, 'v Variant(String, UInt8, Int64)', '\\N\n"string"\n-1\n0\n10000000000\n42d42') format CSV; +select v, variantElement(v, 'Int16') from format(CSV, 'v Variant(String, Int16, Int64)', '\\N\n"string"\n-1\n0\n10000000000\n42d42') format CSV; +select v, variantElement(v, 'UInt16') from format(CSV, 'v Variant(String, UInt16, Int64)', '\\N\n"string"\n-1\n0\n10000000000\n42d42') format CSV; +select v, variantElement(v, 'Int32') from format(CSV, 'v Variant(String, Int32, Int64)', '\\N\n"string"\n-1\n0\n10000000000\n42d42') format CSV; +select v, variantElement(v, 'UInt32') from format(CSV, 'v Variant(String, UInt32, Int64)', '\\N\n"string"\n-1\n0\n10000000000\n42d42') format CSV; +select v, variantElement(v, 'Int64') from format(CSV, 'v Variant(String, Int64, Int128)', '\\N\n"string"\n-1\n0\n10000000000000000000000\n42d42') format CSV; +select v, variantElement(v, 'UInt64') from format(CSV, 'v Variant(String, UInt64, Int128)', '\\N\n"string"\n-1\n0\n10000000000000000000000\n42d42') format CSV; +select v, variantElement(v, 'Int128') from format(CSV, 'v Variant(String, Int128, Int256)', '\\N\n"string"\n-1\n0\n42d42') format CSV; +select v, variantElement(v, 'UInt128') from format(CSV, 'v Variant(String, UInt128, Int256)', '\\N\n"string"\n-1\n0\n42d42') format CSV; + +select 'Floats'; +select v, variantElement(v, 'Float32') from format(CSV, 'v Variant(String, Float32)', '\\N\n"string"\n42.42\n42.d42') format CSV; +select v, variantElement(v, 'Float64') from format(CSV, 'v Variant(String, Float64)', '\\N\n"string"\n42.42\n42.d42') format CSV; + +select 'Decimals'; +select v, variantElement(v, 'Decimal32(6)') from format(CSV, 'v Variant(String, Decimal32(6))', '\\N\n"string"\n42.42\n42d42\n4242424242424242424242424242424242424242424242424242424242424242424242424242424242424242424242.424242424242424242') format CSV; +select v, variantElement(v, 'Decimal64(6)') from format(CSV, 'v Variant(String, Decimal64(6))', '\\N\n"string"\n42.42\n42d42\n4242424242424242424242424242424242424242424242424242424242424242424242424242424242424242424242.424242424242424242') format CSV; +select v, variantElement(v, 'Decimal128(6)') from format(CSV, 'v Variant(String, Decimal128(6))', '\\N\n"string"\n42.42\n42d42\n4242424242424242424242424242424242424242424242424242424242424242424242424242424242424242424242.424242424242424242') format CSV; +select v, variantElement(v, 'Decimal256(6)') from format(CSV, 'v Variant(String, Decimal256(6))', '\\N\n"string"\n42.42\n42d42\n4242424242424242424242424242424242424242424242424242424242424242424242424242424242424242424242.424242424242424242') format CSV; + +select 'Dates and DateTimes'; +select v, variantElement(v, 'Date') from format(CSV, 'v Variant(String, Date, DateTime64)', '\\N\n"string"\n"2020-01-d1"\n"2020-01-01"\n"2020-01-01 00:00:00.999"') format CSV; +select v, variantElement(v, 'Date32') from format(CSV, 'v Variant(String, Date32, DateTime64)', '\\N\n"string"\n"2020-01-d1"\n"1900-01-01"\n"2020-01-01 00:00:00.999"') format CSV; +select v, variantElement(v, 'DateTime') from format(CSV, 'v Variant(String, DateTime, DateTime64)', '\\N\n"string"\n"2020-01-d1"\n"2020-01-01 00:00:00"\n"2020-01-01 00:00:00.999"') format CSV; +select v, variantElement(v, 'DateTime64') from format(CSV, 'v Variant(String, DateTime64)', '\\N\n"string"\n"2020-01-d1"\n"2020-01-01 00:00:00.999"\n"2020-01-01 00:00:00.999999999 ABC"') format CSV; + +select 'UUID'; +select v, variantElement(v, 'UUID') from format(CSV, 'v Variant(String, UUID)', '\\N\n"string"\n"c8619cca-0caa-445e-ae76-1d4f6e0b3927"\nc8619cca-0caa-445e-ae76-1d4f6e0b3927AAA') format CSV; + +select 'IPv4'; +select v, variantElement(v, 'IPv4') from format(CSV, 'v Variant(String, IPv4)', '\\N\n"string"\n"127.0.0.1"\n"127.0.0.1AAA"') format CSV; + +select 'IPv6'; +select v, variantElement(v, 'IPv6') from format(CSV, 'v Variant(String, IPv6)', '\\N\n"string"\n"2001:0db8:85a3:0000:0000:8a2e:0370:7334"\n2001:0db8:85a3:0000:0000:8a2e:0370:7334AAA') format CSV; + +select 'Enum'; +select v, variantElement(v, 'Enum(''a'' = 1)') from format(CSV, 'v Variant(String, UInt32, Enum(''a'' = 1))', '\\N\n"string"\n"a"\n1\n2\naa') format CSV; + +select 'Map'; +select v, variantElement(v, 'Map(String, UInt64)') from format(CSV, 'v Variant(String, Map(String, UInt64))', '\\N\n"string"\n"{''a'' : 42, ''b'' : 43, ''c'' : null}"\n"{''c'' : 44, ''d'' : [1,2,3]}"\n"{''c'' : 44"') format CSV; + +select 'Array'; +select v, variantElement(v, 'Array(UInt64)') from format(CSV, 'v Variant(String, Array(UInt64))', '\\N\n"string"\n"[1, 2, 3]"\n"[null, null, null]"\n"[1, 2, ''hello'']"\n"[1, 2"') format CSV; + +select 'LowCardinality'; +select v, variantElement(v, 'LowCardinality(String)') from format(CSV, 'v Variant(LowCardinality(String), UInt64)', '\\N\n"string"\n42') format CSV; +select v, variantElement(v, 'Array(LowCardinality(Nullable(String)))') from format(CSV, 'v Variant(Array(LowCardinality(Nullable(String))), UInt64, String)', '\\N\n"[''string'', null]"\n"[''string'', nul]"\n42') format CSV; + +select 'Nullable'; +select v, variantElement(v, 'Array(Nullable(String))') from format(CSV, 'v Variant(String, Array(Nullable(String)))', '\\N\n"string"\n"[''hello'', null, ''world'']"\n"[''hello'', nul]"') format CSV; + +select repeat('-', 80) format JSONEachRow; + +select 'TSV'; +select 'String'; +select v, variantElement(v, 'String') from format(TSV, 'v Variant(String, UInt64)', '\\N\nstring\n42') format TSV; + +select 'FixedString'; +select v, variantElement(v, 'FixedString(4)') from format(TSV, 'v Variant(String, FixedString(4))', '\\N\nstring\nabcd') format TSV; + +select 'Bool'; +select v, variantElement(v, 'Bool') from format(TSV, 'v Variant(String, Bool)', '\\N\nTruee\nTrue') format TSV; + +select 'Integers'; +select v, variantElement(v, 'Int8') from format(TSV, 'v Variant(String, Int8, UInt64)', '\\N\nstring\n-1\n0\n10000000000\n42d42') format TSV; +select v, variantElement(v, 'UInt8') from format(TSV, 'v Variant(String, UInt8, Int64)', '\\N\nstring\n-1\n0\n10000000000\n42d42') format TSV; +select v, variantElement(v, 'Int16') from format(TSV, 'v Variant(String, Int16, Int64)', '\\N\nstring\n-1\n0\n10000000000\n42d42') format TSV; +select v, variantElement(v, 'UInt16') from format(TSV, 'v Variant(String, UInt16, Int64)', '\\N\nstring\n-1\n0\n10000000000\n42d42') format TSV; +select v, variantElement(v, 'Int32') from format(TSV, 'v Variant(String, Int32, Int64)', '\\N\nstring\n-1\n0\n10000000000\n42d42') format TSV; +select v, variantElement(v, 'UInt32') from format(TSV, 'v Variant(String, UInt32, Int64)', '\\N\nstring\n-1\n0\n10000000000\n42d42') format TSV; +select v, variantElement(v, 'Int64') from format(TSV, 'v Variant(String, Int64, Int128)', '\\N\nstring\n-1\n0\n10000000000000000000000\n42d42') format TSV; +select v, variantElement(v, 'UInt64') from format(TSV, 'v Variant(String, UInt64, Int128)', '\\N\nstring\n-1\n0\n10000000000000000000000\n42d42') format TSV; +select v, variantElement(v, 'Int128') from format(TSV, 'v Variant(String, Int128, Int256)', '\\N\nstring\n-1\n0\n42d42') format TSV; +select v, variantElement(v, 'UInt128') from format(TSV, 'v Variant(String, UInt128, Int256)', '\\N\nstring\n-1\n0\n42d42') format TSV; + +select 'Floats'; +select v, variantElement(v, 'Float32') from format(TSV, 'v Variant(String, Float32)', '\\N\nstring\n42.42\n42.d42') format TSV; +select v, variantElement(v, 'Float64') from format(TSV, 'v Variant(String, Float64)', '\\N\nstring\n42.42\n42.d42') format TSV; + +select 'Decimals'; +select v, variantElement(v, 'Decimal32(6)') from format(TSV, 'v Variant(String, Decimal32(6))', '\\N\nstring\n42.42\n42d42\n4242424242424242424242424242424242424242424242424242424242424242424242424242424242424242424242.424242424242424242') format TSV; +select v, variantElement(v, 'Decimal64(6)') from format(TSV, 'v Variant(String, Decimal64(6))', '\\N\nstring\n42.42\n42d42\n4242424242424242424242424242424242424242424242424242424242424242424242424242424242424242424242.424242424242424242') format TSV; +select v, variantElement(v, 'Decimal128(6)') from format(TSV, 'v Variant(String, Decimal128(6))', '\\N\nstring\n42.42\n42d42\n4242424242424242424242424242424242424242424242424242424242424242424242424242424242424242424242.424242424242424242') format TSV; +select v, variantElement(v, 'Decimal256(6)') from format(TSV, 'v Variant(String, Decimal256(6))', '\\N\nstring\n42.42\n42d42\n4242424242424242424242424242424242424242424242424242424242424242424242424242424242424242424242.424242424242424242') format TSV; + +select 'Dates and DateTimes'; +select v, variantElement(v, 'Date') from format(TSV, 'v Variant(String, Date, DateTime64)', '\\N\nstring\n2020-01-d1\n2020-01-01\n2020-01-01 00:00:00.999') format TSV; +select v, variantElement(v, 'Date32') from format(TSV, 'v Variant(String, Date32, DateTime64)', '\\N\nstring\n2020-01-d1\n1900-01-01\n2020-01-01 00:00:00.999') format TSV; +select v, variantElement(v, 'DateTime') from format(TSV, 'v Variant(String, DateTime, DateTime64)', '\\N\nstring\n2020-01-d1\n2020-01-01 00:00:00\n2020-01-01 00:00:00.999') format TSV; +select v, variantElement(v, 'DateTime64') from format(TSV, 'v Variant(String, DateTime64)', '\\N\nstring\n2020-01-d1\n2020-01-01 00:00:00.999\n2020-01-01 00:00:00.999999999 ABC') format TSV; + +select 'UUID'; +select v, variantElement(v, 'UUID') from format(TSV, 'v Variant(String, UUID)', '\\N\nstring\nc8619cca-0caa-445e-ae76-1d4f6e0b3927\nc8619cca-0caa-445e-ae76-1d4f6e0b3927AAA') format TSV; + +select 'IPv4'; +select v, variantElement(v, 'IPv4') from format(TSV, 'v Variant(String, IPv4)', '\\N\nstring\n127.0.0.1\n127.0.0.1AAA') format TSV; + +select 'IPv6'; +select v, variantElement(v, 'IPv6') from format(TSV, 'v Variant(String, IPv6)', '\\N\nstring\n2001:0db8:85a3:0000:0000:8a2e:0370:7334\n2001:0db8:85a3:0000:0000:8a2e:0370:7334AAA') format TSV; + +select 'Enum'; +select v, variantElement(v, 'Enum(''a'' = 1)') from format(TSV, 'v Variant(String, UInt32, Enum(''a'' = 1))', '\\N\nstring\na\n1\n2\naa') format TSV; + +select 'Map'; +select v, variantElement(v, 'Map(String, UInt64)') from format(TSV, 'v Variant(String, Map(String, UInt64))', '\\N\nstring\n{''a'' : 42, ''b'' : 43, ''c'' : null}\n{''c'' : 44, ''d'' : [1,2,3]}\n{''c'' : 44') format TSV; + +select 'Array'; +select v, variantElement(v, 'Array(UInt64)') from format(TSV, 'v Variant(String, Array(UInt64))', '\\N\nstring\n[1, 2, 3]\n[null, null, null]\n[1, 2, ''hello'']\n[1, 2') format TSV; + +select 'LowCardinality'; +select v, variantElement(v, 'LowCardinality(String)') from format(TSV, 'v Variant(LowCardinality(String), UInt64)', '\\N\nstring\n42') format TSV; +select v, variantElement(v, 'Array(LowCardinality(Nullable(String)))') from format(TSV, 'v Variant(Array(LowCardinality(Nullable(String))), UInt64, String)', '\\N\n[''string'', null]\n[''string'', nul]\n42') format TSV; + +select 'Nullable'; +select v, variantElement(v, 'Array(Nullable(String))') from format(TSV, 'v Variant(String, Array(Nullable(String)))', '\\N\nstring\n[''hello'', null, ''world'']\n[''hello'', nul]') format TSV; + +select repeat('-', 80) format JSONEachRow; + +select 'Values'; +select 'String'; +select v, variantElement(v, 'String') from format(Values, 'v Variant(String, UInt64)', '(NULL), (''string''), (42)') format Values; + +select 'FixedString'; +select v, variantElement(v, 'FixedString(4)') from format(Values, 'v Variant(String, FixedString(4))', '(NULL), (''string''), (''abcd'')') format Values; + +select 'Bool'; +select v, variantElement(v, 'Bool') from format(Values, 'v Variant(String, Bool)', '(NULL), (True)') format Values; + +select 'Integers'; +select v, variantElement(v, 'Int8') from format(Values, 'v Variant(String, Int8, UInt64)', '(NULL), (''string''), (-1), (0), (10000000000)') format Values; +select v, variantElement(v, 'UInt8') from format(Values, 'v Variant(String, UInt8, Int64)', '(NULL), (''string''), (-1), (0), (10000000000)') format Values; +select v, variantElement(v, 'Int16') from format(Values, 'v Variant(String, Int16, Int64)', '(NULL), (''string''), (-1), (0), (10000000000)') format Values; +select v, variantElement(v, 'UInt16') from format(Values, 'v Variant(String, UInt16, Int64)', '(NULL), (''string''), (-1), (0), (10000000000)') format Values; +select v, variantElement(v, 'Int32') from format(Values, 'v Variant(String, Int32, Int64)', '(NULL), (''string''), (-1), (0), (10000000000)') format Values; +select v, variantElement(v, 'UInt32') from format(Values, 'v Variant(String, UInt32, Int64)', '(NULL), (''string''), (-1), (0), (10000000000)') format Values; +select v, variantElement(v, 'Int64') from format(Values, 'v Variant(String, Int64, Int128)', '(NULL), (''string''), (-1), (0), (10000000000000000000000)') format Values; +select v, variantElement(v, 'UInt64') from format(Values, 'v Variant(String, UInt64, Int128)', '(NULL), (''string''), (-1), (0), (10000000000000000000000)') format Values; +select v, variantElement(v, 'Int128') from format(Values, 'v Variant(String, Int128, Int256)', '(NULL), (''string''), (-1), (0)') format Values; +select v, variantElement(v, 'UInt128') from format(Values, 'v Variant(String, UInt128, Int256)', '(NULL), (''string''), (-1), (0)') format Values; + +select 'Floats'; +select v, variantElement(v, 'Float32') from format(Values, 'v Variant(String, Float32)', '(NULL), (''string''), (42.42)') format Values; +select v, variantElement(v, 'Float64') from format(Values, 'v Variant(String, Float64)', '(NULL), (''string''), (42.42)') format Values; + +select 'Decimals'; +select v, variantElement(v, 'Decimal32(6)') from format(Values, 'v Variant(String, Decimal32(6))', '(NULL), (''string''), (42.42)') format Values; +select v, variantElement(v, 'Decimal64(6)') from format(Values, 'v Variant(String, Decimal64(6))', '(NULL), (''string''), (42.42)') format Values; +select v, variantElement(v, 'Decimal128(6)') from format(Values, 'v Variant(String, Decimal128(6))', '(NULL), (''string''), (42.42)') format Values; +select v, variantElement(v, 'Decimal256(6)') from format(Values, 'v Variant(String, Decimal256(6))', '(NULL), (''string''), (42.42)') format Values; + +select 'Dates and DateTimes'; +select v, variantElement(v, 'Date') from format(Values, 'v Variant(String, Date, DateTime64)', '(NULL), (''string''), (''2020-01-d1''), (''2020-01-01''), (''2020-01-01 00:00:00.999'')') format Values; +select v, variantElement(v, 'Date32') from format(Values, 'v Variant(String, Date32, DateTime64)', '(NULL), (''string''), (''2020-01-d1''), (''1900-01-01''), (''2020-01-01 00:00:00.999'')') format Values; +select v, variantElement(v, 'DateTime') from format(Values, 'v Variant(String, DateTime, DateTime64)', '(NULL), (''string''), (''2020-01-d1''), (''2020-01-01 00:00:00''), (''2020-01-01 00:00:00.999'')') format Values; +select v, variantElement(v, 'DateTime64') from format(Values, 'v Variant(String, DateTime64)', '(NULL), (''string''), (''2020-01-d1''), (''2020-01-01 00:00:00.999''), (''2020-01-01 00:00:00.999999999 ABC'')') format Values; + +select 'UUID'; +select v, variantElement(v, 'UUID') from format(Values, 'v Variant(String, UUID)', '(NULL), (''string''), (''c8619cca-0caa-445e-ae76-1d4f6e0b3927''), (''c8619cca-0caa-445e-ae76-1d4f6e0b3927AAA'')') format Values; + +select 'IPv4'; +select v, variantElement(v, 'IPv4') from format(Values, 'v Variant(String, IPv4)', '(NULL), (''string''), (''127.0.0.1''), (''127.0.0.1AAA'')') format Values; + +select 'IPv6'; +select v, variantElement(v, 'IPv6') from format(Values, 'v Variant(String, IPv6)', '(NULL), (''string''), (''2001:0db8:85a3:0000:0000:8a2e:0370:7334''), (''2001:0db8:85a3:0000:0000:8a2e:0370:7334AAA'')') format Values; + +select 'Enum'; +select v, variantElement(v, 'Enum(''a'' = 1)') from format(Values, 'v Variant(String, UInt32, Enum(''a'' = 1))', '(NULL), (''string''), (''a''), (1), (2), (''aa'')') format Values; + +select 'Map'; +select v, variantElement(v, 'Map(String, UInt64)') from format(Values, 'v Variant(String, Map(String, UInt64))', '(NULL), (''string''), ({''a'' : 42, ''b'' : 43, ''c'' : null})') format Values; + +select 'Array'; +select v, variantElement(v, 'Array(UInt64)') from format(Values, 'v Variant(String, Array(UInt64))', '(NULL), (''string''), ([1, 2, 3]), ([null, null, null])') format Values; + +select 'LowCardinality'; +select v, variantElement(v, 'LowCardinality(String)') from format(Values, 'v Variant(LowCardinality(String), UInt64)', '(NULL), (''string''), (42)') format Values; +select v, variantElement(v, 'Array(LowCardinality(Nullable(String)))') from format(Values, 'v Variant(Array(LowCardinality(Nullable(String))), UInt64, String)', '(NULL), ([''string'', null]), (42)') format Values; + +select 'Nullable'; +select v, variantElement(v, 'Array(Nullable(String))') from format(Values, 'v Variant(String, Array(Nullable(String)))', '(NULL), (''string''), ([''hello'', null, ''world''])') format Values; + +select ''; \ No newline at end of file diff --git a/tests/queries/0_stateless/02941_variant_type_1.reference b/tests/queries/0_stateless/02941_variant_type_1.reference new file mode 100644 index 00000000000..8a6e77d4f6d --- /dev/null +++ b/tests/queries/0_stateless/02941_variant_type_1.reference @@ -0,0 +1,2472 @@ +Memory +test1 insert +test1 select +\N +\N +\N +0 +1 +2 +str_0 +str_1 +str_2 +lc_str_0 +lc_str_1 +lc_str_2 +(0,1) +(1,2) +(2,3) +[0] +[0,1] +[0,1,2] +\N +\N +\N +\N +\N +\N +str_0 +str_1 +str_2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +0 +1 +2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +lc_str_0 +lc_str_1 +lc_str_2 +\N +\N +\N +\N +\N +\N +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,1) +(1,2) +(2,3) +(0,0) +(0,0) +(0,0) +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +0 +1 +2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +1 +2 +3 +\N +\N +\N +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[0] +[0,1] +[0,1,2] +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +1 +2 +3 +----------------------------------------------------------------------------------------------------------- +test2 insert +test2 select +\N +\N +\N +0 +\N +2 +str_0 +\N +str_2 +lc_str_0 +\N +lc_str_2 +(0,1) +\N +(2,3) +[0] +\N +[0,1,2] +\N +\N +\N +\N +\N +\N +str_0 +\N +str_2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +0 +\N +2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +lc_str_0 +\N +lc_str_2 +\N +\N +\N +\N +\N +\N +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,1) +(0,0) +(2,3) +(0,0) +(0,0) +(0,0) +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +0 +\N +2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +1 +\N +3 +\N +\N +\N +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[0] +[] +[0,1,2] +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +1 +\N +3 +----------------------------------------------------------------------------------------------------------- +test3 insert +test3 select +\N +str_1 +2 +lc_str_3 +(4,5) +[0,1,2,3,4,5] +\N +str_7 +8 +lc_str_9 +(10,11) +[0,1,2,3,4,5,6,7,8,9,10,11] +\N +str_13 +14 +lc_str_15 +(16,17) +[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] +\N +str_1 +\N +\N +\N +\N +\N +str_7 +\N +\N +\N +\N +\N +str_13 +\N +\N +\N +\N +\N +\N +2 +\N +\N +\N +\N +\N +8 +\N +\N +\N +\N +\N +14 +\N +\N +\N +\N +\N +\N +lc_str_3 +\N +\N +\N +\N +\N +lc_str_9 +\N +\N +\N +\N +\N +lc_str_15 +\N +\N +(0,0) +(0,0) +(0,0) +(0,0) +(4,5) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(10,11) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(16,17) +(0,0) +\N +\N +\N +\N +4 +\N +\N +\N +\N +\N +10 +\N +\N +\N +\N +\N +16 +\N +\N +\N +\N +\N +5 +\N +\N +\N +\N +\N +11 +\N +\N +\N +\N +\N +17 +\N +[] +[] +[] +[] +[] +[0,1,2,3,4,5] +[] +[] +[] +[] +[] +[0,1,2,3,4,5,6,7,8,9,10,11] +[] +[] +[] +[] +[] +[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] +\N +\N +\N +\N +\N +6 +\N +\N +\N +\N +\N +12 +\N +\N +\N +\N +\N +18 +----------------------------------------------------------------------------------------------------------- +MergeTree compact +test1 insert +test1 select +\N +\N +\N +0 +1 +2 +str_0 +str_1 +str_2 +lc_str_0 +lc_str_1 +lc_str_2 +(0,1) +(1,2) +(2,3) +[0] +[0,1] +[0,1,2] +\N +\N +\N +\N +\N +\N +str_0 +str_1 +str_2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +0 +1 +2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +lc_str_0 +lc_str_1 +lc_str_2 +\N +\N +\N +\N +\N +\N +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,1) +(1,2) +(2,3) +(0,0) +(0,0) +(0,0) +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +0 +1 +2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +1 +2 +3 +\N +\N +\N +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[0] +[0,1] +[0,1,2] +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +1 +2 +3 +----------------------------------------------------------------------------------------------------------- +test1 select +\N +\N +\N +0 +1 +2 +str_0 +str_1 +str_2 +lc_str_0 +lc_str_1 +lc_str_2 +(0,1) +(1,2) +(2,3) +[0] +[0,1] +[0,1,2] +\N +\N +\N +\N +\N +\N +str_0 +str_1 +str_2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +0 +1 +2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +lc_str_0 +lc_str_1 +lc_str_2 +\N +\N +\N +\N +\N +\N +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,1) +(1,2) +(2,3) +(0,0) +(0,0) +(0,0) +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +0 +1 +2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +1 +2 +3 +\N +\N +\N +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[0] +[0,1] +[0,1,2] +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +1 +2 +3 +----------------------------------------------------------------------------------------------------------- +test2 insert +test2 select +\N +\N +\N +0 +\N +2 +str_0 +\N +str_2 +lc_str_0 +\N +lc_str_2 +(0,1) +\N +(2,3) +[0] +\N +[0,1,2] +\N +\N +\N +\N +\N +\N +str_0 +\N +str_2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +0 +\N +2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +lc_str_0 +\N +lc_str_2 +\N +\N +\N +\N +\N +\N +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,1) +(0,0) +(2,3) +(0,0) +(0,0) +(0,0) +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +0 +\N +2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +1 +\N +3 +\N +\N +\N +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[0] +[] +[0,1,2] +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +1 +\N +3 +----------------------------------------------------------------------------------------------------------- +test2 select +\N +\N +\N +0 +\N +2 +str_0 +\N +str_2 +lc_str_0 +\N +lc_str_2 +(0,1) +\N +(2,3) +[0] +\N +[0,1,2] +\N +\N +\N +\N +\N +\N +str_0 +\N +str_2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +0 +\N +2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +lc_str_0 +\N +lc_str_2 +\N +\N +\N +\N +\N +\N +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,1) +(0,0) +(2,3) +(0,0) +(0,0) +(0,0) +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +0 +\N +2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +1 +\N +3 +\N +\N +\N +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[0] +[] +[0,1,2] +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +1 +\N +3 +----------------------------------------------------------------------------------------------------------- +test3 insert +test3 select +\N +str_1 +2 +lc_str_3 +(4,5) +[0,1,2,3,4,5] +\N +str_7 +8 +lc_str_9 +(10,11) +[0,1,2,3,4,5,6,7,8,9,10,11] +\N +str_13 +14 +lc_str_15 +(16,17) +[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] +\N +str_1 +\N +\N +\N +\N +\N +str_7 +\N +\N +\N +\N +\N +str_13 +\N +\N +\N +\N +\N +\N +2 +\N +\N +\N +\N +\N +8 +\N +\N +\N +\N +\N +14 +\N +\N +\N +\N +\N +\N +lc_str_3 +\N +\N +\N +\N +\N +lc_str_9 +\N +\N +\N +\N +\N +lc_str_15 +\N +\N +(0,0) +(0,0) +(0,0) +(0,0) +(4,5) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(10,11) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(16,17) +(0,0) +\N +\N +\N +\N +4 +\N +\N +\N +\N +\N +10 +\N +\N +\N +\N +\N +16 +\N +\N +\N +\N +\N +5 +\N +\N +\N +\N +\N +11 +\N +\N +\N +\N +\N +17 +\N +[] +[] +[] +[] +[] +[0,1,2,3,4,5] +[] +[] +[] +[] +[] +[0,1,2,3,4,5,6,7,8,9,10,11] +[] +[] +[] +[] +[] +[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] +\N +\N +\N +\N +\N +6 +\N +\N +\N +\N +\N +12 +\N +\N +\N +\N +\N +18 +----------------------------------------------------------------------------------------------------------- +test3 select +\N +str_1 +2 +lc_str_3 +(4,5) +[0,1,2,3,4,5] +\N +str_7 +8 +lc_str_9 +(10,11) +[0,1,2,3,4,5,6,7,8,9,10,11] +\N +str_13 +14 +lc_str_15 +(16,17) +[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] +\N +str_1 +\N +\N +\N +\N +\N +str_7 +\N +\N +\N +\N +\N +str_13 +\N +\N +\N +\N +\N +\N +2 +\N +\N +\N +\N +\N +8 +\N +\N +\N +\N +\N +14 +\N +\N +\N +\N +\N +\N +lc_str_3 +\N +\N +\N +\N +\N +lc_str_9 +\N +\N +\N +\N +\N +lc_str_15 +\N +\N +(0,0) +(0,0) +(0,0) +(0,0) +(4,5) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(10,11) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(16,17) +(0,0) +\N +\N +\N +\N +4 +\N +\N +\N +\N +\N +10 +\N +\N +\N +\N +\N +16 +\N +\N +\N +\N +\N +5 +\N +\N +\N +\N +\N +11 +\N +\N +\N +\N +\N +17 +\N +[] +[] +[] +[] +[] +[0,1,2,3,4,5] +[] +[] +[] +[] +[] +[0,1,2,3,4,5,6,7,8,9,10,11] +[] +[] +[] +[] +[] +[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] +\N +\N +\N +\N +\N +6 +\N +\N +\N +\N +\N +12 +\N +\N +\N +\N +\N +18 +----------------------------------------------------------------------------------------------------------- +MergeTree wide +test1 insert +test1 select +\N +\N +\N +0 +1 +2 +str_0 +str_1 +str_2 +lc_str_0 +lc_str_1 +lc_str_2 +(0,1) +(1,2) +(2,3) +[0] +[0,1] +[0,1,2] +\N +\N +\N +\N +\N +\N +str_0 +str_1 +str_2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +0 +1 +2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +lc_str_0 +lc_str_1 +lc_str_2 +\N +\N +\N +\N +\N +\N +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,1) +(1,2) +(2,3) +(0,0) +(0,0) +(0,0) +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +0 +1 +2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +1 +2 +3 +\N +\N +\N +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[0] +[0,1] +[0,1,2] +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +1 +2 +3 +----------------------------------------------------------------------------------------------------------- +test1 select +\N +\N +\N +0 +1 +2 +str_0 +str_1 +str_2 +lc_str_0 +lc_str_1 +lc_str_2 +(0,1) +(1,2) +(2,3) +[0] +[0,1] +[0,1,2] +\N +\N +\N +\N +\N +\N +str_0 +str_1 +str_2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +0 +1 +2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +lc_str_0 +lc_str_1 +lc_str_2 +\N +\N +\N +\N +\N +\N +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,1) +(1,2) +(2,3) +(0,0) +(0,0) +(0,0) +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +0 +1 +2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +1 +2 +3 +\N +\N +\N +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[0] +[0,1] +[0,1,2] +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +1 +2 +3 +----------------------------------------------------------------------------------------------------------- +test2 insert +test2 select +\N +\N +\N +0 +\N +2 +str_0 +\N +str_2 +lc_str_0 +\N +lc_str_2 +(0,1) +\N +(2,3) +[0] +\N +[0,1,2] +\N +\N +\N +\N +\N +\N +str_0 +\N +str_2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +0 +\N +2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +lc_str_0 +\N +lc_str_2 +\N +\N +\N +\N +\N +\N +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,1) +(0,0) +(2,3) +(0,0) +(0,0) +(0,0) +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +0 +\N +2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +1 +\N +3 +\N +\N +\N +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[0] +[] +[0,1,2] +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +1 +\N +3 +----------------------------------------------------------------------------------------------------------- +test2 select +\N +\N +\N +0 +\N +2 +str_0 +\N +str_2 +lc_str_0 +\N +lc_str_2 +(0,1) +\N +(2,3) +[0] +\N +[0,1,2] +\N +\N +\N +\N +\N +\N +str_0 +\N +str_2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +0 +\N +2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +lc_str_0 +\N +lc_str_2 +\N +\N +\N +\N +\N +\N +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(0,1) +(0,0) +(2,3) +(0,0) +(0,0) +(0,0) +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +0 +\N +2 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +1 +\N +3 +\N +\N +\N +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[0] +[] +[0,1,2] +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +1 +\N +3 +----------------------------------------------------------------------------------------------------------- +test3 insert +test3 select +\N +str_1 +2 +lc_str_3 +(4,5) +[0,1,2,3,4,5] +\N +str_7 +8 +lc_str_9 +(10,11) +[0,1,2,3,4,5,6,7,8,9,10,11] +\N +str_13 +14 +lc_str_15 +(16,17) +[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] +\N +str_1 +\N +\N +\N +\N +\N +str_7 +\N +\N +\N +\N +\N +str_13 +\N +\N +\N +\N +\N +\N +2 +\N +\N +\N +\N +\N +8 +\N +\N +\N +\N +\N +14 +\N +\N +\N +\N +\N +\N +lc_str_3 +\N +\N +\N +\N +\N +lc_str_9 +\N +\N +\N +\N +\N +lc_str_15 +\N +\N +(0,0) +(0,0) +(0,0) +(0,0) +(4,5) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(10,11) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(16,17) +(0,0) +\N +\N +\N +\N +4 +\N +\N +\N +\N +\N +10 +\N +\N +\N +\N +\N +16 +\N +\N +\N +\N +\N +5 +\N +\N +\N +\N +\N +11 +\N +\N +\N +\N +\N +17 +\N +[] +[] +[] +[] +[] +[0,1,2,3,4,5] +[] +[] +[] +[] +[] +[0,1,2,3,4,5,6,7,8,9,10,11] +[] +[] +[] +[] +[] +[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] +\N +\N +\N +\N +\N +6 +\N +\N +\N +\N +\N +12 +\N +\N +\N +\N +\N +18 +----------------------------------------------------------------------------------------------------------- +test3 select +\N +str_1 +2 +lc_str_3 +(4,5) +[0,1,2,3,4,5] +\N +str_7 +8 +lc_str_9 +(10,11) +[0,1,2,3,4,5,6,7,8,9,10,11] +\N +str_13 +14 +lc_str_15 +(16,17) +[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] +\N +str_1 +\N +\N +\N +\N +\N +str_7 +\N +\N +\N +\N +\N +str_13 +\N +\N +\N +\N +\N +\N +2 +\N +\N +\N +\N +\N +8 +\N +\N +\N +\N +\N +14 +\N +\N +\N +\N +\N +\N +lc_str_3 +\N +\N +\N +\N +\N +lc_str_9 +\N +\N +\N +\N +\N +lc_str_15 +\N +\N +(0,0) +(0,0) +(0,0) +(0,0) +(4,5) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(10,11) +(0,0) +(0,0) +(0,0) +(0,0) +(0,0) +(16,17) +(0,0) +\N +\N +\N +\N +4 +\N +\N +\N +\N +\N +10 +\N +\N +\N +\N +\N +16 +\N +\N +\N +\N +\N +5 +\N +\N +\N +\N +\N +11 +\N +\N +\N +\N +\N +17 +\N +[] +[] +[] +[] +[] +[0,1,2,3,4,5] +[] +[] +[] +[] +[] +[0,1,2,3,4,5,6,7,8,9,10,11] +[] +[] +[] +[] +[] +[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] +\N +\N +\N +\N +\N +6 +\N +\N +\N +\N +\N +12 +\N +\N +\N +\N +\N +18 +----------------------------------------------------------------------------------------------------------- diff --git a/tests/queries/0_stateless/02941_variant_type_1.sh b/tests/queries/0_stateless/02941_variant_type_1.sh new file mode 100755 index 00000000000..774acb4bbef --- /dev/null +++ b/tests/queries/0_stateless/02941_variant_type_1.sh @@ -0,0 +1,124 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# reset --log_comment +CLICKHOUSE_LOG_COMMENT= +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1" + +function test1_insert() +{ + echo "test1 insert" + $CH_CLIENT -q "insert into test select number, NULL from numbers(3);" + $CH_CLIENT -q "insert into test select number + 3, number from numbers(3);" + $CH_CLIENT -q "insert into test select number + 6, 'str_' || toString(number) from numbers(3);" + $CH_CLIENT -q "insert into test select number + 9, ('lc_str_' || toString(number))::LowCardinality(String) from numbers(3);" + $CH_CLIENT -q "insert into test select number + 12, tuple(number, number + 1)::Tuple(a UInt32, b UInt32) from numbers(3);" + $CH_CLIENT -q "insert into test select number + 15, range(number + 1)::Array(UInt64) from numbers(3);" +} + +function test1_select() +{ + echo "test1 select" + $CH_CLIENT -q "select v from test order by id;" + $CH_CLIENT -q "select v.String from test order by id;" + $CH_CLIENT -q "select v.UInt64 from test order by id;" + $CH_CLIENT -q "select v.\`LowCardinality(String)\` from test order by id;" + $CH_CLIENT -q "select v.\`Tuple(a UInt32, b UInt32)\` from test order by id;" + $CH_CLIENT -q "select v.\`Tuple(a UInt32, b UInt32)\`.a from test order by id;" + $CH_CLIENT -q "select v.\`Tuple(a UInt32, b UInt32)\`.b from test order by id;" + $CH_CLIENT -q "select v.\`Array(UInt64)\` from test order by id;" + $CH_CLIENT -q "select v.\`Array(UInt64)\`.size0 from test order by id;" + echo "-----------------------------------------------------------------------------------------------------------" +} + +function test2_insert() +{ + echo "test2 insert" + $CH_CLIENT -q "insert into test select number, NULL from numbers(3);" + $CH_CLIENT -q "insert into test select number + 3, number % 2 ? NULL : number from numbers(3);" + $CH_CLIENT -q "insert into test select number + 6, number % 2 ? NULL : 'str_' || toString(number) from numbers(3);" + $CH_CLIENT -q "insert into test select number + 9, number % 2 ? CAST(NULL, 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))') : CAST(('lc_str_' || toString(number))::LowCardinality(String), 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))') from numbers(3);" + $CH_CLIENT -q "insert into test select number + 12, number % 2 ? CAST(NULL, 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))') : CAST(tuple(number, number + 1)::Tuple(a UInt32, b UInt32), 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))') from numbers(3);" + $CH_CLIENT -q "insert into test select number + 15, number % 2 ? CAST(NULL, 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))') : CAST(range(number + 1)::Array(UInt64), 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))') from numbers(3);" +} + +function test2_select() +{ + echo "test2 select" + $CH_CLIENT -q "select v from test order by id;" + $CH_CLIENT -q "select v.String from test order by id;" + $CH_CLIENT -q "select v.UInt64 from test order by id;" + $CH_CLIENT -q "select v.\`LowCardinality(String)\` from test order by id;" + $CH_CLIENT -q "select v.\`Tuple(a UInt32, b UInt32)\` from test order by id;" + $CH_CLIENT -q "select v.\`Tuple(a UInt32, b UInt32)\`.a from test order by id;" + $CH_CLIENT -q "select v.\`Tuple(a UInt32, b UInt32)\`.b from test order by id;" + $CH_CLIENT -q "select v.\`Array(UInt64)\` from test order by id;" + $CH_CLIENT -q "select v.\`Array(UInt64)\`.size0 from test order by id;" + echo "-----------------------------------------------------------------------------------------------------------" +} + +function test3_insert() +{ + echo "test3 insert" + $CH_CLIENT -q "insert into test with 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))' as type select number, multiIf(number % 6 == 0, CAST(NULL, type), number % 6 == 1, CAST('str_' || toString(number), type), number % 6 == 2, CAST(number, type), number % 6 == 3, CAST(('lc_str_' || toString(number))::LowCardinality(String), type), number % 6 == 4, CAST(tuple(number, number + 1)::Tuple(a UInt32, b UInt32), type), CAST(range(number + 1)::Array(UInt64), type)) as res from numbers(18);" +} + +function test3_select() +{ + echo "test3 select" + $CH_CLIENT -q "select v from test order by id;" + $CH_CLIENT -q "select v.String from test order by id;" + $CH_CLIENT -q "select v.UInt64 from test order by id;" + $CH_CLIENT -q "select v.\`LowCardinality(String)\` from test order by id;" + $CH_CLIENT -q "select v.\`Tuple(a UInt32, b UInt32)\` from test order by id;" + $CH_CLIENT -q "select v.\`Tuple(a UInt32, b UInt32)\`.a from test order by id;" + $CH_CLIENT -q "select v.\`Tuple(a UInt32, b UInt32)\`.b from test order by id;" + $CH_CLIENT -q "select v.\`Array(UInt64)\` from test order by id;" + $CH_CLIENT -q "select v.\`Array(UInt64)\`.size0 from test order by id;" + echo "-----------------------------------------------------------------------------------------------------------" +} + +function run() +{ + test1_insert + test1_select + if [ $1 == 1 ]; then + $CH_CLIENT -q "optimize table test final;" + test1_select + fi + $CH_CLIENT -q "truncate table test;" + test2_insert + test2_select + if [ $1 == 1 ]; then + $CH_CLIENT -q "optimize table test final;" + test2_select + fi + $CH_CLIENT -q "truncate table test;" + test3_insert + test3_select + if [ $1 == 1 ]; then + $CH_CLIENT -q "optimize table test final;" + test3_select + fi + $CH_CLIENT -q "truncate table test;" +} + +$CH_CLIENT -q "drop table if exists test;" + +echo "Memory" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=Memory;" +run 0 +$CH_CLIENT -q "drop table test;" + +echo "MergeTree compact" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000;" +run 1 +$CH_CLIENT -q "drop table test;" + +echo "MergeTree wide" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" +run 1 +$CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/02941_variant_type_2.reference b/tests/queries/0_stateless/02941_variant_type_2.reference new file mode 100644 index 00000000000..4b6d53c52ac --- /dev/null +++ b/tests/queries/0_stateless/02941_variant_type_2.reference @@ -0,0 +1,51 @@ +Memory +test4 insert +test4 select +1000000 +200000 +200000 +200000 +200000 +200000 +200000 +200000 +MergeTree compact +test4 insert +test4 select +1000000 +200000 +200000 +200000 +200000 +200000 +200000 +200000 +test4 select +1000000 +200000 +200000 +200000 +200000 +200000 +200000 +200000 +MergeTree wide +test4 insert +test4 select +1000000 +200000 +200000 +200000 +200000 +200000 +200000 +200000 +test4 select +1000000 +200000 +200000 +200000 +200000 +200000 +200000 +200000 diff --git a/tests/queries/0_stateless/02941_variant_type_2.sh b/tests/queries/0_stateless/02941_variant_type_2.sh new file mode 100755 index 00000000000..aef5bc3fe02 --- /dev/null +++ b/tests/queries/0_stateless/02941_variant_type_2.sh @@ -0,0 +1,71 @@ +#!/usr/bin/env bash +# tags: long + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# reset --log_comment +CLICKHOUSE_LOG_COMMENT= +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1" + +function test4_insert() +{ + echo "test4 insert" + $CH_CLIENT -q "insert into test select number, NULL from numbers(200000);" + $CH_CLIENT -q "insert into test select number + 200000, number from numbers(200000);" + $CH_CLIENT -q "insert into test select number + 400000, 'str_' || toString(number) from numbers(200000);" + $CH_CLIENT -q "insert into test select number + 600000, ('lc_str_' || toString(number))::LowCardinality(String) from numbers(200000);" + $CH_CLIENT -q "insert into test select number + 800000, tuple(number, number + 1)::Tuple(a UInt32, b UInt32) from numbers(200000);" + $CH_CLIENT -q "insert into test select number + 1000000, range(number % 20 + 1)::Array(UInt64) from numbers(200000);" +} + +function test4_select +{ + echo "test4 select" + $CH_CLIENT -q "select v from test format Null;" + $CH_CLIENT -q "select count() from test where isNotNull(v);" + $CH_CLIENT -q "select v.String from test format Null;" + $CH_CLIENT -q "select count() from test where isNotNull(v.String);" + $CH_CLIENT -q "select v.UInt64 from test format Null;" + $CH_CLIENT -q "select count() from test where isNotNull(v.UInt64);" + $CH_CLIENT -q "select v.\`LowCardinality(String)\` from test format Null;" + $CH_CLIENT -q "select count() from test where isNotNull(v.\`LowCardinality(String)\`);" + $CH_CLIENT -q "select v.\`Tuple(a UInt32, b UInt32)\` from test format Null;" + $CH_CLIENT -q "select v.\`Tuple(a UInt32, b UInt32)\`.a from test format Null;" + $CH_CLIENT -q "select count() from test where isNotNull(v.\`Tuple(a UInt32, b UInt32)\`.a);" + $CH_CLIENT -q "select v.\`Tuple(a UInt32, b UInt32)\`.b from test format Null;" + $CH_CLIENT -q "select count() from test where isNotNull(v.\`Tuple(a UInt32, b UInt32)\`.b);" + $CH_CLIENT -q "select v.\`Array(UInt64)\` from test format Null;" + $CH_CLIENT -q "select count() from test where not empty(v.\`Array(UInt64)\`);" + $CH_CLIENT -q "select v.\`Array(UInt64)\`.size0 from test format Null;" + $CH_CLIENT -q "select count() from test where isNotNull(v.\`Array(UInt64)\`.size0);" +} + +function run() +{ + test4_insert + test4_select + if [ $1 == 1 ]; then + $CH_CLIENT -q "optimize table test final;" + test4_select + fi + $CH_CLIENT -q "truncate table test;" +} + +$CH_CLIENT -q "drop table if exists test;" + +echo "Memory" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=Memory;" +run 0 +$CH_CLIENT -q "drop table test;" + +echo "MergeTree compact" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000;" +run 1 +$CH_CLIENT -q "drop table test;" + +echo "MergeTree wide" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" +run 1 +$CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/02941_variant_type_3.reference b/tests/queries/0_stateless/02941_variant_type_3.reference new file mode 100644 index 00000000000..1ccdb3acdff --- /dev/null +++ b/tests/queries/0_stateless/02941_variant_type_3.reference @@ -0,0 +1,51 @@ +Memory +test5 insert +test5 select +500000 +100000 +100000 +100000 +100000 +100000 +100000 +100000 +MergeTree compact +test5 insert +test5 select +500000 +100000 +100000 +100000 +100000 +100000 +100000 +100000 +test5 select +500000 +100000 +100000 +100000 +100000 +100000 +100000 +100000 +MergeTree wide +test5 insert +test5 select +500000 +100000 +100000 +100000 +100000 +100000 +100000 +100000 +test5 select +500000 +100000 +100000 +100000 +100000 +100000 +100000 +100000 diff --git a/tests/queries/0_stateless/02941_variant_type_3.sh b/tests/queries/0_stateless/02941_variant_type_3.sh new file mode 100755 index 00000000000..d3692270deb --- /dev/null +++ b/tests/queries/0_stateless/02941_variant_type_3.sh @@ -0,0 +1,71 @@ +#!/usr/bin/env bash +# tags: long + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# reset --log_comment +CLICKHOUSE_LOG_COMMENT= +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1" + +function test5_insert() +{ + echo "test5 insert" + $CH_CLIENT -q "insert into test select number, NULL from numbers(200000);" + $CH_CLIENT -q "insert into test select number + 200000, number % 2 ? NULL : number from numbers(200000);" + $CH_CLIENT -q "insert into test select number + 400000, number % 2 ? NULL : 'str_' || toString(number) from numbers(200000);" + $CH_CLIENT -q "insert into test select number + 600000, number % 2 ? CAST(NULL, 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))') : CAST(('lc_str_' || toString(number))::LowCardinality(String), 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))') from numbers(200000);" + $CH_CLIENT -q "insert into test select number + 800000, number % 2 ? CAST(NULL, 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))') : CAST(tuple(number, number + 1)::Tuple(a UInt32, b UInt32), 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))') from numbers(200000);" + $CH_CLIENT -q "insert into test select number + 1000000, number % 2 ? CAST(NULL, 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))') : CAST(range(number % 20 + 1)::Array(UInt64), 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))') from numbers(200000);" +} + +function test5_select() +{ + echo "test5 select" + $CH_CLIENT -q "select v from test format Null;" + $CH_CLIENT -q "select count() from test where isNotNull(v);" + $CH_CLIENT -q "select v.String from test format Null;" + $CH_CLIENT -q "select count() from test where isNotNull(v.String);" + $CH_CLIENT -q "select v.UInt64 from test format Null;" + $CH_CLIENT -q "select count() from test where isNotNull(v.UInt64);" + $CH_CLIENT -q "select v.\`LowCardinality(String)\` from test format Null;" + $CH_CLIENT -q "select count() from test where isNotNull(v.\`LowCardinality(String)\`);" + $CH_CLIENT -q "select v.\`Tuple(a UInt32, b UInt32)\` from test format Null;" + $CH_CLIENT -q "select v.\`Tuple(a UInt32, b UInt32)\`.a from test format Null;" + $CH_CLIENT -q "select count() from test where isNotNull(v.\`Tuple(a UInt32, b UInt32)\`.a);" + $CH_CLIENT -q "select v.\`Tuple(a UInt32, b UInt32)\`.b from test format Null;" + $CH_CLIENT -q "select count() from test where isNotNull(v.\`Tuple(a UInt32, b UInt32)\`.b);" + $CH_CLIENT -q "select v.\`Array(UInt64)\` from test format Null;" + $CH_CLIENT -q "select count() from test where not empty(v.\`Array(UInt64)\`);" + $CH_CLIENT -q "select v.\`Array(UInt64)\`.size0 from test format Null;" + $CH_CLIENT -q "select count() from test where isNotNull(v.\`Array(UInt64)\`.size0);" +} + +function run() +{ + test5_insert + test5_select + if [ $1 == 1 ]; then + $CH_CLIENT -q "optimize table test final;" + test5_select + fi + $CH_CLIENT -q "truncate table test;" +} + +$CH_CLIENT -q "drop table if exists test;" + +echo "Memory" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=Memory;" +run 0 +$CH_CLIENT -q "drop table test;" + +echo "MergeTree compact" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000;" +run 1 +$CH_CLIENT -q "drop table test;" + +echo "MergeTree wide" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" +run 1 +$CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/02941_variant_type_4.reference b/tests/queries/0_stateless/02941_variant_type_4.reference new file mode 100644 index 00000000000..e13d5820343 --- /dev/null +++ b/tests/queries/0_stateless/02941_variant_type_4.reference @@ -0,0 +1,56 @@ +Memory +test6 insert +test6 select +1000000 +200000 +200000 +200000 +200000 +200000 +200000 +200000 +----------------------------------------------------------------------------------------------------------- +MergeTree compact +test6 insert +test6 select +1000000 +200000 +200000 +200000 +200000 +200000 +200000 +200000 +----------------------------------------------------------------------------------------------------------- +test6 select +1000000 +200000 +200000 +200000 +200000 +200000 +200000 +200000 +----------------------------------------------------------------------------------------------------------- +MergeTree wide +test6 insert +test6 select +1000000 +200000 +200000 +200000 +200000 +200000 +200000 +200000 +----------------------------------------------------------------------------------------------------------- +test6 select +1000000 +200000 +200000 +200000 +200000 +200000 +200000 +200000 +----------------------------------------------------------------------------------------------------------- diff --git a/tests/queries/0_stateless/02941_variant_type_4.sh b/tests/queries/0_stateless/02941_variant_type_4.sh new file mode 100755 index 00000000000..b3cc041bcd8 --- /dev/null +++ b/tests/queries/0_stateless/02941_variant_type_4.sh @@ -0,0 +1,66 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# reset --log_comment +CLICKHOUSE_LOG_COMMENT= +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1" + +function test6_insert() +{ + echo "test6 insert" + $CH_CLIENT -q "insert into test with 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))' as type select number, multiIf(number % 6 == 0, CAST(NULL, type), number % 6 == 1, CAST('str_' || toString(number), type), number % 6 == 2, CAST(number, type), number % 6 == 3, CAST(('lc_str_' || toString(number))::LowCardinality(String), type), number % 6 == 4, CAST(tuple(number, number + 1)::Tuple(a UInt32, b UInt32), type), CAST(range(number % 20 + 1)::Array(UInt64), type)) as res from numbers(1200000);" +} + +function test6_select() +{ + echo "test6 select" + $CH_CLIENT -q "select v from test format Null;" + $CH_CLIENT -q "select count() from test where isNotNull(v);" + $CH_CLIENT -q "select v.String from test format Null;" + $CH_CLIENT -q "select count() from test where isNotNull(v.String);" + $CH_CLIENT -q "select v.UInt64 from test format Null;" + $CH_CLIENT -q "select count() from test where isNotNull(v.UInt64);" + $CH_CLIENT -q "select v.\`LowCardinality(String)\` from test format Null;" + $CH_CLIENT -q "select count() from test where isNotNull(v.\`LowCardinality(String)\`);" + $CH_CLIENT -q "select v.\`Tuple(a UInt32, b UInt32)\` from test format Null;" + $CH_CLIENT -q "select v.\`Tuple(a UInt32, b UInt32)\`.a from test format Null;" + $CH_CLIENT -q "select count() from test where isNotNull(v.\`Tuple(a UInt32, b UInt32)\`.a);" + $CH_CLIENT -q "select v.\`Tuple(a UInt32, b UInt32)\`.b from test format Null;" + $CH_CLIENT -q "select count() from test where isNotNull(v.\`Tuple(a UInt32, b UInt32)\`.b);" + $CH_CLIENT -q "select v.\`Array(UInt64)\` from test format Null;" + $CH_CLIENT -q "select count() from test where not empty(v.\`Array(UInt64)\`);" + $CH_CLIENT -q "select v.\`Array(UInt64)\`.size0 from test format Null;" + $CH_CLIENT -q "select count() from test where isNotNull(v.\`Array(UInt64)\`.size0);" + echo "-----------------------------------------------------------------------------------------------------------" +} + +function run() +{ + test6_insert + test6_select + if [ $1 == 1 ]; then + $CH_CLIENT -q "optimize table test final;" + test6_select + fi + $CH_CLIENT -q "truncate table test;" +} + +$CH_CLIENT -q "drop table if exists test;" + +echo "Memory" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=Memory;" +run 0 +$CH_CLIENT -q "drop table test;" + +echo "MergeTree compact" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000;" +run 1 +$CH_CLIENT -q "drop table test;" + +echo "MergeTree wide" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" +run 1 +$CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/02942_variant_cast.reference b/tests/queries/0_stateless/02942_variant_cast.reference new file mode 100644 index 00000000000..f3fd7a9ba33 --- /dev/null +++ b/tests/queries/0_stateless/02942_variant_cast.reference @@ -0,0 +1,25 @@ +\N +42 +0 +\N +2 +\N +Hello +Hello +NULL +Hello +Hello +\N +Hello +\N +0 +\N +42 +\N +Hello +2 +\N +Hello +5 +0 +1 diff --git a/tests/queries/0_stateless/02942_variant_cast.sql b/tests/queries/0_stateless/02942_variant_cast.sql new file mode 100644 index 00000000000..33587e3e438 --- /dev/null +++ b/tests/queries/0_stateless/02942_variant_cast.sql @@ -0,0 +1,23 @@ +set allow_experimental_variant_type=1; + +select NULL::Variant(String, UInt64); +select 42::UInt64::Variant(String, UInt64); +select 42::UInt32::Variant(String, UInt64); -- {serverError CANNOT_CONVERT_TYPE} +select now()::Variant(String, UInt64); -- {serverError CANNOT_CONVERT_TYPE} +select CAST(number % 2 ? NULL : number, 'Variant(String, UInt64)') from numbers(4); +select 'Hello'::LowCardinality(String)::Variant(LowCardinality(String), UInt64); +select 'Hello'::LowCardinality(Nullable(String))::Variant(LowCardinality(String), UInt64); +select 'NULL'::LowCardinality(Nullable(String))::Variant(LowCardinality(String), UInt64); +select 'Hello'::LowCardinality(Nullable(String))::Variant(LowCardinality(String), UInt64); +select CAST(CAST(number % 2 ? NULL : 'Hello', 'LowCardinality(Nullable(String))'), 'Variant(LowCardinality(String), UInt64)') from numbers(4); + +select NULL::Variant(String, UInt64)::UInt64; +select NULL::Variant(String, UInt64)::Nullable(UInt64); +select '42'::Variant(String, UInt64)::UInt64; +select 'str'::Variant(String, UInt64)::UInt64; -- {serverError CANNOT_PARSE_TEXT} +select CAST(multiIf(number % 3 == 0, NULL::Variant(String, UInt64), number % 3 == 1, 'Hello'::Variant(String, UInt64), number::Variant(String, UInt64)), 'Nullable(String)') from numbers(6); +select CAST(multiIf(number == 1, NULL::Variant(String, UInt64), number == 2, 'Hello'::Variant(String, UInt64), number::Variant(String, UInt64)), 'UInt64') from numbers(6); -- {serverError CANNOT_PARSE_TEXT} + + +select number::Variant(UInt64)::Variant(String, UInt64)::Variant(Array(String), String, UInt64) from numbers(2); +select 'str'::Variant(String, UInt64)::Variant(String, Array(UInt64)); -- {serverError CANNOT_CONVERT_TYPE} diff --git a/tests/queries/0_stateless/02943_variant_element.reference b/tests/queries/0_stateless/02943_variant_element.reference new file mode 100644 index 00000000000..ab8aaa8fdef --- /dev/null +++ b/tests/queries/0_stateless/02943_variant_element.reference @@ -0,0 +1,44 @@ +\N +\N +\N +\N +0 +1 +2 +3 +\N +\N +\N +\N +0 +\N +2 +\N +\N +\N +\N +\N +str_0 +\N +str_2 +\N +\N +\N +\N +\N +[] +[] +[] +[] +[] +[] +[] +[] +[0] +[] +[0,1,2] +[] +[[0]] +[[NULL]] +[[2]] +[[NULL]] diff --git a/tests/queries/0_stateless/02943_variant_element.sql b/tests/queries/0_stateless/02943_variant_element.sql new file mode 100644 index 00000000000..c8eff9775ad --- /dev/null +++ b/tests/queries/0_stateless/02943_variant_element.sql @@ -0,0 +1,16 @@ +set allow_experimental_variant_type=1; +set use_variant_when_no_common_type_in_if=1; + +select variantElement(NULL::Variant(String, UInt64), 'UInt64') from numbers(4); +select variantElement(number::Variant(String, UInt64), 'UInt64') from numbers(4); +select variantElement(number::Variant(String, UInt64), 'String') from numbers(4); +select variantElement((number % 2 ? NULL : number)::Variant(String, UInt64), 'UInt64') from numbers(4); +select variantElement((number % 2 ? NULL : number)::Variant(String, UInt64), 'String') from numbers(4); +select variantElement((number % 2 ? NULL : 'str_' || toString(number))::LowCardinality(Nullable(String))::Variant(LowCardinality(String), UInt64), 'LowCardinality(String)') from numbers(4); +select variantElement(NULL::LowCardinality(Nullable(String))::Variant(LowCardinality(String), UInt64), 'LowCardinality(String)') from numbers(4); +select variantElement((number % 2 ? NULL : number)::Variant(Array(UInt64), UInt64), 'Array(UInt64)') from numbers(4); +select variantElement(NULL::Variant(Array(UInt64), UInt64), 'Array(UInt64)') from numbers(4); +select variantElement(number % 2 ? NULL : range(number + 1), 'Array(UInt64)') from numbers(4); + +select variantElement([[(number % 2 ? NULL : number)::Variant(String, UInt64)]], 'UInt64') from numbers(4); + diff --git a/tests/queries/0_stateless/02944_variant_as_if_multi_if_result.reference b/tests/queries/0_stateless/02944_variant_as_if_multi_if_result.reference new file mode 100644 index 00000000000..3803f39253c --- /dev/null +++ b/tests/queries/0_stateless/02944_variant_as_if_multi_if_result.reference @@ -0,0 +1,96 @@ +Array(UInt8) [1,2,3] +Array(UInt8) [1,2,3] +String str_1 +Nullable(String) str_1 +String str_1 +Nullable(String) str_1 +Variant(Array(UInt8), String) str_1 +Variant(Array(UInt8), String) str_1 +Array(UInt8) [1,2,3] +Array(UInt8) [1,2,3] +String str_1 +Nullable(String) str_1 +String str_1 +Nullable(String) str_1 +Variant(Array(UInt8), String) str_1 +Variant(Array(UInt8), String) str_1 +Array(UInt8) [1,2,3] +Array(UInt8) [1,2,3] +String str_1 +Nullable(String) str_1 +String str_1 +Nullable(String) str_1 +Variant(Array(UInt8), String) str_1 +Variant(Array(UInt8), String) str_1 +String str_0 +String str_1 +String str_2 +String str_3 +Nullable(String) str_0 +Nullable(String) str_1 +Nullable(String) str_2 +Nullable(String) str_3 +Array(UInt64) [0] +Array(UInt64) [0,1] +Array(UInt64) [0,1,2] +Array(UInt64) [0,1,2,3] +Array(UInt64) [0] +Array(UInt64) [0,1] +Array(UInt64) [0,1,2] +Array(UInt64) [0,1,2,3] +String str_0 +String str_1 +String str_2 +String str_3 +Nullable(String) str_0 +Nullable(String) str_1 +Nullable(String) str_2 +Nullable(String) str_3 +Variant(Array(UInt64), String) str_0 +Variant(Array(UInt64), String) str_1 +Variant(Array(UInt64), String) str_2 +Variant(Array(UInt64), String) str_3 +Variant(Array(UInt64), String) str_0 +Variant(Array(UInt64), String) str_1 +Variant(Array(UInt64), String) str_2 +Variant(Array(UInt64), String) str_3 +Variant(Array(UInt64), String) str_0 +Variant(Array(UInt64), String) [0,1] +Variant(Array(UInt64), String) str_2 +Variant(Array(UInt64), String) [0,1,2,3] +Variant(Array(UInt64), String) str_0 +Variant(Array(UInt64), String) [0,1] +Variant(Array(UInt64), String) str_2 +Variant(Array(UInt64), String) [0,1,2,3] +Variant(Array(UInt64), String) str_0 +Variant(Array(UInt64), String) [0,1] +Variant(Array(UInt64), String) str_2 +Variant(Array(UInt64), String) [0,1,2,3] +Variant(Array(UInt64), String) str_0 +Variant(Array(UInt64), String) [0,1] +Variant(Array(UInt64), String) str_2 +Variant(Array(UInt64), String) [0,1,2,3] +Variant(Array(UInt64), String, UInt64) [0] +Variant(Array(UInt64), String, UInt64) 1 +Variant(Array(UInt64), String, UInt64) str_2 +Variant(Array(UInt64), String, UInt64) [0,1,2,3] +Variant(Array(UInt64), String, UInt64) 4 +Variant(Array(UInt64), String, UInt64) str_5 +Variant(Array(UInt64), String, UInt64) [0] +Variant(Array(UInt64), String, UInt64) 1 +Variant(Array(UInt64), String, UInt64) str_2 +Variant(Array(UInt64), String, UInt64) [0,1,2,3] +Variant(Array(UInt64), String, UInt64) 4 +Variant(Array(UInt64), String, UInt64) str_5 +Variant(Array(UInt64), String, UInt64) [0] +Variant(Array(UInt64), String, UInt64) 1 +Variant(Array(UInt64), String, UInt64) str_2 +Variant(Array(UInt64), String, UInt64) [0,1,2,3] +Variant(Array(UInt64), String, UInt64) 4 +Variant(Array(UInt64), String, UInt64) str_5 +Variant(Array(UInt64), String, UInt64) [0] +Variant(Array(UInt64), String, UInt64) 1 +Variant(Array(UInt64), String, UInt64) str_2 +Variant(Array(UInt64), String, UInt64) [0,1,2,3] +Variant(Array(UInt64), String, UInt64) 4 +Variant(Array(UInt64), String, UInt64) str_5 diff --git a/tests/queries/0_stateless/02944_variant_as_if_multi_if_result.sql b/tests/queries/0_stateless/02944_variant_as_if_multi_if_result.sql new file mode 100644 index 00000000000..da36863bfda --- /dev/null +++ b/tests/queries/0_stateless/02944_variant_as_if_multi_if_result.sql @@ -0,0 +1,64 @@ +set allow_experimental_variant_type=1; +set use_variant_when_no_common_type_in_if=1; + +select toTypeName(res), if(1, [1,2,3], 'str_1') as res; +select toTypeName(res), if(1, [1,2,3], 'str_1'::Nullable(String)) as res; + +select toTypeName(res), if(0, [1,2,3], 'str_1') as res; +select toTypeName(res), if(0, [1,2,3], 'str_1'::Nullable(String)) as res; + +select toTypeName(res), if(NULL, [1,2,3], 'str_1') as res; +select toTypeName(res), if(NULL, [1,2,3], 'str_1'::Nullable(String)) as res; + +select toTypeName(res), if(materialize(NULL::Nullable(UInt8)), [1,2,3], 'str_1') as res; +select toTypeName(res), if(materialize(NULL::Nullable(UInt8)), [1,2,3], 'str_1'::Nullable(String)) as res; + +select toTypeName(res), if(1, materialize([1,2,3]), 'str_1') as res; +select toTypeName(res), if(1, materialize([1,2,3]), 'str_1'::Nullable(String)) as res; + +select toTypeName(res), if(0, materialize([1,2,3]), 'str_1') as res; +select toTypeName(res), if(0, materialize([1,2,3]), 'str_1'::Nullable(String)) as res; + +select toTypeName(res), if(NULL, materialize([1,2,3]), 'str_1') as res; +select toTypeName(res), if(NULL, materialize([1,2,3]), 'str_1'::Nullable(String)) as res; + +select toTypeName(res), if(materialize(NULL::Nullable(UInt8)), materialize([1,2,3]), 'str_1') as res; +select toTypeName(res), if(materialize(NULL::Nullable(UInt8)), materialize([1,2,3]), 'str_1'::Nullable(String)) as res; + +select toTypeName(res), if(1, [1,2,3], materialize('str_1')) as res; +select toTypeName(res), if(1, [1,2,3], materialize('str_1')::Nullable(String)) as res; + +select toTypeName(res), if(0, [1,2,3], materialize('str_1')) as res; +select toTypeName(res), if(0, [1,2,3], materialize('str_1')::Nullable(String)) as res; + +select toTypeName(res), if(NULL, [1,2,3], materialize('str_1')) as res; +select toTypeName(res), if(NULL, [1,2,3], materialize('str_1')::Nullable(String)) as res; + +select toTypeName(res), if(materialize(NULL::Nullable(UInt8)), [1,2,3], materialize('str_1')) as res; +select toTypeName(res), if(materialize(NULL::Nullable(UInt8)), [1,2,3], materialize('str_1')::Nullable(String)) as res; + + +select toTypeName(res), if(0, range(number + 1), 'str_' || toString(number)) as res from numbers(4); +select toTypeName(res), if(0, range(number + 1), ('str_' || toString(number))::Nullable(String)) as res from numbers(4); + +select toTypeName(res), if(1, range(number + 1), 'str_' || toString(number)) as res from numbers(4); +select toTypeName(res), if(1, range(number + 1), ('str_' || toString(number))::Nullable(String)) as res from numbers(4); + +select toTypeName(res), if(NULL, range(number + 1), 'str_' || toString(number)) as res from numbers(4); +select toTypeName(res), if(NULL, range(number + 1), ('str_' || toString(number))::Nullable(String)) as res from numbers(4); + +select toTypeName(res), if(materialize(NULL::Nullable(UInt8)), range(number + 1), 'str_' || toString(number)) as res from numbers(4); +select toTypeName(res), if(materialize(NULL::Nullable(UInt8)), range(number + 1), ('str_' || toString(number))::Nullable(String)) as res from numbers(4); + +select toTypeName(res), if(number % 2, range(number + 1), 'str_' || toString(number)) as res from numbers(4); +select toTypeName(res), if(number % 2, range(number + 1), ('str_' || toString(number))::Nullable(String)) as res from numbers(4); + +select toTypeName(res), if(number % 2, range(number + 1), ('str_' || toString(number))::LowCardinality(String)) as res from numbers(4); +select toTypeName(res), if(number % 2, range(number + 1), ('str_' || toString(number))::LowCardinality(Nullable(String))) as res from numbers(4); + + +select toTypeName(res), multiIf(number % 3 == 0, range(number + 1), number % 3 == 1, number, 'str_' || toString(number)) as res from numbers(6); +select toTypeName(res), multiIf(number % 3 == 0, range(number + 1), number % 3 == 1, number, ('str_' || toString(number))::Nullable(String)) as res from numbers(6); +select toTypeName(res), multiIf(number % 3 == 0, range(number + 1), number % 3 == 1, number, ('str_' || toString(number))::LowCardinality(String)) as res from numbers(6); +select toTypeName(res), multiIf(number % 3 == 0, range(number + 1), number % 3 == 1, number, ('str_' || toString(number))::LowCardinality(Nullable(String))) as res from numbers(6); + From 0a7ca36e7fbd02b4b64a30371fa3118144179e51 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 19 Dec 2023 17:08:35 +0000 Subject: [PATCH 018/264] Remove unneded changes in IColumn.h --- src/Columns/IColumn.h | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 0dcba5b310c..3f866e6213d 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -631,17 +631,6 @@ struct IsMutableColumns template <> struct IsMutableColumns<> { static const bool value = true; }; -template -struct IsMutableColumnsOrRvalueReferences; - -template -struct IsMutableColumnsOrRvalueReferences -{ - static const bool value = (std::is_assignable::value || std::is_rvalue_reference_v) && IsMutableColumnsOrRvalueReferences::value; -}; - -template <> -struct IsMutableColumnsOrRvalueReferences<> { static const bool value = true; }; template const Type * checkAndGetColumn(const IColumn & column) From bd84799aecb0f8103fd88e9fb1491720f9ec90c8 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 19 Dec 2023 17:21:45 +0000 Subject: [PATCH 019/264] Fix style --- src/Columns/ColumnVariant.h | 4 ++-- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h index 692fdd1709e..702107504f0 100644 --- a/src/Columns/ColumnVariant.h +++ b/src/Columns/ColumnVariant.h @@ -9,7 +9,7 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; + extern const int NOT_IMPLEMENTED; } /** @@ -263,7 +263,7 @@ public: bool hasOnlyNulls() const { /// If all variants are empty, we have only NULL values. - return std::all_of(variants.begin(), variants.end(), [](const auto & v){ return v->empty(); } ); + return std::all_of(variants.begin(), variants.end(), [](const WrappedPtr & v){ return v->empty(); }); } /// Check if local and global order is the same. diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 637ab0ce6d4..bc03f4b39f8 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2594,6 +2594,7 @@ uuid varPop varSamp variadic +variantElement varint varpop varsamp From e74ae96dd006f8ff5fc8150eba5ab0beb47ddba3 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 19 Dec 2023 17:35:13 +0000 Subject: [PATCH 020/264] Fux typo --- src/DataTypes/Serializations/SerializationNullable.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationNullable.cpp b/src/DataTypes/Serializations/SerializationNullable.cpp index e7f0e61f2a5..05c70827c35 100644 --- a/src/DataTypes/Serializations/SerializationNullable.cpp +++ b/src/DataTypes/Serializations/SerializationNullable.cpp @@ -206,7 +206,7 @@ ReturnType safeAppendToNullMap(ColumnNullable & column, bool is_null) } /// Deserialize value into non-nullable column. In case of NULL, insert default and set is_null to true. -/// If ReturnType is bool, return true if parsing was succesfull and false in case of any error. +/// If ReturnType is bool, return true if parsing was successful and false in case of any error. template static ReturnType deserializeImpl(IColumn & column, ReadBuffer & buf, CheckForNull && check_for_null, DeserializeNested && deserialize_nested, bool & is_null) { From 9edbfb3a31e67722a6af3b418a119e9b2bbb164e Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 19 Dec 2023 17:55:42 +0000 Subject: [PATCH 021/264] Fix build after merging with master --- src/DataTypes/Serializations/SerializationEnum.cpp | 10 +++++----- src/DataTypes/Serializations/SerializationEnum.h | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationEnum.cpp b/src/DataTypes/Serializations/SerializationEnum.cpp index 6ad55913738..fb384547d64 100644 --- a/src/DataTypes/Serializations/SerializationEnum.cpp +++ b/src/DataTypes/Serializations/SerializationEnum.cpp @@ -47,7 +47,7 @@ bool SerializationEnum::tryDeserializeTextEscaped(IColumn & column, ReadBu { std::string field_name; readEscapedString(field_name, istr); - if (!this->tryGetValue(x, StringRef(field_name), true)) + if (!ref_enum_values.tryGetValue(x, StringRef(field_name), true)) return false; } @@ -75,7 +75,7 @@ bool SerializationEnum::tryDeserializeTextQuoted(IColumn & column, ReadBuf std::string field_name; readQuotedStringWithSQLStyle(field_name, istr); FieldType x; - if (!this->tryGetValue(x, StringRef(field_name))) + if (!ref_enum_values.tryGetValue(x, StringRef(field_name))) return false; assert_cast(column).getData().push_back(x); return true; @@ -111,7 +111,7 @@ bool SerializationEnum::tryDeserializeWholeText(IColumn & column, ReadBuff { std::string field_name; readStringUntilEOF(field_name, istr); - if (!this->tryGetValue(x, StringRef(field_name), true)) + if (!ref_enum_values.tryGetValue(x, StringRef(field_name), true)) return false; } @@ -157,7 +157,7 @@ bool SerializationEnum::tryDeserializeTextJSON(IColumn & column, ReadBuffe { std::string field_name; readJSONString(field_name, istr); - if (!this->tryGetValue(x, StringRef(field_name))) + if (!ref_enum_values.tryGetValue(x, StringRef(field_name))) return false; } @@ -198,7 +198,7 @@ bool SerializationEnum::tryDeserializeTextCSV(IColumn & column, ReadBuffer { std::string field_name; readCSVString(field_name, istr, settings.csv); - if (!this->tryGetValue(x, StringRef(field_name), true)) + if (!ref_enum_values.tryGetValue(x, StringRef(field_name), true)) return false; } diff --git a/src/DataTypes/Serializations/SerializationEnum.h b/src/DataTypes/Serializations/SerializationEnum.h index 708161dc5fd..5152a3fbc93 100644 --- a/src/DataTypes/Serializations/SerializationEnum.h +++ b/src/DataTypes/Serializations/SerializationEnum.h @@ -60,7 +60,7 @@ public: bool tryReadValue(ReadBuffer & istr, FieldType & x) const { - if (!tryReadText(x, istr) || !this->hasValue(x)) + if (!tryReadText(x, istr) || !ref_enum_values.hasValue(x)) return false; return true; From 3c9dd07f7b2c036f5d299869f16ae0a39621b25f Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 19 Dec 2023 21:17:39 +0000 Subject: [PATCH 022/264] Fix special builds, fix test --- src/Columns/tests/gtest_column_variant.cpp | 5 ++++- src/DataTypes/Serializations/SerializationIPv4andIPv6.cpp | 7 +++---- tests/queries/0_stateless/02941_variant_type_1.sh | 1 + tests/queries/0_stateless/02941_variant_type_2.sh | 2 +- tests/queries/0_stateless/02941_variant_type_3.sh | 2 +- tests/queries/0_stateless/02941_variant_type_4.sh | 1 + 6 files changed, 11 insertions(+), 7 deletions(-) diff --git a/src/Columns/tests/gtest_column_variant.cpp b/src/Columns/tests/gtest_column_variant.cpp index b701e2d3183..0a6512c46b7 100644 --- a/src/Columns/tests/gtest_column_variant.cpp +++ b/src/Columns/tests/gtest_column_variant.cpp @@ -582,7 +582,10 @@ TEST(ColumnVariant, PermuteAndIndexOneColumnNoNulls) ASSERT_EQ((*permuted_column)[2].get(), 2); auto index = ColumnUInt64::create(); - index->getData() = std::move(permutation); + index->getData().push_back(1); + index->getData().push_back(3); + index->getData().push_back(2); + index->getData().push_back(0); auto indexed_column = column->index(*index, 3); ASSERT_EQ(indexed_column->size(), 3); ASSERT_EQ((*indexed_column)[0].get(), 1); diff --git a/src/DataTypes/Serializations/SerializationIPv4andIPv6.cpp b/src/DataTypes/Serializations/SerializationIPv4andIPv6.cpp index 81c4af97401..dfcd24aff58 100644 --- a/src/DataTypes/Serializations/SerializationIPv4andIPv6.cpp +++ b/src/DataTypes/Serializations/SerializationIPv4andIPv6.cpp @@ -15,10 +15,10 @@ void SerializationIP::deserializeText(DB::IColumn & column, DB::ReadBuffer IPv x; readText(x, istr); + assert_cast &>(column).getData().push_back(x); + if (whole && !istr.eof()) throwUnexpectedDataAfterParsedValue(column, istr, settings, TypeName.data()); - - assert_cast &>(column).getData().push_back(x); } template @@ -77,11 +77,10 @@ void SerializationIP::deserializeTextJSON(DB::IColumn & column, DB::ReadBuf /// this code looks weird, but we want to throw specific exception to match original behavior... if (istr.eof()) assertChar('"', istr); + assert_cast &>(column).getData().push_back(x); if (*istr.position() != '"') throwUnexpectedDataAfterParsedValue(column, istr, settings, TypeName.data()); istr.ignore(); - - assert_cast &>(column).getData().push_back(x); } template diff --git a/tests/queries/0_stateless/02941_variant_type_1.sh b/tests/queries/0_stateless/02941_variant_type_1.sh index 774acb4bbef..4cf8ad25122 100755 --- a/tests/queries/0_stateless/02941_variant_type_1.sh +++ b/tests/queries/0_stateless/02941_variant_type_1.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: long CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # reset --log_comment diff --git a/tests/queries/0_stateless/02941_variant_type_2.sh b/tests/queries/0_stateless/02941_variant_type_2.sh index aef5bc3fe02..7064dfbf4ec 100755 --- a/tests/queries/0_stateless/02941_variant_type_2.sh +++ b/tests/queries/0_stateless/02941_variant_type_2.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# tags: long +# Tags: long CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # reset --log_comment diff --git a/tests/queries/0_stateless/02941_variant_type_3.sh b/tests/queries/0_stateless/02941_variant_type_3.sh index d3692270deb..303039edef7 100755 --- a/tests/queries/0_stateless/02941_variant_type_3.sh +++ b/tests/queries/0_stateless/02941_variant_type_3.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# tags: long +# Tags: long CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # reset --log_comment diff --git a/tests/queries/0_stateless/02941_variant_type_4.sh b/tests/queries/0_stateless/02941_variant_type_4.sh index b3cc041bcd8..169e43c6d69 100755 --- a/tests/queries/0_stateless/02941_variant_type_4.sh +++ b/tests/queries/0_stateless/02941_variant_type_4.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: long CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # reset --log_comment From 1efd65b8c73951e60e94f74ccc45141a5b39d85e Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 20 Dec 2023 17:43:04 +0000 Subject: [PATCH 023/264] Fix tests --- src/Columns/ColumnVariant.cpp | 10 ++++++++++ src/Columns/ColumnVariant.h | 1 + src/DataTypes/DataTypeVariant.cpp | 12 ++++++++++++ src/DataTypes/DataTypeVariant.h | 1 + src/DataTypes/IDataType.h | 2 +- .../Serializations/SerializationVariantElement.cpp | 2 +- 6 files changed, 26 insertions(+), 2 deletions(-) diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index 67754e77992..a3a0362b646 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -480,6 +480,16 @@ void ColumnVariant::insertFrom(const IColumn & src_, size_t n) } } +void ColumnVariant::insertIntoVariant(const DB::Field & x, Discriminator global_discr) +{ + if (global_discr > variants.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid global discriminator: {}. The number of variants is {}", size_t(global_discr), variants.size()); + auto & variant = getVariantByGlobalDiscriminator(global_discr); + variant.insert(x); + getLocalDiscriminators().push_back(localDiscriminatorByGlobal(global_discr)); + getOffsets().push_back(variant.size() - 1); +} + void ColumnVariant::insertRangeFrom(const IColumn & src_, size_t start, size_t length) { const size_t num_variants = variants.size(); diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h index 702107504f0..b388b118a69 100644 --- a/src/Columns/ColumnVariant.h +++ b/src/Columns/ColumnVariant.h @@ -174,6 +174,7 @@ public: StringRef getDataAt(size_t n) const override; void insertData(const char * pos, size_t length) override; void insert(const Field & x) override; + void insertIntoVariant(const Field & x, Discriminator global_discr); void insertFrom(const IColumn & src_, size_t n) override; void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; void insertManyFrom(const IColumn & src, size_t position, size_t length) override; diff --git a/src/DataTypes/DataTypeVariant.cpp b/src/DataTypes/DataTypeVariant.cpp index 77e1c504cf8..334ed2c7b10 100644 --- a/src/DataTypes/DataTypeVariant.cpp +++ b/src/DataTypes/DataTypeVariant.cpp @@ -1,9 +1,11 @@ #include +#include #include #include #include #include #include +#include #include #include #include @@ -94,6 +96,16 @@ MutableColumnPtr DataTypeVariant::createColumn() const return ColumnVariant::create(std::move(nested_columns)); } +ColumnPtr DataTypeVariant::createColumnConst(size_t size, const DB::Field & field) const +{ + auto field_type = applyVisitor(FieldToDataType(), field); + auto discr = tryGetVariantDiscriminator(field_type); + if (!discr) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot insert field \"{}\" into column with type {}", toString(field), getName()); + auto column = createColumn(); + assert_cast(*column).insertIntoVariant(field, *discr); + return ColumnConst::create(std::move(column), size); +} Field DataTypeVariant::getDefault() const { diff --git a/src/DataTypes/DataTypeVariant.h b/src/DataTypes/DataTypeVariant.h index 60113a188b0..ca15dff1476 100644 --- a/src/DataTypes/DataTypeVariant.h +++ b/src/DataTypes/DataTypeVariant.h @@ -37,6 +37,7 @@ public: MutableColumnPtr createColumn() const override; + ColumnPtr createColumnConst(size_t size, const Field & field) const override; Field getDefault() const override; bool equals(const IDataType & rhs) const override; diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index ccdf54f57c3..4533c23a89f 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -150,7 +150,7 @@ public: /** Create ColumnConst for corresponding type, with specified size and value. */ - ColumnPtr createColumnConst(size_t size, const Field & field) const; + virtual ColumnPtr createColumnConst(size_t size, const Field & field) const; ColumnPtr createColumnConstWithDefaultValue(size_t size) const; /** Get default value of data type. diff --git a/src/DataTypes/Serializations/SerializationVariantElement.cpp b/src/DataTypes/Serializations/SerializationVariantElement.cpp index 4b24ee5754e..1c0808db2a0 100644 --- a/src/DataTypes/Serializations/SerializationVariantElement.cpp +++ b/src/DataTypes/Serializations/SerializationVariantElement.cpp @@ -204,7 +204,7 @@ ColumnPtr SerializationVariantElement::VariantSubcolumnCreator::create(const DB: /// If this variant is empty, fill result column with default values. if (prev->empty()) { - auto res = IColumn::mutate(makeNullableOrLowCardinalityNullableSafe(prev)); + auto res = makeNullableOrLowCardinalityNullableSafe(prev)->cloneEmpty(); res->insertManyDefaults(local_discriminators->size()); return res; } From 4f8789927db4dd0d9c79a80bebc805895d82297c Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 21 Dec 2023 15:53:21 +0000 Subject: [PATCH 024/264] Fix tests with analyzer, add more tests --- src/DataTypes/DataTypeVariant.cpp | 18 +- ...different_local_and_global_order.reference | 244 ++++++++++++++++++ ...e_with_different_local_and_global_order.sh | 82 ++++++ .../02944_variant_as_if_multi_if_result.sql | 1 + 4 files changed, 340 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.reference create mode 100755 tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.sh diff --git a/src/DataTypes/DataTypeVariant.cpp b/src/DataTypes/DataTypeVariant.cpp index 334ed2c7b10..0575f220f22 100644 --- a/src/DataTypes/DataTypeVariant.cpp +++ b/src/DataTypes/DataTypeVariant.cpp @@ -98,12 +98,20 @@ MutableColumnPtr DataTypeVariant::createColumn() const ColumnPtr DataTypeVariant::createColumnConst(size_t size, const DB::Field & field) const { - auto field_type = applyVisitor(FieldToDataType(), field); - auto discr = tryGetVariantDiscriminator(field_type); - if (!discr) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot insert field \"{}\" into column with type {}", toString(field), getName()); auto column = createColumn(); - assert_cast(*column).insertIntoVariant(field, *discr); + if (field.isNull()) + { + column->insertDefault(); + } + else + { + auto field_type = applyVisitor(FieldToDataType(), field); + auto discr = tryGetVariantDiscriminator(field_type); + if (!discr) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot insert field \"{}\" into column with type {}", toString(field), getName()); + assert_cast(*column).insertIntoVariant(field, *discr); + } + return ColumnConst::create(std::move(column), size); } diff --git a/tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.reference b/tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.reference new file mode 100644 index 00000000000..f2e355824f9 --- /dev/null +++ b/tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.reference @@ -0,0 +1,244 @@ +Memory +test1 insert +test1 select +0 \N 0 +1 \N 1 +2 \N 2 +3 \N 3 +4 \N 4 +5 \N 5 +6 \N 6 +7 \N 7 +8 \N 8 +9 \N 9 +10 \N 10 +\N \N \N +12 \N 12 +\N \N \N +14 \N 14 +\N \N \N +16 \N 16 +\N \N \N +18 \N 18 +\N \N \N +str_20 str_20 \N +\N \N \N +str_22 str_22 \N +\N \N \N +str_24 str_24 \N +\N \N \N +str_26 str_26 \N +\N \N \N +str_28 str_28 \N +\N \N \N +30 \N 30 +\N \N \N +32 \N 32 +\N \N \N +34 \N 34 +\N \N \N +str_36 str_36 \N +\N \N \N +str_38 str_38 \N +\N \N \N +----------------------------------------------------------------------------------------------------------- +test2 insert +test2 select +7000000 +1000000 +6000000 +----------------------------------------------------------------------------------------------------------- +MergeTree compact +test1 insert +test1 select +0 \N 0 +1 \N 1 +2 \N 2 +3 \N 3 +4 \N 4 +5 \N 5 +6 \N 6 +7 \N 7 +8 \N 8 +9 \N 9 +10 \N 10 +\N \N \N +12 \N 12 +\N \N \N +14 \N 14 +\N \N \N +16 \N 16 +\N \N \N +18 \N 18 +\N \N \N +str_20 str_20 \N +\N \N \N +str_22 str_22 \N +\N \N \N +str_24 str_24 \N +\N \N \N +str_26 str_26 \N +\N \N \N +str_28 str_28 \N +\N \N \N +30 \N 30 +\N \N \N +32 \N 32 +\N \N \N +34 \N 34 +\N \N \N +str_36 str_36 \N +\N \N \N +str_38 str_38 \N +\N \N \N +----------------------------------------------------------------------------------------------------------- +test1 select +0 \N 0 +1 \N 1 +2 \N 2 +3 \N 3 +4 \N 4 +5 \N 5 +6 \N 6 +7 \N 7 +8 \N 8 +9 \N 9 +10 \N 10 +\N \N \N +12 \N 12 +\N \N \N +14 \N 14 +\N \N \N +16 \N 16 +\N \N \N +18 \N 18 +\N \N \N +str_20 str_20 \N +\N \N \N +str_22 str_22 \N +\N \N \N +str_24 str_24 \N +\N \N \N +str_26 str_26 \N +\N \N \N +str_28 str_28 \N +\N \N \N +30 \N 30 +\N \N \N +32 \N 32 +\N \N \N +34 \N 34 +\N \N \N +str_36 str_36 \N +\N \N \N +str_38 str_38 \N +\N \N \N +----------------------------------------------------------------------------------------------------------- +test2 insert +test2 select +7000000 +1000000 +6000000 +----------------------------------------------------------------------------------------------------------- +test2 select +7000000 +1000000 +6000000 +----------------------------------------------------------------------------------------------------------- +MergeTree wide +test1 insert +test1 select +0 \N 0 +1 \N 1 +2 \N 2 +3 \N 3 +4 \N 4 +5 \N 5 +6 \N 6 +7 \N 7 +8 \N 8 +9 \N 9 +10 \N 10 +\N \N \N +12 \N 12 +\N \N \N +14 \N 14 +\N \N \N +16 \N 16 +\N \N \N +18 \N 18 +\N \N \N +str_20 str_20 \N +\N \N \N +str_22 str_22 \N +\N \N \N +str_24 str_24 \N +\N \N \N +str_26 str_26 \N +\N \N \N +str_28 str_28 \N +\N \N \N +30 \N 30 +\N \N \N +32 \N 32 +\N \N \N +34 \N 34 +\N \N \N +str_36 str_36 \N +\N \N \N +str_38 str_38 \N +\N \N \N +----------------------------------------------------------------------------------------------------------- +test1 select +0 \N 0 +1 \N 1 +2 \N 2 +3 \N 3 +4 \N 4 +5 \N 5 +6 \N 6 +7 \N 7 +8 \N 8 +9 \N 9 +10 \N 10 +\N \N \N +12 \N 12 +\N \N \N +14 \N 14 +\N \N \N +16 \N 16 +\N \N \N +18 \N 18 +\N \N \N +str_20 str_20 \N +\N \N \N +str_22 str_22 \N +\N \N \N +str_24 str_24 \N +\N \N \N +str_26 str_26 \N +\N \N \N +str_28 str_28 \N +\N \N \N +30 \N 30 +\N \N \N +32 \N 32 +\N \N \N +34 \N 34 +\N \N \N +str_36 str_36 \N +\N \N \N +str_38 str_38 \N +\N \N \N +----------------------------------------------------------------------------------------------------------- +test2 insert +test2 select +7000000 +1000000 +6000000 +----------------------------------------------------------------------------------------------------------- +test2 select +7000000 +1000000 +6000000 +----------------------------------------------------------------------------------------------------------- diff --git a/tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.sh b/tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.sh new file mode 100755 index 00000000000..88bd2d3bd42 --- /dev/null +++ b/tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.sh @@ -0,0 +1,82 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# reset --log_comment +CLICKHOUSE_LOG_COMMENT= +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_when_no_common_type_in_if=1 " + + +function test1_insert() +{ + echo "test1 insert" + $CH_CLIENT -q "insert into test select number, number::Variant(UInt64)::Variant(UInt64, Array(UInt64)) from numbers(10) settings max_block_size=3" + $CH_CLIENT -q "insert into test select number, if(number % 2, NULL, number)::Variant(UInt64)::Variant(UInt64, String, Array(UInt64)) as res from numbers(10, 10) settings max_block_size=3" + $CH_CLIENT -q "insert into test select number, if(number % 2, NULL, 'str_' || toString(number))::Variant(String)::Variant(UInt64, String, Array(UInt64)) as res from numbers(20, 10) settings max_block_size=3" + $CH_CLIENT -q "insert into test select number, if(number < 35, if(number % 2, NULL, number)::Variant(UInt64)::Variant(UInt64, String, Array(UInt64)), if(number % 2, NULL, 'str_' || toString(number))::Variant(String)::Variant(UInt64, String, Array(UInt64))) from numbers(30, 10) settings max_block_size=3" +} + +function test1_select() +{ + echo "test1 select" + $CH_CLIENT -q "select v, v.String, v.UInt64 from test order by id;" + echo "-----------------------------------------------------------------------------------------------------------" +} + +function test2_insert() +{ + echo "test2 insert" + $CH_CLIENT -q "insert into test select number, number::Variant(UInt64)::Variant(UInt64, Array(UInt64)) from numbers(1000000) settings max_insert_block_size = 100000, min_insert_block_size_rows=100000" + $CH_CLIENT -q "insert into test select number, if(number % 2, NULL, number)::Variant(UInt64)::Variant(UInt64, String, Array(UInt64)) as res from numbers(1000000, 10000000) settings max_insert_block_size = 100000, min_insert_block_size_rows=100000" + $CH_CLIENT -q "insert into test select number, if(number % 2, NULL, 'str_' || toString(number))::Variant(String)::Variant(UInt64, String, Array(UInt64)) as res from numbers(2000000, 1000000) settings max_insert_block_size = 100000, min_insert_block_size_rows=100000" + $CH_CLIENT -q "insert into test select number, if(number < 5, if(number % 2, NULL, number)::Variant(UInt64)::Variant(UInt64, String, Array(UInt64)), if(number % 2, NULL, 'str_' || toString(number))::Variant(String)::Variant(UInt64, String, Array(UInt64))) from numbers(3000000, 1000000) settings max_insert_block_size = 100000, min_insert_block_size_rows=100000" + } + +function test2_select() +{ + echo "test2 select" + $CH_CLIENT -q "select v from test format Null;" + $CH_CLIENT -q "select count() from test where isNotNull(v);" + $CH_CLIENT -q "select v.String from test format Null;" + $CH_CLIENT -q "select count() from test where isNotNull(v.String);" + $CH_CLIENT -q "select v.UInt64 from test format Null;" + $CH_CLIENT -q "select count() from test where isNotNull(v.UInt64);" + echo "-----------------------------------------------------------------------------------------------------------" +} + +function run() +{ + test1_insert + test1_select + if [ $1 == 1 ]; then + $CH_CLIENT -q "optimize table test final;" + test1_select + fi + $CH_CLIENT -q "truncate table test;" + test2_insert + test2_select + if [ $1 == 1 ]; then + $CH_CLIENT -q "optimize table test final;" + test2_select + fi + $CH_CLIENT -q "truncate table test;" +} + +$CH_CLIENT -q "drop table if exists test;" + +echo "Memory" +$CH_CLIENT -q "create table test (id UInt64, v Variant(UInt64, String, Array(UInt64))) engine=Memory;" +run 0 +$CH_CLIENT -q "drop table test;" + +echo "MergeTree compact" +$CH_CLIENT -q "create table test (id UInt64, v Variant(UInt64, String, Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000;" +run 1 +$CH_CLIENT -q "drop table test;" + +echo "MergeTree wide" +$CH_CLIENT -q "create table test (id UInt64, v Variant(UInt64, String, Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" +run 1 +$CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/02944_variant_as_if_multi_if_result.sql b/tests/queries/0_stateless/02944_variant_as_if_multi_if_result.sql index da36863bfda..1121b21e383 100644 --- a/tests/queries/0_stateless/02944_variant_as_if_multi_if_result.sql +++ b/tests/queries/0_stateless/02944_variant_as_if_multi_if_result.sql @@ -1,3 +1,4 @@ +set allow_experimental_analyzer=0; -- The result type for if function with constant is different with analyzer. set allow_experimental_variant_type=1; set use_variant_when_no_common_type_in_if=1; From 38ec9b5f719740b4e94758f9e5578acd562df939 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 22 Dec 2023 00:11:39 +0000 Subject: [PATCH 025/264] Fix variant element deserialization --- .../Serializations/SerializationVariant.cpp | 27 ++-- .../SerializationVariantElement.cpp | 149 ++++++++++-------- ...different_local_and_global_order.reference | 30 ++-- ...e_with_different_local_and_global_order.sh | 8 +- 4 files changed, 117 insertions(+), 97 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationVariant.cpp b/src/DataTypes/Serializations/SerializationVariant.cpp index ebd44fd6955..910ad1da303 100644 --- a/src/DataTypes/Serializations/SerializationVariant.cpp +++ b/src/DataTypes/Serializations/SerializationVariant.cpp @@ -277,13 +277,10 @@ void SerializationVariant::deserializeBinaryBulkWithMultipleStreams( throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to deserialize data into Variant column with not global variants order"); /// First, deserialize new discriminators. - /// We deserialize them into a separate column to be able to use substream cache, - /// so if we also need to deserialize some of sub columns, we will read discriminators only once. settings.path.push_back(Substream::VariantDiscriminators); - ColumnPtr discriminators; if (auto cached_discriminators = getFromSubstreamsCache(cache, settings.path)) { - discriminators = cached_discriminators; + col.getLocalDiscriminatorsPtr() = cached_discriminators; } else { @@ -291,29 +288,31 @@ void SerializationVariant::deserializeBinaryBulkWithMultipleStreams( if (!discriminators_stream) throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty stream for VariantDiscriminators in SerializationVariant::deserializeBinaryBulkWithMultipleStreams"); - discriminators = ColumnVariant::ColumnDiscriminators::create(); - SerializationNumber().deserializeBinaryBulk(*discriminators->assumeMutable(), *discriminators_stream, limit, 0); - addToSubstreamsCache(cache, settings.path, discriminators); + SerializationNumber().deserializeBinaryBulk(*col.getLocalDiscriminatorsPtr()->assumeMutable(), *discriminators_stream, limit, 0); + addToSubstreamsCache(cache, settings.path, col.getLocalDiscriminatorsPtr()); } settings.path.pop_back(); - /// Iterate through new discriminators, append them to column and calculate the limit for each variant. + /// Iterate through new discriminators and calculate the limit for each variant. /// While calculating limits we can also fill offsets column (we store offsets only in memory). - const auto & discriminators_data = assert_cast(*discriminators).getData(); - auto & local_discriminators = col.getLocalDiscriminators(); - local_discriminators.reserve(local_discriminators.size() + limit); + auto & discriminators_data = col.getLocalDiscriminators(); auto & offsets = col.getOffsets(); offsets.reserve(offsets.size() + limit); std::vector variant_limits(variants.size(), 0); - for (size_t i = 0; i != limit; ++i) + size_t discriminators_offset = discriminators_data.size() - limit; + for (size_t i = discriminators_offset; i != discriminators_data.size(); ++i) { ColumnVariant::Discriminator discr = discriminators_data[i]; - local_discriminators.push_back(discr); if (discr == ColumnVariant::NULL_DISCRIMINATOR) + { offsets.emplace_back(); + } else - offsets.push_back(col.getVariantByLocalDiscriminator(discr).size() + variant_limits[discr]++); + { + offsets.push_back(col.getVariantByLocalDiscriminator(discr).size() + variant_limits[discr]); + ++variant_limits[discr]; + } } /// Now we can deserialize variants according to their limits. diff --git a/src/DataTypes/Serializations/SerializationVariantElement.cpp b/src/DataTypes/Serializations/SerializationVariantElement.cpp index 1c0808db2a0..e06a20d2990 100644 --- a/src/DataTypes/Serializations/SerializationVariantElement.cpp +++ b/src/DataTypes/Serializations/SerializationVariantElement.cpp @@ -40,11 +40,31 @@ void SerializationVariantElement::serializeBinaryBulkStateSuffix(SerializeBinary ErrorCodes::NOT_IMPLEMENTED, "Method serializeBinaryBulkStateSuffix is not implemented for SerializationVariantElement"); } +struct DeserializeBinaryBulkStateVariantElement : public ISerialization::DeserializeBinaryBulkState +{ + /// During deserialization discriminators and variant streams can be shared. + /// For example we can read several variant elements together: "select v.UInt32, v.String from table", + /// or we can read the whole variant and some of variant elements: "select v, v.UInt32 from table". + /// To read the same column from the same stream more than once we use substream cache, + /// but this cache stores the whole column, not only the current range. + /// During deserialization of variant element discriminators and variant columns are not stored + /// in the result column, so we need to store them inside deserialization state, so we can use + /// substream cache correctly. + ColumnPtr discriminators; + ColumnPtr variant; + + ISerialization::DeserializeBinaryBulkStatePtr variant_element_state; +}; + void SerializationVariantElement::deserializeBinaryBulkStatePrefix(DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const { + auto variant_element_state = std::make_shared(); + addVariantToPath(settings.path); - nested_serialization->deserializeBinaryBulkStatePrefix(settings, state); + nested_serialization->deserializeBinaryBulkStatePrefix(settings, variant_element_state->variant_element_state); removeVariantFromPath(settings.path); + + state = std::move(variant_element_state); } void SerializationVariantElement::serializeBinaryBulkWithMultipleStreams(const IColumn &, size_t, size_t, SerializeBinaryBulkSettings &, SerializeBinaryBulkStatePtr &) const @@ -53,22 +73,19 @@ void SerializationVariantElement::serializeBinaryBulkWithMultipleStreams(const I } void SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams( - ColumnPtr & column, + ColumnPtr & result_column, size_t limit, DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state, SubstreamsCache * cache) const { - auto mutable_column = column->assumeMutable(); - ColumnNullable * nullable_col = typeid_cast(mutable_column.get()); - NullMap * null_map = nullable_col ? &nullable_col->getNullMapData() : nullptr; + auto * variant_element_state = checkAndGetState(state); /// First, deserialize discriminators from Variant column. settings.path.push_back(Substream::VariantDiscriminators); - ColumnPtr discriminators; if (auto cached_discriminators = getFromSubstreamsCache(cache, settings.path)) { - discriminators = cached_discriminators; + variant_element_state->discriminators = cached_discriminators; } else { @@ -76,85 +93,87 @@ void SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams( if (!discriminators_stream) throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty stream for VariantDiscriminators in SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams"); - discriminators = ColumnVariant::ColumnDiscriminators::create(); - SerializationNumber().deserializeBinaryBulk(*discriminators->assumeMutable(), *discriminators_stream, limit, 0); - addToSubstreamsCache(cache, settings.path, discriminators); + /// If we started to read a new column, reinitialize discriminators column in deserialization state. + if (!variant_element_state->discriminators || result_column->empty()) + variant_element_state->discriminators = ColumnVariant::ColumnDiscriminators::create(); + + SerializationNumber().deserializeBinaryBulk(*variant_element_state->discriminators->assumeMutable(), *discriminators_stream, limit, 0); + addToSubstreamsCache(cache, settings.path, variant_element_state->discriminators); } settings.path.pop_back(); - /// Iterate through discriminators to calculate the size of the variant. - const auto & discriminators_data = assert_cast(*discriminators).getData(); - size_t variant_size = 0; - for (auto discr : discriminators_data) - variant_size += discr == variant_discriminator; + /// Iterate through new discriminators to calculate the limit for our variant. + const auto & discriminators_data = assert_cast(*variant_element_state->discriminators).getData(); + size_t discriminators_offset = variant_element_state->discriminators->size() - limit; + size_t variant_limit = 0; + for (size_t i = discriminators_offset; i != discriminators_data.size(); ++i) + variant_limit += (discriminators_data[i] == variant_discriminator); - /// Now we know the size of the variant and can deserialize it. + /// Now we know the limit for our variant and can deserialize it. - /// If the size of variant column is the same as the size of discriminators, - /// we can deserialize new values directly into our column. - if (variant_size == discriminators_data.size()) + /// If result column is Nullable, fill null map and extract nested column. + MutableColumnPtr mutable_column = result_column->assumeMutable(); + if (isColumnNullable(*mutable_column)) { - addVariantToPath(settings.path); - /// Special case when our result column is LowCardinality(Nullable(T)). - /// In this case the variant type is LowCardinality(T), and we cannot just - /// deserialize its values directly into LowCardinality(Nullable(T)) column. - /// We create a separate column with type LowCardinality(T), deserialize - /// values into it and then insert into result column using insertRangeFrom. - if (isColumnLowCardinalityNullable(*column)) + auto & nullable_column = assert_cast(*mutable_column); + NullMap & null_map = nullable_column.getNullMapData(); + /// If we have only our discriminator in range, fill null map with 0. + if (variant_limit == limit) { - ColumnPtr variant_col = mutable_column->cloneEmpty(); - /// LowCardinality(Nullable(T)) -> LowCardinality(T) - assert_cast(*variant_col->assumeMutable()).nestedRemoveNullable(); - nested_serialization->deserializeBinaryBulkWithMultipleStreams(variant_col, limit, settings, state, cache); - mutable_column->insertRangeFrom(*variant_col, 0, variant_col->size()); + null_map.resize_fill(null_map.size() + limit, 0); } + /// If no our discriminator in current range, fill null map with 1. + else if (variant_limit == 0) + { + null_map.resize_fill(null_map.size() + limit, 1); + } + /// Otherwise we should iterate through discriminators to fill null map. else { - nested_serialization->deserializeBinaryBulkWithMultipleStreams(nullable_col ? nullable_col->getNestedColumnPtr() : column, limit, settings, state, cache); + null_map.reserve(null_map.size() + limit); + for (size_t i = discriminators_offset; i != discriminators_data.size(); ++i) + null_map.push_back(discriminators_data[i] != variant_discriminator); } - if (nullable_col) - null_map->resize_fill(null_map->size() + limit, 0); - removeVariantFromPath(settings.path); - return; + + mutable_column = nullable_column.getNestedColumnPtr()->assumeMutable(); } - /// If variant size is 0, just fill column with default values. - if (variant_size == 0) + /// If we started to read a new column, reinitialize variant column in deserialization state. + if (!variant_element_state->variant || result_column->empty()) { - mutable_column->insertManyDefaults(limit); - return; + variant_element_state->variant = mutable_column->cloneEmpty(); + + /// When result column is LowCardinality(Nullable(T)) we should + /// remove Nullable from variant column before deserialization. + if (isColumnLowCardinalityNullable(*mutable_column)) + assert_cast(*variant_element_state->variant->assumeMutable()).nestedRemoveNullable(); } - /// In general case we should deserialize variant into a separate column, - /// iterate through discriminators and insert values from variant only when - /// row contains its discriminator and default value otherwise. - mutable_column->reserve(mutable_column->size() + limit); - mutable_column = nullable_col ? nullable_col->getNestedColumnPtr()->assumeMutable() : std::move(mutable_column); - ColumnPtr variant_col = mutable_column->cloneEmpty(); - - /// Special case when our result column is LowCardinality(Nullable(T)). - /// We should remove Nullable from variant column before deserialization. - if (isColumnLowCardinalityNullable(*column)) - assert_cast(*variant_col->assumeMutable()).nestedRemoveNullable(); - addVariantToPath(settings.path); - nested_serialization->deserializeBinaryBulkWithMultipleStreams(variant_col, variant_size, settings, state, cache); + nested_serialization->deserializeBinaryBulkWithMultipleStreams(variant_element_state->variant, variant_limit, settings, variant_element_state->variant_element_state, cache); removeVariantFromPath(settings.path); - size_t variant_index = 0; - for (auto discr : discriminators_data) + size_t variant_offset = variant_element_state->variant->size() - variant_limit; + + /// If don't have our discriminator in range, just insert defaults. + if (variant_limit == 0) { - if (discr == variant_discriminator) + mutable_column->insertManyDefaults(limit); + } + /// If we have only our discriminator in range, insert the whole range to result column. + else if (variant_limit == limit) + { + mutable_column->insertRangeFrom(*variant_element_state->variant, variant_offset, variant_limit); + } + /// Otherwise iterate through discriminators and insert value from variant or default value depending on the discriminator. + else + { + for (size_t i = discriminators_offset; i != discriminators_data.size(); ++i) { - if (null_map) - null_map->push_back(0); - mutable_column->insertFrom(*variant_col, variant_index++); - } - else - { - if (null_map) - null_map->push_back(1); - mutable_column->insertDefault(); + if (discriminators_data[i] == variant_discriminator) + mutable_column->insertFrom(*variant_element_state->variant, variant_offset++); + else + mutable_column->insertDefault(); } } } diff --git a/tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.reference b/tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.reference index f2e355824f9..1736a307c42 100644 --- a/tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.reference +++ b/tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.reference @@ -44,9 +44,9 @@ str_38 str_38 \N ----------------------------------------------------------------------------------------------------------- test2 insert test2 select -7000000 -1000000 -6000000 +2500000 +750000 +1750000 ----------------------------------------------------------------------------------------------------------- MergeTree compact test1 insert @@ -136,14 +136,14 @@ str_38 str_38 \N ----------------------------------------------------------------------------------------------------------- test2 insert test2 select -7000000 -1000000 -6000000 +2500000 +750000 +1750000 ----------------------------------------------------------------------------------------------------------- test2 select -7000000 -1000000 -6000000 +2500000 +750000 +1750000 ----------------------------------------------------------------------------------------------------------- MergeTree wide test1 insert @@ -233,12 +233,12 @@ str_38 str_38 \N ----------------------------------------------------------------------------------------------------------- test2 insert test2 select -7000000 -1000000 -6000000 +2500000 +750000 +1750000 ----------------------------------------------------------------------------------------------------------- test2 select -7000000 -1000000 -6000000 +2500000 +750000 +1750000 ----------------------------------------------------------------------------------------------------------- diff --git a/tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.sh b/tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.sh index 88bd2d3bd42..9f4df8d7466 100755 --- a/tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.sh +++ b/tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: long CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # reset --log_comment @@ -29,14 +30,15 @@ function test2_insert() { echo "test2 insert" $CH_CLIENT -q "insert into test select number, number::Variant(UInt64)::Variant(UInt64, Array(UInt64)) from numbers(1000000) settings max_insert_block_size = 100000, min_insert_block_size_rows=100000" - $CH_CLIENT -q "insert into test select number, if(number % 2, NULL, number)::Variant(UInt64)::Variant(UInt64, String, Array(UInt64)) as res from numbers(1000000, 10000000) settings max_insert_block_size = 100000, min_insert_block_size_rows=100000" + $CH_CLIENT -q "insert into test select number, if(number % 2, NULL, number)::Variant(UInt64)::Variant(UInt64, String, Array(UInt64)) as res from numbers(1000000, 1000000) settings max_insert_block_size = 100000, min_insert_block_size_rows=100000" $CH_CLIENT -q "insert into test select number, if(number % 2, NULL, 'str_' || toString(number))::Variant(String)::Variant(UInt64, String, Array(UInt64)) as res from numbers(2000000, 1000000) settings max_insert_block_size = 100000, min_insert_block_size_rows=100000" - $CH_CLIENT -q "insert into test select number, if(number < 5, if(number % 2, NULL, number)::Variant(UInt64)::Variant(UInt64, String, Array(UInt64)), if(number % 2, NULL, 'str_' || toString(number))::Variant(String)::Variant(UInt64, String, Array(UInt64))) from numbers(3000000, 1000000) settings max_insert_block_size = 100000, min_insert_block_size_rows=100000" - } + $CH_CLIENT -q "insert into test select number, if(number < 3500000, if(number % 2, NULL, number)::Variant(UInt64)::Variant(UInt64, String, Array(UInt64)), if(number % 2, NULL, 'str_' || toString(number))::Variant(String)::Variant(UInt64, String, Array(UInt64))) from numbers(3000000, 1000000) settings max_insert_block_size = 100000, min_insert_block_size_rows=100000" +} function test2_select() { echo "test2 select" + $CH_CLIENT -q "select v, v.String, v.UInt64 from test format Null;" $CH_CLIENT -q "select v from test format Null;" $CH_CLIENT -q "select count() from test where isNotNull(v);" $CH_CLIENT -q "select v.String from test format Null;" From 319c20091efe8eebee5bde9bb8bae67e58a589d9 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 22 Dec 2023 00:15:44 +0000 Subject: [PATCH 026/264] Fix comments --- src/DataTypes/Serializations/SerializationVariant.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationVariant.cpp b/src/DataTypes/Serializations/SerializationVariant.cpp index 910ad1da303..3b51c51872f 100644 --- a/src/DataTypes/Serializations/SerializationVariant.cpp +++ b/src/DataTypes/Serializations/SerializationVariant.cpp @@ -276,7 +276,7 @@ void SerializationVariant::deserializeBinaryBulkWithMultipleStreams( if (!col.hasGlobalVariantsOrder()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to deserialize data into Variant column with not global variants order"); - /// First, deserialize new discriminators. + /// First, deserialize discriminators. settings.path.push_back(Substream::VariantDiscriminators); if (auto cached_discriminators = getFromSubstreamsCache(cache, settings.path)) { @@ -451,7 +451,7 @@ std::unordered_map getTypesTextDeserializePriorityMap() /// then for types with the same depth we sort by the types priority, and last we sort by the depth of LowCardinality/Nullable types, /// so if we have types with the same level of nesting and the same priority, we will first try to deserialize LowCardinality/Nullable types /// (for example if we have types Array(Array(String)) and Array(Array(Nullable(String))). -/// This is just a batch of heuristics, +/// This is just a batch of heuristics. std::tuple getTypeTextDeserializePriority(const DataTypePtr & type, size_t nested_depth, size_t simple_nested_depth, std::unordered_map & priority_map) { if (const auto * nullable_type = typeid_cast(type.get())) @@ -553,7 +553,7 @@ bool SerializationVariant::tryDeserializeImpl( for (size_t global_discr : deserialize_text_order) { ReadBufferFromString variant_buf(field); - /// Usually try_deserialize_variant should not throw an exception, but let's use try/catch just in case. + /// Usually try_deserialize_variant should not throw any exception, but let's use try/catch just in case. try { auto & variant_column = column_variant.getVariantByGlobalDiscriminator(global_discr); From a61efedba8854e8f06b549deb595315ee40eb303 Mon Sep 17 00:00:00 2001 From: avogar Date: Sat, 23 Dec 2023 00:10:58 +0000 Subject: [PATCH 027/264] Fix serialization again, add more tests --- src/Columns/ColumnVariant.h | 2 +- src/Columns/ColumnVector.cpp | 2 +- src/DataTypes/DataTypeVariant.cpp | 15 +++++- .../Serializations/ISerialization.cpp | 7 +++ src/DataTypes/Serializations/ISerialization.h | 1 + .../Serializations/SerializationVariant.cpp | 52 ++++++++++++++----- .../02943_variant_read_subcolumns_1.reference | 6 +++ .../02943_variant_read_subcolumns_1.sh | 38 ++++++++++++++ .../02943_variant_read_subcolumns_2.reference | 6 +++ .../02943_variant_read_subcolumns_2.sh | 38 ++++++++++++++ 10 files changed, 150 insertions(+), 17 deletions(-) create mode 100644 tests/queries/0_stateless/02943_variant_read_subcolumns_1.reference create mode 100755 tests/queries/0_stateless/02943_variant_read_subcolumns_1.sh create mode 100644 tests/queries/0_stateless/02943_variant_read_subcolumns_2.reference create mode 100755 tests/queries/0_stateless/02943_variant_read_subcolumns_2.sh diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h index b388b118a69..ec58553f5f3 100644 --- a/src/Columns/ColumnVariant.h +++ b/src/Columns/ColumnVariant.h @@ -163,7 +163,7 @@ public: size_t size() const override { - return local_discriminators->size(); + return offsets->size(); } Field operator[](size_t n) const override; diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 37e62c76596..b4e3fee5e42 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -469,7 +469,7 @@ void ColumnVector::insertRangeFrom(const IColumn & src, size_t start, size_t const ColumnVector & src_vec = assert_cast(src); if (start + length > src_vec.data.size()) - throw Exception(ErrorCodes::PARAMETER_OUT_OF_BOUND, + throw Exception(ErrorCodes::LOGICAL_ERROR, "Parameters start = {}, length = {} are out of bound " "in ColumnVector::insertRangeFrom method (data.size() = {}).", toString(start), toString(length), toString(src_vec.data.size())); diff --git a/src/DataTypes/DataTypeVariant.cpp b/src/DataTypes/DataTypeVariant.cpp index 0575f220f22..5dc42cc7443 100644 --- a/src/DataTypes/DataTypeVariant.cpp +++ b/src/DataTypes/DataTypeVariant.cpp @@ -108,7 +108,20 @@ ColumnPtr DataTypeVariant::createColumnConst(size_t size, const DB::Field & fiel auto field_type = applyVisitor(FieldToDataType(), field); auto discr = tryGetVariantDiscriminator(field_type); if (!discr) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot insert field \"{}\" into column with type {}", toString(field), getName()); + { + for (size_t i = 0; i != variants.size(); ++i) + { + if (field.getType() == variants[i]->getDefault().getType()) + { + discr = i; + break; + } + } + } + + if (!discr) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot insert field \"{}\" with type {} into column with type {}", toString(field), field.getTypeName(), getName()); + assert_cast(*column).insertIntoVariant(field, *discr); } diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index 86a37949dc8..46353fffb48 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -124,15 +124,20 @@ void ISerialization::deserializeBinaryBulkWithMultipleStreams( DeserializeBinaryBulkStatePtr & /* state */, SubstreamsCache * cache) const { + LOG_DEBUG(&Poco::Logger::get("ISerialization"), "Deserialize path {}. Initial column size: {}", settings.path.toString(), column->size()); + auto cached_column = getFromSubstreamsCache(cache, settings.path); if (cached_column) { column = cached_column; + LOG_DEBUG(&Poco::Logger::get("ISerialization"), "Use column from cache. Size: {}", cached_column->size()); } else if (ReadBuffer * stream = settings.getter(settings.path)) { auto mutable_column = column->assumeMutable(); + LOG_DEBUG(&Poco::Logger::get("ISerialization"), "Deserialize column. Initial size: {}", mutable_column->size()); deserializeBinaryBulk(*mutable_column, *stream, limit, settings.avg_value_size_hint); + LOG_DEBUG(&Poco::Logger::get("ISerialization"), "Deserialized column. Size: {}", mutable_column->size()); column = std::move(mutable_column); addToSubstreamsCache(cache, settings.path, column); } @@ -177,6 +182,8 @@ String getNameForSubstreamPath( } else if (it->type == Substream::VariantDiscriminators) stream_name += ".discr"; + else if (it->type == Substream::VariantOffsets) + stream_name += ".variant_offsets"; else if (it->type == Substream::VariantElement) stream_name += "." + it->variant_element_name; } diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index f0273f59d1f..5c6fe31ed9e 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -153,6 +153,7 @@ public: ObjectData, VariantDiscriminators, + VariantOffsets, VariantElements, VariantElement, diff --git a/src/DataTypes/Serializations/SerializationVariant.cpp b/src/DataTypes/Serializations/SerializationVariant.cpp index 3b51c51872f..d36151fe8e9 100644 --- a/src/DataTypes/Serializations/SerializationVariant.cpp +++ b/src/DataTypes/Serializations/SerializationVariant.cpp @@ -291,28 +291,17 @@ void SerializationVariant::deserializeBinaryBulkWithMultipleStreams( SerializationNumber().deserializeBinaryBulk(*col.getLocalDiscriminatorsPtr()->assumeMutable(), *discriminators_stream, limit, 0); addToSubstreamsCache(cache, settings.path, col.getLocalDiscriminatorsPtr()); } - settings.path.pop_back(); - /// Iterate through new discriminators and calculate the limit for each variant. - /// While calculating limits we can also fill offsets column (we store offsets only in memory). - auto & discriminators_data = col.getLocalDiscriminators(); - auto & offsets = col.getOffsets(); - offsets.reserve(offsets.size() + limit); + /// Second, calculate limits for each variant by iterating through new discriminators. std::vector variant_limits(variants.size(), 0); + auto & discriminators_data = col.getLocalDiscriminators(); size_t discriminators_offset = discriminators_data.size() - limit; for (size_t i = discriminators_offset; i != discriminators_data.size(); ++i) { ColumnVariant::Discriminator discr = discriminators_data[i]; - if (discr == ColumnVariant::NULL_DISCRIMINATOR) - { - offsets.emplace_back(); - } - else - { - offsets.push_back(col.getVariantByLocalDiscriminator(discr).size() + variant_limits[discr]); + if (discr != ColumnVariant::NULL_DISCRIMINATOR) ++variant_limits[discr]; - } } /// Now we can deserialize variants according to their limits. @@ -325,6 +314,41 @@ void SerializationVariant::deserializeBinaryBulkWithMultipleStreams( settings.path.pop_back(); } settings.path.pop_back(); + + /// Fill offsets column. + /// It's important to do it after deserialization of all variants, because to fill offsets we need + /// initial variants sizes without values in current range, but some variants can be shared with + /// other columns via substream cache and they can already contain values from this range even + /// before we call deserialize for them. So, before deserialize we cannot know for sure if + /// variant columns already contain values from current range or not. But after calling deserialize + /// we know for sure that they contain these values, so we can use valiant limits and their + /// new sizes to calculate correct offsets. + settings.path.push_back(Substream::VariantOffsets); + if (auto cached_offsets = getFromSubstreamsCache(cache, settings.path)) + { + col.getOffsetsPtr() = cached_offsets; + } + else + { + auto & offsets = col.getOffsets(); + offsets.reserve(offsets.size() + limit); + std::vector variant_offsets; + variant_offsets.reserve(variants.size()); + for (size_t i = 0; i != variants.size(); ++i) + variant_offsets.push_back(col.getVariantByLocalDiscriminator(i).size() - variant_limits[i]); + + for (size_t i = discriminators_offset; i != discriminators_data.size(); ++i) + { + ColumnVariant::Discriminator discr = discriminators_data[i]; + if (discr == ColumnVariant::NULL_DISCRIMINATOR) + offsets.emplace_back(); + else + offsets.push_back(variant_offsets[discr]++); + } + + addToSubstreamsCache(cache, settings.path, col.getOffsetsPtr()); + } + settings.path.pop_back(); } void SerializationVariant::addVariantElementToPath(DB::ISerialization::SubstreamPath & path, size_t i) const diff --git a/tests/queries/0_stateless/02943_variant_read_subcolumns_1.reference b/tests/queries/0_stateless/02943_variant_read_subcolumns_1.reference new file mode 100644 index 00000000000..4b93782cddf --- /dev/null +++ b/tests/queries/0_stateless/02943_variant_read_subcolumns_1.reference @@ -0,0 +1,6 @@ +Memory +test +MergeTree compact +test +MergeTree wide +test diff --git a/tests/queries/0_stateless/02943_variant_read_subcolumns_1.sh b/tests/queries/0_stateless/02943_variant_read_subcolumns_1.sh new file mode 100755 index 00000000000..9ccad55191f --- /dev/null +++ b/tests/queries/0_stateless/02943_variant_read_subcolumns_1.sh @@ -0,0 +1,38 @@ +#!/usr/bin/env bash +# Tags: long + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# reset --log_comment +CLICKHOUSE_LOG_COMMENT= +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_when_no_common_type_in_if=1 " + + +function test() +{ + echo "test" + $CH_CLIENT -q "insert into test select number, multiIf(number % 3 == 2, NULL, number % 3 == 1, number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10))) from numbers(1000000) settings min_insert_block_size_rows=100000" + $CH_CLIENT -q "select v, v.UInt64, v.\`Array(Variant(String, UInt64))\`, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64 from test order by id format Null" + $CH_CLIENT -q "select v.UInt64, v.\`Array(Variant(String, UInt64))\`, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64 from test order by id format Null" + $CH_CLIENT -q "select v.\`Array(Variant(String, UInt64))\`, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64, v.\`Array(Variant(String, UInt64))\`.String from test order by id format Null" +} + +$CH_CLIENT -q "drop table if exists test;" + +echo "Memory" +$CH_CLIENT -q "create table test (id UInt64, v Variant(UInt64, Array(Variant(String, UInt64)))) engine=Memory" +test +$CH_CLIENT -q "drop table test;" + +echo "MergeTree compact" +$CH_CLIENT -q "create table test (id UInt64, v Variant(UInt64, Array(Variant(String, UInt64)))) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000;" +test +$CH_CLIENT -q "drop table test;" + +echo "MergeTree wide" +$CH_CLIENT -q "create table test (id UInt64, v Variant(UInt64, Array(Variant(String, UInt64)))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" +test +$CH_CLIENT -q "drop table test;" + diff --git a/tests/queries/0_stateless/02943_variant_read_subcolumns_2.reference b/tests/queries/0_stateless/02943_variant_read_subcolumns_2.reference new file mode 100644 index 00000000000..4b93782cddf --- /dev/null +++ b/tests/queries/0_stateless/02943_variant_read_subcolumns_2.reference @@ -0,0 +1,6 @@ +Memory +test +MergeTree compact +test +MergeTree wide +test diff --git a/tests/queries/0_stateless/02943_variant_read_subcolumns_2.sh b/tests/queries/0_stateless/02943_variant_read_subcolumns_2.sh new file mode 100755 index 00000000000..9ccad55191f --- /dev/null +++ b/tests/queries/0_stateless/02943_variant_read_subcolumns_2.sh @@ -0,0 +1,38 @@ +#!/usr/bin/env bash +# Tags: long + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# reset --log_comment +CLICKHOUSE_LOG_COMMENT= +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_when_no_common_type_in_if=1 " + + +function test() +{ + echo "test" + $CH_CLIENT -q "insert into test select number, multiIf(number % 3 == 2, NULL, number % 3 == 1, number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10))) from numbers(1000000) settings min_insert_block_size_rows=100000" + $CH_CLIENT -q "select v, v.UInt64, v.\`Array(Variant(String, UInt64))\`, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64 from test order by id format Null" + $CH_CLIENT -q "select v.UInt64, v.\`Array(Variant(String, UInt64))\`, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64 from test order by id format Null" + $CH_CLIENT -q "select v.\`Array(Variant(String, UInt64))\`, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64, v.\`Array(Variant(String, UInt64))\`.String from test order by id format Null" +} + +$CH_CLIENT -q "drop table if exists test;" + +echo "Memory" +$CH_CLIENT -q "create table test (id UInt64, v Variant(UInt64, Array(Variant(String, UInt64)))) engine=Memory" +test +$CH_CLIENT -q "drop table test;" + +echo "MergeTree compact" +$CH_CLIENT -q "create table test (id UInt64, v Variant(UInt64, Array(Variant(String, UInt64)))) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000;" +test +$CH_CLIENT -q "drop table test;" + +echo "MergeTree wide" +$CH_CLIENT -q "create table test (id UInt64, v Variant(UInt64, Array(Variant(String, UInt64)))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" +test +$CH_CLIENT -q "drop table test;" + From 4931b363079aa5dd4fbc35ff6faea62efaf218de Mon Sep 17 00:00:00 2001 From: avogar Date: Sat, 23 Dec 2023 00:32:28 +0000 Subject: [PATCH 028/264] Fix style --- src/Columns/ColumnVector.cpp | 2 +- src/DataTypes/Serializations/SerializationArray.cpp | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index b4e3fee5e42..37e62c76596 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -469,7 +469,7 @@ void ColumnVector::insertRangeFrom(const IColumn & src, size_t start, size_t const ColumnVector & src_vec = assert_cast(src); if (start + length > src_vec.data.size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, + throw Exception(ErrorCodes::PARAMETER_OUT_OF_BOUND, "Parameters start = {}, length = {} are out of bound " "in ColumnVector::insertRangeFrom method (data.size() = {}).", toString(start), toString(length), toString(src_vec.data.size())); diff --git a/src/DataTypes/Serializations/SerializationArray.cpp b/src/DataTypes/Serializations/SerializationArray.cpp index be23278ef25..bb22af16c69 100644 --- a/src/DataTypes/Serializations/SerializationArray.cpp +++ b/src/DataTypes/Serializations/SerializationArray.cpp @@ -348,6 +348,7 @@ void SerializationArray::deserializeBinaryBulkWithMultipleStreams( { auto mutable_column = column->assumeMutable(); ColumnArray & column_array = typeid_cast(*mutable_column); + size_t prev_last_offset = column_array.getOffsets().back(); settings.path.push_back(Substream::ArraySizes); if (auto cached_column = getFromSubstreamsCache(cache, settings.path)) @@ -371,9 +372,9 @@ void SerializationArray::deserializeBinaryBulkWithMultipleStreams( /// Number of values corresponding with `offset_values` must be read. size_t last_offset = offset_values.back(); - if (last_offset < nested_column->size()) + if (last_offset < prev_last_offset) throw Exception(ErrorCodes::LOGICAL_ERROR, "Nested column is longer than last offset"); - size_t nested_limit = last_offset - nested_column->size(); + size_t nested_limit = last_offset - prev_last_offset; if (unlikely(nested_limit > MAX_ARRAYS_SIZE)) throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array sizes are too large: {}", nested_limit); From 4e4aa90430d02f1fcc17b517946799f23c59b83e Mon Sep 17 00:00:00 2001 From: avogar Date: Sat, 23 Dec 2023 00:35:20 +0000 Subject: [PATCH 029/264] Remove debug logging --- src/DataTypes/Serializations/ISerialization.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index 46353fffb48..08575f06f2a 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -124,20 +124,15 @@ void ISerialization::deserializeBinaryBulkWithMultipleStreams( DeserializeBinaryBulkStatePtr & /* state */, SubstreamsCache * cache) const { - LOG_DEBUG(&Poco::Logger::get("ISerialization"), "Deserialize path {}. Initial column size: {}", settings.path.toString(), column->size()); - auto cached_column = getFromSubstreamsCache(cache, settings.path); if (cached_column) { column = cached_column; - LOG_DEBUG(&Poco::Logger::get("ISerialization"), "Use column from cache. Size: {}", cached_column->size()); } else if (ReadBuffer * stream = settings.getter(settings.path)) { auto mutable_column = column->assumeMutable(); - LOG_DEBUG(&Poco::Logger::get("ISerialization"), "Deserialize column. Initial size: {}", mutable_column->size()); deserializeBinaryBulk(*mutable_column, *stream, limit, settings.avg_value_size_hint); - LOG_DEBUG(&Poco::Logger::get("ISerialization"), "Deserialized column. Size: {}", mutable_column->size()); column = std::move(mutable_column); addToSubstreamsCache(cache, settings.path, column); } From f594ab34f50c1bcd860bd3b950c8d74ffe09662d Mon Sep 17 00:00:00 2001 From: avogar Date: Sat, 23 Dec 2023 12:56:11 +0000 Subject: [PATCH 030/264] Fix special build --- src/Columns/ColumnVariant.cpp | 8 +++--- src/Columns/ColumnVariant.h | 2 +- .../Serializations/SerializationArray.cpp | 5 ++-- .../Serializations/SerializationVariant.cpp | 26 +++++++------------ src/Functions/if.cpp | 2 +- 5 files changed, 17 insertions(+), 26 deletions(-) diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index a3a0362b646..f90ebfc54bb 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -102,7 +102,7 @@ ColumnVariant::ColumnVariant(MutableColumnPtr local_discriminators_, MutableColu { } -ColumnVariant::ColumnVariant(MutableColumnPtr local_discriminators_, MutableColumns && variants_, const std::vector & global_discriminators) : ColumnVariant(std::move(local_discriminators_), nullptr, std::move(variants_), global_discriminators) +ColumnVariant::ColumnVariant(MutableColumnPtr local_discriminators_, MutableColumns && variants_, const std::vector & local_to_global_discriminators_) : ColumnVariant(std::move(local_discriminators_), nullptr, std::move(variants_), local_to_global_discriminators_) { } @@ -449,12 +449,12 @@ void ColumnVariant::insertData(const char *, size_t) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertData is not supported for {}", getName()); } -void ColumnVariant::insert(const Field & field) +void ColumnVariant::insert(const Field & x) { - if (field.isNull()) + if (x.isNull()) insertDefault(); else - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot insert field {} to column {}", toString(field), getName()); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot insert field {} to column {}", toString(x), getName()); } void ColumnVariant::insertFrom(const IColumn & src_, size_t n) diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h index ec58553f5f3..eb96205924c 100644 --- a/src/Columns/ColumnVariant.h +++ b/src/Columns/ColumnVariant.h @@ -194,7 +194,7 @@ public: template ColumnPtr indexImpl(const PaddedPODArray & indexes, size_t limit) const; ColumnPtr replicate(const Offsets & replicate_offsets) const override; - MutableColumns scatter(ColumnIndex num_variants, const Selector & selector) const override; + MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override; void gather(ColumnGathererStream & gatherer_stream) override; /// Variant type is not comparable. diff --git a/src/DataTypes/Serializations/SerializationArray.cpp b/src/DataTypes/Serializations/SerializationArray.cpp index bb22af16c69..be23278ef25 100644 --- a/src/DataTypes/Serializations/SerializationArray.cpp +++ b/src/DataTypes/Serializations/SerializationArray.cpp @@ -348,7 +348,6 @@ void SerializationArray::deserializeBinaryBulkWithMultipleStreams( { auto mutable_column = column->assumeMutable(); ColumnArray & column_array = typeid_cast(*mutable_column); - size_t prev_last_offset = column_array.getOffsets().back(); settings.path.push_back(Substream::ArraySizes); if (auto cached_column = getFromSubstreamsCache(cache, settings.path)) @@ -372,9 +371,9 @@ void SerializationArray::deserializeBinaryBulkWithMultipleStreams( /// Number of values corresponding with `offset_values` must be read. size_t last_offset = offset_values.back(); - if (last_offset < prev_last_offset) + if (last_offset < nested_column->size()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Nested column is longer than last offset"); - size_t nested_limit = last_offset - prev_last_offset; + size_t nested_limit = last_offset - nested_column->size(); if (unlikely(nested_limit > MAX_ARRAYS_SIZE)) throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array sizes are too large: {}", nested_limit); diff --git a/src/DataTypes/Serializations/SerializationVariant.cpp b/src/DataTypes/Serializations/SerializationVariant.cpp index d36151fe8e9..c88dd8e9e0d 100644 --- a/src/DataTypes/Serializations/SerializationVariant.cpp +++ b/src/DataTypes/Serializations/SerializationVariant.cpp @@ -564,7 +564,7 @@ bool SerializationVariant::tryDeserializeImpl( IColumn & column, const String & field, std::function check_for_null, - std::function try_deserialize_variant) const + std::function try_deserialize_nested) const { auto & column_variant = assert_cast(column); ReadBufferFromString null_buf(field); @@ -577,25 +577,17 @@ bool SerializationVariant::tryDeserializeImpl( for (size_t global_discr : deserialize_text_order) { ReadBufferFromString variant_buf(field); - /// Usually try_deserialize_variant should not throw any exception, but let's use try/catch just in case. - try + auto & variant_column = column_variant.getVariantByGlobalDiscriminator(global_discr); + size_t prev_size = variant_column.size(); + if (try_deserialize_nested(variant_column, variants[global_discr], variant_buf) && variant_buf.eof()) { - auto & variant_column = column_variant.getVariantByGlobalDiscriminator(global_discr); - size_t prev_size = variant_column.size(); - if (try_deserialize_variant(variant_column, variants[global_discr], variant_buf) && variant_buf.eof()) - { - column_variant.getLocalDiscriminators().push_back(column_variant.localDiscriminatorByGlobal(global_discr)); - column_variant.getOffsets().push_back(prev_size); - return true; - } - else if (variant_column.size() > prev_size) - { - variant_column.popBack(1); - } + column_variant.getLocalDiscriminators().push_back(column_variant.localDiscriminatorByGlobal(global_discr)); + column_variant.getOffsets().push_back(prev_size); + return true; } - catch (...) + else if (variant_column.size() > prev_size) { - /// Try next variant. + variant_column.popBack(1); } } diff --git a/src/Functions/if.cpp b/src/Functions/if.cpp index b15bc5938be..9ca4b487119 100644 --- a/src/Functions/if.cpp +++ b/src/Functions/if.cpp @@ -224,7 +224,7 @@ public: return std::make_shared(context->getSettingsRef().allow_experimental_variant_type && context->getSettingsRef().use_variant_when_no_common_type_in_if); } - FunctionIf(bool use_variant_when_no_common_type_ = false) : FunctionIfBase(), use_variant_when_no_common_type(use_variant_when_no_common_type_) {} + explicit FunctionIf(bool use_variant_when_no_common_type_ = false) : FunctionIfBase(), use_variant_when_no_common_type(use_variant_when_no_common_type_) {} private: bool use_variant_when_no_common_type = false; From 5497fa79edfa6fdc2559d516486f80f88af40c68 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 27 Dec 2023 14:11:53 +0000 Subject: [PATCH 031/264] Fix tests --- src/DataTypes/Serializations/SerializationEnum.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationEnum.cpp b/src/DataTypes/Serializations/SerializationEnum.cpp index fb384547d64..14b1a33e2ce 100644 --- a/src/DataTypes/Serializations/SerializationEnum.cpp +++ b/src/DataTypes/Serializations/SerializationEnum.cpp @@ -73,7 +73,9 @@ template bool SerializationEnum::tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { std::string field_name; - readQuotedStringWithSQLStyle(field_name, istr); + if (!tryReadQuotedStringWithSQLStyle(field_name, istr)) + return false; + FieldType x; if (!ref_enum_values.tryGetValue(x, StringRef(field_name))) return false; From 4b2a0b99fc094e6b70e516af0360f126f62a886d Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 27 Dec 2023 20:02:50 +0100 Subject: [PATCH 032/264] Update docs/en/sql-reference/functions/other-functions.md --- docs/en/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index d69d692d055..ebc80e4d308 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2839,7 +2839,7 @@ Extracts a column with specified type from a `Variant` column. **Syntax** ```sql -tupleElement(variant, type_name, [, default_value]) +variantElement(variant, type_name, [, default_value]) ``` **Arguments** From 275fbe3e986c8faee3bd396e3ed87e3707f0f25f Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 27 Dec 2023 20:04:14 +0000 Subject: [PATCH 033/264] Support function to subcolumns optimization for Variant, better text priority for reading Bool --- .../Passes/FunctionToSubcolumnsPass.cpp | 17 +++++++++++++++++ .../Serializations/SerializationVariant.cpp | 4 ++++ .../RewriteFunctionToSubcolumnVisitor.cpp | 15 +++++++++++++++ 3 files changed, 36 insertions(+) diff --git a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp index cd635f87e0e..c74c1038173 100644 --- a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp +++ b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp @@ -176,6 +176,23 @@ public: node = std::make_shared(column, column_source); } + else if (function_name == "variantElement" && isVariant(column_type) && second_argument_constant_node) + { + /// Replace `variantElement(variant_argument, type_name)` with `variant_argument.type_name`. + const auto & variant_element_constant_value = second_argument_constant_node->getValue(); + String subcolumn_name; + + if (variant_element_constant_value.getType() != Field::Types::String) + return; + + subcolumn_name = variant_element_constant_value.get(); + + column.name += '.'; + column.name += subcolumn_name; + column.type = function_node->getResultType(); + + node = std::make_shared(column, column_source); + } else if (function_name == "mapContains" && column_type.isMap()) { const auto & data_type_map = assert_cast(*column.type); diff --git a/src/DataTypes/Serializations/SerializationVariant.cpp b/src/DataTypes/Serializations/SerializationVariant.cpp index c88dd8e9e0d..49ecb2fc546 100644 --- a/src/DataTypes/Serializations/SerializationVariant.cpp +++ b/src/DataTypes/Serializations/SerializationVariant.cpp @@ -534,6 +534,10 @@ std::tuple getTypeTextDeserializePriority(const DataType return {max_depth, max_priority, max_simple_nested_depth}; } + /// Bool type should have priority higher then all integers. + if (isBool(type)) + return {nested_depth, priority_map[TypeIndex::Int8] + 1 , simple_nested_depth}; + return {nested_depth, priority_map[type->getTypeId()], simple_nested_depth}; } diff --git a/src/Interpreters/RewriteFunctionToSubcolumnVisitor.cpp b/src/Interpreters/RewriteFunctionToSubcolumnVisitor.cpp index 506fa13b7ba..0717abd4782 100644 --- a/src/Interpreters/RewriteFunctionToSubcolumnVisitor.cpp +++ b/src/Interpreters/RewriteFunctionToSubcolumnVisitor.cpp @@ -122,6 +122,21 @@ void RewriteFunctionToSubcolumnData::visit(ASTFunction & function, ASTPtr & ast) ast = transformToSubcolumn(name_in_storage, subcolumn_name); ast->setAlias(alias); } + else if (function.name == "variantElement" && column_type_id == TypeIndex::Variant) + { + const auto * literal = arguments[1]->as(); + if (!literal) + return; + + String subcolumn_name; + auto value_type = literal->value.getType(); + if (value_type != Field::Types::String) + return; + + subcolumn_name = literal->value.get(); + ast = transformToSubcolumn(name_in_storage, subcolumn_name); + ast->setAlias(alias); + } else { auto it = binary_function_to_subcolumn.find(function.name); From 8b4157141c0501d4498278947b468d03638cdf8a Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 27 Dec 2023 20:36:10 +0000 Subject: [PATCH 034/264] Fix style --- src/DataTypes/Serializations/SerializationVariant.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationVariant.cpp b/src/DataTypes/Serializations/SerializationVariant.cpp index 49ecb2fc546..9cfc4b9e26f 100644 --- a/src/DataTypes/Serializations/SerializationVariant.cpp +++ b/src/DataTypes/Serializations/SerializationVariant.cpp @@ -536,7 +536,7 @@ std::tuple getTypeTextDeserializePriority(const DataType /// Bool type should have priority higher then all integers. if (isBool(type)) - return {nested_depth, priority_map[TypeIndex::Int8] + 1 , simple_nested_depth}; + return {nested_depth, priority_map[TypeIndex::Int8] + 1, simple_nested_depth}; return {nested_depth, priority_map[type->getTypeId()], simple_nested_depth}; } From d5b138eee4668018330d423aef33bd38a105298c Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 28 Dec 2023 13:41:07 +0100 Subject: [PATCH 035/264] Log exception --- src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp index 48e12c7b9b9..44ca67b5df8 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp @@ -231,6 +231,8 @@ void CachedOnDiskWriteBufferFromFile::nextImpl() } catch (...) { + tryLogCurrentException(__PRETTY_FUNCTION__); + /// If something was already written to cache, remove it. cache_writer.reset(); cache->removeKeyIfExists(key); From 3de5b27c48483962285de0b16f152cc35eadd1a6 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 8 Jan 2024 16:50:17 +0100 Subject: [PATCH 036/264] Fix conflicts --- .../Serializations/SerializationString.cpp | 22 ++++++++++++++++++- 1 file changed, 21 insertions(+), 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationString.cpp b/src/DataTypes/Serializations/SerializationString.cpp index a6bf29336b7..b2c254e63c5 100644 --- a/src/DataTypes/Serializations/SerializationString.cpp +++ b/src/DataTypes/Serializations/SerializationString.cpp @@ -381,7 +381,7 @@ void SerializationString::deserializeTextJSON(IColumn & column, ReadBuffer & ist str_value = "false"; } - read(column, [&](ColumnString::Chars & data) { data.insert(str_value.begin(), str_value.end()); }); + read(column, [&](ColumnString::Chars & data) { data.insert(str_value.begin(), str_value.end()); }); } else if (settings.json.read_numbers_as_strings && !istr.eof() && *istr.position() != '"') { @@ -406,6 +406,26 @@ bool SerializationString::tryDeserializeTextJSON(IColumn & column, ReadBuffer & if (settings.json.read_arrays_as_strings && !istr.eof() && *istr.position() == '[') return read(column, [&](ColumnString::Chars & data) { return readJSONArrayInto(data, istr); }); + if (settings.json.read_bools_as_strings && !istr.eof() && (*istr.position() == 't' || *istr.position() == 'f')) + { + String str_value; + if (*istr.position() == 't') + { + if (!checkString("true", istr)) + return false; + str_value = "true"; + } + else if (*istr.position() == 'f') + { + if (!checkString("false", istr)) + return false; + str_value = "false"; + } + + read(column, [&](ColumnString::Chars & data) { data.insert(str_value.begin(), str_value.end()); }); + return true; + } + if (settings.json.read_numbers_as_strings && !istr.eof() && *istr.position() != '"') { String field; From 21e4b453dfc7df905ed304c5513b50f57ef19228 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 8 Jan 2024 22:02:40 +0100 Subject: [PATCH 037/264] Fix pretty type name --- src/DataTypes/DataTypeVariant.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/DataTypes/DataTypeVariant.cpp b/src/DataTypes/DataTypeVariant.cpp index 5dc42cc7443..2bc4dfa5a7a 100644 --- a/src/DataTypes/DataTypeVariant.cpp +++ b/src/DataTypes/DataTypeVariant.cpp @@ -71,17 +71,17 @@ std::string DataTypeVariant::doGetPrettyName(size_t indent) const { size_t size = variants.size(); WriteBufferFromOwnString s; - s << "Variant(\n"; + s << "Variant("; for (size_t i = 0; i != size; ++i) { if (i != 0) - s << ",\n"; + s << ", "; - s << fourSpaceIndent(indent + 1) << variants[i]->getPrettyName(indent + 1); + s << variants[i]->getPrettyName(indent); } - s << '\n' << fourSpaceIndent(indent) << ')'; + s << ')'; return s.str(); } From 633b4a5dcfcf63bec8e2b5a1b5f38e648348639d Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 9 Jan 2024 19:23:34 +0100 Subject: [PATCH 038/264] Apply suggestions from code review Co-authored-by: Antonio Andelic --- src/Columns/ColumnNullable.cpp | 2 +- src/Columns/ColumnVariant.cpp | 8 +++++--- src/Columns/ColumnVariant.h | 2 +- src/DataTypes/EnumValues.cpp | 4 +--- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index d2a579d6800..25b0e35e15e 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -928,7 +928,7 @@ ColumnPtr makeNullableOrLowCardinalityNullableSafe(const ColumnPtr & column) return assert_cast(*column).cloneNullable(); if (column->canBeInsideNullable()) - return makeNullableSafe(column); + return makeNullable(column); return column; } diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index f90ebfc54bb..10d79f59d37 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -631,9 +631,9 @@ void ColumnVariant::popBack(size_t n) size_t size = local_discriminators_data.size(); const size_t num_variants = variants.size(); std::vector nested_n(num_variants, 0); - for (size_t i = 0; i != n; ++i) + for (size_t i = size - n; i < size; ++i) { - Discriminator discr = local_discriminators_data[size - i - 1]; + Discriminator discr = local_discriminators_data[i]; if (discr != NULL_DISCRIMINATOR) ++nested_n[discr]; } @@ -966,7 +966,7 @@ ColumnPtr ColumnVariant::replicate(const Offsets & replicate_offsets) const { new_offsets_data.reserve(new_size); for (size_t i = old_size; i < new_size; ++i) - new_offsets_data.push_back(new_offsets_data[i - 1] + 1); + new_offsets_data.push_back(i); } else { @@ -1260,6 +1260,8 @@ std::optional ColumnVariant::getLocalDiscriminator { if (variants[i]->size() == local_discriminators->size()) return i; + if (!variants[i]->empty()) + return std::nullopt } return std::nullopt; diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h index eb96205924c..8f0c5a6eef9 100644 --- a/src/Columns/ColumnVariant.h +++ b/src/Columns/ColumnVariant.h @@ -205,7 +205,7 @@ public: void compareColumn(const IColumn &, size_t, PaddedPODArray *, PaddedPODArray &, int, int) const override { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method compareColumn is not supported for ColumnAggregateFunction"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method compareColumn is not supported for ColumnVariant"); } bool hasEqualValues() const override; diff --git a/src/DataTypes/EnumValues.cpp b/src/DataTypes/EnumValues.cpp index 8a4b1304d5e..a15136b9335 100644 --- a/src/DataTypes/EnumValues.cpp +++ b/src/DataTypes/EnumValues.cpp @@ -85,9 +85,7 @@ bool EnumValues::tryGetValue(T & x, StringRef field_name, bool try_treat_as_i if (try_treat_as_id) { ReadBufferFromMemory tmp_buf(field_name.data, field_name.size); - if (!tryReadText(x, tmp_buf) || !tmp_buf.eof() || !value_to_name_map.contains(x)) - return false; - return true; + return tryReadText(x, tmp_buf) && tmp_buf.eof() && value_to_name_map.contains(x); } return false; } From fb758e48b04c5f799a5169af584f6a562866640d Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 9 Jan 2024 19:02:20 +0000 Subject: [PATCH 039/264] Apply suggestions --- src/Columns/ColumnVariant.cpp | 172 +++++++++++++++------------------- 1 file changed, 74 insertions(+), 98 deletions(-) diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index 10d79f59d37..a707ec8e153 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -204,10 +204,13 @@ ColumnVariant::ColumnVariant(DB::MutableColumnPtr local_discriminators_, DB::Mut } } -ColumnVariant::Ptr ColumnVariant::create(const Columns & variants, const std::vector & local_to_global_discriminators) +namespace +{ + +MutableColumns getVariantsAssumeMutable(const Columns & variants) { MutableColumns mutable_variants; - mutable_variants.reserve(variants.size()); + for (const auto & variant : variants) { if (isColumnConst(*variant)) @@ -215,35 +218,24 @@ ColumnVariant::Ptr ColumnVariant::create(const Columns & variants, const std::ve mutable_variants.emplace_back(variant->assumeMutable()); } - return ColumnVariant::create(std::move(mutable_variants), local_to_global_discriminators); + return mutable_variants; +} + +} + +ColumnVariant::Ptr ColumnVariant::create(const Columns & variants, const std::vector & local_to_global_discriminators) +{ + return ColumnVariant::create(getVariantsAssumeMutable(variants), local_to_global_discriminators); } ColumnVariant::Ptr ColumnVariant::create(const DB::ColumnPtr & local_discriminators, const DB::Columns & variants, const std::vector & local_to_global_discriminators) { - MutableColumns mutable_variants; - mutable_variants.reserve(variants.size()); - for (const auto & variant : variants) - { - if (isColumnConst(*variant)) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "ColumnVariant cannot have ColumnConst as its element"); - mutable_variants.emplace_back(variant->assumeMutable()); - } - - return ColumnVariant::create(local_discriminators->assumeMutable(), std::move(mutable_variants), local_to_global_discriminators); + return ColumnVariant::create(local_discriminators->assumeMutable(), getVariantsAssumeMutable(variants), local_to_global_discriminators); } ColumnVariant::Ptr ColumnVariant::create(const DB::ColumnPtr & local_discriminators, const DB::ColumnPtr & offsets, const DB::Columns & variants, const std::vector & local_to_global_discriminators) { - MutableColumns mutable_variants; - mutable_variants.reserve(variants.size()); - for (const auto & variant : variants) - { - if (isColumnConst(*variant)) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "ColumnVariant cannot have ColumnConst as its element"); - mutable_variants.emplace_back(variant->assumeMutable()); - } - - return ColumnVariant::create(local_discriminators->assumeMutable(), offsets->assumeMutable(), std::move(mutable_variants), local_to_global_discriminators); + return ColumnVariant::create(local_discriminators->assumeMutable(), offsets->assumeMutable(), getVariantsAssumeMutable(variants), local_to_global_discriminators); } MutableColumnPtr ColumnVariant::cloneEmpty() const @@ -309,104 +301,88 @@ MutableColumnPtr ColumnVariant::cloneResized(size_t new_size) const const auto & local_discriminators_data = getLocalDiscriminators(); const auto & offsets_data = getOffsets(); - /// We can find all variants sizes by scanning all new_size local_discriminators and calculating - /// sizes for all new variants. This code is below and commented. - -// std::vector new_nested_sizes(num_variants, 0); -// for (size_t i = 0; i != new_size; ++i) -// { -// Discriminator discr = local_discriminators_data[i]; -// if (discr != NULL_DISCRIMINATOR) -// ++new_nested_sizes[discr]; -// } -// -// MutableColumns new_variants; -// new_variants.reserve(num_variants); -// for (size_t i = 0; i != num_variants; ++i) -// { -// if (new_nested_sizes[i]) -// new_variants.emplace_back(variants[i]->cloneResized(new_nested_sizes[i])); -// else -// new_variants.emplace_back(variants[i]->cloneEmpty()); -// } -// -// return ColumnVariant::create(local_discriminators->cloneResized(new_size), offsets->cloneResized(new_size), std::move(new_variants), local_to_global_discriminators); - + /// We can find all variants sizes by scanning all new_size local_discriminators and calculating sizes for all new variants. /// But instead we are trying to optimize it using offsets column: /// For all non-empty variants we are trying to find last occurrence of its discriminator in local_discriminators[:new_size] or - /// first occurrence in local_discriminators[new_size:]. The same row in offsets column will contain the desired size (or size - 1) of variant. + /// first occurrence in local_discriminators[new_size:] depending on what range is smaller. The same row in offsets column will + /// contain the desired size (or size - 1) of variant. /// All empty variants will remain empty. - /// Not sure how good this optimization is, feel free to remove it and use simpler version above. + /// Not sure how good this optimization is, feel free to remove it and use simpler version without using offsets. MutableColumns new_variants(num_variants); - std::unordered_set seen_variants; + std::vector seen_variants(num_variants, 0); + size_t number_of_seen_variants = 0; /// First, check which variants are empty. They will remain empty. for (Discriminator i = 0; i != num_variants; ++i) { if (variants[i]->empty()) { - seen_variants.insert(i); + seen_variants[i] = 1; + ++number_of_seen_variants; new_variants[i] = variants[i]->cloneEmpty(); } } - /// Now, iterate through local discriminators using two pointers. - /// First will go from new_size - 1 to 0, second from new_size to size. - /// Finish when we find all variants or hit lower or upper bound. - ssize_t i = new_size - 1; - size_t j = new_size; - while (i != -1 && j != size) + /// Now, choose what range is smaller and use it. + /// [0, new_size) + if (2 * new_size <= size) { - Discriminator i_discr = local_discriminators_data[i]; - if (i_discr != NULL_DISCRIMINATOR) + for (ssize_t i = new_size - 1; i > -1; --i) { - auto [_, inserted] = seen_variants.insert(i_discr); - /// If this is the first occurrence of this discriminator, - /// we can get new size for this variant. - if (inserted) + Discriminator discr = local_discriminators_data[i]; + if (discr != NULL_DISCRIMINATOR) { - new_variants[i_discr] = variants[i_discr]->cloneResized(offsets_data[i] + 1); - if (seen_variants.size() == num_variants) - break; + /// If this is the first occurrence of this discriminator, + /// we can get new size for this variant. + if (!seen_variants[discr]) + { + seen_variants[discr] = 1; + ++number_of_seen_variants; + new_variants[discr] = variants[discr]->cloneResized(offsets_data[i] + 1); + /// Break if we found sizes for all variants. + if (number_of_seen_variants == num_variants) + break; + } } } - Discriminator j_discr = local_discriminators_data[j]; - if (j_discr != NULL_DISCRIMINATOR) + /// All variants that weren't found in range [0, new_size] will be empty in the result column. + if (number_of_seen_variants != num_variants) { - auto [_, inserted] = seen_variants.insert(j_discr); - /// If this is the first occurrence of this discriminator, - /// we can get new size for this variant. - if (inserted) - { - new_variants[j_discr] = variants[j_discr]->cloneResized(offsets_data[j]); - if (seen_variants.size() == num_variants) - break; - } - } - - --i; - ++j; - } - - /// We can finish in 3 cases: - /// 1) seen_variants.size() == num_variants - we found local_discriminators of all variants, nothing to do. - /// 2) i == -1 - we scanned all values in local_discriminators[:new_size]. Not found variants doesn't have - /// values in local_discriminators[:new_size], so they should be empty in the resized version. - /// 3) j == size - we scanned all values in local_discriminators[new_size:]. Not found variants doesn't have - /// values in local_discriminators[new_size:], so, we should use the full variant in the resized version. - if (seen_variants.size() != num_variants) - { - for (size_t discr = 0; discr != num_variants; ++discr) - { - if (!seen_variants.contains(discr)) - { - if (i == -1) + for (size_t discr = 0; discr != num_variants; ++discr) + if (!seen_variants[discr]) new_variants[discr] = variants[discr]->cloneEmpty(); - else - new_variants[discr] = IColumn::mutate(variants[discr]); + } + } + /// [new_size, size) + else + { + for (size_t i = new_size; i < size; ++i) + { + Discriminator discr = local_discriminators_data[i]; + if (discr != NULL_DISCRIMINATOR) + { + /// If this is the first occurrence of this discriminator, + /// we can get new size for this variant. + if (!seen_variants[discr]) + { + seen_variants[discr] = 1; + ++number_of_seen_variants; + new_variants[discr] = variants[discr]->cloneResized(offsets_data[i]); + /// Break if we found sizes for all variants. + if (number_of_seen_variants == num_variants) + break; + } } } + + if (number_of_seen_variants != num_variants) + { + /// All variants that weren't found in range [new_size, size) will not change their sizes. + for (size_t discr = 0; discr != num_variants; ++discr) + if (!seen_variants[discr]) + new_variants[discr] = IColumn::mutate(variants[discr]); + } } return ColumnVariant::create(local_discriminators->cloneResized(new_size), offsets->cloneResized(new_size), std::move(new_variants), local_to_global_discriminators); @@ -1261,7 +1237,7 @@ std::optional ColumnVariant::getLocalDiscriminator if (variants[i]->size() == local_discriminators->size()) return i; if (!variants[i]->empty()) - return std::nullopt + return std::nullopt; } return std::nullopt; From 1deaaf5466a2633d58fba87521435491546df0a2 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 11 Jan 2024 15:20:06 +0100 Subject: [PATCH 040/264] Apply suggestions from code review Co-authored-by: Antonio Andelic --- .../Serializations/SerializationDateTime64.cpp | 6 +++--- src/DataTypes/Serializations/SerializationEnum.h | 5 +---- .../Serializations/SerializationNamed.cpp | 1 - .../Serializations/SerializationTuple.cpp | 15 +++------------ 4 files changed, 7 insertions(+), 20 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationDateTime64.cpp b/src/DataTypes/Serializations/SerializationDateTime64.cpp index a19619bf8d3..442e29edd52 100644 --- a/src/DataTypes/Serializations/SerializationDateTime64.cpp +++ b/src/DataTypes/Serializations/SerializationDateTime64.cpp @@ -50,7 +50,7 @@ void SerializationDateTime64::deserializeText(IColumn & column, ReadBuffer & ist bool SerializationDateTime64::tryDeserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const { DateTime64 result = 0; - if (tryReadDateTime64Text(result, scale, istr, time_zone) || (whole && istr.eof())) + if (!tryReadDateTime64Text(result, scale, istr, time_zone) || (whole && !istr.eof())) return false; assert_cast(column).getData().push_back(result); @@ -151,7 +151,7 @@ bool SerializationDateTime64::tryDeserializeTextQuoted(IColumn & column, ReadBuf DateTime64 x = 0; if (checkChar('\'', istr)) /// Cases: '2017-08-31 18:36:48' or '1504193808' { - if (tryReadText(x, scale, istr, settings, time_zone, utc_time_zone) || !checkChar('\'', istr)) + if (!tryReadText(x, scale, istr, settings, time_zone, utc_time_zone) || !checkChar('\'', istr)) return false; } else /// Just 1504193808 or 01504193808 @@ -265,7 +265,7 @@ bool SerializationDateTime64::tryDeserializeTextCSV(IColumn & column, ReadBuffer { if (settings.csv.delimiter != ',' || settings.date_time_input_format == FormatSettings::DateTimeInputFormat::Basic) { - if (tryReadText(x, scale, istr, settings, time_zone, utc_time_zone)) + if (!tryReadText(x, scale, istr, settings, time_zone, utc_time_zone)) return false; } else diff --git a/src/DataTypes/Serializations/SerializationEnum.h b/src/DataTypes/Serializations/SerializationEnum.h index 5152a3fbc93..bb720ee9b1f 100644 --- a/src/DataTypes/Serializations/SerializationEnum.h +++ b/src/DataTypes/Serializations/SerializationEnum.h @@ -60,10 +60,7 @@ public: bool tryReadValue(ReadBuffer & istr, FieldType & x) const { - if (!tryReadText(x, istr) || !ref_enum_values.hasValue(x)) - return false; - - return true; + return tryReadText(x, istr) && ref_enum_values.hasValue(x); } std::optional> own_enum_values; diff --git a/src/DataTypes/Serializations/SerializationNamed.cpp b/src/DataTypes/Serializations/SerializationNamed.cpp index 1a9cbe9a37d..ca60948ce68 100644 --- a/src/DataTypes/Serializations/SerializationNamed.cpp +++ b/src/DataTypes/Serializations/SerializationNamed.cpp @@ -1,5 +1,4 @@ #include -#include namespace DB { diff --git a/src/DataTypes/Serializations/SerializationTuple.cpp b/src/DataTypes/Serializations/SerializationTuple.cpp index c0b0658e6b4..79b7fa84242 100644 --- a/src/DataTypes/Serializations/SerializationTuple.cpp +++ b/src/DataTypes/Serializations/SerializationTuple.cpp @@ -212,10 +212,7 @@ ReturnType SerializationTuple::deserializeTextImpl(IColumn & column, ReadBuffer return ReturnType(true); }; - if constexpr (throw_exception) - addElementSafe(elems.size(), column, impl); - else - return addElementSafe(elems.size(), column, impl); + return addElementSafe(elems.size(), column, impl); } void SerializationTuple::deserializeText(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings & settings, bool whole) const @@ -457,10 +454,7 @@ ReturnType SerializationTuple::deserializeTextJSONImpl(IColumn & column, ReadBuf return ReturnType(true); }; - if constexpr (throw_exception) - addElementSafe(elems.size(), column, impl); - else - return addElementSafe(elems.size(), column, impl); + return addElementSafe(elems.size(), column, impl); } else { @@ -502,10 +496,7 @@ ReturnType SerializationTuple::deserializeTextJSONImpl(IColumn & column, ReadBuf return ReturnType(true); }; - if constexpr (throw_exception) - addElementSafe(elems.size(), column, impl); - else - return addElementSafe(elems.size(), column, impl); + return addElementSafe(elems.size(), column, impl); } } From f05d89bc2b26206b1b6854ad48dd35840b82a123 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 11 Jan 2024 14:48:57 +0000 Subject: [PATCH 041/264] Apply review suggestions --- .../Serializations/ISerialization.cpp | 47 +++--- .../Serializations/SerializationTuple.cpp | 3 + .../Serializations/SerializationVariant.cpp | 135 +++++++++--------- 3 files changed, 101 insertions(+), 84 deletions(-) diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index 08575f06f2a..c699b3b0748 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -176,7 +176,7 @@ String getNameForSubstreamPath( stream_name += "." + it->tuple_element_name; } else if (it->type == Substream::VariantDiscriminators) - stream_name += ".discr"; + stream_name += ".variant_discr"; else if (it->type == Substream::VariantOffsets) stream_name += ".variant_offsets"; else if (it->type == Substream::VariantElement) @@ -261,43 +261,51 @@ bool ISerialization::isSpecialCompressionAllowed(const SubstreamPath & path) return true; } -#define TRY_DESERIALIZE_TEXT(deserialize) \ - size_t prev_size = column.size(); \ - try \ - { \ - deserialize(column, istr, settings); \ - return true; \ - } \ - catch (...) \ - { \ - if (column.size() > prev_size) \ - column.popBack(column.size() - prev_size); \ - return false; \ - } \ +namespace +{ + +template +bool tryDeserializeText(const F deserialize, DB::IColumn & column) +{ + size_t prev_size = column.size(); + try + { + deserialize(column); + return true; + } + catch (...) + { + if (column.size() > prev_size) + column.popBack(column.size() - prev_size); + return false; + } +} + +} bool ISerialization::tryDeserializeTextCSV(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings & settings) const { - TRY_DESERIALIZE_TEXT(deserializeTextCSV) + return tryDeserializeText([&](DB::IColumn & my_column) { deserializeTextCSV(my_column, istr, settings); }, column); } bool ISerialization::tryDeserializeTextEscaped(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings & settings) const { - TRY_DESERIALIZE_TEXT(deserializeTextEscaped) + return tryDeserializeText([&](DB::IColumn & my_column) { deserializeTextEscaped(my_column, istr, settings); }, column); } bool ISerialization::tryDeserializeTextJSON(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings & settings) const { - TRY_DESERIALIZE_TEXT(deserializeTextJSON) + return tryDeserializeText([&](DB::IColumn & my_column) { deserializeTextJSON(my_column, istr, settings); }, column); } bool ISerialization::tryDeserializeTextQuoted(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings & settings) const { - TRY_DESERIALIZE_TEXT(deserializeTextQuoted) + return tryDeserializeText([&](DB::IColumn & my_column) { deserializeTextQuoted(my_column, istr, settings); }, column); } bool ISerialization::tryDeserializeWholeText(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings & settings) const { - TRY_DESERIALIZE_TEXT(deserializeWholeText) + return tryDeserializeText([&](DB::IColumn & my_column) { deserializeWholeText(my_column, istr, settings); }, column); } void ISerialization::deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const @@ -346,7 +354,6 @@ bool ISerialization::hasSubcolumnForPath(const SubstreamPath & path, size_t pref return path[last_elem].type == Substream::NullMap || path[last_elem].type == Substream::TupleElement || path[last_elem].type == Substream::ArraySizes - || path[last_elem].type == Substream::VariantDiscriminators || path[last_elem].type == Substream::VariantElement; } diff --git a/src/DataTypes/Serializations/SerializationTuple.cpp b/src/DataTypes/Serializations/SerializationTuple.cpp index 79b7fa84242..c249ee69e46 100644 --- a/src/DataTypes/Serializations/SerializationTuple.cpp +++ b/src/DataTypes/Serializations/SerializationTuple.cpp @@ -76,7 +76,10 @@ static ReturnType addElementSafe(size_t num_elems, IColumn & column, F && impl) { auto & element_column = extractElementColumn(column, i); if (element_column.size() > old_size) + { + chassert(old_size - element_column.size() == 1); element_column.popBack(1); + } } }; diff --git a/src/DataTypes/Serializations/SerializationVariant.cpp b/src/DataTypes/Serializations/SerializationVariant.cpp index 9cfc4b9e26f..64fcb63d604 100644 --- a/src/DataTypes/Serializations/SerializationVariant.cpp +++ b/src/DataTypes/Serializations/SerializationVariant.cpp @@ -397,70 +397,76 @@ void SerializationVariant::deserializeBinary(IColumn & column, ReadBuffer & istr namespace { -std::unordered_map getTypesTextDeserializePriorityMap() +const std::unordered_map & getTypesTextDeserializePriorityMap() { - static const std::vector priorities = { - /// Complex types have highest priority. - TypeIndex::Array, - TypeIndex::Tuple, - TypeIndex::Map, - TypeIndex::AggregateFunction, + static std::unordered_map priority_map = [] + { + static constexpr std::array priorities = { + /// Complex types have highest priority. + TypeIndex::Array, + TypeIndex::Tuple, + TypeIndex::Map, + TypeIndex::AggregateFunction, - /// Enums can be parsed both from strings and numbers. - /// So they have high enough priority. - TypeIndex::Enum8, - TypeIndex::Enum16, + /// Enums can be parsed both from strings and numbers. + /// So they have high enough priority. + TypeIndex::Enum8, + TypeIndex::Enum16, - /// Types that can be parsed from strings. - TypeIndex::UUID, - TypeIndex::IPv4, - TypeIndex::IPv6, + /// Types that can be parsed from strings. + TypeIndex::UUID, + TypeIndex::IPv4, + TypeIndex::IPv6, - /// Types that can be parsed from numbers. - /// The order: - /// 1) Integers - /// 2) Big Integers - /// 3) Decimals - /// 4) Floats - /// In each group small types have higher priority. - TypeIndex::Int8, - TypeIndex::UInt8, - TypeIndex::Int16, - TypeIndex::UInt16, - TypeIndex::Int32, - TypeIndex::UInt32, - TypeIndex::Int64, - TypeIndex::UInt64, - TypeIndex::Int128, - TypeIndex::UInt128, - TypeIndex::Int256, - TypeIndex::UInt256, - TypeIndex::Decimal32, - TypeIndex::Decimal64, - TypeIndex::Decimal128, - TypeIndex::Decimal256, - TypeIndex::Float32, - TypeIndex::Float64, + /// Types that can be parsed from numbers. + /// The order: + /// 1) Integers + /// 2) Big Integers + /// 3) Decimals + /// 4) Floats + /// In each group small types have higher priority. + TypeIndex::Int8, + TypeIndex::UInt8, + TypeIndex::Int16, + TypeIndex::UInt16, + TypeIndex::Int32, + TypeIndex::UInt32, + TypeIndex::Int64, + TypeIndex::UInt64, + TypeIndex::Int128, + TypeIndex::UInt128, + TypeIndex::Int256, + TypeIndex::UInt256, + TypeIndex::Decimal32, + TypeIndex::Decimal64, + TypeIndex::Decimal128, + TypeIndex::Decimal256, + TypeIndex::Float32, + TypeIndex::Float64, - /// Dates and DateTimes. More simple Date types have higher priority. - /// They have lower priority as numbers as some DateTimes sometimes can - /// be also parsed from numbers, but we don't want it usually. - TypeIndex::Date, - TypeIndex::Date32, - TypeIndex::DateTime, - TypeIndex::DateTime64, + /// Dates and DateTimes. More simple Date types have higher priority. + /// They have lower priority as numbers as some DateTimes sometimes can + /// be also parsed from numbers, but we don't want it usually. + TypeIndex::Date, + TypeIndex::Date32, + TypeIndex::DateTime, + TypeIndex::DateTime64, - /// String types have almost the lowest priority, - /// as in text formats almost all data can - /// be deserialized into String type. - TypeIndex::FixedString, - TypeIndex::String, - }; + /// String types have almost the lowest priority, + /// as in text formats almost all data can + /// be deserialized into String type. + TypeIndex::FixedString, + TypeIndex::String, + }; + + std::unordered_map pm; + + pm.reserve(priorities.size()); + for (size_t i = 0; i != priorities.size(); ++i) + pm[priorities[i]] = priorities.size() - i; + return pm; + }(); - std::unordered_map priority_map; - priority_map.reserve(priorities.size()); - for (size_t i = 0; i != priorities.size(); ++i) - priority_map[priorities[i]] = priorities.size() - i; return priority_map; } @@ -476,7 +482,7 @@ std::unordered_map getTypesTextDeserializePriorityMap() /// so if we have types with the same level of nesting and the same priority, we will first try to deserialize LowCardinality/Nullable types /// (for example if we have types Array(Array(String)) and Array(Array(Nullable(String))). /// This is just a batch of heuristics. -std::tuple getTypeTextDeserializePriority(const DataTypePtr & type, size_t nested_depth, size_t simple_nested_depth, std::unordered_map & priority_map) +std::tuple getTypeTextDeserializePriority(const DataTypePtr & type, size_t nested_depth, size_t simple_nested_depth, const std::unordered_map & priority_map) { if (const auto * nullable_type = typeid_cast(type.get())) return getTypeTextDeserializePriority(nullable_type->getNestedType(), nested_depth, simple_nested_depth + 1, priority_map); @@ -487,7 +493,7 @@ std::tuple getTypeTextDeserializePriority(const DataType if (const auto * array_type = typeid_cast(type.get())) { auto [elements_nested_depth, elements_priority, elements_simple_nested_depth] = getTypeTextDeserializePriority(array_type->getNestedType(), nested_depth + 1, simple_nested_depth, priority_map); - return {elements_nested_depth, elements_priority + priority_map[TypeIndex::Array], elements_simple_nested_depth}; + return {elements_nested_depth, elements_priority + priority_map.at(TypeIndex::Array), elements_simple_nested_depth}; } if (const auto * tuple_type = typeid_cast(type.get())) @@ -505,14 +511,14 @@ std::tuple getTypeTextDeserializePriority(const DataType max_simple_nested_depth = elem_simple_nested_depth; } - return {max_nested_depth, sum_priority + priority_map[TypeIndex::Tuple], max_simple_nested_depth}; + return {max_nested_depth, sum_priority + priority_map.at(TypeIndex::Tuple), max_simple_nested_depth}; } if (const auto * map_type = typeid_cast(type.get())) { auto [key_max_depth, key_priority, key_simple_nested_depth] = getTypeTextDeserializePriority(map_type->getKeyType(), nested_depth + 1, simple_nested_depth, priority_map); auto [value_max_depth, value_priority, value_simple_nested_depth] = getTypeTextDeserializePriority(map_type->getValueType(), nested_depth + 1, simple_nested_depth, priority_map); - return {std::max(key_max_depth, value_max_depth), key_priority + value_priority + priority_map[TypeIndex::Map], std::max(key_simple_nested_depth, value_simple_nested_depth)}; + return {std::max(key_max_depth, value_max_depth), key_priority + value_priority + priority_map.at(TypeIndex::Map), std::max(key_simple_nested_depth, value_simple_nested_depth)}; } if (const auto * variant_type = typeid_cast(type.get())) @@ -536,9 +542,10 @@ std::tuple getTypeTextDeserializePriority(const DataType /// Bool type should have priority higher then all integers. if (isBool(type)) - return {nested_depth, priority_map[TypeIndex::Int8] + 1, simple_nested_depth}; + return {nested_depth, priority_map.at(TypeIndex::Int8) + 1, simple_nested_depth}; - return {nested_depth, priority_map[type->getTypeId()], simple_nested_depth}; + auto it = priority_map.find(type->getTypeId()); + return {nested_depth, it == priority_map.end() ? 0 : it->second, simple_nested_depth}; } } @@ -549,7 +556,7 @@ std::vector SerializationVariant::getVariantsDeserializeTextOrder(const priorities.reserve(variant_types.size()); std::vector order; order.reserve(variant_types.size()); - auto priority_map = getTypesTextDeserializePriorityMap(); + const auto & priority_map = getTypesTextDeserializePriorityMap(); for (size_t i = 0; i != variant_types.size(); ++i) { priorities.push_back(getTypeTextDeserializePriority(variant_types[i], 0, 0, priority_map)); From 9e639df12e69c7373e400115977c432b8fdf31f2 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 11 Jan 2024 18:44:05 +0000 Subject: [PATCH 042/264] Add fixes, add new mode to getLeastSupertype and use it in if/multiIf --- .../Serializations/SerializationTuple.cpp | 2 +- src/DataTypes/getLeastSupertype.cpp | 73 ++++++++++++++++--- src/DataTypes/getLeastSupertype.h | 12 +++ src/Functions/if.cpp | 14 +--- src/Functions/multiIf.cpp | 8 +- ...940_variant_text_deserialization.reference | 2 +- 6 files changed, 79 insertions(+), 32 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationTuple.cpp b/src/DataTypes/Serializations/SerializationTuple.cpp index c249ee69e46..5d8c84b70bf 100644 --- a/src/DataTypes/Serializations/SerializationTuple.cpp +++ b/src/DataTypes/Serializations/SerializationTuple.cpp @@ -77,7 +77,7 @@ static ReturnType addElementSafe(size_t num_elems, IColumn & column, F && impl) auto & element_column = extractElementColumn(column, i); if (element_column.size() > old_size) { - chassert(old_size - element_column.size() == 1); + chassert(element_column.size() - old_size == 1); element_column.popBack(1); } } diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index e5bdb4b267f..5d67f888c4b 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -18,6 +18,7 @@ #include #include #include +#include namespace DB @@ -58,6 +59,25 @@ DataTypePtr throwOrReturn(const DataTypes & types, std::string_view message_suff if constexpr (on_error == LeastSupertypeOnError::String) return std::make_shared(); + if constexpr (on_error == LeastSupertypeOnError::Variant && std::is_same_v>) + { + DataTypes variants; + for (const auto & type : types) + { + if (isVariant(type)) + { + const DataTypes & nested_variants = assert_cast(*type).getVariants(); + variants.insert(variants.end(), nested_variants.begin(), nested_variants.end()); + } + else + { + variants.push_back(removeNullableOrLowCardinalityNullable(type)); + } + } + + return std::make_shared(variants); + } + if constexpr (on_error == LeastSupertypeOnError::Null) return nullptr; @@ -67,8 +87,8 @@ DataTypePtr throwOrReturn(const DataTypes & types, std::string_view message_suff throw Exception(error_code, "There is no supertype for types {} {}", getExceptionMessagePrefix(types), message_suffix); } -template -DataTypePtr getNumericType(const TypeIndexSet & types) +template +DataTypePtr getNumericType(const TypeIndexSet & types, ThrowOrReturnFunc throwOrReturnFunc) { bool all_numbers = true; @@ -119,7 +139,7 @@ DataTypePtr getNumericType(const TypeIndexSet & types) if (max_bits_of_signed_integer || max_bits_of_unsigned_integer || max_mantissa_bits_of_floating) { if (!all_numbers) - return throwOrReturn(types, "because some of them are numbers and some of them are not", ErrorCodes::NO_COMMON_TYPE); + return throwOrReturnFunc(types, "because some of them are numbers and some of them are not", ErrorCodes::NO_COMMON_TYPE); /// If there are signed and unsigned types of same bit-width, the result must be signed number with at least one more bit. /// Example, common of Int32, UInt32 = Int64. @@ -134,7 +154,7 @@ DataTypePtr getNumericType(const TypeIndexSet & types) if (min_bit_width_of_integer != 64) ++min_bit_width_of_integer; else - return throwOrReturn(types, + return throwOrReturnFunc(types, "because some of them are signed integers and some are unsigned integers," " but there is no signed integer type, that can exactly represent all required unsigned integer values", ErrorCodes::NO_COMMON_TYPE); @@ -149,7 +169,7 @@ DataTypePtr getNumericType(const TypeIndexSet & types) else if (min_mantissa_bits <= 53) return std::make_shared(); else - return throwOrReturn(types, + return throwOrReturnFunc(types, " because some of them are integers and some are floating point," " but there is no floating point type, that can exactly represent all required integers", ErrorCodes::NO_COMMON_TYPE); } @@ -170,7 +190,7 @@ DataTypePtr getNumericType(const TypeIndexSet & types) else if (min_bit_width_of_integer <= 256) return std::make_shared(); else - return throwOrReturn(types, + return throwOrReturnFunc(types, " because some of them are signed integers and some are unsigned integers," " but there is no signed integer type, that can exactly represent all required unsigned integer values", ErrorCodes::NO_COMMON_TYPE); } @@ -190,7 +210,7 @@ DataTypePtr getNumericType(const TypeIndexSet & types) else if (min_bit_width_of_integer <= 256) return std::make_shared(); else - return throwOrReturn(types, + return throwOrReturnFunc(types, " but as all data types are unsigned integers, we must have found maximum unsigned integer type", ErrorCodes::NO_COMMON_TYPE); } } @@ -382,7 +402,18 @@ DataTypePtr getLeastSupertype(const DataTypes & types) if (!all_maps) return throwOrReturn(types, "because some of them are Maps and some of them are not", ErrorCodes::NO_COMMON_TYPE); - auto keys_common_type = getLeastSupertype(key_types); + DataTypePtr keys_common_type; + if constexpr (on_error == LeastSupertypeOnError::Variant) + { + keys_common_type = getLeastSupertype(key_types); + if (!keys_common_type) + return throwOrReturn(types, "", ErrorCodes::NO_COMMON_TYPE); + } + else + { + keys_common_type = getLeastSupertype(key_types); + } + auto values_common_type = getLeastSupertype(value_types); /// When on_error == LeastSupertypeOnError::Null and we cannot get least supertype for keys or values, /// keys_common_type or values_common_type will be nullptr, we should return nullptr in this case. @@ -423,7 +454,18 @@ DataTypePtr getLeastSupertype(const DataTypes & types) return getLeastSupertype(nested_types); else { - auto nested_type = getLeastSupertype(nested_types); + DataTypePtr nested_type; + if constexpr (on_error == LeastSupertypeOnError::Variant) + { + nested_type = getLeastSupertype(nested_types); + if (!nested_type) + return throwOrReturn(types, "", ErrorCodes::NO_COMMON_TYPE); + } + else + { + nested_type = getLeastSupertype(nested_types); + } + /// When on_error == LeastSupertypeOnError::Null and we cannot get least supertype, /// nested_type will be nullptr, we should return nullptr in this case. if (!nested_type) @@ -456,6 +498,8 @@ DataTypePtr getLeastSupertype(const DataTypes & types) if (have_nullable) { auto nested_type = getLeastSupertype(nested_types); + if (isVariant(nested_type)) + return nested_type; /// When on_error == LeastSupertypeOnError::Null and we cannot get least supertype, /// nested_type will be nullptr, we should return nullptr in this case. if (!nested_type) @@ -623,7 +667,8 @@ DataTypePtr getLeastSupertype(const DataTypes & types) { /// First, if we have signed integers, try to convert all UInt64 to Int64 if possible. convertUInt64toInt64IfPossible(types, type_ids); - auto numeric_type = getNumericType(type_ids); + auto throw_or_return = [&](const TypeIndexSet &, std::string_view message_suffix, int error_code){ return throwOrReturn(types, message_suffix, error_code); }; + auto numeric_type = getNumericType(type_ids, throw_or_return); if (numeric_type) return numeric_type; } @@ -637,6 +682,11 @@ DataTypePtr getLeastSupertypeOrString(const DataTypes & types) return getLeastSupertype(types); } +DataTypePtr getLeastSupertypeOrVariant(const DataTypes & types) +{ + return getLeastSupertype(types); +} + DataTypePtr tryGetLeastSupertype(const DataTypes & types) { return getLeastSupertype(types); @@ -676,7 +726,8 @@ DataTypePtr getLeastSupertype(const TypeIndexSet & types) return std::make_shared(); } - auto numeric_type = getNumericType(types); + auto throw_or_return = [](const TypeIndexSet & type_ids, std::string_view message_suffix, int error_code){ return throwOrReturn(type_ids, message_suffix, error_code); }; + auto numeric_type = getNumericType(types, throw_or_return); if (numeric_type) return numeric_type; diff --git a/src/DataTypes/getLeastSupertype.h b/src/DataTypes/getLeastSupertype.h index 2ef4a0e6850..d949fad69c5 100644 --- a/src/DataTypes/getLeastSupertype.h +++ b/src/DataTypes/getLeastSupertype.h @@ -8,6 +8,7 @@ enum class LeastSupertypeOnError { Throw, String, + Variant, Null, }; @@ -24,6 +25,17 @@ DataTypePtr getLeastSupertype(const DataTypes & types); /// All types can be casted to String, because they can be serialized to String. DataTypePtr getLeastSupertypeOrString(const DataTypes & types); +/// Same as getLeastSupertype but in case when there is no supertype for some types +/// it uses Variant of these types as a supertype. Any type can be casted to a Variant +/// that contains this type. +/// As nested Variants are not allowed, if one of the types is Variant, it's variants +/// are used in the resulting Variant. +/// Examples: +/// (UInt64, String) -> Variant(UInt64, String) +/// (Array(UInt64), Array(String)) -> Array(Variant(UInt64, String)) +/// (Variant(UInt64, String), Array(UInt32)) -> Variant(UInt64, String, Array(UInt32)) +DataTypePtr getLeastSupertypeOrVariant(const DataTypes & types); + /// Same as above but return nullptr instead of throwing exception. DataTypePtr tryGetLeastSupertype(const DataTypes & types); diff --git a/src/Functions/if.cpp b/src/Functions/if.cpp index 1dc7443f124..c247938f885 100644 --- a/src/Functions/if.cpp +++ b/src/Functions/if.cpp @@ -688,15 +688,9 @@ private: DataTypePtr common_type; if (use_variant_when_no_common_type) - { - common_type = tryGetLeastSupertype(DataTypes{arg1.type, arg2.type}); - if (!common_type) - common_type = std::make_shared(DataTypes{removeNullableOrLowCardinalityNullable(arg1.type), removeNullableOrLowCardinalityNullable(arg2.type)}); - } + common_type = getLeastSupertypeOrVariant(DataTypes{arg1.type, arg2.type}); else - { common_type = getLeastSupertype(DataTypes{arg1.type, arg2.type}); - } ColumnPtr col_then = castColumn(arg1, common_type); ColumnPtr col_else = castColumn(arg2, common_type); @@ -1118,11 +1112,7 @@ public: "Must be UInt8.", arguments[0]->getName()); if (use_variant_when_no_common_type) - { - if (auto res = tryGetLeastSupertype(DataTypes{arguments[1], arguments[2]})) - return res; - return std::make_shared(DataTypes{removeNullableOrLowCardinalityNullable(arguments[1]), removeNullableOrLowCardinalityNullable(arguments[2])}); - } + return getLeastSupertypeOrVariant(DataTypes{arguments[1], arguments[2]}); return getLeastSupertype(DataTypes{arguments[1], arguments[2]}); } diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index 7a2e9444b2c..cefbea9f352 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -119,13 +119,7 @@ public: }); if (context->getSettingsRef().allow_experimental_variant_type && context->getSettingsRef().use_variant_when_no_common_type_in_if) - { - if (auto res = tryGetLeastSupertype(types_of_branches)) - return res; - for (auto & type : types_of_branches) - type = removeNullableOrLowCardinalityNullable(type); - return std::make_shared(types_of_branches); - } + return getLeastSupertypeOrVariant(types_of_branches); return getLeastSupertype(types_of_branches); } diff --git a/tests/queries/0_stateless/02940_variant_text_deserialization.reference b/tests/queries/0_stateless/02940_variant_text_deserialization.reference index 98725917567..8836e6c4e57 100644 --- a/tests/queries/0_stateless/02940_variant_text_deserialization.reference +++ b/tests/queries/0_stateless/02940_variant_text_deserialization.reference @@ -505,7 +505,7 @@ String (NULL,NULL),('string',NULL),(-1,-1),(0,0),(10000000000,NULL)(NULL,NULL),('string',NULL),(-1,NULL),(0,0),(10000000000,NULL)(NULL,NULL),('string',NULL),(-1,-1),(0,0),(10000000000,NULL)(NULL,NULL),('string',NULL),(-1,NULL),(0,0),(10000000000,NULL)(NULL,NULL),('string',NULL),(-1,-1),(0,0),(10000000000,NULL)(NULL,NULL),('string',NULL),(-1,NULL),(0,0),(10000000000,NULL)(NULL,NULL),('string',NULL),(-1,-1),(0,0),(10000000000000000000000,NULL)(NULL,NULL),('string',NULL),(-1,NULL),(0,0),(10000000000000000000000,NULL)(NULL,NULL),('string',NULL),(-1,-1),(0,0)(NULL,NULL),('string',NULL),(-1,NULL),(0,0)Floats (NULL,NULL),('string',NULL),(42.42,42.42)(NULL,NULL),('string',NULL),(42.42,42.42)Decimals (NULL,NULL),('string',NULL),(42.42,42.42)(NULL,NULL),('string',NULL),(42.42,42.42)(NULL,NULL),('string',NULL),(42.42,42.42)(NULL,NULL),('string',NULL),(42.42,42.42)Dates and DateTimes -(NULL,NULL),('string',NULL),('1970-01-01 00:00:00.000',NULL),('2020-01-01','2020-01-01'),('2020-01-01 00:00:00.999',NULL)(NULL,NULL),('string',NULL),('1970-01-01 00:00:00.000',NULL),('1900-01-01','1900-01-01'),('2020-01-01 00:00:00.999',NULL)(NULL,NULL),('string',NULL),('1970-01-01 00:00:00.000',NULL),('2020-01-01 00:00:00','2020-01-01 00:00:00'),('2020-01-01 00:00:00.999',NULL)(NULL,NULL),('string',NULL),('1970-01-01 00:00:00.000','1970-01-01 00:00:00.000'),('2020-01-01 00:00:00.999',NULL),('2020-01-01 00:00:00.999999999 ABC',NULL)UUID +(NULL,NULL),('string',NULL),('2020-01-d1',NULL),('2020-01-01','2020-01-01'),('2020-01-01 00:00:00.999',NULL)(NULL,NULL),('string',NULL),('2020-01-d1',NULL),('1900-01-01','1900-01-01'),('2020-01-01 00:00:00.999',NULL)(NULL,NULL),('string',NULL),('2020-01-d1',NULL),('2020-01-01 00:00:00','2020-01-01 00:00:00'),('2020-01-01 00:00:00.999',NULL)(NULL,NULL),('string',NULL),('2020-01-d1',NULL),('2020-01-01 00:00:00.999','2020-01-01 00:00:00.999'),('2020-01-01 00:00:00.999999999 ABC',NULL)UUID (NULL,NULL),('string',NULL),('c8619cca-0caa-445e-ae76-1d4f6e0b3927','c8619cca-0caa-445e-ae76-1d4f6e0b3927'),('c8619cca-0caa-445e-ae76-1d4f6e0b3927AAA',NULL)IPv4 (NULL,NULL),('string',NULL),('127.0.0.1','127.0.0.1'),('127.0.0.1AAA',NULL)IPv6 (NULL,NULL),('string',NULL),('2001:db8:85a3::8a2e:370:7334','2001:db8:85a3::8a2e:370:7334'),('2001:0db8:85a3:0000:0000:8a2e:0370:7334AAA',NULL)Enum From c14a0e80c638f69f7787583d44f4ca3620629b23 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 12 Jan 2024 13:23:36 +0100 Subject: [PATCH 043/264] Fxi --- src/Interpreters/Cache/FileSegment.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index ca2112e526b..17f86b6b5ba 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -820,7 +820,7 @@ bool FileSegment::assertCorrectnessUnlocked(const FileSegmentGuard::Lock & lock) std::lock_guard lk(write_mutex); if (downloaded_size == 0) { - if (fs::exists(file_path)) + if (download_state != State::DOWNLOADING && fs::exists(file_path)) throw_logical("Expected file " + file_path + " not to exist"); } else if (!fs::exists(file_path)) From 18e31df388a7130c03488733c5a9b90ca78e1f1d Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 12 Jan 2024 13:24:59 +0100 Subject: [PATCH 044/264] One more fi --- src/Interpreters/Cache/FileSegment.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 17f86b6b5ba..d2644843927 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -807,7 +807,7 @@ bool FileSegment::assertCorrectnessUnlocked(const FileSegmentGuard::Lock & lock) return; const auto & entry = it->getEntry(); - if (entry.size != reserved_size) + if (download_state != State::DOWNLOADING && entry.size != reserved_size) throw_logical(fmt::format("Expected entry.size == reserved_size ({} == {})", entry.size, reserved_size)); chassert(entry.key == key()); From f755e775349ac10207378e0774c4b76559fbd7c8 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Sun, 14 Jan 2024 09:36:32 +0000 Subject: [PATCH 045/264] move column level setting to the end of declaration Signed-off-by: Duc Canh Le --- src/Interpreters/InterpreterCreateQuery.cpp | 16 ++++++++-------- src/Parsers/ASTColumnDeclaration.cpp | 12 ++++++------ 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 01eedca2184..f3ef1e3359e 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -453,14 +453,6 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns) column_declaration->children.push_back(column_declaration->codec); } - if (!column.settings.empty()) - { - auto per_column_settings = std::make_shared(); - per_column_settings->is_standalone = false; - per_column_settings->changes = column.settings; - column_declaration->per_column_settings = std::move(per_column_settings); - } - if (column.stat) { column_declaration->stat_type = column.stat->ast; @@ -473,6 +465,14 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns) column_declaration->children.push_back(column_declaration->ttl); } + if (!column.settings.empty()) + { + auto per_column_settings = std::make_shared(); + per_column_settings->is_standalone = false; + per_column_settings->changes = column.settings; + column_declaration->per_column_settings = std::move(per_column_settings); + } + columns_list->children.push_back(column_declaration_ptr); } diff --git a/src/Parsers/ASTColumnDeclaration.cpp b/src/Parsers/ASTColumnDeclaration.cpp index 196780ccf29..dd3bf6ae8d8 100644 --- a/src/Parsers/ASTColumnDeclaration.cpp +++ b/src/Parsers/ASTColumnDeclaration.cpp @@ -39,12 +39,6 @@ ASTPtr ASTColumnDeclaration::clone() const res->children.push_back(res->codec); } - if (per_column_settings) - { - res->per_column_settings = per_column_settings->clone(); - res->children.push_back(res->per_column_settings); - } - if (stat_type) { res->stat_type = stat_type->clone(); @@ -63,6 +57,12 @@ ASTPtr ASTColumnDeclaration::clone() const res->children.push_back(res->collation); } + if (per_column_settings) + { + res->per_column_settings = per_column_settings->clone(); + res->children.push_back(res->per_column_settings); + } + return res; } From 826b29ef4d52a30c3fc06d8ba0e171808e0a4aee Mon Sep 17 00:00:00 2001 From: zhongyuankai <872237106@qq.com> Date: Wed, 20 Dec 2023 13:14:08 +0800 Subject: [PATCH 046/264] Max materialized views size for table --- .../settings.md | 7 +++ src/Common/ErrorCodes.cpp | 1 + src/Core/ServerSettings.h | 1 + src/Storages/StorageMaterializedView.cpp | 9 ++++ .../__init__.py | 0 .../configs/max_num_limit.xml | 3 ++ .../test.py | 48 +++++++++++++++++++ 7 files changed, 69 insertions(+) create mode 100644 tests/integration/test_limit_materialized_view_count/__init__.py create mode 100644 tests/integration/test_limit_materialized_view_count/configs/max_num_limit.xml create mode 100644 tests/integration/test_limit_materialized_view_count/test.py diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 48434d992e2..1cbf9deccc6 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -2866,3 +2866,10 @@ This also allows a mix of resolver types can be used. ### disable_tunneling_for_https_requests_over_http_proxy {#disable_tunneling_for_https_requests_over_http_proxy} By default, tunneling (i.e, `HTTP CONNECT`) is used to make `HTTPS` requests over `HTTP` proxy. This setting can be used to disable it. + +## max_materialized_views_count_for_table {#max_materialized_views_count_for_table} + +A limit on the number of materialized views attached to a table. +Note that only directly dependent views are considered here, and the creation of one view on top of another view is not considered. + +Default value: `0`. diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 577a83e40b9..5b0b2fad3fa 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -590,6 +590,7 @@ M(708, ILLEGAL_STATISTIC) \ M(709, CANNOT_GET_REPLICATED_DATABASE_SNAPSHOT) \ M(710, FAULT_INJECTED) \ + M(711, TOO_MANY_MATERIALIZED_VIEWS) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 2a9fa8e744c..dced2129160 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -112,6 +112,7 @@ namespace DB M(UInt64, total_memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \ M(Bool, validate_tcp_client_information, false, "Validate client_information in the query packet over the native TCP protocol.", 0) \ M(Bool, storage_metadata_write_full_object_key, false, "Write disk metadata files with VERSION_FULL_OBJECT_KEY format", 0) \ + M(UInt64, max_materialized_views_count_for_table, 0, "A limit on the number of materialized views attached to a table.", 0) \ /// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index ae616b1df04..569b36a9780 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -21,6 +21,7 @@ #include #include +#include #include #include #include @@ -38,6 +39,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int NOT_IMPLEMENTED; extern const int INCORRECT_QUERY; + extern const int TOO_MANY_MATERIALIZED_VIEWS; } namespace ActionLocks @@ -87,6 +89,13 @@ StorageMaterializedView::StorageMaterializedView( "either ENGINE or an existing table in a TO clause"); auto select = SelectQueryDescription::getSelectQueryFromASTForMatView(query.select->clone(), query.refresh_strategy != nullptr, local_context); + auto select_table_dependent_views = DatabaseCatalog::instance().getDependentViews(select.select_table_id); + + auto max_materialized_views_count_for_table = getContext()->getServerSettings().max_materialized_views_count_for_table; + if (max_materialized_views_count_for_table && select_table_dependent_views.size() >= max_materialized_views_count_for_table) + throw Exception(ErrorCodes::TOO_MANY_MATERIALIZED_VIEWS, + "Too many materialized views, maximum: {}", max_materialized_views_count_for_table); + storage_metadata.setSelectQuery(select); if (!comment.empty()) storage_metadata.setComment(comment); diff --git a/tests/integration/test_limit_materialized_view_count/__init__.py b/tests/integration/test_limit_materialized_view_count/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_limit_materialized_view_count/configs/max_num_limit.xml b/tests/integration/test_limit_materialized_view_count/configs/max_num_limit.xml new file mode 100644 index 00000000000..7fcf9005a08 --- /dev/null +++ b/tests/integration/test_limit_materialized_view_count/configs/max_num_limit.xml @@ -0,0 +1,3 @@ + + 1 + diff --git a/tests/integration/test_limit_materialized_view_count/test.py b/tests/integration/test_limit_materialized_view_count/test.py new file mode 100644 index 00000000000..18d30a8fc2e --- /dev/null +++ b/tests/integration/test_limit_materialized_view_count/test.py @@ -0,0 +1,48 @@ +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node", + main_configs=["configs/max_num_limit.xml"], +) + +config = """ + 2 + +""" + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + for _, node in cluster.instances.items(): + node.query( + f""" + CREATE TABLE test_tb (a String) ENGINE = MergeTree ORDER BY a; + """ + ) + yield cluster + finally: + cluster.shutdown() + + +def test_limit_materialized_view_count(started_cluster): + node.query( + "CREATE MATERIALIZED VIEW test_view1 ENGINE = MergeTree ORDER BY a AS SELECT * FROM test_tb;" + ) + assert "Too many materialized views" in node.query_and_get_error( + "CREATE MATERIALIZED VIEW test_view2 ENGINE = MergeTree ORDER BY a AS SELECT * FROM test_tb;" + ) + + node.replace_config("/etc/clickhouse-server/config.d/max_num_limit.xml", config) + node.query("SYSTEM RELOAD CONFIG;") + + node.query( + "CREATE MATERIALIZED VIEW test_view2 ENGINE = MergeTree ORDER BY a AS SELECT * FROM test_tb;" + ) + assert "Too many materialized views" in node.query_and_get_error( + "CREATE MATERIALIZED VIEW test_view3 ENGINE = MergeTree ORDER BY a AS SELECT * FROM test_tb;" + ) From cca2fa69767b09011db8fe89a43f4763061d3a96 Mon Sep 17 00:00:00 2001 From: zhongyuankai <872237106@qq.com> Date: Mon, 15 Jan 2024 20:22:31 +0800 Subject: [PATCH 047/264] fix test --- src/Storages/StorageMaterializedView.cpp | 13 ++++++++----- .../test_limit_materialized_view_count/test.py | 3 ++- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 569b36a9780..050e76c9205 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -89,12 +89,15 @@ StorageMaterializedView::StorageMaterializedView( "either ENGINE or an existing table in a TO clause"); auto select = SelectQueryDescription::getSelectQueryFromASTForMatView(query.select->clone(), query.refresh_strategy != nullptr, local_context); - auto select_table_dependent_views = DatabaseCatalog::instance().getDependentViews(select.select_table_id); + if (select.select_table_id) + { + auto select_table_dependent_views = DatabaseCatalog::instance().getDependentViews(select.select_table_id); - auto max_materialized_views_count_for_table = getContext()->getServerSettings().max_materialized_views_count_for_table; - if (max_materialized_views_count_for_table && select_table_dependent_views.size() >= max_materialized_views_count_for_table) - throw Exception(ErrorCodes::TOO_MANY_MATERIALIZED_VIEWS, - "Too many materialized views, maximum: {}", max_materialized_views_count_for_table); + auto max_materialized_views_count_for_table = getContext()->getServerSettings().max_materialized_views_count_for_table; + if (max_materialized_views_count_for_table && select_table_dependent_views.size() >= max_materialized_views_count_for_table) + throw Exception(ErrorCodes::TOO_MANY_MATERIALIZED_VIEWS, + "Too many materialized views, maximum: {}", max_materialized_views_count_for_table); + } storage_metadata.setSelectQuery(select); if (!comment.empty()) diff --git a/tests/integration/test_limit_materialized_view_count/test.py b/tests/integration/test_limit_materialized_view_count/test.py index 18d30a8fc2e..c14c5b2055e 100644 --- a/tests/integration/test_limit_materialized_view_count/test.py +++ b/tests/integration/test_limit_materialized_view_count/test.py @@ -6,6 +6,7 @@ cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", main_configs=["configs/max_num_limit.xml"], + stay_alive=True, ) config = """ @@ -38,7 +39,7 @@ def test_limit_materialized_view_count(started_cluster): ) node.replace_config("/etc/clickhouse-server/config.d/max_num_limit.xml", config) - node.query("SYSTEM RELOAD CONFIG;") + node.restart_clickhouse() node.query( "CREATE MATERIALIZED VIEW test_view2 ENGINE = MergeTree ORDER BY a AS SELECT * FROM test_tb;" From e6c699cf77fda32dab63f68d1ccab3f2660d1bae Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 16 Jan 2024 21:14:47 +0100 Subject: [PATCH 048/264] Revert "Merge pull request #57640 from ClickHouse/remove-bad-test-7" This reverts commit 9c4a1ff63ecaf5db547a1b8710b32eeba06acbbe, reversing changes made to c97d33abf967cc747785d8a87dd2dbe6dad2e3d0. --- ...avoiding_when_reading_from_cache.reference | 1 + ...o_seek_avoiding_when_reading_from_cache.sh | 36 +++++++++++++++++++ 2 files changed, 37 insertions(+) create mode 100644 tests/queries/1_stateful/00180_no_seek_avoiding_when_reading_from_cache.reference create mode 100755 tests/queries/1_stateful/00180_no_seek_avoiding_when_reading_from_cache.sh diff --git a/tests/queries/1_stateful/00180_no_seek_avoiding_when_reading_from_cache.reference b/tests/queries/1_stateful/00180_no_seek_avoiding_when_reading_from_cache.reference new file mode 100644 index 00000000000..d05b1f927f4 --- /dev/null +++ b/tests/queries/1_stateful/00180_no_seek_avoiding_when_reading_from_cache.reference @@ -0,0 +1 @@ +0 0 diff --git a/tests/queries/1_stateful/00180_no_seek_avoiding_when_reading_from_cache.sh b/tests/queries/1_stateful/00180_no_seek_avoiding_when_reading_from_cache.sh new file mode 100755 index 00000000000..2e1b807c496 --- /dev/null +++ b/tests/queries/1_stateful/00180_no_seek_avoiding_when_reading_from_cache.sh @@ -0,0 +1,36 @@ +#!/usr/bin/env bash + +# Tags: no-parallel, no-random-settings, long + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +# Test assumes that the whole table is residing in the cache, but `hits_s3` has only 128Mi of cache. +# So we need to create a smaller table. +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS hits_s3_sampled" +$CLICKHOUSE_CLIENT -q "CREATE TABLE hits_s3_sampled AS test.hits_s3" +$CLICKHOUSE_CLIENT -q "INSERT INTO hits_s3_sampled SELECT * FROM test.hits_s3 SAMPLE 0.01" +$CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE hits_s3_sampled FINAL" + +$CLICKHOUSE_CLIENT -q "SYSTEM DROP FILESYSTEM CACHE" + +# Warm up the cache +$CLICKHOUSE_CLIENT -q "SELECT * FROM hits_s3_sampled WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10 FORMAT Null" +$CLICKHOUSE_CLIENT -q "SELECT * FROM hits_s3_sampled WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10 FORMAT Null" + +query_id=02906_read_from_cache_$RANDOM +$CLICKHOUSE_CLIENT --query_id ${query_id} -q "SELECT * FROM hits_s3_sampled WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10 FORMAT Null" + +$CLICKHOUSE_CLIENT -nq " + SYSTEM FLUSH LOGS; + + -- AsynchronousReaderIgnoredBytes = 0: no seek-avoiding happened + -- CachedReadBufferReadFromSourceBytes = 0: sanity check to ensure we read only from cache + SELECT ProfileEvents['AsynchronousReaderIgnoredBytes'], ProfileEvents['CachedReadBufferReadFromSourceBytes'] + FROM system.query_log + WHERE query_id = '$query_id' AND type = 'QueryFinish' AND event_date >= yesterday() AND current_database = currentDatabase() +" + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS hits_s3_sampled" From 299293c0889eaaef9dc7576d9a5ef33ae02e3e0c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 16 Jan 2024 23:19:00 +0000 Subject: [PATCH 049/264] better interface in read buffers --- src/Disks/DiskEncrypted.cpp | 4 +- src/Disks/DiskEncryptedTransaction.cpp | 1 - src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 6 +- src/Disks/IO/ReadBufferFromRemoteFSGather.h | 4 +- src/Disks/IO/createReadBufferFromFileBase.cpp | 2 +- .../ObjectStorages/DiskObjectStorage.cpp | 2 +- src/IO/Archives/LibArchiveReader.cpp | 7 ++- src/IO/Archives/ZipArchiveReader.cpp | 6 ++ src/IO/BoundedReadBuffer.cpp | 3 +- src/IO/BoundedReadBuffer.h | 10 +++- src/IO/LimitSeekableReadBuffer.h | 1 - src/IO/MMapReadBufferFromFileDescriptor.cpp | 5 ++ src/IO/MMapReadBufferFromFileDescriptor.h | 2 + src/IO/MMapReadBufferFromFileWithCache.cpp | 5 ++ src/IO/MMapReadBufferFromFileWithCache.h | 2 +- src/IO/ReadBufferFromEmptyFile.h | 8 ++- src/IO/ReadBufferFromEncryptedFile.cpp | 12 ++++ src/IO/ReadBufferFromEncryptedFile.h | 4 +- src/IO/ReadBufferFromFileBase.h | 6 ++ src/IO/ReadBufferFromFileDecorator.cpp | 60 ------------------- src/IO/ReadBufferFromFileDecorator.h | 37 ------------ src/IO/ReadBufferFromMemory.h | 1 - src/IO/SeekableReadBuffer.h | 6 -- src/IO/WithFileSize.cpp | 23 +------ .../HDFS/AsynchronousReadBufferFromHDFS.cpp | 2 +- .../HDFS/AsynchronousReadBufferFromHDFS.h | 2 +- 26 files changed, 74 insertions(+), 147 deletions(-) delete mode 100644 src/IO/ReadBufferFromFileDecorator.cpp delete mode 100644 src/IO/ReadBufferFromFileDecorator.h diff --git a/src/Disks/DiskEncrypted.cpp b/src/Disks/DiskEncrypted.cpp index ac81899156a..01710fb2b63 100644 --- a/src/Disks/DiskEncrypted.cpp +++ b/src/Disks/DiskEncrypted.cpp @@ -4,9 +4,9 @@ #include #include #include -#include #include #include +#include #include #include #include @@ -374,7 +374,7 @@ std::unique_ptr DiskEncrypted::readFile( { /// File is empty, that's a normal case, see DiskEncrypted::truncateFile(). /// There is no header so we just return `ReadBufferFromString("")`. - return std::make_unique(std::make_unique(std::string_view{}), wrapped_path); + return std::make_unique(wrapped_path); } auto encryption_settings = current_settings.get(); FileEncryption::Header header = readHeader(*buffer); diff --git a/src/Disks/DiskEncryptedTransaction.cpp b/src/Disks/DiskEncryptedTransaction.cpp index daeab7aae6c..3da2e6f925a 100644 --- a/src/Disks/DiskEncryptedTransaction.cpp +++ b/src/Disks/DiskEncryptedTransaction.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 5ff02ae464c..05093b13375 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -1,6 +1,6 @@ #include "ReadBufferFromRemoteFSGather.h" -#include +#include #include #include @@ -61,7 +61,7 @@ ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather( current_object = blobs_to_read.front(); } -SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(const StoredObject & object) +std::unique_ptr ReadBufferFromRemoteFSGather::createImplementationBuffer(const StoredObject & object) { if (current_buf && !with_cache) { @@ -78,7 +78,7 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(c if (with_cache) { auto cache_key = settings.remote_fs_cache->createKeyForPath(object_path); - return std::make_shared( + return std::make_unique( object_path, cache_key, settings.remote_fs_cache, diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h index 2764ed2d021..c5886dea603 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.h @@ -53,7 +53,7 @@ public: bool isContentCached(size_t offset, size_t size) override; private: - SeekableReadBufferPtr createImplementationBuffer(const StoredObject & object); + std::unique_ptr createImplementationBuffer(const StoredObject & object); bool nextImpl() override; @@ -80,7 +80,7 @@ private: StoredObject current_object; size_t current_buf_idx = 0; - SeekableReadBufferPtr current_buf; + std::unique_ptr current_buf; Poco::Logger * log; }; diff --git a/src/Disks/IO/createReadBufferFromFileBase.cpp b/src/Disks/IO/createReadBufferFromFileBase.cpp index 236dd43e9ee..f27352dcd01 100644 --- a/src/Disks/IO/createReadBufferFromFileBase.cpp +++ b/src/Disks/IO/createReadBufferFromFileBase.cpp @@ -40,7 +40,7 @@ std::unique_ptr createReadBufferFromFileBase( bool use_external_buffer) { if (file_size.has_value() && !*file_size) - return std::make_unique(); + return std::make_unique(filename); size_t estimated_size = 0; if (read_hint.has_value()) diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 6962248c7e1..48d6163f895 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -524,7 +524,7 @@ std::unique_ptr DiskObjectStorage::readFile( const bool file_can_be_empty = !file_size.has_value() || *file_size == 0; if (storage_objects.empty() && file_can_be_empty) - return std::make_unique(); + return std::make_unique(path); return object_storage->readObjects( storage_objects, diff --git a/src/IO/Archives/LibArchiveReader.cpp b/src/IO/Archives/LibArchiveReader.cpp index a411b4bb4b6..8442c03637b 100644 --- a/src/IO/Archives/LibArchiveReader.cpp +++ b/src/IO/Archives/LibArchiveReader.cpp @@ -226,7 +226,12 @@ public: off_t getPosition() override { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "getPosition not supported when reading from archive"); + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "getPosition is not supported when reading from archive"); + } + + size_t getFileOffsetOfBufferEnd() const override + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "getFileOffsetOfBufferEnd is not supported when reading from archive"); } String getFileName() const override { return handle.getFileName(); } diff --git a/src/IO/Archives/ZipArchiveReader.cpp b/src/IO/Archives/ZipArchiveReader.cpp index fd7a09c4f20..ab7dff3e147 100644 --- a/src/IO/Archives/ZipArchiveReader.cpp +++ b/src/IO/Archives/ZipArchiveReader.cpp @@ -15,6 +15,7 @@ namespace ErrorCodes extern const int CANNOT_UNPACK_ARCHIVE; extern const int LOGICAL_ERROR; extern const int SEEK_POSITION_OUT_OF_BOUND; + extern const int UNSUPPORTED_METHOD; } using RawHandle = unzFile; @@ -251,6 +252,11 @@ public: checkResult(err); } + size_t getFileOffsetOfBufferEnd() const override + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "getFileOffsetOfBufferEnd is not supported when reading from zip archive"); + } + off_t seek(off_t off, int whence) override { off_t current_pos = getPosition(); diff --git a/src/IO/BoundedReadBuffer.cpp b/src/IO/BoundedReadBuffer.cpp index bda79d82ad3..f3b176a963c 100644 --- a/src/IO/BoundedReadBuffer.cpp +++ b/src/IO/BoundedReadBuffer.cpp @@ -4,8 +4,7 @@ namespace DB { -BoundedReadBuffer::BoundedReadBuffer(std::unique_ptr impl_) - : ReadBufferFromFileDecorator(std::move(impl_)) +BoundedReadBuffer::BoundedReadBuffer(std::unique_ptr impl_) : impl(std::move(impl_)) { } diff --git a/src/IO/BoundedReadBuffer.h b/src/IO/BoundedReadBuffer.h index eb65857e83a..22a6471a9a1 100644 --- a/src/IO/BoundedReadBuffer.h +++ b/src/IO/BoundedReadBuffer.h @@ -1,5 +1,5 @@ #pragma once -#include +#include namespace DB @@ -7,10 +7,10 @@ namespace DB /// A buffer which allows to make an underlying buffer as right bounded, /// e.g. the buffer cannot return data beyond offset specified in `setReadUntilPosition`. -class BoundedReadBuffer : public ReadBufferFromFileDecorator +class BoundedReadBuffer : public ReadBufferFromFileBase { public: - explicit BoundedReadBuffer(std::unique_ptr impl_); + explicit BoundedReadBuffer(std::unique_ptr impl_); bool supportsRightBoundedReads() const override { return true; } @@ -23,6 +23,8 @@ public: off_t seek(off_t off, int whence) override; size_t getFileOffsetOfBufferEnd() const override { return file_offset_of_buffer_end; } + String getFileName() const override { return impl->getFileName(); } + size_t getFileSize() override { return impl->getFileSize(); } /// file_offset_of_buffer_end can differ from impl's file_offset_of_buffer_end /// because of resizing of the tail. => Need to also override getPosition() as @@ -30,6 +32,8 @@ public: off_t getPosition() override; private: + std::unique_ptr impl; + std::optional read_until_position; /// atomic because can be used in log or exception messages while being updated. std::atomic file_offset_of_buffer_end = 0; diff --git a/src/IO/LimitSeekableReadBuffer.h b/src/IO/LimitSeekableReadBuffer.h index 61b307c522c..5624388dd7e 100644 --- a/src/IO/LimitSeekableReadBuffer.h +++ b/src/IO/LimitSeekableReadBuffer.h @@ -18,7 +18,6 @@ public: /// Returns adjusted position, i.e. returns `3` if the position in the nested buffer is `start_offset + 3`. off_t getPosition() override; - off_t seek(off_t off, int whence) override; private: diff --git a/src/IO/MMapReadBufferFromFileDescriptor.cpp b/src/IO/MMapReadBufferFromFileDescriptor.cpp index 9b1c132cc01..56a094bb1a3 100644 --- a/src/IO/MMapReadBufferFromFileDescriptor.cpp +++ b/src/IO/MMapReadBufferFromFileDescriptor.cpp @@ -92,6 +92,11 @@ size_t MMapReadBufferFromFileDescriptor::getFileSize() return getSizeFromFileDescriptor(getFD(), getFileName()); } +size_t MMapReadBufferFromFileDescriptor::getFileOffsetOfBufferEnd() const +{ + return mapped.getOffset() + mapped.getLength(); +} + size_t MMapReadBufferFromFileDescriptor::readBigAt(char * to, size_t n, size_t offset, const std::function &) { if (offset >= mapped.getLength()) diff --git a/src/IO/MMapReadBufferFromFileDescriptor.h b/src/IO/MMapReadBufferFromFileDescriptor.h index 2a039e04971..97d8bbe224d 100644 --- a/src/IO/MMapReadBufferFromFileDescriptor.h +++ b/src/IO/MMapReadBufferFromFileDescriptor.h @@ -36,6 +36,8 @@ public: std::string getFileName() const override; + size_t getFileOffsetOfBufferEnd() const override; + int getFD() const; size_t getFileSize() override; diff --git a/src/IO/MMapReadBufferFromFileWithCache.cpp b/src/IO/MMapReadBufferFromFileWithCache.cpp index d53f3bc325d..f3c4d6f4e01 100644 --- a/src/IO/MMapReadBufferFromFileWithCache.cpp +++ b/src/IO/MMapReadBufferFromFileWithCache.cpp @@ -76,4 +76,9 @@ off_t MMapReadBufferFromFileWithCache::seek(off_t offset, int whence) return new_pos; } +size_t MMapReadBufferFromFileWithCache::getFileOffsetOfBufferEnd() const +{ + return mapped->getOffset() + mapped->getLength(); +} + } diff --git a/src/IO/MMapReadBufferFromFileWithCache.h b/src/IO/MMapReadBufferFromFileWithCache.h index cb87b03df8d..ce5da29831e 100644 --- a/src/IO/MMapReadBufferFromFileWithCache.h +++ b/src/IO/MMapReadBufferFromFileWithCache.h @@ -19,7 +19,7 @@ public: off_t getPosition() override; std::string getFileName() const override; off_t seek(off_t offset, int whence) override; - + size_t getFileOffsetOfBufferEnd() const override; bool isRegularLocalFile(size_t * /* out_view_offset */) override { return true; } private: diff --git a/src/IO/ReadBufferFromEmptyFile.h b/src/IO/ReadBufferFromEmptyFile.h index f21f2f507dc..e2765765c47 100644 --- a/src/IO/ReadBufferFromEmptyFile.h +++ b/src/IO/ReadBufferFromEmptyFile.h @@ -14,12 +14,18 @@ namespace DB /// - ThreadPoolReader class ReadBufferFromEmptyFile : public ReadBufferFromFileBase { +public: + explicit ReadBufferFromEmptyFile(const String & file_name_) : file_name(file_name_) {} + private: + String file_name; + bool nextImpl() override { return false; } - std::string getFileName() const override { return ""; } + std::string getFileName() const override { return file_name; } off_t seek(off_t /*off*/, int /*whence*/) override { return 0; } off_t getPosition() override { return 0; } size_t getFileSize() override { return 0; } + size_t getFileOffsetOfBufferEnd() const override { return 0; } }; } diff --git a/src/IO/ReadBufferFromEncryptedFile.cpp b/src/IO/ReadBufferFromEncryptedFile.cpp index f9cf1597153..6861ae06dd8 100644 --- a/src/IO/ReadBufferFromEncryptedFile.cpp +++ b/src/IO/ReadBufferFromEncryptedFile.cpp @@ -101,6 +101,18 @@ bool ReadBufferFromEncryptedFile::nextImpl() return true; } +size_t ReadBufferFromEncryptedFile::getFileSize() +{ + size_t size = in->getFileSize(); + return size > FileEncryption::Header::kSize ? size - FileEncryption::Header::kSize : size; +} + +size_t ReadBufferFromEncryptedFile::getFileOffsetOfBufferEnd() const +{ + size_t file_offset = in->getFileOffsetOfBufferEnd(); + return file_offset > FileEncryption::Header::kSize ? file_offset - FileEncryption::Header::kSize : file_offset; +} + } #endif diff --git a/src/IO/ReadBufferFromEncryptedFile.h b/src/IO/ReadBufferFromEncryptedFile.h index 3626daccb3e..2f5093153ea 100644 --- a/src/IO/ReadBufferFromEncryptedFile.h +++ b/src/IO/ReadBufferFromEncryptedFile.h @@ -27,10 +27,10 @@ public: std::string getFileName() const override { return in->getFileName(); } void setReadUntilPosition(size_t position) override { in->setReadUntilPosition(position + FileEncryption::Header::kSize); } - void setReadUntilEnd() override { in->setReadUntilEnd(); } - size_t getFileSize() override { return in->getFileSize(); } + size_t getFileSize() override; + size_t getFileOffsetOfBufferEnd() const override; private: bool nextImpl() override; diff --git a/src/IO/ReadBufferFromFileBase.h b/src/IO/ReadBufferFromFileBase.h index 296edf9c689..b9288ce6636 100644 --- a/src/IO/ReadBufferFromFileBase.h +++ b/src/IO/ReadBufferFromFileBase.h @@ -60,6 +60,12 @@ public: /// file offset and what getPosition() returns. virtual bool isRegularLocalFile(size_t * /* out_view_offset */ = nullptr) { return false; } + /// NOTE: This method should be thread-safe against seek(), since it can be + /// used in CachedOnDiskReadBufferFromFile from multiple threads (because + /// it first releases the buffer, and then do logging, and so other thread + /// can already call seek() which will lead to data-race). + virtual size_t getFileOffsetOfBufferEnd() const = 0; + protected: std::optional file_size; ProfileCallback profile_callback; diff --git a/src/IO/ReadBufferFromFileDecorator.cpp b/src/IO/ReadBufferFromFileDecorator.cpp deleted file mode 100644 index 9ac0fb4e475..00000000000 --- a/src/IO/ReadBufferFromFileDecorator.cpp +++ /dev/null @@ -1,60 +0,0 @@ -#include - - -namespace DB -{ - -ReadBufferFromFileDecorator::ReadBufferFromFileDecorator(std::unique_ptr impl_) - : ReadBufferFromFileDecorator(std::move(impl_), "") -{ -} - - -ReadBufferFromFileDecorator::ReadBufferFromFileDecorator(std::unique_ptr impl_, const String & file_name_) - : impl(std::move(impl_)), file_name(file_name_) -{ - swap(*impl); -} - - -std::string ReadBufferFromFileDecorator::getFileName() const -{ - if (!file_name.empty()) - return file_name; - - return getFileNameFromReadBuffer(*impl); -} - - -off_t ReadBufferFromFileDecorator::getPosition() -{ - swap(*impl); - auto position = impl->getPosition(); - swap(*impl); - return position; -} - - -off_t ReadBufferFromFileDecorator::seek(off_t off, int whence) -{ - swap(*impl); - auto result = impl->seek(off, whence); - swap(*impl); - return result; -} - - -bool ReadBufferFromFileDecorator::nextImpl() -{ - swap(*impl); - auto result = impl->next(); - swap(*impl); - return result; -} - -size_t ReadBufferFromFileDecorator::getFileSize() -{ - return getFileSizeFromReadBuffer(*impl); -} - -} diff --git a/src/IO/ReadBufferFromFileDecorator.h b/src/IO/ReadBufferFromFileDecorator.h deleted file mode 100644 index 6e62c7f741b..00000000000 --- a/src/IO/ReadBufferFromFileDecorator.h +++ /dev/null @@ -1,37 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -/// Delegates all reads to underlying buffer. Doesn't have own memory. -class ReadBufferFromFileDecorator : public ReadBufferFromFileBase -{ -public: - explicit ReadBufferFromFileDecorator(std::unique_ptr impl_); - ReadBufferFromFileDecorator(std::unique_ptr impl_, const String & file_name_); - - std::string getFileName() const override; - - off_t getPosition() override; - - off_t seek(off_t off, int whence) override; - - bool nextImpl() override; - - bool isWithFileSize() const { return dynamic_cast(impl.get()) != nullptr; } - - const ReadBuffer & getWrappedReadBuffer() const { return *impl; } - - ReadBuffer & getWrappedReadBuffer() { return *impl; } - - size_t getFileSize() override; - -protected: - std::unique_ptr impl; - String file_name; -}; - -} diff --git a/src/IO/ReadBufferFromMemory.h b/src/IO/ReadBufferFromMemory.h index ad96e4bfa28..6d3f1a2c6e5 100644 --- a/src/IO/ReadBufferFromMemory.h +++ b/src/IO/ReadBufferFromMemory.h @@ -20,7 +20,6 @@ public: : SeekableReadBuffer(const_cast(str.data()), str.size(), 0) {} off_t seek(off_t off, int whence) override; - off_t getPosition() override; }; diff --git a/src/IO/SeekableReadBuffer.h b/src/IO/SeekableReadBuffer.h index c002d30e633..1fb66a5aa9f 100644 --- a/src/IO/SeekableReadBuffer.h +++ b/src/IO/SeekableReadBuffer.h @@ -44,12 +44,6 @@ public: virtual String getInfoForLog() { return ""; } - /// NOTE: This method should be thread-safe against seek(), since it can be - /// used in CachedOnDiskReadBufferFromFile from multiple threads (because - /// it first releases the buffer, and then do logging, and so other thread - /// can already call seek() which will lead to data-race). - virtual size_t getFileOffsetOfBufferEnd() const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getFileOffsetOfBufferEnd() not implemented"); } - /// If true, setReadUntilPosition() guarantees that eof will be reported at the given position. virtual bool supportsRightBoundedReads() const { return false; } diff --git a/src/IO/WithFileSize.cpp b/src/IO/WithFileSize.cpp index 3660d962c08..435789652dc 100644 --- a/src/IO/WithFileSize.cpp +++ b/src/IO/WithFileSize.cpp @@ -2,7 +2,6 @@ #include #include #include -#include #include namespace DB @@ -17,23 +16,15 @@ template static size_t getFileSize(T & in) { if (auto * with_file_size = dynamic_cast(&in)) - { return with_file_size->getFileSize(); - } throw Exception(ErrorCodes::UNKNOWN_FILE_SIZE, "Cannot find out file size"); } size_t getFileSizeFromReadBuffer(ReadBuffer & in) { - if (auto * delegate = dynamic_cast(&in)) - { - return getFileSize(delegate->getWrappedReadBuffer()); - } - else if (auto * compressed = dynamic_cast(&in)) - { + if (auto * compressed = dynamic_cast(&in)) return getFileSize(compressed->getWrappedReadBuffer()); - } return getFileSize(in); } @@ -52,11 +43,7 @@ std::optional tryGetFileSizeFromReadBuffer(ReadBuffer & in) bool isBufferWithFileSize(const ReadBuffer & in) { - if (const auto * delegate = dynamic_cast(&in)) - { - return delegate->isWithFileSize(); - } - else if (const auto * compressed = dynamic_cast(&in)) + if (const auto * compressed = dynamic_cast(&in)) { return isBufferWithFileSize(compressed->getWrappedReadBuffer()); } @@ -66,11 +53,7 @@ bool isBufferWithFileSize(const ReadBuffer & in) size_t getDataOffsetMaybeCompressed(const ReadBuffer & in) { - if (const auto * delegate = dynamic_cast(&in)) - { - return getDataOffsetMaybeCompressed(delegate->getWrappedReadBuffer()); - } - else if (const auto * compressed = dynamic_cast(&in)) + if (const auto * compressed = dynamic_cast(&in)) { return getDataOffsetMaybeCompressed(compressed->getWrappedReadBuffer()); } diff --git a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp index b490c5cac63..553473fcc9e 100644 --- a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp @@ -37,7 +37,7 @@ namespace ErrorCodes AsynchronousReadBufferFromHDFS::AsynchronousReadBufferFromHDFS( IAsynchronousReader & reader_, const ReadSettings & settings_, std::shared_ptr impl_) - : BufferWithOwnMemory(settings_.remote_fs_buffer_size) + : ReadBufferFromFileBase(settings_.remote_fs_buffer_size, nullptr, 0) , reader(reader_) , base_priority(settings_.priority) , impl(std::move(impl_)) diff --git a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h index d89aa60ab71..9c01bd6e434 100644 --- a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h +++ b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h @@ -21,7 +21,7 @@ namespace DB class IAsynchronousReader; -class AsynchronousReadBufferFromHDFS : public BufferWithOwnMemory, public WithFileName, public WithFileSize +class AsynchronousReadBufferFromHDFS : public ReadBufferFromFileBase { public: AsynchronousReadBufferFromHDFS( From 74f2864f207dd45be30a1a87618a81dc1f3dcdd5 Mon Sep 17 00:00:00 2001 From: avinzhang <42455944+avinzhang@users.noreply.github.com> Date: Wed, 17 Jan 2024 17:42:59 +1000 Subject: [PATCH 050/264] Update executable.md Adding the sentence on the requirement to run the script. --- docs/en/sql-reference/table-functions/executable.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/executable.md b/docs/en/sql-reference/table-functions/executable.md index d377c5d4d0c..4bbf3313caf 100644 --- a/docs/en/sql-reference/table-functions/executable.md +++ b/docs/en/sql-reference/table-functions/executable.md @@ -7,7 +7,7 @@ keywords: [udf, user defined function, clickhouse, executable, table, function] # executable Table Function for UDFs -The `executable` table function creates a table based on the output of a user-defined function (UDF) that you define in a script that outputs rows to **stdout**. The executable script is stored in the `users_scripts` directory and can read data from any source. +The `executable` table function creates a table based on the output of a user-defined function (UDF) that you define in a script that outputs rows to **stdout**. The executable script is stored in the `users_scripts` directory and can read data from any source. Make sure the clickhouse server meets the requirement to run the executable script. e.g. If it's a python script, make sure the server has installed python packages to run the script. You can optionally include one or more input queries that stream their results to **stdin** for the script to read. From 9fbe2296db82993fc2eba513362d1ac6580e3b67 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 17 Jan 2024 13:04:17 +0100 Subject: [PATCH 051/264] Update docs/en/sql-reference/table-functions/executable.md --- docs/en/sql-reference/table-functions/executable.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/executable.md b/docs/en/sql-reference/table-functions/executable.md index 4bbf3313caf..4df6a4728b9 100644 --- a/docs/en/sql-reference/table-functions/executable.md +++ b/docs/en/sql-reference/table-functions/executable.md @@ -7,7 +7,7 @@ keywords: [udf, user defined function, clickhouse, executable, table, function] # executable Table Function for UDFs -The `executable` table function creates a table based on the output of a user-defined function (UDF) that you define in a script that outputs rows to **stdout**. The executable script is stored in the `users_scripts` directory and can read data from any source. Make sure the clickhouse server meets the requirement to run the executable script. e.g. If it's a python script, make sure the server has installed python packages to run the script. +The `executable` table function creates a table based on the output of a user-defined function (UDF) that you define in a script that outputs rows to **stdout**. The executable script is stored in the `users_scripts` directory and can read data from any source. Make sure your ClickHouse server has all the required packages to run the executable script. For example, if it's a Python script, ensure that the server has the necessary Python packages installed. You can optionally include one or more input queries that stream their results to **stdin** for the script to read. From 8dce581ae74f29d5eee7451d20b8cf91aef6678c Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 17 Jan 2024 14:14:15 +0100 Subject: [PATCH 052/264] update ci --- docs/en/sql-reference/table-functions/executable.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/executable.md b/docs/en/sql-reference/table-functions/executable.md index 4df6a4728b9..b83e0020524 100644 --- a/docs/en/sql-reference/table-functions/executable.md +++ b/docs/en/sql-reference/table-functions/executable.md @@ -7,7 +7,7 @@ keywords: [udf, user defined function, clickhouse, executable, table, function] # executable Table Function for UDFs -The `executable` table function creates a table based on the output of a user-defined function (UDF) that you define in a script that outputs rows to **stdout**. The executable script is stored in the `users_scripts` directory and can read data from any source. Make sure your ClickHouse server has all the required packages to run the executable script. For example, if it's a Python script, ensure that the server has the necessary Python packages installed. +The `executable` table function creates a table based on the output of a user-defined function (UDF) that you define in a script that outputs rows to **stdout**. The executable script is stored in the `users_scripts` directory and can read data from any source. Make sure your ClickHouse server has all the required packages to run the executable script. For example, if it is a Python script, ensure that the server has the necessary Python packages installed. You can optionally include one or more input queries that stream their results to **stdin** for the script to read. From 7dda18c209c29ef10da2af611437fb1ea73ebd9e Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Wed, 17 Jan 2024 16:07:53 +0100 Subject: [PATCH 053/264] Spans in mergeBlock and finalizeImpl --- src/IO/WriteBufferFromS3.cpp | 2 ++ src/Storages/MergeTree/MergeTreeDataWriter.cpp | 3 +++ 2 files changed, 5 insertions(+) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 8605fdc004a..c928c43f8ee 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -178,6 +178,8 @@ void WriteBufferFromS3::preFinalize() void WriteBufferFromS3::finalizeImpl() { + OpenTelemetry::SpanHolder span("WriteBufferFromS3::finalizeImpl"); + LOG_TRACE(limitedLog, "finalizeImpl WriteBufferFromS3. {}.", getShortLogDetails()); if (!is_prefinalized) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 2a381afa805..2a6eb1ae4f7 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -314,6 +315,8 @@ Block MergeTreeDataWriter::mergeBlock( IColumn::Permutation *& permutation, const MergeTreeData::MergingParams & merging_params) { + OpenTelemetry::SpanHolder span("MergeTreeDataWriter::mergeBlock"); + size_t block_size = block.rows(); auto get_merging_algorithm = [&]() -> std::shared_ptr From 0a42dbafe847732f32553d51287e8edc94d9ad01 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Wed, 17 Jan 2024 17:46:15 +0100 Subject: [PATCH 054/264] Add attributes --- src/IO/WriteBufferFromS3.cpp | 5 +++++ src/Storages/MergeTree/MergeTreeDataWriter.cpp | 5 +++++ 2 files changed, 10 insertions(+) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index c928c43f8ee..a7b674b7a75 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -179,6 +179,9 @@ void WriteBufferFromS3::preFinalize() void WriteBufferFromS3::finalizeImpl() { OpenTelemetry::SpanHolder span("WriteBufferFromS3::finalizeImpl"); + span.addAttribute("clickhouse.s3_bucket", bucket); + span.addAttribute("clickhouse.s3_key", key); + span.addAttribute("clickhouse.total_size", total_size); LOG_TRACE(limitedLog, "finalizeImpl WriteBufferFromS3. {}.", getShortLogDetails()); @@ -190,6 +193,8 @@ void WriteBufferFromS3::finalizeImpl() task_tracker->waitAll(); + span.addAttributeIfNotZero("clickhouse.multipart_upload_parts", multipart_tags.size()); + if (!multipart_upload_id.empty()) { completeMultipartUpload(); diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 2a6eb1ae4f7..3c0b2d2b42e 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -319,6 +319,9 @@ Block MergeTreeDataWriter::mergeBlock( size_t block_size = block.rows(); + span.addAttribute("clickhouse.rows", block_size); + span.addAttribute("clickhouse.columns", block.columns()); + auto get_merging_algorithm = [&]() -> std::shared_ptr { switch (merging_params.mode) @@ -354,6 +357,8 @@ Block MergeTreeDataWriter::mergeBlock( if (!merging_algorithm) return block; + span.addAttribute("clickhouse.merging_algorithm", merging_algorithm->getName()); + Chunk chunk(block.getColumns(), block_size); IMergingAlgorithm::Input input; From 22afcd46f44eb531fa67eebd766d73b00324360a Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Thu, 18 Jan 2024 05:17:07 +0000 Subject: [PATCH 055/264] add ALTER COLUMN MODIFY/RESET SETTING Signed-off-by: Duc Canh Le --- src/Interpreters/InterpreterCreateQuery.cpp | 53 +++++++--------- src/Parsers/ASTColumnDeclaration.cpp | 52 ++++++++-------- src/Parsers/ASTColumnDeclaration.h | 4 +- src/Parsers/ParserAlterQuery.cpp | 20 ++++--- src/Parsers/ParserCreateQuery.h | 33 +++++----- src/Storages/AlterCommands.cpp | 60 ++++++++++--------- src/Storages/AlterCommands.h | 9 ++- src/Storages/ColumnsDescription.cpp | 4 +- .../02870_per_column_compress_block.reference | 8 ++- .../02870_per_column_compress_block.sql | 18 ++++-- 10 files changed, 137 insertions(+), 124 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index f3ef1e3359e..d002cc6d980 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -467,10 +467,10 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns) if (!column.settings.empty()) { - auto per_column_settings = std::make_shared(); - per_column_settings->is_standalone = false; - per_column_settings->changes = column.settings; - column_declaration->per_column_settings = std::move(per_column_settings); + auto settings = std::make_shared(); + settings->is_standalone = false; + settings->changes = column.settings; + column_declaration->settings = std::move(settings); } columns_list->children.push_back(column_declaration_ptr); @@ -680,9 +680,9 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( if (col_decl.ttl) column.ttl = col_decl.ttl; - if (col_decl.per_column_settings) + if (col_decl.settings) { - column.settings = col_decl.per_column_settings->as().changes; + column.settings = col_decl.settings->as().changes; MergeTreeColumnSettings::validate(column.settings); } @@ -890,33 +890,22 @@ void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & creat throw Exception(ErrorCodes::DUPLICATE_COLUMN, "Column {} already exists", backQuoteIfNeed(column.name)); } - if (create.storage && create.storage->engine) + /// Check if _row_exists for lightweight delete column in column_lists for merge tree family. + if (create.storage && create.storage->engine && endsWith(create.storage->engine->name, "MergeTree")) { - /// Check if _row_exists for lightweight delete column in column_lists for merge tree family. - if (endsWith(create.storage->engine->name, "MergeTree")) - { - auto search = all_columns.find(LightweightDeleteDescription::FILTER_COLUMN.name); - if (search != all_columns.end()) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "Cannot create table with column '{}' for *MergeTree engines because it " - "is reserved for lightweight delete feature", - LightweightDeleteDescription::FILTER_COLUMN.name); - auto search_block_number = all_columns.find(BlockNumberColumn::name); - if (search_block_number != all_columns.end()) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "Cannot create table with column '{}' for *MergeTree engines because it " - "is reserved for storing block number", - BlockNumberColumn::name); - } - else - { - /// Only merge tree family supports column with custom column setting - if (std::any_of( - properties.columns.begin(), - properties.columns.end(), - [](const ColumnDescription & column) { return !column.settings.empty(); })) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot create table with column level settings for non-MergeTree engines"); - } + auto search = all_columns.find(LightweightDeleteDescription::FILTER_COLUMN.name); + if (search != all_columns.end()) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Cannot create table with column '{}' for *MergeTree engines because it " + "is reserved for lightweight delete feature", + LightweightDeleteDescription::FILTER_COLUMN.name); + + auto search_block_number = all_columns.find(BlockNumberColumn::name); + if (search_block_number != all_columns.end()) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Cannot create table with column '{}' for *MergeTree engines because it " + "is reserved for storing block number", + BlockNumberColumn::name); } const auto & settings = getContext()->getSettingsRef(); diff --git a/src/Parsers/ASTColumnDeclaration.cpp b/src/Parsers/ASTColumnDeclaration.cpp index dd3bf6ae8d8..a6db1453743 100644 --- a/src/Parsers/ASTColumnDeclaration.cpp +++ b/src/Parsers/ASTColumnDeclaration.cpp @@ -57,83 +57,83 @@ ASTPtr ASTColumnDeclaration::clone() const res->children.push_back(res->collation); } - if (per_column_settings) + if (settings) { - res->per_column_settings = per_column_settings->clone(); - res->children.push_back(res->per_column_settings); + res->settings = settings->clone(); + res->children.push_back(res->settings); } return res; } -void ASTColumnDeclaration::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const +void ASTColumnDeclaration::formatImpl(const FormatSettings & format_settings, FormatState & state, FormatStateStacked frame) const { frame.need_parens = false; /// We have to always backquote column names to avoid ambiguouty with INDEX and other declarations in CREATE query. - settings.ostr << backQuote(name); + format_settings.ostr << backQuote(name); if (type) { - settings.ostr << ' '; + format_settings.ostr << ' '; FormatStateStacked type_frame = frame; type_frame.indent = 0; - type->formatImpl(settings, state, type_frame); + type->formatImpl(format_settings, state, type_frame); } if (null_modifier) { - settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") - << (*null_modifier ? "" : "NOT ") << "NULL" << (settings.hilite ? hilite_none : ""); + format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") + << (*null_modifier ? "" : "NOT ") << "NULL" << (format_settings.hilite ? hilite_none : ""); } if (default_expression) { - settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << default_specifier << (settings.hilite ? hilite_none : ""); + format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << default_specifier << (format_settings.hilite ? hilite_none : ""); if (!ephemeral_default) { - settings.ostr << ' '; - default_expression->formatImpl(settings, state, frame); + format_settings.ostr << ' '; + default_expression->formatImpl(format_settings, state, frame); } } if (comment) { - settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "COMMENT" << (settings.hilite ? hilite_none : "") << ' '; - comment->formatImpl(settings, state, frame); + format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << "COMMENT" << (format_settings.hilite ? hilite_none : "") << ' '; + comment->formatImpl(format_settings, state, frame); } if (codec) { - settings.ostr << ' '; - codec->formatImpl(settings, state, frame); + format_settings.ostr << ' '; + codec->formatImpl(format_settings, state, frame); } if (stat_type) { - settings.ostr << ' '; - stat_type->formatImpl(settings, state, frame); + format_settings.ostr << ' '; + stat_type->formatImpl(format_settings, state, frame); } if (ttl) { - settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "TTL" << (settings.hilite ? hilite_none : "") << ' '; - ttl->formatImpl(settings, state, frame); + format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << "TTL" << (format_settings.hilite ? hilite_none : "") << ' '; + ttl->formatImpl(format_settings, state, frame); } if (collation) { - settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "COLLATE" << (settings.hilite ? hilite_none : "") << ' '; - collation->formatImpl(settings, state, frame); + format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << "COLLATE" << (format_settings.hilite ? hilite_none : "") << ' '; + collation->formatImpl(format_settings, state, frame); } - if (per_column_settings) + if (settings) { - settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "SETTINGS" << (settings.hilite ? hilite_none : "") << ' ' << '('; - per_column_settings->formatImpl(settings, state, frame); - settings.ostr << ')'; + format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << "SETTINGS" << (format_settings.hilite ? hilite_none : "") << ' ' << '('; + settings->formatImpl(format_settings, state, frame); + format_settings.ostr << ')'; } } diff --git a/src/Parsers/ASTColumnDeclaration.h b/src/Parsers/ASTColumnDeclaration.h index 980f6252953..d775928d05c 100644 --- a/src/Parsers/ASTColumnDeclaration.h +++ b/src/Parsers/ASTColumnDeclaration.h @@ -22,13 +22,13 @@ public: ASTPtr stat_type; ASTPtr ttl; ASTPtr collation; - ASTPtr per_column_settings; + ASTPtr settings; bool primary_key_specifier = false; String getID(char delim) const override { return "ColumnDeclaration" + (delim + name); } ASTPtr clone() const override; - void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; + void formatImpl(const FormatSettings & format_settings, FormatState & state, FormatStateStacked frame) const override; }; } diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index e41055ef8d8..527f5af829a 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -110,7 +110,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_comment("COMMENT"); ParserKeyword s_codec("CODEC"); ParserKeyword s_ttl("TTL"); - ParserKeyword s_setting("SETTING"); + ParserKeyword s_settings("SETTINGS"); ParserKeyword s_remove_ttl("REMOVE TTL"); ParserKeyword s_remove_sample_by("REMOVE SAMPLE BY"); @@ -702,15 +702,21 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->remove_property = "CODEC"; else if (s_ttl.ignore(pos, expected)) command->remove_property = "TTL"; - else if (s_setting.ignore(pos, expected)) - { - command->remove_property = "SETTING"; - if (!parser_reset_setting.parse(pos, command->settings_resets, expected)) - return false; - } + else if (s_settings.ignore(pos, expected)) + command->remove_property = "SETTINGS"; else return false; } + else if (s_modify_setting.ignore(pos, expected)) + { + if (!parser_settings.parse(pos, command->settings_changes, expected)) + return false; + } + else if (s_reset_setting.ignore(pos, expected)) + { + if (!parser_reset_setting.parse(pos, command->settings_resets, expected)) + return false; + } else { if (s_first.ignore(pos, expected)) diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index ad1f24d36f7..80cf541cbdc 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -135,6 +135,8 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E ParserKeyword s_stat{"STATISTIC"}; ParserKeyword s_ttl{"TTL"}; ParserKeyword s_remove{"REMOVE"}; + ParserKeyword s_modify_setting("MODIFY SETTING"); + ParserKeyword s_reset_setting("RESET SETTING"); ParserKeyword s_type{"TYPE"}; ParserKeyword s_collate{"COLLATE"}; ParserKeyword s_primary_key{"PRIMARY KEY"}; @@ -159,10 +161,12 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E /// This keyword may occur only in MODIFY COLUMN query. We check it here /// because ParserDataType parses types as an arbitrary identifiers and /// doesn't check that parsed string is existing data type. In this way - /// REMOVE keyword can be parsed as data type and further parsing will fail. - /// So we just check this keyword and in case of success return column - /// declaration with name only. - if (!require_type && s_remove.checkWithoutMoving(pos, expected)) + /// REMOVE, MODIFY SETTING, or RESET SETTING can be parsed as data type + /// and further parsing will fail. So we just check these keyword and in + /// case of success return column declaration with name only. + if (!require_type + && (s_remove.checkWithoutMoving(pos, expected) || s_modify_setting.checkWithoutMoving(pos, expected) + || s_reset_setting.checkWithoutMoving(pos, expected))) { if (!check_keywords_after_name) return false; @@ -181,10 +185,10 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E ASTPtr default_expression; ASTPtr comment_expression; ASTPtr codec_expression; - ASTPtr per_column_settings; ASTPtr stat_type_expression; ASTPtr ttl_expression; ASTPtr collation_expression; + ASTPtr settings; bool primary_key_specifier = false; auto null_check_without_moving = [&]() -> bool @@ -325,24 +329,19 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E primary_key_specifier = true; } - auto old_pos = pos; if (s_settings.ignore(pos, expected)) { ParserToken parser_opening_bracket(TokenType::OpeningRoundBracket); - if (parser_opening_bracket.ignore(pos, expected)) + if (parser_opening_bracket.check(pos, expected)) { - if (!settings_parser.parse(pos, per_column_settings, expected)) + if (!settings_parser.parse(pos, settings, expected)) return false; ParserToken parser_closing_bracket(TokenType::ClosingRoundBracket); if (!parser_closing_bracket.ignore(pos, expected)) return false; } - else - { - /// This could be settings in alter query - /// E.g: ALTER TABLE alter_enum_array MODIFY COLUMN x String SETTINGS mutations_sync=2; - pos = old_pos; - } + /// This could be settings in alter query + /// E.g: ALTER TABLE alter_enum_array MODIFY COLUMN x String SETTINGS mutations_sync=2; } node = column_declaration; @@ -375,10 +374,10 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E column_declaration->children.push_back(std::move(codec_expression)); } - if (per_column_settings) + if (settings) { - column_declaration->per_column_settings = per_column_settings; - column_declaration->children.push_back(std::move(per_column_settings)); + column_declaration->settings = settings; + column_declaration->children.push_back(std::move(settings)); } if (stat_type_expression) diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 7638732d2ba..7a9a013e85d 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -75,8 +75,8 @@ AlterCommand::RemoveProperty removePropertyFromString(const String & property) return AlterCommand::RemoveProperty::CODEC; else if (property == "TTL") return AlterCommand::RemoveProperty::TTL; - else if (property == "SETTING") - return AlterCommand::RemoveProperty::SETTING; + else if (property == "SETTINGS") + return AlterCommand::RemoveProperty::SETTINGS; throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot remove unknown property '{}'", property); } @@ -152,16 +152,6 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ const auto & ast_col_decl = command_ast->col_decl->as(); command.column_name = ast_col_decl.name; command.to_remove = removePropertyFromString(command_ast->remove_property); - if (command.to_remove == RemoveProperty::SETTING) - { - for (const ASTPtr & identifier_ast : command_ast->settings_resets->children) - { - const auto & identifier = identifier_ast->as(); - auto insertion = command.settings_resets.emplace(identifier.name()); - if (!insertion.second) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Duplicate setting name {}", backQuote(identifier.name())); - } - } if (ast_col_decl.type) { @@ -186,8 +176,24 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ if (ast_col_decl.codec) command.codec = ast_col_decl.codec; - if (ast_col_decl.per_column_settings) - command.settings_changes = ast_col_decl.per_column_settings->as().changes; + if (ast_col_decl.settings) + command.settings_changes = ast_col_decl.settings->as().changes; + + /// At most only one of ast_col_decl.settings or command_ast->settings_changes is non-null + if (command_ast->settings_changes) + { + command.settings_changes = command_ast->settings_changes->as().changes; + command.append_column_setting = true; + } + + if (command_ast->settings_resets) + { + for (const ASTPtr & identifier_ast : command_ast->settings_resets->children) + { + const auto & identifier = identifier_ast->as(); + command.settings_resets.emplace(identifier.name()); + } + } if (command_ast->column) command.after_column = getIdentifierName(command_ast->column); @@ -517,10 +523,9 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) { column.ttl.reset(); } - else if (to_remove == RemoveProperty::SETTING) + else if (to_remove == RemoveProperty::SETTINGS) { - for (const auto & setting : settings_resets) - column.settings.removeSetting(setting); + column.settings.clear(); } else { @@ -539,7 +544,17 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) if (!settings_changes.empty()) { MergeTreeColumnSettings::validate(settings_changes); - column.settings = settings_changes; + if (append_column_setting) + for (const auto & change : settings_changes) + column.settings.setSetting(change.name, change.value); + else + column.settings = settings_changes; + } + + if (!settings_resets.empty()) + { + for (const auto & setting : settings_resets) + column.settings.removeSetting(setting); } /// User specified default expression or changed @@ -1384,15 +1399,6 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const ErrorCodes::BAD_ARGUMENTS, "Column {} doesn't have COMMENT, cannot remove it", backQuote(column_name)); - if (command.to_remove == AlterCommand::RemoveProperty::SETTING) - { - for (const auto & setting : command.settings_resets) - { - if (!column_from_table.settings.tryGet(setting)) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, "Column {} doesn't have SETTINGS, cannot remove it", backQuote(column_name)); - } - } } modified_columns.emplace(column_name); diff --git a/src/Storages/AlterCommands.h b/src/Storages/AlterCommands.h index fbd2f811630..f7ab1385f1a 100644 --- a/src/Storages/AlterCommands.h +++ b/src/Storages/AlterCommands.h @@ -65,7 +65,7 @@ struct AlterCommand COMMENT, CODEC, TTL, - SETTING + SETTINGS }; Type type = UNKNOWN; @@ -138,10 +138,10 @@ struct AlterCommand /// For ADD and MODIFY ASTPtr codec = nullptr; - /// For MODIFY SETTING or MODIFY COLUMN with SETTINGS (...) + /// For MODIFY SETTING or MODIFY COLUMN MODIFY SETTING SettingsChanges settings_changes; - /// For RESET SETTING or MODIFY COLUMN REMOVE SETTING (...) + /// For RESET SETTING or MODIFY COLUMN RESET SETTING std::set settings_resets; /// For MODIFY_QUERY @@ -156,6 +156,9 @@ struct AlterCommand /// What to remove from column (or TTL) RemoveProperty to_remove = RemoveProperty::NO_PROPERTY; + /// Is this MODIFY COLUMN MODIFY SETTING or MODIFY COLUMN parse(const ASTAlterCommand * command); void apply(StorageInMemoryMetadata & metadata, ContextPtr context) const; diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 6e86dc4ddff..94f04c1af1c 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -159,8 +159,8 @@ void ColumnDescription::readText(ReadBuffer & buf) if (col_ast->ttl) ttl = col_ast->ttl; - if (col_ast->per_column_settings) - settings = col_ast->per_column_settings->as().changes; + if (col_ast->settings) + settings = col_ast->settings->as().changes; } else throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse column description"); diff --git a/tests/queries/0_stateless/02870_per_column_compress_block.reference b/tests/queries/0_stateless/02870_per_column_compress_block.reference index 96e01669414..c6b396a11c2 100644 --- a/tests/queries/0_stateless/02870_per_column_compress_block.reference +++ b/tests/queries/0_stateless/02870_per_column_compress_block.reference @@ -1,7 +1,9 @@ -CREATE TABLE db_02780.t\n(\n `id` UInt64 CODEC(ZSTD(1)),\n `long_string` String CODEC(ZSTD(9, 24)) SETTINGS (min_compress_block_size = 163840, max_compress_block_size = 163840),\n `v1` String CODEC(ZSTD(1)),\n `v2` UInt64 CODEC(ZSTD(1)),\n `v3` Float32 CODEC(ZSTD(1)),\n `v4` Float64 CODEC(ZSTD(1))\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/db_02780/t/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192 +CREATE TABLE default.t\n(\n `id` UInt64 CODEC(ZSTD(1)),\n `long_string` String CODEC(ZSTD(9, 24)) SETTINGS (min_compress_block_size = 163840, max_compress_block_size = 163840),\n `v1` String CODEC(ZSTD(1)),\n `v2` UInt64 CODEC(ZSTD(1)),\n `v3` Float32 CODEC(ZSTD(1)),\n `v4` Float64 CODEC(ZSTD(1))\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/t/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192 1000 -CREATE TABLE db_02780.t\n(\n `id` UInt64 CODEC(ZSTD(1)),\n `long_string` String CODEC(ZSTD(9, 24)),\n `v1` String CODEC(ZSTD(1)),\n `v2` UInt64 CODEC(ZSTD(1)),\n `v3` Float32 CODEC(ZSTD(1)),\n `v4` Float64 CODEC(ZSTD(1))\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/db_02780/t/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192 -CREATE TABLE db_02780.t\n(\n `id` UInt64 CODEC(ZSTD(1)),\n `long_string` String CODEC(ZSTD(9, 24)) SETTINGS (min_compress_block_size = 163840, max_compress_block_size = 163840),\n `v1` String CODEC(ZSTD(1)),\n `v2` UInt64 CODEC(ZSTD(1)),\n `v3` Float32 CODEC(ZSTD(1)),\n `v4` Float64 CODEC(ZSTD(1))\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/db_02780/t/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192 +CREATE TABLE default.t\n(\n `id` UInt64 CODEC(ZSTD(1)),\n `long_string` String CODEC(ZSTD(9, 24)) SETTINGS (min_compress_block_size = 8192, max_compress_block_size = 163840),\n `v1` String CODEC(ZSTD(1)),\n `v2` UInt64 CODEC(ZSTD(1)),\n `v3` Float32 CODEC(ZSTD(1)),\n `v4` Float64 CODEC(ZSTD(1))\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/t/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192 +CREATE TABLE default.t\n(\n `id` UInt64 CODEC(ZSTD(1)),\n `long_string` String CODEC(ZSTD(9, 24)) SETTINGS (max_compress_block_size = 163840),\n `v1` String CODEC(ZSTD(1)),\n `v2` UInt64 CODEC(ZSTD(1)),\n `v3` Float32 CODEC(ZSTD(1)),\n `v4` Float64 CODEC(ZSTD(1))\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/t/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192 +CREATE TABLE default.t\n(\n `id` UInt64 CODEC(ZSTD(1)),\n `long_string` String CODEC(ZSTD(9, 24)),\n `v1` String CODEC(ZSTD(1)),\n `v2` UInt64 CODEC(ZSTD(1)),\n `v3` Float32 CODEC(ZSTD(1)),\n `v4` Float64 CODEC(ZSTD(1))\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/t/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192 +CREATE TABLE default.t\n(\n `id` UInt64 CODEC(ZSTD(1)),\n `long_string` String CODEC(ZSTD(9, 24)) SETTINGS (min_compress_block_size = 163840, max_compress_block_size = 163840),\n `v1` String CODEC(ZSTD(1)),\n `v2` UInt64 CODEC(ZSTD(1)),\n `v3` Float32 CODEC(ZSTD(1)),\n `v4` Float64 CODEC(ZSTD(1))\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/t/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192 (0,0) 0 (1,1) 1 (2,2) 2 diff --git a/tests/queries/0_stateless/02870_per_column_compress_block.sql b/tests/queries/0_stateless/02870_per_column_compress_block.sql index 65556edeeb8..079ece80fa0 100644 --- a/tests/queries/0_stateless/02870_per_column_compress_block.sql +++ b/tests/queries/0_stateless/02870_per_column_compress_block.sql @@ -1,7 +1,5 @@ -- Tags: no-random-merge-tree-settings -DROP DATABASE IF EXISTS db_02780; -CREATE DATABASE db_02780; -USE db_02780; + CREATE TABLE t ( `id` UInt64 CODEC(ZSTD(1)), @@ -21,7 +19,15 @@ INSERT INTO TABLE t SELECT number, randomPrintableASCII(1000), randomPrintableAS SELECT count() FROM t; -ALTER TABLE t MODIFY COLUMN long_string REMOVE SETTING min_compress_block_size, max_compress_block_size; +ALTER TABLE t MODIFY COLUMN long_string MODIFY SETTING min_compress_block_size = 8192; + +SHOW CREATE t; + +ALTER TABLE t MODIFY COLUMN long_string RESET SETTING min_compress_block_size; + +SHOW CREATE t; + +ALTER TABLE t MODIFY COLUMN long_string REMOVE SETTINGS; SHOW CREATE t; @@ -46,7 +52,9 @@ SETTINGS min_bytes_for_wide_part = 1; INSERT INTO TABLE t2 SELECT number, tuple(number, number), concat('{"key": ', toString(number), ' ,"value": ', toString(rand(number+1)), '}') FROM numbers(1000); SELECT tup, json.key AS key FROM t2 ORDER BY key LIMIT 10; +DROP TABLE t2; +-- Non-supported column setting CREATE TABLE t3 ( `id` UInt64 CODEC(ZSTD(1)), @@ -56,6 +64,7 @@ ENGINE = MergeTree ORDER BY id SETTINGS min_bytes_for_wide_part = 1; -- {serverError 115} +-- Invalid setting values CREATE TABLE t4 ( `id` UInt64 CODEC(ZSTD(1)), @@ -64,4 +73,3 @@ CREATE TABLE t4 ENGINE = TinyLog ORDER BY id; -- {serverError 44} -DROP DATABASE db_02780; \ No newline at end of file From a5d15c10e8190fa4fd74f72e62b0e14d8c189fd1 Mon Sep 17 00:00:00 2001 From: Tim Liou Date: Thu, 18 Jan 2024 14:51:21 +0800 Subject: [PATCH 056/264] use getPriorityName --- src/Loggers/OwnJSONPatternFormatter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Loggers/OwnJSONPatternFormatter.cpp b/src/Loggers/OwnJSONPatternFormatter.cpp index e4f8e0facae..4263ad5925a 100644 --- a/src/Loggers/OwnJSONPatternFormatter.cpp +++ b/src/Loggers/OwnJSONPatternFormatter.cpp @@ -118,7 +118,7 @@ void OwnJSONPatternFormatter::formatExtended(const DB::ExtendedLogMessage & msg_ writeJSONString(level, wb, settings); DB::writeChar(':', wb); int priority = static_cast(msg.getPriority()); - writeJSONString(std::to_string(priority), wb, settings); + writeJSONString(getPriorityName(priority), wb, settings); } if (!query_id.empty()) From b16a4cf36104907ce2eb255fdf5148ea53543667 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Thu, 18 Jan 2024 07:55:44 +0000 Subject: [PATCH 057/264] add document Signed-off-by: Duc Canh Le --- .../mergetree-family/mergetree.md | 43 ++++++++++++++++- .../sql-reference/statements/alter/column.md | 46 +++++++++++++++++-- src/Parsers/ASTColumnDeclaration.cpp | 2 +- 3 files changed, 84 insertions(+), 7 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 6d60611ae4b..95c962757b7 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -39,8 +39,8 @@ If you need to update rows frequently, we recommend using the [`ReplacingMergeTr ``` sql CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ( - name1 [type1] [[NOT] NULL] [DEFAULT|MATERIALIZED|ALIAS|EPHEMERAL expr1] [COMMENT ...] [CODEC(codec1)] [STATISTIC(stat1)] [TTL expr1] [PRIMARY KEY], - name2 [type2] [[NOT] NULL] [DEFAULT|MATERIALIZED|ALIAS|EPHEMERAL expr2] [COMMENT ...] [CODEC(codec2)] [STATISTIC(stat2)] [TTL expr2] [PRIMARY KEY], + name1 [type1] [[NOT] NULL] [DEFAULT|MATERIALIZED|ALIAS|EPHEMERAL expr1] [COMMENT ...] [CODEC(codec1)] [STATISTIC(stat1)] [TTL expr1] [PRIMARY KEY] [SETTINGS (name = value, ...)], + name2 [type2] [[NOT] NULL] [DEFAULT|MATERIALIZED|ALIAS|EPHEMERAL expr2] [COMMENT ...] [CODEC(codec2)] [STATISTIC(stat2)] [TTL expr2] [PRIMARY KEY] [SETTINGS (name = value, ...)], ... INDEX index_name1 expr1 TYPE type1(...) [GRANULARITY value1], INDEX index_name2 expr2 TYPE type2(...) [GRANULARITY value2], @@ -1390,3 +1390,42 @@ They can be used for query optimization when we enable `set allow_statistic_opti - `tdigest` Stores distribution of values from numeric columns in [TDigest](https://github.com/tdunning/t-digest) sketch. + +## Column Settings {#column-settings} + +Some table parameters can be override at column level by column settings. + +- `max_compress_block_size` — Maximum size of blocks of uncompressed data before compressing for writing to a table. +- `min_compress_block_size` — Minimum size of blocks of uncompressed data required for compression when writing the next mark. + +Example of creating table with column settings: + +```sql +CREATE TABLE example_table +( + id Int64, + document String CODEC(ZSTD(9,24)) SETTINGS (min_compress_block_size = 16777216, max_compress_block_size = 16777216) +) +ENGINE = MergeTree +ORDER BY id +``` + +Column settings can be modified or removed via [ALTER MODIFY COLUMN](/docs/en/sql-reference/statements/alter/column.md) query, for example: + +Remove `SETTINGS` from column declaration: + +```sql +ALTER TABLE example_table MODIFY COLUMN document REMOVE SETTINGS; +``` + +Modify a setting: + +```sql +ALTER TABLE example_table MODIFY COLUMN document MODIFY SETTING min_compress_block_size = 8192; +``` + +Reset one or more settings, after reset the setting expression is also removed from column declaration: + +```sql +ALTER TABLE example_table MODIFY COLUMN document RESET SETTING min_compress_block_size; +``` diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index 2cb802c863b..87cf8ca9914 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -23,10 +23,11 @@ The following actions are supported: - [RENAME COLUMN](#rename-column) — Renames an existing column. - [CLEAR COLUMN](#clear-column) — Resets column values. - [COMMENT COLUMN](#comment-column) — Adds a text comment to the column. -- [MODIFY COLUMN](#modify-column) — Changes column’s type, default expression and TTL. +- [MODIFY COLUMN](#modify-column) — Changes column’s type, default expression, TTL, and settings. - [MODIFY COLUMN REMOVE](#modify-column-remove) — Removes one of the column properties. +- [MODIFY COLUMN MODIFY SETTING](#modify-column-modify-setting) - Changes column settings. +- [MODIFY COLUMN RESET SETTING](#modify-column-reset-setting) - Reset column settings. - [MATERIALIZE COLUMN](#materialize-column) — Materializes the column in the parts where the column is missing. - These actions are described in detail below. ## ADD COLUMN @@ -75,7 +76,7 @@ Deletes the column with the name `name`. If the `IF EXISTS` clause is specified, Deletes data from the file system. Since this deletes entire files, the query is completed almost instantly. -:::tip +:::tip You can’t delete a column if it is referenced by [materialized view](/docs/en/sql-reference/statements/create/view.md/#materialized). Otherwise, it returns an error. ::: @@ -208,7 +209,7 @@ The `ALTER` query for changing columns is replicated. The instructions are saved ## MODIFY COLUMN REMOVE -Removes one of the column properties: `DEFAULT`, `ALIAS`, `MATERIALIZED`, `CODEC`, `COMMENT`, `TTL`. +Removes one of the column properties: `DEFAULT`, `ALIAS`, `MATERIALIZED`, `CODEC`, `COMMENT`, `TTL`, `SETTING`. Syntax: @@ -228,6 +229,43 @@ ALTER TABLE table_with_ttl MODIFY COLUMN column_ttl REMOVE TTL; - [REMOVE TTL](ttl.md). + +## MODIFY COLUMN MODIFY SETTING + +Modify a column level setting. + +Syntax: + +```sql +ALTER TABLE table_name MODIFY COLUMN MODIFY SETTING name=value,...; +``` + +**Example** + +Modify column's `max_compress_block_size` to `1MB`: + +```sql +ALTER TABLE table_name MODIFY COLUMN MODIFY SETTING max_compress_block_size = 1048576; +``` + +## MODIFY COLUMN RESET SETTING + +Reset a column setting, also remove the setting declaration in column expression in table create query. + +Syntax: + +```sql +ALTER TABLE table_name MODIFY COLUMN RESET SETTING name,...; +``` + +**Example** + +Remove column setting `max_compress_block_size` to `1MB`: + +```sql +ALTER TABLE table_name MODIFY COLUMN REMOVE SETTING max_compress_block_size; +``` + ## MATERIALIZE COLUMN Materializes or updates a column with an expression for a default value (`DEFAULT` or `MATERIALIZED`). diff --git a/src/Parsers/ASTColumnDeclaration.cpp b/src/Parsers/ASTColumnDeclaration.cpp index a6db1453743..6c29e0bf9d5 100644 --- a/src/Parsers/ASTColumnDeclaration.cpp +++ b/src/Parsers/ASTColumnDeclaration.cpp @@ -132,7 +132,7 @@ void ASTColumnDeclaration::formatImpl(const FormatSettings & format_settings, Fo if (settings) { format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << "SETTINGS" << (format_settings.hilite ? hilite_none : "") << ' ' << '('; - settings->formatImpl(format_settings, state, frame); + settings->formatImpl(format_settings, state, frame); format_settings.ostr << ')'; } } From e9919f47c11067095103bb3fd5e58fc3204e3bb9 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Thu, 18 Jan 2024 14:26:19 +0100 Subject: [PATCH 058/264] Move paramaters from span name to attributes --- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 5 +++-- src/Interpreters/Cache/FileSegment.cpp | 4 +++- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 217d13ae398..e4d906d8150 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -558,8 +558,9 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegment & file_segment) ProfileEvents::FileSegmentPredownloadMicroseconds, predownload_watch.elapsedMicroseconds()); }); - OpenTelemetry::SpanHolder span{ - fmt::format("CachedOnDiskReadBufferFromFile::predownload(key={}, size={})", file_segment.key().toString(), bytes_to_predownload)}; + OpenTelemetry::SpanHolder span("CachedOnDiskReadBufferFromFile::predownload"); + span.addAttribute("clickhouse.key", file_segment.key().toString()); + span.addAttribute("clickhouse.size", bytes_to_predownload); if (bytes_to_predownload) { diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 184c682e193..c6b4399feb8 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -424,7 +424,9 @@ void FileSegment::write(const char * from, size_t size, size_t offset) FileSegment::State FileSegment::wait(size_t offset) { - OpenTelemetry::SpanHolder span{fmt::format("FileSegment::wait({})", key().toString())}; + OpenTelemetry::SpanHolder span("FileSegment::wait"); + span.addAttribute("clickhouse.key", key().toString()); + span.addAttribute("clickhouse.offset", offset); auto lock = lockFileSegment(); From 61826cf21a67f2f5fd5d956884c6e5022252accc Mon Sep 17 00:00:00 2001 From: Tim Liou Date: Thu, 18 Jan 2024 21:40:23 +0800 Subject: [PATCH 059/264] Validate log level values --- .../test_structured_logging_json/test.py | 26 ++++++++++++++++++- 1 file changed, 25 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_structured_logging_json/test.py b/tests/integration/test_structured_logging_json/test.py index cbd066abc91..313fd0e205d 100644 --- a/tests/integration/test_structured_logging_json/test.py +++ b/tests/integration/test_structured_logging_json/test.py @@ -32,6 +32,30 @@ def is_json(log_json): return True +def validate_log_level(config, logs): + root = ET.fromstring(config) + key = root.findtext(".//names/level") or "level" + + valid_level_values = { + "Fatal", + "Critical", + "Error", + "Warning", + "Notice", + "Information", + "Debug", + "Trace", + "Test", + } + + length = min(10, len(logs)) + for i in range(0, length): + json_log = json.loads(logs[i]) + if json_log[key] not in valid_level_values: + return False + return True + + def validate_log_config_relation(config, logs, config_type): root = ET.fromstring(config) keys_in_config = set() @@ -80,7 +104,7 @@ def valiade_everything(config, node, config_type): logs = node.grep_in_log("").split("\n") return validate_logs(logs) and validate_log_config_relation( config, logs, config_type - ) + ) and validate_log_level(config, logs) def test_structured_logging_json_format(start_cluster): From 8ac04c6dd8a945e0f189aae572c54ee4458f75dd Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 18 Jan 2024 15:15:57 +0000 Subject: [PATCH 060/264] Address comments --- src/DataTypes/DataTypeVariant.cpp | 9 ++++ .../Serializations/SerializationVariant.cpp | 48 +++++++------------ .../Serializations/SerializationVariant.h | 23 +++++++++ .../SerializationVariantElement.cpp | 14 +++--- 4 files changed, 57 insertions(+), 37 deletions(-) diff --git a/src/DataTypes/DataTypeVariant.cpp b/src/DataTypes/DataTypeVariant.cpp index 2bc4dfa5a7a..e0510373960 100644 --- a/src/DataTypes/DataTypeVariant.cpp +++ b/src/DataTypes/DataTypeVariant.cpp @@ -105,6 +105,15 @@ ColumnPtr DataTypeVariant::createColumnConst(size_t size, const DB::Field & fiel } else { + /// We don't have exact mapping Field type -> Data type, so we cannot + /// always know in which variant we need to insert the field by it's type. + /// Examples: + /// Field(42) and Variant(UInt16, String). Type of the Field - UInt64, but we can insert it in UInt16 + /// Field(42) and Variant(Date, String). Type of the Field - UInt64, but we can insert it in Date + + /// Let's first apply FieldToDataType visitor to find best Data type for this field. + /// If we have variant with such type, we will insert this field into it. + /// Otherwise we will try to find the first variant that has default Field value with the same type. auto field_type = applyVisitor(FieldToDataType(), field); auto discr = tryGetVariantDiscriminator(field_type); if (!discr) diff --git a/src/DataTypes/Serializations/SerializationVariant.cpp b/src/DataTypes/Serializations/SerializationVariant.cpp index 64fcb63d604..78ec0a5e2da 100644 --- a/src/DataTypes/Serializations/SerializationVariant.cpp +++ b/src/DataTypes/Serializations/SerializationVariant.cpp @@ -200,19 +200,12 @@ void SerializationVariant::serializeBinaryBulkWithMultipleStreams( for (size_t i = 0; i != limit; ++i) writeBinaryLittleEndian(non_empty_global_discr, *discriminators_stream); - /// Second, serialize variants in global order. + /// Second, serialize non-empty variant (other variants are empty and we can skip their serialization). settings.path.push_back(Substream::VariantElements); - for (size_t i = 0; i != variants.size(); ++i) - { - addVariantElementToPath(settings.path, i); - /// For non empty variant use the same offset/limit as for whole Variant column - if (i == non_empty_global_discr) - variants[i]->serializeBinaryBulkWithMultipleStreams(col.getVariantByGlobalDiscriminator(i), offset, limit, settings, variant_state->states[i]); - /// For empty variants, use just 0/0, they won't serialize anything. - else - variants[i]->serializeBinaryBulkWithMultipleStreams(col.getVariantByGlobalDiscriminator(i), 0, 0, settings, variant_state->states[i]); - settings.path.pop_back(); - } + addVariantElementToPath(settings.path, non_empty_global_discr); + /// We can use the same offset/limit as for whole Variant column + variants[non_empty_global_discr]->serializeBinaryBulkWithMultipleStreams(col.getVariantByGlobalDiscriminator(non_empty_global_discr), offset, limit, settings, variant_state->states[non_empty_global_discr]); + settings.path.pop_back(); settings.path.pop_back(); return; } @@ -237,26 +230,22 @@ void SerializationVariant::serializeBinaryBulkWithMultipleStreams( } } - /// If limit for some variant is 0, it means that we don't have its discriminator in the range. - /// Set offset to the size of column for such variants, so we won't serialize values from them. - for (size_t i = 0; i != variant_offsets_and_limits.size(); ++i) - { - if (!variant_offsets_and_limits[i].second) - variant_offsets_and_limits[i].first = col.getVariantByGlobalDiscriminator(i).size(); - } - /// Serialize variants in global order. settings.path.push_back(Substream::VariantElements); for (size_t i = 0; i != variants.size(); ++i) { - addVariantElementToPath(settings.path, i); - variants[i]->serializeBinaryBulkWithMultipleStreams( - col.getVariantByGlobalDiscriminator(i), - variant_offsets_and_limits[i].first, - variant_offsets_and_limits[i].second, - settings, - variant_state->states[i]); - settings.path.pop_back(); + /// Serialize variant only if we have its discriminator in the range. + if (variant_offsets_and_limits[i].second) + { + addVariantElementToPath(settings.path, i); + variants[i]->serializeBinaryBulkWithMultipleStreams( + col.getVariantByGlobalDiscriminator(i), + variant_offsets_and_limits[i].first, + variant_offsets_and_limits[i].second, + settings, + variant_state->states[i]); + settings.path.pop_back(); + } } settings.path.pop_back(); } @@ -564,9 +553,6 @@ std::vector SerializationVariant::getVariantsDeserializeTextOrder(const } std::sort(order.begin(), order.end(), [&](size_t left, size_t right) { return priorities[left] > priorities[right]; }); - String types_order; - for (auto i : order) - types_order += " " + variant_types[i]->getName(); return order; } diff --git a/src/DataTypes/Serializations/SerializationVariant.h b/src/DataTypes/Serializations/SerializationVariant.h index b6bee94c65f..3f53dcf1339 100644 --- a/src/DataTypes/Serializations/SerializationVariant.h +++ b/src/DataTypes/Serializations/SerializationVariant.h @@ -6,6 +6,29 @@ namespace DB { +/// Class for serializing/deserializing column with Variant type. +/// It supports both text and binary bulk serializations/deserializations. +/// +/// During text serialization it checks discriminator of the current row and +/// uses corresponding text serialization of this variant. +/// +/// During text deserialization it tries all variants deserializations +/// (using tryDeserializeText* methods of ISerialization) in predefined order +/// and inserts data in the first variant with succeeded deserialization. +/// +/// During binary bulk serialization it transforms local discriminators +/// to global and serializes them into a separate stream VariantDiscriminators. +/// Each variant is serialized into a separate stream with path VariantElements/VariantElement +/// (VariantElements stream is needed for correct sub-columns creation). We store and serialize +/// variants in a sparse form (the size of a variant column equals to the number of its discriminator +/// in the discriminators column), so during deserialization the limit for each variant is +/// calculated according to discriminators column. +/// Offsets column is not serialized and stored only in memory. +/// +/// During binary bulk deserialization we first deserialize discriminators from corresponding stream +/// and use them to calculate the limit for each variant. Each variant is deserialized from +/// corresponding stream using calculated limit. Offsets column is not deserialized and constructed +/// according to discriminators. class SerializationVariant : public ISerialization { public: diff --git a/src/DataTypes/Serializations/SerializationVariantElement.cpp b/src/DataTypes/Serializations/SerializationVariantElement.cpp index e06a20d2990..be91e0ba2ee 100644 --- a/src/DataTypes/Serializations/SerializationVariantElement.cpp +++ b/src/DataTypes/Serializations/SerializationVariantElement.cpp @@ -149,19 +149,21 @@ void SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams( assert_cast(*variant_element_state->variant->assumeMutable()).nestedRemoveNullable(); } + /// If nothing to deserialize, just insert defaults. + if (variant_limit == 0) + { + mutable_column->insertManyDefaults(limit); + return; + } + addVariantToPath(settings.path); nested_serialization->deserializeBinaryBulkWithMultipleStreams(variant_element_state->variant, variant_limit, settings, variant_element_state->variant_element_state, cache); removeVariantFromPath(settings.path); size_t variant_offset = variant_element_state->variant->size() - variant_limit; - /// If don't have our discriminator in range, just insert defaults. - if (variant_limit == 0) - { - mutable_column->insertManyDefaults(limit); - } /// If we have only our discriminator in range, insert the whole range to result column. - else if (variant_limit == limit) + if (variant_limit == limit) { mutable_column->insertRangeFrom(*variant_element_state->variant, variant_offset, variant_limit); } From 483bbaf14d16bb4b3871781c1bbc95eee7311a8f Mon Sep 17 00:00:00 2001 From: Tim Liou Date: Thu, 18 Jan 2024 23:42:10 +0800 Subject: [PATCH 061/264] Run black locally to format the code --- .../test_structured_logging_json/test.py | 24 ++++++++++--------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/tests/integration/test_structured_logging_json/test.py b/tests/integration/test_structured_logging_json/test.py index 313fd0e205d..6d1455f6e0e 100644 --- a/tests/integration/test_structured_logging_json/test.py +++ b/tests/integration/test_structured_logging_json/test.py @@ -37,14 +37,14 @@ def validate_log_level(config, logs): key = root.findtext(".//names/level") or "level" valid_level_values = { - "Fatal", - "Critical", - "Error", - "Warning", - "Notice", - "Information", - "Debug", - "Trace", + "Fatal", + "Critical", + "Error", + "Warning", + "Notice", + "Information", + "Debug", + "Trace", "Test", } @@ -102,9 +102,11 @@ def validate_logs(logs): def valiade_everything(config, node, config_type): node.query("SELECT 1") logs = node.grep_in_log("").split("\n") - return validate_logs(logs) and validate_log_config_relation( - config, logs, config_type - ) and validate_log_level(config, logs) + return ( + validate_logs(logs) + and validate_log_config_relation(config, logs, config_type) + and validate_log_level(config, logs) + ) def test_structured_logging_json_format(start_cluster): From 0ec385064c93bf6375b37354a7ffa09bb1368dbe Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Thu, 18 Jan 2024 17:28:47 +0100 Subject: [PATCH 062/264] Add SIGNED/UNSIGNED aliases for Int64/UInt64 --- src/DataTypes/DataTypesNumber.cpp | 6 ++- .../02969_mysql_cast_type_aliases.reference | 54 +++++++++++++++++++ .../02969_mysql_cast_type_aliases.sh | 52 ++++++++++++++++++ 3 files changed, 110 insertions(+), 2 deletions(-) create mode 100755 tests/queries/0_stateless/02969_mysql_cast_type_aliases.reference create mode 100755 tests/queries/0_stateless/02969_mysql_cast_type_aliases.sh diff --git a/src/DataTypes/DataTypesNumber.cpp b/src/DataTypes/DataTypesNumber.cpp index 1c0c418411b..e472021e756 100644 --- a/src/DataTypes/DataTypesNumber.cpp +++ b/src/DataTypes/DataTypesNumber.cpp @@ -87,6 +87,7 @@ void registerDataTypeNumbers(DataTypeFactory & factory) factory.registerAlias("MEDIUMINT SIGNED", "Int32", DataTypeFactory::CaseInsensitive); factory.registerAlias("INT SIGNED", "Int32", DataTypeFactory::CaseInsensitive); factory.registerAlias("INTEGER SIGNED", "Int32", DataTypeFactory::CaseInsensitive); + factory.registerAlias("SIGNED", "Int64", DataTypeFactory::CaseInsensitive); factory.registerAlias("BIGINT SIGNED", "Int64", DataTypeFactory::CaseInsensitive); factory.registerAlias("TINYINT UNSIGNED", "UInt8", DataTypeFactory::CaseInsensitive); factory.registerAlias("INT1 UNSIGNED", "UInt8", DataTypeFactory::CaseInsensitive); @@ -94,9 +95,10 @@ void registerDataTypeNumbers(DataTypeFactory & factory) factory.registerAlias("MEDIUMINT UNSIGNED", "UInt32", DataTypeFactory::CaseInsensitive); factory.registerAlias("INT UNSIGNED", "UInt32", DataTypeFactory::CaseInsensitive); factory.registerAlias("INTEGER UNSIGNED", "UInt32", DataTypeFactory::CaseInsensitive); + factory.registerAlias("UNSIGNED", "UInt64", DataTypeFactory::CaseInsensitive); factory.registerAlias("BIGINT UNSIGNED", "UInt64", DataTypeFactory::CaseInsensitive); - factory.registerAlias("BIT", "UInt64", DataTypeFactory::CaseInsensitive); /// MySQL - factory.registerAlias("SET", "UInt64", DataTypeFactory::CaseInsensitive); /// MySQL + factory.registerAlias("BIT", "UInt64", DataTypeFactory::CaseInsensitive); + factory.registerAlias("SET", "UInt64", DataTypeFactory::CaseInsensitive); factory.registerAlias("YEAR", "UInt16", DataTypeFactory::CaseInsensitive); factory.registerAlias("TIME", "Int64", DataTypeFactory::CaseInsensitive); } diff --git a/tests/queries/0_stateless/02969_mysql_cast_type_aliases.reference b/tests/queries/0_stateless/02969_mysql_cast_type_aliases.reference new file mode 100755 index 00000000000..b24a4bf5886 --- /dev/null +++ b/tests/queries/0_stateless/02969_mysql_cast_type_aliases.reference @@ -0,0 +1,54 @@ +-- Uppercase tests +type result +Binary(N) foo +type result +Char 44 +type result +Date 2021-02-03 +type result +DateTime 2021-02-03 12:01:02 +type result +Decimal 45 +type result +Decimal(M) 46 +type result +Decimal(M, D) 47.21 +type result +Double 48.11 +type result +JSON {"foo":"bar"} +type result +Real 49.22 +type result +Signed 50 +type result +Unsigned 52 +type result +Year 2007 +-- Lowercase tests +type result +Binary(N) foo +type result +Char 44 +type result +Date 2021-02-03 +type result +DateTime 2021-02-03 12:01:02 +type result +Decimal 45 +type result +Decimal(M) 46 +type result +Decimal(M, D) 47.21 +type result +Double 48.11 +type result +JSON {"foo":"bar"} +type result +Real 49.22 +type result +Signed 50 +type result +Unsigned 52 +type result +Year 2007 diff --git a/tests/queries/0_stateless/02969_mysql_cast_type_aliases.sh b/tests/queries/0_stateless/02969_mysql_cast_type_aliases.sh new file mode 100755 index 00000000000..175d12e2045 --- /dev/null +++ b/tests/queries/0_stateless/02969_mysql_cast_type_aliases.sh @@ -0,0 +1,52 @@ +#!/usr/bin/env bash +# Tags: no-fasttest +# Tag no-fasttest: requires mysql client + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# See https://dev.mysql.com/doc/refman/8.0/en/cast-functions.html#function_cast +# Tests are in order of the type appearance in the docs + +echo "-- Uppercase tests" +#### Not supported as it is translated to FixedString without arguments +# ${MYSQL_CLIENT} --execute "SELECT 'Binary' AS type, CAST('' AS BINARY) AS result;" +${MYSQL_CLIENT} --execute "SELECT 'Binary(N)' AS type, CAST('foo' AS BINARY(3)) AS result;" +${MYSQL_CLIENT} --execute "SELECT 'Char' AS type, CAST(44 AS CHAR) AS result;" +${MYSQL_CLIENT} --execute "SELECT 'Date' AS type, CAST('2021-02-03' AS DATE) AS result;" +${MYSQL_CLIENT} --execute "SELECT 'DateTime' AS type, CAST('2021-02-03 12:01:02' AS DATETIME) AS result;" +${MYSQL_CLIENT} --execute "SELECT 'Decimal' AS type, CAST(45.1 AS DECIMAL) AS result;" +${MYSQL_CLIENT} --execute "SELECT 'Decimal(M)' AS type, CAST(46.2 AS DECIMAL(4)) AS result;" +${MYSQL_CLIENT} --execute "SELECT 'Decimal(M, D)' AS type, CAST(47.21 AS DECIMAL(4, 2)) AS result;" +${MYSQL_CLIENT} --execute "SELECT 'Double' AS type, CAST(48.11 AS DOUBLE) AS result;" +${MYSQL_CLIENT} --execute "SET allow_experimental_object_type = 1; SELECT 'JSON' AS type, CAST('{\"foo\":\"bar\"}' AS JSON) AS result;" +${MYSQL_CLIENT} --execute "SELECT 'Real' AS type, CAST(49.22 AS REAL) AS result;" +${MYSQL_CLIENT} --execute "SELECT 'Signed' AS type, CAST(50 AS SIGNED) AS result;" +${MYSQL_CLIENT} --execute "SELECT 'Unsigned' AS type, CAST(52 AS UNSIGNED) AS result;" +#### Could be added as an alias, but SIGNED INTEGER in CAST context means UInt64, +#### while INTEGER SIGNED as a column definition means UInt32. +# ${MYSQL_CLIENT} --execute "SELECT 'Signed integer' AS type, CAST(51 AS SIGNED INTEGER) AS result;" +# ${MYSQL_CLIENT} --execute "SELECT 'Unsigned integer' AS type, CAST(53 AS UNSIGNED INTEGER) AS result;" +${MYSQL_CLIENT} --execute "SELECT 'Year' AS type, CAST(2007 AS YEAR) AS result;" +#### Currently, expects UInt64 as an argument +# ${MYSQL_CLIENT} --execute "SELECT 'Time' AS type, CAST('12:45' AS TIME) AS result;" + +echo "-- Lowercase tests" +# ${MYSQL_CLIENT} --execute "select 'Binary' as type, cast('' as binary) as result;" +${MYSQL_CLIENT} --execute "select 'Binary(N)' as type, cast('foo' as binary(3)) as result;" +${MYSQL_CLIENT} --execute "select 'Char' as type, cast(44 as char) as result;" +${MYSQL_CLIENT} --execute "select 'Date' as type, cast('2021-02-03' as date) as result;" +${MYSQL_CLIENT} --execute "select 'DateTime' as type, cast('2021-02-03 12:01:02' as datetime) as result;" +${MYSQL_CLIENT} --execute "select 'Decimal' as type, cast(45.1 as decimal) as result;" +${MYSQL_CLIENT} --execute "select 'Decimal(M)' as type, cast(46.2 as decimal(4)) as result;" +${MYSQL_CLIENT} --execute "select 'Decimal(M, D)' as type, cast(47.21 as decimal(4, 2)) as result;" +${MYSQL_CLIENT} --execute "select 'Double' as type, cast(48.11 as double) as result;" +${MYSQL_CLIENT} --execute "set allow_experimental_object_type = 1; select 'JSON' as type, cast('{\"foo\":\"bar\"}' as json) as result;" +${MYSQL_CLIENT} --execute "select 'Real' as type, cast(49.22 as real) as result;" +${MYSQL_CLIENT} --execute "select 'Signed' as type, cast(50 as signed) as result;" +${MYSQL_CLIENT} --execute "select 'Unsigned' as type, cast(52 as unsigned) as result;" +# ${MYSQL_CLIENT} --execute "select 'Signed integer' as type, cast(51 as signed integer) as result;" +# ${MYSQL_CLIENT} --execute "select 'Unsigned integer' as type, cast(53 as unsigned integer) as result;" +${MYSQL_CLIENT} --execute "select 'Year' as type, cast(2007 as year) as result;" +# ${MYSQL_CLIENT} --execute "select 'Time' as type, cast('12:45' as time) as result;" From cee5a99bcad35759e8c92f01af499f150b6b0875 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Thu, 18 Jan 2024 17:49:18 +0100 Subject: [PATCH 063/264] Add native type checks to the tests --- .../02969_mysql_cast_type_aliases.reference | 104 +++++++++--------- .../02969_mysql_cast_type_aliases.sh | 68 ++++++------ 2 files changed, 86 insertions(+), 86 deletions(-) diff --git a/tests/queries/0_stateless/02969_mysql_cast_type_aliases.reference b/tests/queries/0_stateless/02969_mysql_cast_type_aliases.reference index b24a4bf5886..cd9ed5eb39f 100755 --- a/tests/queries/0_stateless/02969_mysql_cast_type_aliases.reference +++ b/tests/queries/0_stateless/02969_mysql_cast_type_aliases.reference @@ -1,54 +1,54 @@ -- Uppercase tests -type result -Binary(N) foo -type result -Char 44 -type result -Date 2021-02-03 -type result -DateTime 2021-02-03 12:01:02 -type result -Decimal 45 -type result -Decimal(M) 46 -type result -Decimal(M, D) 47.21 -type result -Double 48.11 -type result -JSON {"foo":"bar"} -type result -Real 49.22 -type result -Signed 50 -type result -Unsigned 52 -type result -Year 2007 +mysql_type result native_type +Binary(N) foo FixedString(3) +mysql_type result native_type +Char 44 String +mysql_type result native_type +Date 2021-02-03 Date +mysql_type result native_type +DateTime 2021-02-03 12:01:02 DateTime +mysql_type result native_type +Decimal 45 Decimal(10, 0) +mysql_type result native_type +Decimal(M) 46 Decimal(4, 0) +mysql_type result native_type +Decimal(M, D) 47.21 Decimal(4, 2) +mysql_type result native_type +Double 48.11 Float64 +mysql_type result native_type +JSON {"foo":"bar"} Object('json') +mysql_type result native_type +Real 49.22 Float32 +mysql_type result native_type +Signed 50 Int64 +mysql_type result native_type +Unsigned 52 UInt64 +mysql_type result native_type +Year 2007 UInt16 -- Lowercase tests -type result -Binary(N) foo -type result -Char 44 -type result -Date 2021-02-03 -type result -DateTime 2021-02-03 12:01:02 -type result -Decimal 45 -type result -Decimal(M) 46 -type result -Decimal(M, D) 47.21 -type result -Double 48.11 -type result -JSON {"foo":"bar"} -type result -Real 49.22 -type result -Signed 50 -type result -Unsigned 52 -type result -Year 2007 +mysql_type result native_type +Binary(N) foo FixedString(3) +mysql_type result native_type +Char 44 String +mysql_type result native_type +Date 2021-02-03 Date +mysql_type result native_type +DateTime 2021-02-03 12:01:02 DateTime +mysql_type result native_type +Decimal 45 Decimal(10, 0) +mysql_type result native_type +Decimal(M) 46 Decimal(4, 0) +mysql_type result native_type +Decimal(M, D) 47.21 Decimal(4, 2) +mysql_type result native_type +Double 48.11 Float64 +mysql_type result native_type +JSON {"foo":"bar"} Object('json') +mysql_type result native_type +Real 49.22 Float32 +mysql_type result native_type +Signed 50 Int64 +mysql_type result native_type +Unsigned 52 UInt64 +mysql_type result native_type +Year 2007 UInt16 diff --git a/tests/queries/0_stateless/02969_mysql_cast_type_aliases.sh b/tests/queries/0_stateless/02969_mysql_cast_type_aliases.sh index 175d12e2045..35218bec549 100755 --- a/tests/queries/0_stateless/02969_mysql_cast_type_aliases.sh +++ b/tests/queries/0_stateless/02969_mysql_cast_type_aliases.sh @@ -11,42 +11,42 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) echo "-- Uppercase tests" #### Not supported as it is translated to FixedString without arguments -# ${MYSQL_CLIENT} --execute "SELECT 'Binary' AS type, CAST('' AS BINARY) AS result;" -${MYSQL_CLIENT} --execute "SELECT 'Binary(N)' AS type, CAST('foo' AS BINARY(3)) AS result;" -${MYSQL_CLIENT} --execute "SELECT 'Char' AS type, CAST(44 AS CHAR) AS result;" -${MYSQL_CLIENT} --execute "SELECT 'Date' AS type, CAST('2021-02-03' AS DATE) AS result;" -${MYSQL_CLIENT} --execute "SELECT 'DateTime' AS type, CAST('2021-02-03 12:01:02' AS DATETIME) AS result;" -${MYSQL_CLIENT} --execute "SELECT 'Decimal' AS type, CAST(45.1 AS DECIMAL) AS result;" -${MYSQL_CLIENT} --execute "SELECT 'Decimal(M)' AS type, CAST(46.2 AS DECIMAL(4)) AS result;" -${MYSQL_CLIENT} --execute "SELECT 'Decimal(M, D)' AS type, CAST(47.21 AS DECIMAL(4, 2)) AS result;" -${MYSQL_CLIENT} --execute "SELECT 'Double' AS type, CAST(48.11 AS DOUBLE) AS result;" -${MYSQL_CLIENT} --execute "SET allow_experimental_object_type = 1; SELECT 'JSON' AS type, CAST('{\"foo\":\"bar\"}' AS JSON) AS result;" -${MYSQL_CLIENT} --execute "SELECT 'Real' AS type, CAST(49.22 AS REAL) AS result;" -${MYSQL_CLIENT} --execute "SELECT 'Signed' AS type, CAST(50 AS SIGNED) AS result;" -${MYSQL_CLIENT} --execute "SELECT 'Unsigned' AS type, CAST(52 AS UNSIGNED) AS result;" +# ${MYSQL_CLIENT} --execute "SELECT 'Binary' AS mysql_type, CAST('' AS BINARY) AS result, toTypeName(result) AS native_type;" +${MYSQL_CLIENT} --execute "SELECT 'Binary(N)' AS mysql_type, CAST('foo' AS BINARY(3)) AS result, toTypeName(result) AS native_type;" +${MYSQL_CLIENT} --execute "SELECT 'Char' AS mysql_type, CAST(44 AS CHAR) AS result, toTypeName(result) AS native_type;" +${MYSQL_CLIENT} --execute "SELECT 'Date' AS mysql_type, CAST('2021-02-03' AS DATE) AS result, toTypeName(result) AS native_type;" +${MYSQL_CLIENT} --execute "SELECT 'DateTime' AS mysql_type, CAST('2021-02-03 12:01:02' AS DATETIME) AS result, toTypeName(result) AS native_type;" +${MYSQL_CLIENT} --execute "SELECT 'Decimal' AS mysql_type, CAST(45.1 AS DECIMAL) AS result, toTypeName(result) AS native_type;" +${MYSQL_CLIENT} --execute "SELECT 'Decimal(M)' AS mysql_type, CAST(46.2 AS DECIMAL(4)) AS result, toTypeName(result) AS native_type;" +${MYSQL_CLIENT} --execute "SELECT 'Decimal(M, D)' AS mysql_type, CAST(47.21 AS DECIMAL(4, 2)) AS result, toTypeName(result) AS native_type;" +${MYSQL_CLIENT} --execute "SELECT 'Double' AS mysql_type, CAST(48.11 AS DOUBLE) AS result, toTypeName(result) AS native_type;" +${MYSQL_CLIENT} --execute "SET allow_experimental_object_type = 1; SELECT 'JSON' AS mysql_type, CAST('{\"foo\":\"bar\"}' AS JSON) AS result, toTypeName(result) AS native_type;" +${MYSQL_CLIENT} --execute "SELECT 'Real' AS mysql_type, CAST(49.22 AS REAL) AS result, toTypeName(result) AS native_type;" +${MYSQL_CLIENT} --execute "SELECT 'Signed' AS mysql_type, CAST(50 AS SIGNED) AS result, toTypeName(result) AS native_type;" +${MYSQL_CLIENT} --execute "SELECT 'Unsigned' AS mysql_type, CAST(52 AS UNSIGNED) AS result, toTypeName(result) AS native_type;" #### Could be added as an alias, but SIGNED INTEGER in CAST context means UInt64, #### while INTEGER SIGNED as a column definition means UInt32. -# ${MYSQL_CLIENT} --execute "SELECT 'Signed integer' AS type, CAST(51 AS SIGNED INTEGER) AS result;" -# ${MYSQL_CLIENT} --execute "SELECT 'Unsigned integer' AS type, CAST(53 AS UNSIGNED INTEGER) AS result;" -${MYSQL_CLIENT} --execute "SELECT 'Year' AS type, CAST(2007 AS YEAR) AS result;" +# ${MYSQL_CLIENT} --execute "SELECT 'Signed integer' AS mysql_type, CAST(51 AS SIGNED INTEGER) AS result, toTypeName(result) AS native_type;" +# ${MYSQL_CLIENT} --execute "SELECT 'Unsigned integer' AS mysql_type, CAST(53 AS UNSIGNED INTEGER) AS result, toTypeName(result) AS native_type;" +${MYSQL_CLIENT} --execute "SELECT 'Year' AS mysql_type, CAST(2007 AS YEAR) AS result, toTypeName(result) AS native_type;" #### Currently, expects UInt64 as an argument -# ${MYSQL_CLIENT} --execute "SELECT 'Time' AS type, CAST('12:45' AS TIME) AS result;" +# ${MYSQL_CLIENT} --execute "SELECT 'Time' AS mysql_type, CAST('12:45' AS TIME) AS result, toTypeName(result) AS native_type;" echo "-- Lowercase tests" -# ${MYSQL_CLIENT} --execute "select 'Binary' as type, cast('' as binary) as result;" -${MYSQL_CLIENT} --execute "select 'Binary(N)' as type, cast('foo' as binary(3)) as result;" -${MYSQL_CLIENT} --execute "select 'Char' as type, cast(44 as char) as result;" -${MYSQL_CLIENT} --execute "select 'Date' as type, cast('2021-02-03' as date) as result;" -${MYSQL_CLIENT} --execute "select 'DateTime' as type, cast('2021-02-03 12:01:02' as datetime) as result;" -${MYSQL_CLIENT} --execute "select 'Decimal' as type, cast(45.1 as decimal) as result;" -${MYSQL_CLIENT} --execute "select 'Decimal(M)' as type, cast(46.2 as decimal(4)) as result;" -${MYSQL_CLIENT} --execute "select 'Decimal(M, D)' as type, cast(47.21 as decimal(4, 2)) as result;" -${MYSQL_CLIENT} --execute "select 'Double' as type, cast(48.11 as double) as result;" -${MYSQL_CLIENT} --execute "set allow_experimental_object_type = 1; select 'JSON' as type, cast('{\"foo\":\"bar\"}' as json) as result;" -${MYSQL_CLIENT} --execute "select 'Real' as type, cast(49.22 as real) as result;" -${MYSQL_CLIENT} --execute "select 'Signed' as type, cast(50 as signed) as result;" -${MYSQL_CLIENT} --execute "select 'Unsigned' as type, cast(52 as unsigned) as result;" -# ${MYSQL_CLIENT} --execute "select 'Signed integer' as type, cast(51 as signed integer) as result;" -# ${MYSQL_CLIENT} --execute "select 'Unsigned integer' as type, cast(53 as unsigned integer) as result;" -${MYSQL_CLIENT} --execute "select 'Year' as type, cast(2007 as year) as result;" -# ${MYSQL_CLIENT} --execute "select 'Time' as type, cast('12:45' as time) as result;" +# ${MYSQL_CLIENT} --execute "select 'Binary' as mysql_type, cast('' as binary) as result, toTypeName(result) as native_type;" +${MYSQL_CLIENT} --execute "select 'Binary(N)' as mysql_type, cast('foo' as binary(3)) as result, toTypeName(result) as native_type;" +${MYSQL_CLIENT} --execute "select 'Char' as mysql_type, cast(44 as char) as result, toTypeName(result) as native_type;" +${MYSQL_CLIENT} --execute "select 'Date' as mysql_type, cast('2021-02-03' as date) as result, toTypeName(result) as native_type;" +${MYSQL_CLIENT} --execute "select 'DateTime' as mysql_type, cast('2021-02-03 12:01:02' as datetime) as result, toTypeName(result) as native_type;" +${MYSQL_CLIENT} --execute "select 'Decimal' as mysql_type, cast(45.1 as decimal) as result, toTypeName(result) as native_type;" +${MYSQL_CLIENT} --execute "select 'Decimal(M)' as mysql_type, cast(46.2 as decimal(4)) as result, toTypeName(result) as native_type;" +${MYSQL_CLIENT} --execute "select 'Decimal(M, D)' as mysql_type, cast(47.21 as decimal(4, 2)) as result, toTypeName(result) as native_type;" +${MYSQL_CLIENT} --execute "select 'Double' as mysql_type, cast(48.11 as double) as result, toTypeName(result) as native_type;" +${MYSQL_CLIENT} --execute "set allow_experimental_object_type = 1; select 'JSON' as mysql_type, cast('{\"foo\":\"bar\"}' as json) as result, toTypeName(result) as native_type;" +${MYSQL_CLIENT} --execute "select 'Real' as mysql_type, cast(49.22 as real) as result, toTypeName(result) as native_type;" +${MYSQL_CLIENT} --execute "select 'Signed' as mysql_type, cast(50 as signed) as result, toTypeName(result) as native_type;" +${MYSQL_CLIENT} --execute "select 'Unsigned' as mysql_type, cast(52 as unsigned) as result, toTypeName(result) as native_type;" +# ${MYSQL_CLIENT} --execute "select 'Signed integer' as mysql_type, cast(51 as signed integer) as result, toTypeName(result) as native_type;" +# ${MYSQL_CLIENT} --execute "select 'Unsigned integer' as mysql_type, cast(53 as unsigned integer) as result, toTypeName(result) as native_type;" +${MYSQL_CLIENT} --execute "select 'Year' as mysql_type, cast(2007 as year) as result, toTypeName(result) as native_type;" +# ${MYSQL_CLIENT} --execute "select 'Time' as mysql_type, cast('12:45' as time) as result, toTypeName(result) as native_type;" From f202d713711857c083de5aaba1198198d2eaa3a4 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 18 Jan 2024 17:29:36 +0000 Subject: [PATCH 064/264] Make alter with variant espansion as no-op, add tests for alters --- src/DataTypes/DataTypeVariant.cpp | 22 ++ src/DataTypes/DataTypeVariant.h | 4 + .../Serializations/SerializationVariant.cpp | 2 +- .../SerializationVariantElement.cpp | 16 +- src/Storages/MergeTree/MutateTask.cpp | 20 +- .../02941_variant_type_alters.reference | 330 ++++++++++++++++++ .../0_stateless/02941_variant_type_alters.sh | 61 ++++ 7 files changed, 452 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02941_variant_type_alters.reference create mode 100755 tests/queries/0_stateless/02941_variant_type_alters.sh diff --git a/src/DataTypes/DataTypeVariant.cpp b/src/DataTypes/DataTypeVariant.cpp index e0510373960..3a39fdf9ea8 100644 --- a/src/DataTypes/DataTypeVariant.cpp +++ b/src/DataTypes/DataTypeVariant.cpp @@ -230,6 +230,28 @@ static DataTypePtr create(const ASTPtr & arguments) return std::make_shared(nested_types); } +bool isVariantExtension(const DataTypePtr & from_type, const DataTypePtr & to_type) +{ + const auto * from_variant = typeid_cast(from_type.get()); + const auto * to_variant = typeid_cast(to_type.get()); + if (!from_variant || !to_variant) + return false; + + const auto & to_variants = to_variant->getVariants(); + std::unordered_set to_variant_types; + to_variant_types.reserve(to_variants.size()); + for (const auto & variant : to_variants) + to_variant_types.insert(variant->getName()); + + for (const auto & variant : from_variant->getVariants()) + { + if (!to_variant_types.contains(variant->getName())) + return false; + } + + return true; +} + void registerDataTypeVariant(DataTypeFactory & factory) { diff --git a/src/DataTypes/DataTypeVariant.h b/src/DataTypes/DataTypeVariant.h index ca15dff1476..1a1cb6c12f2 100644 --- a/src/DataTypes/DataTypeVariant.h +++ b/src/DataTypes/DataTypeVariant.h @@ -61,5 +61,9 @@ private: SerializationPtr doGetDefaultSerialization() const override; }; +/// Check if conversion from from_type to to_type is Variant extension +/// (both types are Variants and to_type contains all variants from from_type). +bool isVariantExtension(const DataTypePtr & from_type, const DataTypePtr & to_type); + } diff --git a/src/DataTypes/Serializations/SerializationVariant.cpp b/src/DataTypes/Serializations/SerializationVariant.cpp index 78ec0a5e2da..48a78dd54a9 100644 --- a/src/DataTypes/Serializations/SerializationVariant.cpp +++ b/src/DataTypes/Serializations/SerializationVariant.cpp @@ -275,7 +275,7 @@ void SerializationVariant::deserializeBinaryBulkWithMultipleStreams( { auto * discriminators_stream = settings.getter(settings.path); if (!discriminators_stream) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty stream for VariantDiscriminators in SerializationVariant::deserializeBinaryBulkWithMultipleStreams"); + return; SerializationNumber().deserializeBinaryBulk(*col.getLocalDiscriminatorsPtr()->assumeMutable(), *discriminators_stream, limit, 0); addToSubstreamsCache(cache, settings.path, col.getLocalDiscriminatorsPtr()); diff --git a/src/DataTypes/Serializations/SerializationVariantElement.cpp b/src/DataTypes/Serializations/SerializationVariantElement.cpp index be91e0ba2ee..80524cbd814 100644 --- a/src/DataTypes/Serializations/SerializationVariantElement.cpp +++ b/src/DataTypes/Serializations/SerializationVariantElement.cpp @@ -91,7 +91,7 @@ void SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams( { auto * discriminators_stream = settings.getter(settings.path); if (!discriminators_stream) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty stream for VariantDiscriminators in SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams"); + return; /// If we started to read a new column, reinitialize discriminators column in deserialization state. if (!variant_element_state->discriminators || result_column->empty()) @@ -156,10 +156,24 @@ void SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams( return; } + size_t prev_variant_size = variant_element_state->variant->size(); addVariantToPath(settings.path); nested_serialization->deserializeBinaryBulkWithMultipleStreams(variant_element_state->variant, variant_limit, settings, variant_element_state->variant_element_state, cache); removeVariantFromPath(settings.path); + /// If nothing was deserialized when variant_limit > 0 + /// it means that we don't have a stream for such sub-column. + /// It may happen during ALTER MODIFY column with Variant extension. + /// In this case we should just insert default values. + if (variant_element_state->variant->empty()) + { + mutable_column->insertManyDefaults(limit); + return; + } + + if (variant_element_state->variant->size() != prev_variant_size + variant_limit) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected variant column size after deserialization. Expected {}, got {}", prev_variant_size + variant_limit, variant_element_state->variant->size()); + size_t variant_offset = variant_element_state->variant->size() - variant_limit; /// If we have only our discriminator in range, insert the whole range to result column. diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index e4070aa8262..44734ec98c0 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -22,6 +22,7 @@ #include #include #include +#include #include #include @@ -1921,7 +1922,7 @@ static bool canSkipConversionToNullable(const MergeTreeDataPartPtr & part, const if (!part_column) return false; - /// For ALTER MODIFY COLUMN from 'Type' to 'Nullable(Type)' we can skip mutatation and + /// For ALTER MODIFY COLUMN from 'Type' to 'Nullable(Type)' we can skip mutation and /// apply only metadata conversion. But it doesn't work for custom serialization. const auto * to_nullable = typeid_cast(command.data_type.get()); if (!to_nullable) @@ -1937,6 +1938,20 @@ static bool canSkipConversionToNullable(const MergeTreeDataPartPtr & part, const return true; } +static bool canSkipConversionToVariant(const MergeTreeDataPartPtr & part, const MutationCommand & command) +{ + if (command.type != MutationCommand::READ_COLUMN) + return false; + + auto part_column = part->tryGetColumn(command.column_name); + if (!part_column) + return false; + + /// For ALTER MODIFY COLUMN with Variant extension (like 'Variant(T1, T2)' to 'Variant(T1, T2, T3, ...)') + /// we can skip mutation and apply only metadata conversion. + return isVariantExtension(part_column->type, command.data_type); +} + static bool canSkipMutationCommandForPart(const MergeTreeDataPartPtr & part, const MutationCommand & command, const ContextPtr & context) { if (command.partition) @@ -1952,6 +1967,9 @@ static bool canSkipMutationCommandForPart(const MergeTreeDataPartPtr & part, con if (canSkipConversionToNullable(part, command)) return true; + if (canSkipConversionToVariant(part, command)) + return true; + return false; } diff --git a/tests/queries/0_stateless/02941_variant_type_alters.reference b/tests/queries/0_stateless/02941_variant_type_alters.reference new file mode 100644 index 00000000000..52c834e455b --- /dev/null +++ b/tests/queries/0_stateless/02941_variant_type_alters.reference @@ -0,0 +1,330 @@ +Memory +initial insert +alter add column 1 +0 0 \N \N \N +1 1 \N \N \N +2 2 \N \N \N +insert after alter add column 1 +0 0 \N \N \N +1 1 \N \N \N +2 2 \N \N \N +3 3 3 \N 3 +4 4 4 \N 4 +5 5 5 \N 5 +6 6 str_6 str_6 \N +7 7 str_7 str_7 \N +8 8 str_8 str_8 \N +9 9 \N \N \N +10 10 \N \N \N +11 11 \N \N \N +12 12 12 \N 12 +13 13 str_13 str_13 \N +14 14 \N \N \N +alter modify column 1 +0 0 \N \N \N \N +1 1 \N \N \N \N +2 2 \N \N \N \N +3 3 3 \N 3 \N +4 4 4 \N 4 \N +5 5 5 \N 5 \N +6 6 str_6 str_6 \N \N +7 7 str_7 str_7 \N \N +8 8 str_8 str_8 \N \N +9 9 \N \N \N \N +10 10 \N \N \N \N +11 11 \N \N \N \N +12 12 12 \N 12 \N +13 13 str_13 str_13 \N \N +14 14 \N \N \N \N +insert after alter modify column 1 +0 0 \N \N \N \N +1 1 \N \N \N \N +2 2 \N \N \N \N +3 3 3 \N 3 \N +4 4 4 \N 4 \N +5 5 5 \N 5 \N +6 6 str_6 str_6 \N \N +7 7 str_7 str_7 \N \N +8 8 str_8 str_8 \N \N +9 9 \N \N \N \N +10 10 \N \N \N \N +11 11 \N \N \N \N +12 12 12 \N 12 \N +13 13 str_13 str_13 \N \N +14 14 \N \N \N \N +15 15 1970-01-16 \N \N 1970-01-16 +16 16 1970-01-17 \N \N 1970-01-17 +17 17 1970-01-18 \N \N 1970-01-18 +18 18 1970-01-19 \N \N 1970-01-19 +19 19 \N \N \N \N +20 20 20 \N 20 \N +21 21 str_21 str_21 \N \N +alter modify column 2 +0 0 \N \N \N \N \N \N +1 1 \N \N \N \N \N \N +2 2 \N \N \N \N \N \N +3 3 \N \N 3 \N 3 \N +4 4 \N \N 4 \N 4 \N +5 5 \N \N 5 \N 5 \N +6 6 \N \N str_6 str_6 \N \N +7 7 \N \N str_7 str_7 \N \N +8 8 \N \N str_8 str_8 \N \N +9 9 \N \N \N \N \N \N +10 10 \N \N \N \N \N \N +11 11 \N \N \N \N \N \N +12 12 \N \N 12 \N 12 \N +13 13 \N \N str_13 str_13 \N \N +14 14 \N \N \N \N \N \N +15 15 \N \N 1970-01-16 \N \N 1970-01-16 +16 16 \N \N 1970-01-17 \N \N 1970-01-17 +17 17 \N \N 1970-01-18 \N \N 1970-01-18 +18 18 \N \N 1970-01-19 \N \N 1970-01-19 +19 19 \N \N \N \N \N \N +20 20 \N \N 20 \N 20 \N +21 21 \N \N str_21 str_21 \N \N +insert after alter modify column 2 +0 0 \N \N \N \N \N \N +1 1 \N \N \N \N \N \N +2 2 \N \N \N \N \N \N +3 3 \N \N 3 \N 3 \N +4 4 \N \N 4 \N 4 \N +5 5 \N \N 5 \N 5 \N +6 6 \N \N str_6 str_6 \N \N +7 7 \N \N str_7 str_7 \N \N +8 8 \N \N str_8 str_8 \N \N +9 9 \N \N \N \N \N \N +10 10 \N \N \N \N \N \N +11 11 \N \N \N \N \N \N +12 12 \N \N 12 \N 12 \N +13 13 \N \N str_13 str_13 \N \N +14 14 \N \N \N \N \N \N +15 15 \N \N 1970-01-16 \N \N 1970-01-16 +16 16 \N \N 1970-01-17 \N \N 1970-01-17 +17 17 \N \N 1970-01-18 \N \N 1970-01-18 +18 18 \N \N 1970-01-19 \N \N 1970-01-19 +19 19 \N \N \N \N \N \N +20 20 \N \N 20 \N 20 \N +21 21 \N \N str_21 str_21 \N \N +22 str_22 \N str_22 \N \N \N \N +23 \N \N \N \N \N \N \N +24 24 24 \N \N \N \N \N +MergeTree compact +initial insert +alter add column 1 +0 0 \N \N \N +1 1 \N \N \N +2 2 \N \N \N +insert after alter add column 1 +0 0 \N \N \N +1 1 \N \N \N +2 2 \N \N \N +3 3 3 \N 3 +4 4 4 \N 4 +5 5 5 \N 5 +6 6 str_6 str_6 \N +7 7 str_7 str_7 \N +8 8 str_8 str_8 \N +9 9 \N \N \N +10 10 \N \N \N +11 11 \N \N \N +12 12 12 \N 12 +13 13 str_13 str_13 \N +14 14 \N \N \N +alter modify column 1 +0 0 \N \N \N \N +1 1 \N \N \N \N +2 2 \N \N \N \N +3 3 3 \N 3 \N +4 4 4 \N 4 \N +5 5 5 \N 5 \N +6 6 str_6 str_6 \N \N +7 7 str_7 str_7 \N \N +8 8 str_8 str_8 \N \N +9 9 \N \N \N \N +10 10 \N \N \N \N +11 11 \N \N \N \N +12 12 12 \N 12 \N +13 13 str_13 str_13 \N \N +14 14 \N \N \N \N +insert after alter modify column 1 +0 0 \N \N \N \N +1 1 \N \N \N \N +2 2 \N \N \N \N +3 3 3 \N 3 \N +4 4 4 \N 4 \N +5 5 5 \N 5 \N +6 6 str_6 str_6 \N \N +7 7 str_7 str_7 \N \N +8 8 str_8 str_8 \N \N +9 9 \N \N \N \N +10 10 \N \N \N \N +11 11 \N \N \N \N +12 12 12 \N 12 \N +13 13 str_13 str_13 \N \N +14 14 \N \N \N \N +15 15 1970-01-16 \N \N 1970-01-16 +16 16 1970-01-17 \N \N 1970-01-17 +17 17 1970-01-18 \N \N 1970-01-18 +18 18 1970-01-19 \N \N 1970-01-19 +19 19 \N \N \N \N +20 20 20 \N 20 \N +21 21 str_21 str_21 \N \N +alter modify column 2 +0 0 0 \N \N \N \N \N +1 1 1 \N \N \N \N \N +2 2 2 \N \N \N \N \N +3 3 3 \N 3 \N 3 \N +4 4 4 \N 4 \N 4 \N +5 5 5 \N 5 \N 5 \N +6 6 6 \N str_6 str_6 \N \N +7 7 7 \N str_7 str_7 \N \N +8 8 8 \N str_8 str_8 \N \N +9 9 9 \N \N \N \N \N +10 10 10 \N \N \N \N \N +11 11 11 \N \N \N \N \N +12 12 12 \N 12 \N 12 \N +13 13 13 \N str_13 str_13 \N \N +14 14 14 \N \N \N \N \N +15 15 15 \N 1970-01-16 \N \N 1970-01-16 +16 16 16 \N 1970-01-17 \N \N 1970-01-17 +17 17 17 \N 1970-01-18 \N \N 1970-01-18 +18 18 18 \N 1970-01-19 \N \N 1970-01-19 +19 19 19 \N \N \N \N \N +20 20 20 \N 20 \N 20 \N +21 21 21 \N str_21 str_21 \N \N +insert after alter modify column 2 +0 0 0 \N \N \N \N \N +1 1 1 \N \N \N \N \N +2 2 2 \N \N \N \N \N +3 3 3 \N 3 \N 3 \N +4 4 4 \N 4 \N 4 \N +5 5 5 \N 5 \N 5 \N +6 6 6 \N str_6 str_6 \N \N +7 7 7 \N str_7 str_7 \N \N +8 8 8 \N str_8 str_8 \N \N +9 9 9 \N \N \N \N \N +10 10 10 \N \N \N \N \N +11 11 11 \N \N \N \N \N +12 12 12 \N 12 \N 12 \N +13 13 13 \N str_13 str_13 \N \N +14 14 14 \N \N \N \N \N +15 15 15 \N 1970-01-16 \N \N 1970-01-16 +16 16 16 \N 1970-01-17 \N \N 1970-01-17 +17 17 17 \N 1970-01-18 \N \N 1970-01-18 +18 18 18 \N 1970-01-19 \N \N 1970-01-19 +19 19 19 \N \N \N \N \N +20 20 20 \N 20 \N 20 \N +21 21 21 \N str_21 str_21 \N \N +22 str_22 \N str_22 \N \N \N \N +23 \N \N \N \N \N \N \N +24 24 24 \N \N \N \N \N +MergeTree wide +initial insert +alter add column 1 +0 0 \N \N \N +1 1 \N \N \N +2 2 \N \N \N +insert after alter add column 1 +0 0 \N \N \N +1 1 \N \N \N +2 2 \N \N \N +3 3 3 \N 3 +4 4 4 \N 4 +5 5 5 \N 5 +6 6 str_6 str_6 \N +7 7 str_7 str_7 \N +8 8 str_8 str_8 \N +9 9 \N \N \N +10 10 \N \N \N +11 11 \N \N \N +12 12 12 \N 12 +13 13 str_13 str_13 \N +14 14 \N \N \N +alter modify column 1 +0 0 \N \N \N \N +1 1 \N \N \N \N +2 2 \N \N \N \N +3 3 3 \N 3 \N +4 4 4 \N 4 \N +5 5 5 \N 5 \N +6 6 str_6 str_6 \N \N +7 7 str_7 str_7 \N \N +8 8 str_8 str_8 \N \N +9 9 \N \N \N \N +10 10 \N \N \N \N +11 11 \N \N \N \N +12 12 12 \N 12 \N +13 13 str_13 str_13 \N \N +14 14 \N \N \N \N +insert after alter modify column 1 +0 0 \N \N \N \N +1 1 \N \N \N \N +2 2 \N \N \N \N +3 3 3 \N 3 \N +4 4 4 \N 4 \N +5 5 5 \N 5 \N +6 6 str_6 str_6 \N \N +7 7 str_7 str_7 \N \N +8 8 str_8 str_8 \N \N +9 9 \N \N \N \N +10 10 \N \N \N \N +11 11 \N \N \N \N +12 12 12 \N 12 \N +13 13 str_13 str_13 \N \N +14 14 \N \N \N \N +15 15 1970-01-16 \N \N 1970-01-16 +16 16 1970-01-17 \N \N 1970-01-17 +17 17 1970-01-18 \N \N 1970-01-18 +18 18 1970-01-19 \N \N 1970-01-19 +19 19 \N \N \N \N +20 20 20 \N 20 \N +21 21 str_21 str_21 \N \N +alter modify column 2 +0 0 0 \N \N \N \N \N +1 1 1 \N \N \N \N \N +2 2 2 \N \N \N \N \N +3 3 3 \N 3 \N 3 \N +4 4 4 \N 4 \N 4 \N +5 5 5 \N 5 \N 5 \N +6 6 6 \N str_6 str_6 \N \N +7 7 7 \N str_7 str_7 \N \N +8 8 8 \N str_8 str_8 \N \N +9 9 9 \N \N \N \N \N +10 10 10 \N \N \N \N \N +11 11 11 \N \N \N \N \N +12 12 12 \N 12 \N 12 \N +13 13 13 \N str_13 str_13 \N \N +14 14 14 \N \N \N \N \N +15 15 15 \N 1970-01-16 \N \N 1970-01-16 +16 16 16 \N 1970-01-17 \N \N 1970-01-17 +17 17 17 \N 1970-01-18 \N \N 1970-01-18 +18 18 18 \N 1970-01-19 \N \N 1970-01-19 +19 19 19 \N \N \N \N \N +20 20 20 \N 20 \N 20 \N +21 21 21 \N str_21 str_21 \N \N +insert after alter modify column 2 +0 0 0 \N \N \N \N \N +1 1 1 \N \N \N \N \N +2 2 2 \N \N \N \N \N +3 3 3 \N 3 \N 3 \N +4 4 4 \N 4 \N 4 \N +5 5 5 \N 5 \N 5 \N +6 6 6 \N str_6 str_6 \N \N +7 7 7 \N str_7 str_7 \N \N +8 8 8 \N str_8 str_8 \N \N +9 9 9 \N \N \N \N \N +10 10 10 \N \N \N \N \N +11 11 11 \N \N \N \N \N +12 12 12 \N 12 \N 12 \N +13 13 13 \N str_13 str_13 \N \N +14 14 14 \N \N \N \N \N +15 15 15 \N 1970-01-16 \N \N 1970-01-16 +16 16 16 \N 1970-01-17 \N \N 1970-01-17 +17 17 17 \N 1970-01-18 \N \N 1970-01-18 +18 18 18 \N 1970-01-19 \N \N 1970-01-19 +19 19 19 \N \N \N \N \N +20 20 20 \N 20 \N 20 \N +21 21 21 \N str_21 str_21 \N \N +22 str_22 \N str_22 \N \N \N \N +23 \N \N \N \N \N \N \N +24 24 24 \N \N \N \N \N diff --git a/tests/queries/0_stateless/02941_variant_type_alters.sh b/tests/queries/0_stateless/02941_variant_type_alters.sh new file mode 100755 index 00000000000..9b0d4febd65 --- /dev/null +++ b/tests/queries/0_stateless/02941_variant_type_alters.sh @@ -0,0 +1,61 @@ +#!/usr/bin/env bash +# Tags: long + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# reset --log_comment +CLICKHOUSE_LOG_COMMENT= +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_when_no_common_type_in_if=1 " + +function run() +{ + echo "initial insert" + $CH_CLIENT -q "insert into test select number, number from numbers(3)" + + echo "alter add column 1" + $CH_CLIENT -q "alter table test add column v Variant(UInt64, String) settings mutations_sync=1" + $CH_CLIENT -q "select x, y, v, v.String, v.UInt64 from test order by x" + + echo "insert after alter add column 1" + $CH_CLIENT -q "insert into test select number, number, number from numbers(3, 3)" + $CH_CLIENT -q "insert into test select number, number, 'str_' || toString(number) from numbers(6, 3)" + $CH_CLIENT -q "insert into test select number, number, NULL from numbers(9, 3)" + $CH_CLIENT -q "insert into test select number, number, multiIf(number % 3 == 0, number, number % 3 == 1, 'str_' || toString(number), NULL) from numbers(12, 3)" + $CH_CLIENT -q "select x, y, v, v.String, v.UInt64 from test order by x" + + echo "alter modify column 1" + $CH_CLIENT -q "alter table test modify column v Variant(UInt64, String, Date) settings mutations_sync=1" + $CH_CLIENT -q "select x, y, v, v.String, v.UInt64, v.Date from test order by x" + + echo "insert after alter modify column 1" + $CH_CLIENT -q "insert into test select number, number, toDate(number) from numbers(15, 3)" + $CH_CLIENT -q "insert into test select number, number, multiIf(number % 4 == 0, number, number % 4 == 1, 'str_' || toString(number), number % 4 == 2, toDate(number), NULL) from numbers(18, 4)" + $CH_CLIENT -q "select x, y, v, v.String, v.UInt64, v.Date from test order by x" + + echo "alter modify column 2" + $CH_CLIENT -q "alter table test modify column y Variant(UInt64, String) settings mutations_sync=1" + $CH_CLIENT -q "select x, y, y.UInt64, y.String, v, v.String, v.UInt64, v.Date from test order by x" + + echo "insert after alter modify column 2" + $CH_CLIENT -q "insert into test select number, multiIf(number % 3 == 0, number, number % 3 == 1, 'str_' || toString(number), NULL), NULL from numbers(22, 3)" + $CH_CLIENT -q "select x, y, y.UInt64, y.String, v, v.String, v.UInt64, v.Date from test order by x" +} + +$CH_CLIENT -q "drop table if exists test;" + +echo "Memory" +$CH_CLIENT -q "create table test (x UInt64, y UInt64) engine=Memory" +run +$CH_CLIENT -q "drop table test;" + +echo "MergeTree compact" +$CH_CLIENT -q "create table test (x UInt64, y UInt64) engine=MergeTree order by x settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000;" +run +$CH_CLIENT -q "drop table test;" + +echo "MergeTree wide" +$CH_CLIENT -q "create table test (x UInt64, y UInt64 ) engine=MergeTree order by x settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" +run +$CH_CLIENT -q "drop table test;" From 4109b6608186b1b9d9dce60f1821313294b7e7c4 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 18 Jan 2024 17:30:32 +0000 Subject: [PATCH 065/264] Remove unneded tag from test --- tests/queries/0_stateless/02941_variant_type_alters.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02941_variant_type_alters.sh b/tests/queries/0_stateless/02941_variant_type_alters.sh index 9b0d4febd65..7e2ecbd67aa 100755 --- a/tests/queries/0_stateless/02941_variant_type_alters.sh +++ b/tests/queries/0_stateless/02941_variant_type_alters.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: long CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # reset --log_comment From bb1058e2b28401b8189c2d21db9476fca19286cb Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 18 Jan 2024 17:59:23 +0000 Subject: [PATCH 066/264] fix storage replacement with insertion block for analyzer --- src/Analyzer/Utils.cpp | 66 +++++++++++++++- src/Analyzer/Utils.h | 3 + .../InterpreterSelectQueryAnalyzer.cpp | 78 ++++++------------- 3 files changed, 93 insertions(+), 54 deletions(-) diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index 53fcf534f64..af40b7f766f 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -326,6 +326,69 @@ void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_q } } +QueryTreeNodes extractTrueTableExpressions(const QueryTreeNodePtr & tree) +{ + QueryTreeNodes result; + + std::deque nodes_to_process; + nodes_to_process.push_back(tree); + + while (!nodes_to_process.empty()) + { + auto node_to_process = std::move(nodes_to_process.front()); + nodes_to_process.pop_front(); + + auto node_type = node_to_process->getNodeType(); + + switch (node_type) + { + case QueryTreeNodeType::TABLE: + { + result.push_back(std::move(node_to_process)); + break; + } + case QueryTreeNodeType::QUERY: + { + nodes_to_process.push_front(node_to_process->as()->getJoinTree()); + break; + } + case QueryTreeNodeType::UNION: + { + for (auto union_node : node_to_process->as()->getQueries().getNodes()) + nodes_to_process.push_front(union_node); + break; + } + case QueryTreeNodeType::TABLE_FUNCTION: + { + for (auto argument_node : node_to_process->as()->getArgumentsNode()->getChildren()) + nodes_to_process.push_front(argument_node); + break; + } + case QueryTreeNodeType::ARRAY_JOIN: + { + nodes_to_process.push_front(node_to_process->as()->getTableExpression()); + break; + } + case QueryTreeNodeType::JOIN: + { + auto & join_node = node_to_process->as(); + nodes_to_process.push_front(join_node.getRightTableExpression()); + nodes_to_process.push_front(join_node.getLeftTableExpression()); + break; + } + default: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Unexpected node type for table expression. " + "Expected table, table function, query, union, join or array join. Actual {}", + node_to_process->getNodeTypeName()); + } + } + } + + return result; +} + QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node, bool add_array_join) { QueryTreeNodes result; @@ -383,6 +446,7 @@ QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node, QueryTreeNodePtr extractLeftTableExpression(const QueryTreeNodePtr & join_tree_node) { +std::cout << "\033[1;31m" << "+++ JOO extractLeftTableExpression 0" << "\033[0m" << std::endl; QueryTreeNodePtr result; std::deque nodes_to_process; @@ -394,7 +458,7 @@ QueryTreeNodePtr extractLeftTableExpression(const QueryTreeNodePtr & join_tree_n nodes_to_process.pop_front(); auto node_type = node_to_process->getNodeType(); - +std::cout << "\033[1;31m" << "+++ JOO extractLeftTableExpression " << static_cast(node_type) << "\033[0m" << std::endl; switch (node_type) { case QueryTreeNodeType::TABLE: diff --git a/src/Analyzer/Utils.h b/src/Analyzer/Utils.h index d3eb6ba3cc2..04a5e460974 100644 --- a/src/Analyzer/Utils.h +++ b/src/Analyzer/Utils.h @@ -50,6 +50,9 @@ std::optional tryExtractConstantFromConditionNode(const QueryTreeNodePtr & */ void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_query_ast, const QueryTreeNodePtr & table_expression, const IQueryTreeNode::ConvertToASTOptions & convert_to_ast_options); +/// Extract table expressions from tree +QueryTreeNodes extractTrueTableExpressions(const QueryTreeNodePtr & tree); + /// Extract table, table function, query, union from join tree QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node, bool add_array_join = false); diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index 868ef170f7c..a44ff1a79df 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -1,3 +1,5 @@ +#include "Analyzer/IQueryTreeNode.h" +#include "Analyzer/JoinNode.h" #include #include @@ -74,60 +76,28 @@ ContextMutablePtr buildContext(const ContextPtr & context, const SelectQueryOpti void replaceStorageInQueryTree(QueryTreeNodePtr & query_tree, const ContextPtr & context, const StoragePtr & storage) { - auto query_to_replace_table_expression = query_tree; - QueryTreeNodePtr table_expression_to_replace; - - while (!table_expression_to_replace) + auto nodes = extractTrueTableExpressions(query_tree); + IQueryTreeNode::ReplacementMap replacement_map; + + for (auto & node : nodes) { - if (auto * union_node = query_to_replace_table_expression->as()) - query_to_replace_table_expression = union_node->getQueries().getNodes().at(0); + auto & table_node = node->as(); - auto & query_to_replace_table_expression_typed = query_to_replace_table_expression->as(); - auto left_table_expression = extractLeftTableExpression(query_to_replace_table_expression_typed.getJoinTree()); - auto left_table_expression_node_type = left_table_expression->getNodeType(); + /// Don't replace storage if table name differs + if (table_node.getStorageID().getFullNameNotQuoted() != storage->getStorageID().getFullNameNotQuoted()) + continue; - switch (left_table_expression_node_type) - { - case QueryTreeNodeType::QUERY: - case QueryTreeNodeType::UNION: - { - query_to_replace_table_expression = std::move(left_table_expression); - break; - } - case QueryTreeNodeType::TABLE: - case QueryTreeNodeType::TABLE_FUNCTION: - case QueryTreeNodeType::IDENTIFIER: - { - table_expression_to_replace = std::move(left_table_expression); - break; - } - default: - { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Expected table, table function or identifier node to replace with storage. Actual {}", - left_table_expression->formatASTForErrorMessage()); - } - } + auto replacement_table_expression = std::make_shared(storage, context); + + if (auto table_expression_modifiers = table_node.getTableExpressionModifiers()) + replacement_table_expression->setTableExpressionModifiers(*table_expression_modifiers); + + if (node->hasAlias()) + replacement_table_expression->setAlias(node->getAlias() + "_replacement"); + + replacement_map.emplace(node.get(), std::move(replacement_table_expression)); } - - /// Don't replace storage if table name differs - if (auto * table_node = table_expression_to_replace->as(); table_node && table_node->getStorageID().getFullNameNotQuoted() != storage->getStorageID().getFullNameNotQuoted()) - return; - - auto replacement_table_expression = std::make_shared(storage, context); - std::optional table_expression_modifiers; - - if (auto * table_node = table_expression_to_replace->as()) - table_expression_modifiers = table_node->getTableExpressionModifiers(); - else if (auto * table_function_node = table_expression_to_replace->as()) - table_expression_modifiers = table_function_node->getTableExpressionModifiers(); - else if (auto * identifier_node = table_expression_to_replace->as()) - table_expression_modifiers = identifier_node->getTableExpressionModifiers(); - - if (table_expression_modifiers) - replacement_table_expression->setTableExpressionModifiers(*table_expression_modifiers); - - query_tree = query_tree->cloneAndReplace(table_expression_to_replace, std::move(replacement_table_expression)); + query_tree = query_tree->cloneAndReplace(replacement_map); } QueryTreeNodePtr buildQueryTreeAndRunPasses(const ASTPtr & query, @@ -147,10 +117,12 @@ QueryTreeNodePtr buildQueryTreeAndRunPasses(const ASTPtr & query, query_tree_pass_manager.runOnlyResolve(query_tree); else query_tree_pass_manager.run(query_tree); - - if (storage) +std::cout << "\033[1;31m" << "+++ JOO query_tree\n" << query_tree->dumpTree() << "\033[0m" << std::endl; + if (storage) { replaceStorageInQueryTree(query_tree, context, storage); - +// replaceStorageInQueryTree(query_tree, context, storage, false); + } +std::cout << "\033[1;31m" << "+++ JOO query_tree NEW\n" << query_tree->dumpTree() << "\033[0m" << std::endl; return query_tree; } From b914938c4ca27a3eb41fe6f4b5d0c6de2aff3f2a Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 18 Jan 2024 18:10:54 +0000 Subject: [PATCH 067/264] clenup --- src/Analyzer/Utils.cpp | 3 +-- src/Interpreters/InterpreterSelectQueryAnalyzer.cpp | 10 ++++------ 2 files changed, 5 insertions(+), 8 deletions(-) diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index af40b7f766f..26067141b1d 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -446,7 +446,6 @@ QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node, QueryTreeNodePtr extractLeftTableExpression(const QueryTreeNodePtr & join_tree_node) { -std::cout << "\033[1;31m" << "+++ JOO extractLeftTableExpression 0" << "\033[0m" << std::endl; QueryTreeNodePtr result; std::deque nodes_to_process; @@ -458,7 +457,7 @@ std::cout << "\033[1;31m" << "+++ JOO extractLeftTableExpression 0" << "\033[0m" nodes_to_process.pop_front(); auto node_type = node_to_process->getNodeType(); -std::cout << "\033[1;31m" << "+++ JOO extractLeftTableExpression " << static_cast(node_type) << "\033[0m" << std::endl; + switch (node_type) { case QueryTreeNodeType::TABLE: diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index a44ff1a79df..0aeafb8a979 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -78,7 +78,7 @@ void replaceStorageInQueryTree(QueryTreeNodePtr & query_tree, const ContextPtr & { auto nodes = extractTrueTableExpressions(query_tree); IQueryTreeNode::ReplacementMap replacement_map; - + for (auto & node : nodes) { auto & table_node = node->as(); @@ -117,12 +117,10 @@ QueryTreeNodePtr buildQueryTreeAndRunPasses(const ASTPtr & query, query_tree_pass_manager.runOnlyResolve(query_tree); else query_tree_pass_manager.run(query_tree); -std::cout << "\033[1;31m" << "+++ JOO query_tree\n" << query_tree->dumpTree() << "\033[0m" << std::endl; - if (storage) { + + if (storage) replaceStorageInQueryTree(query_tree, context, storage); -// replaceStorageInQueryTree(query_tree, context, storage, false); - } -std::cout << "\033[1;31m" << "+++ JOO query_tree NEW\n" << query_tree->dumpTree() << "\033[0m" << std::endl; + return query_tree; } From 1144f6051519afb06d6728ce2adf694aea47c13c Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 18 Jan 2024 18:17:07 +0000 Subject: [PATCH 068/264] clenup --- src/Interpreters/InterpreterSelectQueryAnalyzer.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index 0aeafb8a979..8e6d48102f3 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -1,5 +1,4 @@ -#include "Analyzer/IQueryTreeNode.h" -#include "Analyzer/JoinNode.h" +#include #include #include From 43aaccdaa165ef6c56937247c7a1bcba337f8d52 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 18 Jan 2024 19:17:05 +0000 Subject: [PATCH 069/264] clang tidy --- src/Analyzer/Utils.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index 26067141b1d..9c3a7c2fb55 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -354,13 +354,13 @@ QueryTreeNodes extractTrueTableExpressions(const QueryTreeNodePtr & tree) } case QueryTreeNodeType::UNION: { - for (auto union_node : node_to_process->as()->getQueries().getNodes()) + for (const auto & union_node : node_to_process->as()->getQueries().getNodes()) nodes_to_process.push_front(union_node); break; } case QueryTreeNodeType::TABLE_FUNCTION: { - for (auto argument_node : node_to_process->as()->getArgumentsNode()->getChildren()) + for (const auto & argument_node : node_to_process->as()->getArgumentsNode()->getChildren()) nodes_to_process.push_front(argument_node); break; } From 3dcc2056a59f9b374b4de3b72c30107dd7825d47 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 18 Jan 2024 20:39:36 +0000 Subject: [PATCH 070/264] Fix conflicts --- src/DataTypes/Serializations/ISerialization.cpp | 1 + src/DataTypes/Serializations/ISerialization.h | 1 + src/DataTypes/Serializations/SerializationVariant.cpp | 2 +- 3 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index 2f1eb1887af..7d57d72090b 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -54,6 +54,7 @@ const std::set ISerialization::Substream::named_types TupleElement, NamedOffsets, NamedNullMap, + NamedVariantDiscriminators, }; String ISerialization::Substream::toString() const diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index 64a7a889640..7fba9db4acf 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -155,6 +155,7 @@ public: ObjectData, VariantDiscriminators, + NamedVariantDiscriminators, VariantOffsets, VariantElements, VariantElement, diff --git a/src/DataTypes/Serializations/SerializationVariant.cpp b/src/DataTypes/Serializations/SerializationVariant.cpp index 48a78dd54a9..5af94364167 100644 --- a/src/DataTypes/Serializations/SerializationVariant.cpp +++ b/src/DataTypes/Serializations/SerializationVariant.cpp @@ -36,7 +36,7 @@ void SerializationVariant::enumerateStreams( const auto * type_variant = data.type ? &assert_cast(*data.type) : nullptr; const auto * column_variant = data.column ? &assert_cast(*data.column) : nullptr; - auto discriminators_serialization = std::make_shared(std::make_shared>(), "discr", false); + auto discriminators_serialization = std::make_shared(std::make_shared>(), "discr", SubstreamType::NamedVariantDiscriminators); auto local_discriminators = column_variant ? column_variant->getLocalDiscriminatorsPtr() : nullptr; settings.path.push_back(Substream::VariantDiscriminators); From 1bfeee1954b2578374e34f5d2f877c96bae3c1a8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 18 Jan 2024 19:52:13 +0000 Subject: [PATCH 071/264] Some fixups --- .../mergetree-family/mergetree.md | 42 +++++----- .../sql-reference/statements/alter/column.md | 6 +- src/Parsers/ParserAlterQuery.cpp | 4 +- src/Parsers/ParserCreateQuery.h | 12 +-- src/Storages/AlterCommands.h | 2 +- src/Storages/MergeTree/MergeTreeSettings.cpp | 1 + src/Storages/MergeTree/MergeTreeSettings.h | 3 +- .../02870_per_column_compress_block.reference | 16 ---- .../02870_per_column_compress_block.sql | 75 ------------------ .../02870_per_column_settings.reference | 19 +++++ .../0_stateless/02870_per_column_settings.sql | 78 +++++++++++++++++++ 11 files changed, 133 insertions(+), 125 deletions(-) delete mode 100644 tests/queries/0_stateless/02870_per_column_compress_block.reference delete mode 100644 tests/queries/0_stateless/02870_per_column_compress_block.sql create mode 100644 tests/queries/0_stateless/02870_per_column_settings.reference create mode 100644 tests/queries/0_stateless/02870_per_column_settings.sql diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 658f57ceb9a..f185c11bab3 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -56,7 +56,7 @@ ORDER BY expr [DELETE|TO DISK 'xxx'|TO VOLUME 'xxx' [, ...] ] [WHERE conditions] [GROUP BY key_expr [SET v1 = aggr_func(v1) [, v2 = aggr_func(v2) ...]] ] ] -[SETTINGS name=value, ...] +[SETTINGS name = value, ...] ``` For a description of parameters, see the [CREATE query description](/docs/en/sql-reference/statements/create/table.md). @@ -620,7 +620,7 @@ The `TTL` clause can’t be used for key columns. #### Creating a table with `TTL`: ``` sql -CREATE TABLE example_table +CREATE TABLE tab ( d DateTime, a Int TTL d + INTERVAL 1 MONTH, @@ -635,7 +635,7 @@ ORDER BY d; #### Adding TTL to a column of an existing table ``` sql -ALTER TABLE example_table +ALTER TABLE tab MODIFY COLUMN c String TTL d + INTERVAL 1 DAY; ``` @@ -643,7 +643,7 @@ ALTER TABLE example_table #### Altering TTL of the column ``` sql -ALTER TABLE example_table +ALTER TABLE tab MODIFY COLUMN c String TTL d + INTERVAL 1 MONTH; ``` @@ -681,7 +681,7 @@ If a column is not part of the `GROUP BY` expression and is not set explicitly i #### Creating a table with `TTL`: ``` sql -CREATE TABLE example_table +CREATE TABLE tab ( d DateTime, a Int @@ -697,7 +697,7 @@ TTL d + INTERVAL 1 MONTH DELETE, #### Altering `TTL` of the table: ``` sql -ALTER TABLE example_table +ALTER TABLE tab MODIFY TTL d + INTERVAL 1 DAY; ``` @@ -1366,7 +1366,7 @@ In this sample configuration: The statistic declaration is in the columns section of the `CREATE` query for tables from the `*MergeTree*` Family when we enable `set allow_experimental_statistic = 1`. ``` sql -CREATE TABLE example_table +CREATE TABLE tab ( a Int64 STATISTIC(tdigest), b Float64 @@ -1378,8 +1378,8 @@ ORDER BY a We can also manipulate statistics with `ALTER` statements. ```sql -ALTER TABLE example_table ADD STATISTIC b TYPE tdigest; -ALTER TABLE example_table DROP STATISTIC a TYPE tdigest; +ALTER TABLE tab ADD STATISTIC b TYPE tdigest; +ALTER TABLE tab DROP STATISTIC a TYPE tdigest; ``` These lightweight statistics aggregate information about distribution of values in columns. @@ -1391,41 +1391,41 @@ They can be used for query optimization when we enable `set allow_statistic_opti Stores distribution of values from numeric columns in [TDigest](https://github.com/tdunning/t-digest) sketch. -## Column Settings {#column-settings} +## Column-level Settings {#column-level-settings} -Some table parameters can be override at column level by column settings. +Certain MergeTree settings can be override at column level: - `max_compress_block_size` — Maximum size of blocks of uncompressed data before compressing for writing to a table. - `min_compress_block_size` — Minimum size of blocks of uncompressed data required for compression when writing the next mark. -Example of creating table with column settings: +Example: ```sql -CREATE TABLE example_table +CREATE TABLE tab ( id Int64, - document String CODEC(ZSTD(9,24)) SETTINGS (min_compress_block_size = 16777216, max_compress_block_size = 16777216) + document String SETTINGS (min_compress_block_size = 16777216, max_compress_block_size = 16777216) ) ENGINE = MergeTree ORDER BY id ``` -Column settings can be modified or removed via [ALTER MODIFY COLUMN](/docs/en/sql-reference/statements/alter/column.md) query, for example: +Column-level settings can be modified or removed using [ALTER MODIFY COLUMN](/docs/en/sql-reference/statements/alter/column.md), for example: -Remove `SETTINGS` from column declaration: +- Remove `SETTINGS` from column declaration: ```sql -ALTER TABLE example_table MODIFY COLUMN document REMOVE SETTINGS; +ALTER TABLE tab MODIFY COLUMN document REMOVE SETTINGS; ``` -Modify a setting: +- Modify a setting: ```sql -ALTER TABLE example_table MODIFY COLUMN document MODIFY SETTING min_compress_block_size = 8192; +ALTER TABLE tab MODIFY COLUMN document MODIFY SETTING min_compress_block_size = 8192; ``` -Reset one or more settings, after reset the setting expression is also removed from column declaration: +- Reset one or more settings, also removes the setting declaration in the column expression of the table's CREATE query. ```sql -ALTER TABLE example_table MODIFY COLUMN document RESET SETTING min_compress_block_size; +ALTER TABLE tab MODIFY COLUMN document RESET SETTING min_compress_block_size; ``` diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index 87cf8ca9914..676d30f5e44 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -23,7 +23,7 @@ The following actions are supported: - [RENAME COLUMN](#rename-column) — Renames an existing column. - [CLEAR COLUMN](#clear-column) — Resets column values. - [COMMENT COLUMN](#comment-column) — Adds a text comment to the column. -- [MODIFY COLUMN](#modify-column) — Changes column’s type, default expression, TTL, and settings. +- [MODIFY COLUMN](#modify-column) — Changes column’s type, default expression, TTL, and column settings. - [MODIFY COLUMN REMOVE](#modify-column-remove) — Removes one of the column properties. - [MODIFY COLUMN MODIFY SETTING](#modify-column-modify-setting) - Changes column settings. - [MODIFY COLUMN RESET SETTING](#modify-column-reset-setting) - Reset column settings. @@ -232,7 +232,7 @@ ALTER TABLE table_with_ttl MODIFY COLUMN column_ttl REMOVE TTL; ## MODIFY COLUMN MODIFY SETTING -Modify a column level setting. +Modify a column setting. Syntax: @@ -250,7 +250,7 @@ ALTER TABLE table_name MODIFY COLUMN MODIFY SETTING max_compress_block_size = 10 ## MODIFY COLUMN RESET SETTING -Reset a column setting, also remove the setting declaration in column expression in table create query. +Reset a column setting, also removes the setting declaration in the column expression of the table's CREATE query. Syntax: diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index 7eb1ed2eff5..d72fb493368 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -733,12 +733,12 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected } else if (s_modify_setting.ignore(pos, expected)) { - if (!parser_settings.parse(pos, command->settings_changes, expected)) + if (!parser_settings.parse(pos, command_settings_changes, expected)) return false; } else if (s_reset_setting.ignore(pos, expected)) { - if (!parser_reset_setting.parse(pos, command->settings_resets, expected)) + if (!parser_reset_setting.parse(pos, command_settings_resets, expected)) return false; } else diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index 80cf541cbdc..06973573023 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -121,8 +121,6 @@ using ParserCompoundColumnDeclaration = IParserColumnDeclaration bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - NameParser name_parser; - ParserDataType type_parser; ParserKeyword s_default{"DEFAULT"}; ParserKeyword s_null{"NULL"}; ParserKeyword s_not{"NOT"}; @@ -137,10 +135,13 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E ParserKeyword s_remove{"REMOVE"}; ParserKeyword s_modify_setting("MODIFY SETTING"); ParserKeyword s_reset_setting("RESET SETTING"); + ParserKeyword s_settings("SETTINGS"); ParserKeyword s_type{"TYPE"}; ParserKeyword s_collate{"COLLATE"}; ParserKeyword s_primary_key{"PRIMARY KEY"}; - ParserKeyword s_settings("SETTINGS"); + + NameParser name_parser; + ParserDataType type_parser; ParserExpression expr_parser; ParserStringLiteral string_literal_parser; ParserLiteral literal_parser; @@ -160,13 +161,12 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E /// This keyword may occur only in MODIFY COLUMN query. We check it here /// because ParserDataType parses types as an arbitrary identifiers and - /// doesn't check that parsed string is existing data type. In this way + /// doesn't check that parsed string is existing data type. In this way, /// REMOVE, MODIFY SETTING, or RESET SETTING can be parsed as data type /// and further parsing will fail. So we just check these keyword and in /// case of success return column declaration with name only. if (!require_type - && (s_remove.checkWithoutMoving(pos, expected) || s_modify_setting.checkWithoutMoving(pos, expected) - || s_reset_setting.checkWithoutMoving(pos, expected))) + && (s_remove.checkWithoutMoving(pos, expected) || s_modify_setting.checkWithoutMoving(pos, expected) || s_reset_setting.checkWithoutMoving(pos, expected))) { if (!check_keywords_after_name) return false; diff --git a/src/Storages/AlterCommands.h b/src/Storages/AlterCommands.h index f7ab1385f1a..d0d5d02b5f7 100644 --- a/src/Storages/AlterCommands.h +++ b/src/Storages/AlterCommands.h @@ -156,7 +156,7 @@ struct AlterCommand /// What to remove from column (or TTL) RemoveProperty to_remove = RemoveProperty::NO_PROPERTY; - /// Is this MODIFY COLUMN MODIFY SETTING or MODIFY COLUMN parse(const ASTAlterCommand * command); diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index b7e366855db..cefee36c124 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -221,6 +221,7 @@ void MergeTreeColumnSettings::validate(const SettingsChanges & changes) "min_compress_block_size", "max_compress_block_size" }; + for (const auto & change : changes) { if (!allowed_column_level_settings.contains(change.name)) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 11aaec3ac34..b8ab682de08 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -275,9 +275,10 @@ struct MergeTreeSettings : public BaseSettings, public std::vector getAllRegisteredNames() const override; }; - using MergeTreeSettingsPtr = std::shared_ptr; + +/// Column-level Merge-Tree settings which overwrite MergeTree settings namespace MergeTreeColumnSettings { void validate(const SettingsChanges & changes); diff --git a/tests/queries/0_stateless/02870_per_column_compress_block.reference b/tests/queries/0_stateless/02870_per_column_compress_block.reference deleted file mode 100644 index c6b396a11c2..00000000000 --- a/tests/queries/0_stateless/02870_per_column_compress_block.reference +++ /dev/null @@ -1,16 +0,0 @@ -CREATE TABLE default.t\n(\n `id` UInt64 CODEC(ZSTD(1)),\n `long_string` String CODEC(ZSTD(9, 24)) SETTINGS (min_compress_block_size = 163840, max_compress_block_size = 163840),\n `v1` String CODEC(ZSTD(1)),\n `v2` UInt64 CODEC(ZSTD(1)),\n `v3` Float32 CODEC(ZSTD(1)),\n `v4` Float64 CODEC(ZSTD(1))\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/t/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192 -1000 -CREATE TABLE default.t\n(\n `id` UInt64 CODEC(ZSTD(1)),\n `long_string` String CODEC(ZSTD(9, 24)) SETTINGS (min_compress_block_size = 8192, max_compress_block_size = 163840),\n `v1` String CODEC(ZSTD(1)),\n `v2` UInt64 CODEC(ZSTD(1)),\n `v3` Float32 CODEC(ZSTD(1)),\n `v4` Float64 CODEC(ZSTD(1))\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/t/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192 -CREATE TABLE default.t\n(\n `id` UInt64 CODEC(ZSTD(1)),\n `long_string` String CODEC(ZSTD(9, 24)) SETTINGS (max_compress_block_size = 163840),\n `v1` String CODEC(ZSTD(1)),\n `v2` UInt64 CODEC(ZSTD(1)),\n `v3` Float32 CODEC(ZSTD(1)),\n `v4` Float64 CODEC(ZSTD(1))\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/t/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192 -CREATE TABLE default.t\n(\n `id` UInt64 CODEC(ZSTD(1)),\n `long_string` String CODEC(ZSTD(9, 24)),\n `v1` String CODEC(ZSTD(1)),\n `v2` UInt64 CODEC(ZSTD(1)),\n `v3` Float32 CODEC(ZSTD(1)),\n `v4` Float64 CODEC(ZSTD(1))\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/t/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192 -CREATE TABLE default.t\n(\n `id` UInt64 CODEC(ZSTD(1)),\n `long_string` String CODEC(ZSTD(9, 24)) SETTINGS (min_compress_block_size = 163840, max_compress_block_size = 163840),\n `v1` String CODEC(ZSTD(1)),\n `v2` UInt64 CODEC(ZSTD(1)),\n `v3` Float32 CODEC(ZSTD(1)),\n `v4` Float64 CODEC(ZSTD(1))\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/t/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192 -(0,0) 0 -(1,1) 1 -(2,2) 2 -(3,3) 3 -(4,4) 4 -(5,5) 5 -(6,6) 6 -(7,7) 7 -(8,8) 8 -(9,9) 9 diff --git a/tests/queries/0_stateless/02870_per_column_compress_block.sql b/tests/queries/0_stateless/02870_per_column_compress_block.sql deleted file mode 100644 index 079ece80fa0..00000000000 --- a/tests/queries/0_stateless/02870_per_column_compress_block.sql +++ /dev/null @@ -1,75 +0,0 @@ --- Tags: no-random-merge-tree-settings - -CREATE TABLE t -( - `id` UInt64 CODEC(ZSTD(1)), - `long_string` String CODEC(ZSTD(9, 24)) SETTINGS (min_compress_block_size = 163840, max_compress_block_size = 163840), - `v1` String CODEC(ZSTD(1)), - `v2` UInt64 CODEC(ZSTD(1)), - `v3` Float32 CODEC(ZSTD(1)), - `v4` Float64 CODEC(ZSTD(1)) -) -ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/t/2870', 'r1') -ORDER BY id -SETTINGS min_bytes_for_wide_part = 1; - -SHOW CREATE t; - -INSERT INTO TABLE t SELECT number, randomPrintableASCII(1000), randomPrintableASCII(10), rand(number), rand(number+1), rand(number+2) FROM numbers(1000); - -SELECT count() FROM t; - -ALTER TABLE t MODIFY COLUMN long_string MODIFY SETTING min_compress_block_size = 8192; - -SHOW CREATE t; - -ALTER TABLE t MODIFY COLUMN long_string RESET SETTING min_compress_block_size; - -SHOW CREATE t; - -ALTER TABLE t MODIFY COLUMN long_string REMOVE SETTINGS; - -SHOW CREATE t; - -ALTER TABLE t MODIFY COLUMN long_string String CODEC(ZSTD(9, 24)) SETTINGS (min_compress_block_size = 163840, max_compress_block_size = 163840); - -SHOW CREATE t; - -DROP TABLE t; - -SET allow_experimental_object_type = 1; - -CREATE TABLE t2 -( - `id` UInt64 CODEC(ZSTD(1)), - `tup` Tuple(UInt64, UInt64) CODEC(ZSTD(1)) SETTINGS (min_compress_block_size = 81920, max_compress_block_size = 163840), - `json` JSON CODEC(ZSTD(9, 24)) SETTINGS (min_compress_block_size = 81920, max_compress_block_size = 163840), -) -ENGINE = MergeTree -ORDER BY id -SETTINGS min_bytes_for_wide_part = 1; - -INSERT INTO TABLE t2 SELECT number, tuple(number, number), concat('{"key": ', toString(number), ' ,"value": ', toString(rand(number+1)), '}') FROM numbers(1000); -SELECT tup, json.key AS key FROM t2 ORDER BY key LIMIT 10; - -DROP TABLE t2; - --- Non-supported column setting -CREATE TABLE t3 -( - `id` UInt64 CODEC(ZSTD(1)), - `long_string` String CODEC(ZSTD(1)) SETTINGS (min_block_size = 81920, max_compress_block_size = 163840), -) -ENGINE = MergeTree -ORDER BY id -SETTINGS min_bytes_for_wide_part = 1; -- {serverError 115} - --- Invalid setting values -CREATE TABLE t4 -( - `id` UInt64 CODEC(ZSTD(1)), - `long_string` String CODEC(ZSTD(1)) SETTINGS (min_compress_block_size = 81920, max_compress_block_size = 163840), -) -ENGINE = TinyLog -ORDER BY id; -- {serverError 44} - diff --git a/tests/queries/0_stateless/02870_per_column_settings.reference b/tests/queries/0_stateless/02870_per_column_settings.reference new file mode 100644 index 00000000000..a57407e4f0e --- /dev/null +++ b/tests/queries/0_stateless/02870_per_column_settings.reference @@ -0,0 +1,19 @@ +CREATE TABLE default.tab\n(\n `id` UInt64,\n `long_string` String SETTINGS (min_compress_block_size = 163840, max_compress_block_size = 163840),\n `v1` String,\n `v2` UInt64,\n `v3` Float32,\n `v4` Float64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/tab/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192 +1000 +CREATE TABLE default.tab\n(\n `id` UInt64,\n `long_string` String SETTINGS (min_compress_block_size = 8192, max_compress_block_size = 163840),\n `v1` String,\n `v2` UInt64,\n `v3` Float32,\n `v4` Float64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/tab/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192 +CREATE TABLE default.tab\n(\n `id` UInt64,\n `long_string` String SETTINGS (max_compress_block_size = 163840),\n `v1` String,\n `v2` UInt64,\n `v3` Float32,\n `v4` Float64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/tab/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192 +CREATE TABLE default.tab\n(\n `id` UInt64,\n `long_string` String,\n `v1` String,\n `v2` UInt64,\n `v3` Float32,\n `v4` Float64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/tab/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192 +CREATE TABLE default.tab\n(\n `id` UInt64,\n `long_string` String SETTINGS (min_compress_block_size = 163840, max_compress_block_size = 163840),\n `v1` String,\n `v2` UInt64,\n `v3` Float32,\n `v4` Float64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/tab/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192 +--- +(0,0) 0 +(1,1) 1 +(2,2) 2 +(3,3) 3 +(4,4) 4 +(5,5) 5 +(6,6) 6 +(7,7) 7 +(8,8) 8 +(9,9) 9 +--- +--- diff --git a/tests/queries/0_stateless/02870_per_column_settings.sql b/tests/queries/0_stateless/02870_per_column_settings.sql new file mode 100644 index 00000000000..bd115885330 --- /dev/null +++ b/tests/queries/0_stateless/02870_per_column_settings.sql @@ -0,0 +1,78 @@ +-- Tags: no-random-merge-tree-settings + +-- Tests column-level settings for MergeTree* tables + +DROP TABLE IF EXISTS tab; + +CREATE TABLE tab +( + id UInt64, + long_string String SETTINGS (min_compress_block_size = 163840, max_compress_block_size = 163840), + v1 String, + v2 UInt64, + v3 Float32, + v4 Float64 +) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/tab/2870', 'r1') +ORDER BY id +SETTINGS min_bytes_for_wide_part = 1; + +SHOW CREATE tab; + +INSERT INTO TABLE tab SELECT number, randomPrintableASCII(1000), randomPrintableASCII(10), rand(number), rand(number+1), rand(number+2) FROM numbers(1000); +SELECT count() FROM tab; + +ALTER TABLE tab MODIFY COLUMN long_string MODIFY SETTING min_compress_block_size = 8192; +SHOW CREATE tab; + +ALTER TABLE tab MODIFY COLUMN long_string RESET SETTING min_compress_block_size; +SHOW CREATE tab; + +ALTER TABLE tab MODIFY COLUMN long_string REMOVE SETTINGS; +SHOW CREATE tab; + +ALTER TABLE tab MODIFY COLUMN long_string String SETTINGS (min_compress_block_size = 163840, max_compress_block_size = 163840); +SHOW CREATE tab; + +DROP TABLE tab; + +SELECT '--- '; + +SET allow_experimental_object_type = 1; + +CREATE TABLE tab +( + id UInt64, + tup Tuple(UInt64, UInt64) SETTINGS (min_compress_block_size = 81920, max_compress_block_size = 163840), + json JSON SETTINGS (min_compress_block_size = 81920, max_compress_block_size = 163840), +) +ENGINE = MergeTree +ORDER BY id +SETTINGS min_bytes_for_wide_part = 1; + +INSERT INTO TABLE tab SELECT number, tuple(number, number), concat('{"key": ', toString(number), ' ,"value": ', toString(rand(number+1)), '}') FROM numbers(1000); +SELECT tup, json.key AS key FROM tab ORDER BY key LIMIT 10; + +DROP TABLE tab; + +SELECT '--- '; + +-- Unsupported column-level settings are rejected +CREATE TABLE tab +( + id UInt64, + long_string String SETTINGS (min_block_size = 81920, max_compress_block_size = 163840), +) +ENGINE = MergeTree +ORDER BY id +SETTINGS min_bytes_for_wide_part = 1; -- {serverError UNKNOWN_SETTING} + +SELECT '--- '; + +-- Column-level settings are only supported for MergeTree* tables +CREATE TABLE tab +( + id UInt64 CODEC(ZSTD), + long_string String CODEC(ZSTD) SETTINGS (min_compress_block_size = 81920, max_compress_block_size = 163840), +) +ENGINE = TinyLog; -- {serverError ILLEGAL_COLUMN} From cfc8c60aa70917e48281e3583adc922967326d50 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 18 Jan 2024 21:26:55 +0000 Subject: [PATCH 072/264] Fix build --- src/DataTypes/Serializations/SerializationVariantElement.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationVariantElement.cpp b/src/DataTypes/Serializations/SerializationVariantElement.cpp index 80524cbd814..8d0acee1c2b 100644 --- a/src/DataTypes/Serializations/SerializationVariantElement.cpp +++ b/src/DataTypes/Serializations/SerializationVariantElement.cpp @@ -208,8 +208,8 @@ void SerializationVariantElement::removeVariantFromPath(DB::ISerialization::Subs } SerializationVariantElement::VariantSubcolumnCreator::VariantSubcolumnCreator( - const DB::ColumnPtr & local_discriminators_, - const DB::String & variant_element_name_, + const ColumnPtr & local_discriminators_, + const String & variant_element_name_, const ColumnVariant::Discriminator global_variant_discriminator_, const ColumnVariant::Discriminator local_variant_discriminator_) : local_discriminators(local_discriminators_) From 7ae631de1ed1ff4bcb8bac5e06c2026db3ff972c Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 19 Jan 2024 10:23:40 +0000 Subject: [PATCH 073/264] Remove wron check, remove duplicate tests --- .../SerializationVariantElement.cpp | 4 -- ...> 02943_variant_read_subcolumns.reference} | 0 ..._1.sh => 02943_variant_read_subcolumns.sh} | 0 .../02943_variant_read_subcolumns_2.reference | 6 --- .../02943_variant_read_subcolumns_2.sh | 38 ------------------- 5 files changed, 48 deletions(-) rename tests/queries/0_stateless/{02943_variant_read_subcolumns_1.reference => 02943_variant_read_subcolumns.reference} (100%) rename tests/queries/0_stateless/{02943_variant_read_subcolumns_1.sh => 02943_variant_read_subcolumns.sh} (100%) delete mode 100644 tests/queries/0_stateless/02943_variant_read_subcolumns_2.reference delete mode 100755 tests/queries/0_stateless/02943_variant_read_subcolumns_2.sh diff --git a/src/DataTypes/Serializations/SerializationVariantElement.cpp b/src/DataTypes/Serializations/SerializationVariantElement.cpp index 8d0acee1c2b..56f0e5d77be 100644 --- a/src/DataTypes/Serializations/SerializationVariantElement.cpp +++ b/src/DataTypes/Serializations/SerializationVariantElement.cpp @@ -156,7 +156,6 @@ void SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams( return; } - size_t prev_variant_size = variant_element_state->variant->size(); addVariantToPath(settings.path); nested_serialization->deserializeBinaryBulkWithMultipleStreams(variant_element_state->variant, variant_limit, settings, variant_element_state->variant_element_state, cache); removeVariantFromPath(settings.path); @@ -171,9 +170,6 @@ void SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams( return; } - if (variant_element_state->variant->size() != prev_variant_size + variant_limit) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected variant column size after deserialization. Expected {}, got {}", prev_variant_size + variant_limit, variant_element_state->variant->size()); - size_t variant_offset = variant_element_state->variant->size() - variant_limit; /// If we have only our discriminator in range, insert the whole range to result column. diff --git a/tests/queries/0_stateless/02943_variant_read_subcolumns_1.reference b/tests/queries/0_stateless/02943_variant_read_subcolumns.reference similarity index 100% rename from tests/queries/0_stateless/02943_variant_read_subcolumns_1.reference rename to tests/queries/0_stateless/02943_variant_read_subcolumns.reference diff --git a/tests/queries/0_stateless/02943_variant_read_subcolumns_1.sh b/tests/queries/0_stateless/02943_variant_read_subcolumns.sh similarity index 100% rename from tests/queries/0_stateless/02943_variant_read_subcolumns_1.sh rename to tests/queries/0_stateless/02943_variant_read_subcolumns.sh diff --git a/tests/queries/0_stateless/02943_variant_read_subcolumns_2.reference b/tests/queries/0_stateless/02943_variant_read_subcolumns_2.reference deleted file mode 100644 index 4b93782cddf..00000000000 --- a/tests/queries/0_stateless/02943_variant_read_subcolumns_2.reference +++ /dev/null @@ -1,6 +0,0 @@ -Memory -test -MergeTree compact -test -MergeTree wide -test diff --git a/tests/queries/0_stateless/02943_variant_read_subcolumns_2.sh b/tests/queries/0_stateless/02943_variant_read_subcolumns_2.sh deleted file mode 100755 index 9ccad55191f..00000000000 --- a/tests/queries/0_stateless/02943_variant_read_subcolumns_2.sh +++ /dev/null @@ -1,38 +0,0 @@ -#!/usr/bin/env bash -# Tags: long - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# reset --log_comment -CLICKHOUSE_LOG_COMMENT= -# shellcheck source=../shell_config.sh -. "$CUR_DIR"/../shell_config.sh - -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_when_no_common_type_in_if=1 " - - -function test() -{ - echo "test" - $CH_CLIENT -q "insert into test select number, multiIf(number % 3 == 2, NULL, number % 3 == 1, number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10))) from numbers(1000000) settings min_insert_block_size_rows=100000" - $CH_CLIENT -q "select v, v.UInt64, v.\`Array(Variant(String, UInt64))\`, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64 from test order by id format Null" - $CH_CLIENT -q "select v.UInt64, v.\`Array(Variant(String, UInt64))\`, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64 from test order by id format Null" - $CH_CLIENT -q "select v.\`Array(Variant(String, UInt64))\`, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64, v.\`Array(Variant(String, UInt64))\`.String from test order by id format Null" -} - -$CH_CLIENT -q "drop table if exists test;" - -echo "Memory" -$CH_CLIENT -q "create table test (id UInt64, v Variant(UInt64, Array(Variant(String, UInt64)))) engine=Memory" -test -$CH_CLIENT -q "drop table test;" - -echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, v Variant(UInt64, Array(Variant(String, UInt64)))) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000;" -test -$CH_CLIENT -q "drop table test;" - -echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, v Variant(UInt64, Array(Variant(String, UInt64)))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" -test -$CH_CLIENT -q "drop table test;" - From 0c85339ddb26e00ac64d6c763a0f5019b7ee2619 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 19 Jan 2024 10:50:13 +0000 Subject: [PATCH 074/264] Fix style --- src/DataTypes/Serializations/SerializationVariantElement.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationVariantElement.cpp b/src/DataTypes/Serializations/SerializationVariantElement.cpp index 56f0e5d77be..053f8d22d5a 100644 --- a/src/DataTypes/Serializations/SerializationVariantElement.cpp +++ b/src/DataTypes/Serializations/SerializationVariantElement.cpp @@ -9,7 +9,6 @@ namespace DB namespace ErrorCodes { extern const int NOT_IMPLEMENTED; - extern const int LOGICAL_ERROR; } void SerializationVariantElement::enumerateStreams( From e0bf32dfbc9c49ab07ab5cf240c8dba12a882fea Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 19 Jan 2024 11:59:28 +0100 Subject: [PATCH 075/264] Update install.sh --- tests/config/install.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/install.sh b/tests/config/install.sh index ebb09c07d32..a68a4c19501 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -44,7 +44,7 @@ ln -sf $SRC_PATH/config.d/transactions.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/encryption.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/CORS.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/zookeeper_log.xml $DEST_SERVER_PATH/config.d/ -ln -sf $SRC_PATH/config.d/logger_test.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/logger_trace.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/named_collection.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/ssl_certs.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/filesystem_cache_log.xml $DEST_SERVER_PATH/config.d/ From 580501c2b42231eacc4e843968aeb876ff784297 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 19 Jan 2024 11:08:56 +0000 Subject: [PATCH 076/264] Add new settings to settings changes history --- src/Core/SettingsChangesHistory.h | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 62ffd837a33..af213983b66 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -83,7 +83,9 @@ static std::map sett { {"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}, {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}, - {"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"}}}, + {"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"}, + {"allow_experimental_variant_type", false, false, "Add new experimental Variant type"}, + {"use_variant_when_no_common_type_in_if", false, false, "Allow to use Variant in if/multiIf if there is no common type"}}}, {"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}, {"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, From f6f30eb9901805b3651a7e0c32870a8208bace27 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 19 Jan 2024 14:15:49 +0100 Subject: [PATCH 077/264] Fix DB::Field operators for Float64 --- src/Columns/ColumnVector.h | 94 +---------- src/Core/CompareHelper.h | 93 +++++++++++ src/Core/Field.h | 28 ++-- src/Core/FieldHash.h | 147 ++++++++++++++++++ .../02968_sumMap_with_nan.reference | 2 + .../0_stateless/02968_sumMap_with_nan.sql | 4 + 6 files changed, 268 insertions(+), 100 deletions(-) create mode 100644 src/Core/CompareHelper.h create mode 100644 src/Core/FieldHash.h create mode 100644 tests/queries/0_stateless/02968_sumMap_with_nan.reference create mode 100644 tests/queries/0_stateless/02968_sumMap_with_nan.sql diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index fab2d5f06aa..517375f8eb4 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -1,15 +1,16 @@ #pragma once #include +#include #include #include -#include -#include +#include #include -#include -#include #include #include +#include +#include +#include #include "config.h" @@ -26,91 +27,6 @@ namespace ErrorCodes } -/** Stuff for comparing numbers. - * Integer values are compared as usual. - * Floating-point numbers are compared this way that NaNs always end up at the end - * (if you don't do this, the sort would not work at all). - */ -template -struct CompareHelper -{ - static constexpr bool less(T a, U b, int /*nan_direction_hint*/) { return a < b; } - static constexpr bool greater(T a, U b, int /*nan_direction_hint*/) { return a > b; } - static constexpr bool equals(T a, U b, int /*nan_direction_hint*/) { return a == b; } - - /** Compares two numbers. Returns a number less than zero, equal to zero, or greater than zero if a < b, a == b, a > b, respectively. - * If one of the values is NaN, then - * - if nan_direction_hint == -1 - NaN are considered less than all numbers; - * - if nan_direction_hint == 1 - NaN are considered to be larger than all numbers; - * Essentially: nan_direction_hint == -1 says that the comparison is for sorting in descending order. - */ - static constexpr int compare(T a, U b, int /*nan_direction_hint*/) - { - return a > b ? 1 : (a < b ? -1 : 0); - } -}; - -template -struct FloatCompareHelper -{ - static constexpr bool less(T a, T b, int nan_direction_hint) - { - const bool isnan_a = std::isnan(a); - const bool isnan_b = std::isnan(b); - - if (isnan_a && isnan_b) - return false; - if (isnan_a) - return nan_direction_hint < 0; - if (isnan_b) - return nan_direction_hint > 0; - - return a < b; - } - - static constexpr bool greater(T a, T b, int nan_direction_hint) - { - const bool isnan_a = std::isnan(a); - const bool isnan_b = std::isnan(b); - - if (isnan_a && isnan_b) - return false; - if (isnan_a) - return nan_direction_hint > 0; - if (isnan_b) - return nan_direction_hint < 0; - - return a > b; - } - - static constexpr bool equals(T a, T b, int nan_direction_hint) - { - return compare(a, b, nan_direction_hint) == 0; - } - - static constexpr int compare(T a, T b, int nan_direction_hint) - { - const bool isnan_a = std::isnan(a); - const bool isnan_b = std::isnan(b); - - if (unlikely(isnan_a || isnan_b)) - { - if (isnan_a && isnan_b) - return 0; - - return isnan_a - ? nan_direction_hint - : -nan_direction_hint; - } - - return (T(0) < (a - b)) - ((a - b) < T(0)); - } -}; - -template struct CompareHelper : public FloatCompareHelper {}; -template struct CompareHelper : public FloatCompareHelper {}; - - /** A template for columns that use a simple array to store. */ template diff --git a/src/Core/CompareHelper.h b/src/Core/CompareHelper.h new file mode 100644 index 00000000000..2e7aaf1f605 --- /dev/null +++ b/src/Core/CompareHelper.h @@ -0,0 +1,93 @@ +#pragma once + +#include +#include + +#include + +namespace DB +{ + +/** Stuff for comparing numbers. + * Integer values are compared as usual. + * Floating-point numbers are compared this way that NaNs always end up at the end + * (if you don't do this, the sort would not work at all). + */ +template +struct CompareHelper +{ + static constexpr bool less(T a, U b, int /*nan_direction_hint*/) { return a < b; } + static constexpr bool greater(T a, U b, int /*nan_direction_hint*/) { return a > b; } + static constexpr bool equals(T a, U b, int /*nan_direction_hint*/) { return a == b; } + + /** Compares two numbers. Returns a number less than zero, equal to zero, or greater than zero if a < b, a == b, a > b, respectively. + * If one of the values is NaN, then + * - if nan_direction_hint == -1 - NaN are considered less than all numbers; + * - if nan_direction_hint == 1 - NaN are considered to be larger than all numbers; + * Essentially: nan_direction_hint == -1 says that the comparison is for sorting in descending order. + */ + static constexpr int compare(T a, U b, int /*nan_direction_hint*/) { return a > b ? 1 : (a < b ? -1 : 0); } +}; + +template +struct FloatCompareHelper +{ + static constexpr bool less(T a, T b, int nan_direction_hint) + { + const bool isnan_a = std::isnan(a); + const bool isnan_b = std::isnan(b); + + if (isnan_a && isnan_b) + return false; + if (isnan_a) + return nan_direction_hint < 0; + if (isnan_b) + return nan_direction_hint > 0; + + return a < b; + } + + static constexpr bool greater(T a, T b, int nan_direction_hint) + { + const bool isnan_a = std::isnan(a); + const bool isnan_b = std::isnan(b); + + if (isnan_a && isnan_b) + return false; + if (isnan_a) + return nan_direction_hint > 0; + if (isnan_b) + return nan_direction_hint < 0; + + return a > b; + } + + static constexpr bool equals(T a, T b, int nan_direction_hint) { return compare(a, b, nan_direction_hint) == 0; } + + static constexpr int compare(T a, T b, int nan_direction_hint) + { + const bool isnan_a = std::isnan(a); + const bool isnan_b = std::isnan(b); + + if (unlikely(isnan_a || isnan_b)) + { + if (isnan_a && isnan_b) + return 0; + + return isnan_a ? nan_direction_hint : -nan_direction_hint; + } + + return (T(0) < (a - b)) - ((a - b) < T(0)); + } +}; + +template +struct CompareHelper : public FloatCompareHelper +{ +}; +template +struct CompareHelper : public FloatCompareHelper +{ +}; + +} diff --git a/src/Core/Field.h b/src/Core/Field.h index 6afa98ed9c0..445a5850ca4 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -7,14 +7,15 @@ #include #include -#include -#include -#include -#include +#include #include +#include +#include #include -#include #include +#include +#include +#include namespace DB { @@ -305,6 +306,7 @@ static constexpr auto DBMS_MIN_FIELD_SIZE = 32; */ class Field { + static constexpr int nan_direction_hint = 1; // When comparing Floats NaN are considered to be larger than all numbers public: struct Types { @@ -508,7 +510,8 @@ public: case Types::UUID: return get() < rhs.get(); case Types::IPv4: return get() < rhs.get(); case Types::IPv6: return get() < rhs.get(); - case Types::Float64: return get() < rhs.get(); + case Types::Float64: + return FloatCompareHelper::less(get(), rhs.get(), nan_direction_hint); case Types::String: return get() < rhs.get(); case Types::Array: return get() < rhs.get(); case Types::Tuple: return get() < rhs.get(); @@ -550,7 +553,13 @@ public: case Types::UUID: return get().toUnderType() <= rhs.get().toUnderType(); case Types::IPv4: return get() <= rhs.get(); case Types::IPv6: return get() <= rhs.get(); - case Types::Float64: return get() <= rhs.get(); + case Types::Float64: + { + Float64 f1 = get(); + Float64 f2 = get(); + return FloatCompareHelper::less(f1, f2, nan_direction_hint) + || FloatCompareHelper::equals(f1, f2, nan_direction_hint); + } case Types::String: return get() <= rhs.get(); case Types::Array: return get() <= rhs.get(); case Types::Tuple: return get() <= rhs.get(); @@ -586,10 +595,7 @@ public: case Types::UInt64: return get() == rhs.get(); case Types::Int64: return get() == rhs.get(); case Types::Float64: - { - // Compare as UInt64 so that NaNs compare as equal. - return std::bit_cast(get()) == std::bit_cast(rhs.get()); - } + return FloatCompareHelper::equals(get(), rhs.get(), nan_direction_hint); case Types::UUID: return get() == rhs.get(); case Types::IPv4: return get() == rhs.get(); case Types::IPv6: return get() == rhs.get(); diff --git a/src/Core/FieldHash.h b/src/Core/FieldHash.h new file mode 100644 index 00000000000..e0d1b997065 --- /dev/null +++ b/src/Core/FieldHash.h @@ -0,0 +1,147 @@ +#pragma once + +#include +#include + +#include +#include + +namespace +{ +static_assert(sizeof(size_t) == 4 || sizeof(size_t) == 8); +using size_t_equivalent = std::conditional_t; + +inline size_t hash64(UInt64 x) +{ + return static_cast(intHash64(x)); +} + +inline size_t hash32(UInt32 x) +{ + /// It's ok to just treat it as 64bits + return static_cast(hash64(x)); +} + +inline size_t hash_size_t(size_t x) +{ + if constexpr (sizeof(size_t) == 4) + return hash32(std::bit_cast(x)); + else + return hash64(std::bit_cast(x)); +} + +inline size_t combine_hashes(size_t h1, size_t h2) +{ + return hash_size_t(h1) ^ h2; +} +} + + +namespace std +{ +/// Extends std::hash to support DB::Field so it can be used in std::unordered_map and similar structures +template <> +struct hash +{ + size_t operator()(const DB::Field & field) const noexcept + { + using Which = DB::Field::Types::Which; + size_t type_hash = ::hash32(field.getType()); + switch (field.getType()) + { + case Which::Null: + return 0; + case Which::Bool: + [[fallthrough]]; + case Which::UInt64: + return ::combine_hashes(type_hash, ::hash64(field.get())); + case Which::IPv6: + [[fallthrough]]; + case Which::UUID: + [[fallthrough]]; + case Which::UInt128: { + UInt128 n = field.get(); + size_t h1 = ::hash64(n.items[0]); + size_t h2 = ::hash64(n.items[1]); + return ::combine_hashes(type_hash, ::combine_hashes(h1, h2)); + } + case Which::UInt256: { + UInt256 n = field.get(); + size_t h1 = ::hash64(n.items[0]); + size_t h2 = ::hash64(n.items[1]); + size_t h3 = ::hash64(n.items[2]); + size_t h4 = ::hash64(n.items[3]); + return ::combine_hashes(::combine_hashes(type_hash, ::combine_hashes(h1, h2)), ::combine_hashes(h3, h4)); + } + case Which::Int64: { + UInt64 n = std::bit_cast(&field.get()); + return ::combine_hashes(type_hash, n); + } + case Which::Int128: { + UInt128 n = std::bit_cast(field.get()); + size_t h1 = ::hash64(n.items[0]); + size_t h2 = ::hash64(n.items[1]); + return ::combine_hashes(type_hash, ::combine_hashes(h1, h2)); + } + case Which::Int256: { + UInt256 n = std::bit_cast(field.get()); + size_t h1 = ::hash64(n.items[0]); + size_t h2 = ::hash64(n.items[1]); + size_t h3 = ::hash64(n.items[2]); + size_t h4 = ::hash64(n.items[3]); + return ::combine_hashes(::combine_hashes(type_hash, ::combine_hashes(h1, h2)), ::combine_hashes(h3, h4)); + } + case Which::IPv4: { + UInt32 n = field.get(); + return ::combine_hashes(type_hash, ::hash32(n)); + } + case Which::Float64: + return ::combine_hashes(type_hash, std::hash{}(field.get())); + case Which::String: + return ::combine_hashes(type_hash, std::hash{}(field.get())); + case Which::Array: { + auto const & array = field.get(); + size_t res = type_hash; + for (const auto & e : array) + res = ::combine_hashes(res, std::hash{}(e)); + return res; + } + case Which::Tuple: { + auto const & tuple = field.get(); + size_t res = type_hash; + for (const auto & e : tuple) + res = ::combine_hashes(res, std::hash{}(e)); + return res; + } + case Which::Map: { + auto const & map = field.get(); + size_t res = type_hash; + for (const auto & e : map) + res = ::combine_hashes(res, std::hash{}(e)); + return res; + } + case Which::Object: { + auto const & object = field.get(); + size_t res = type_hash; + for (const auto & e : object) + res = ::combine_hashes(res, ::combine_hashes(std::hash{}(e.first), std::hash{}(e.second))); + return res; + } + case Which::Decimal32: + return ::combine_hashes(type_hash, std::hash{}(field.get())); + case Which::Decimal64: + return ::combine_hashes(type_hash, std::hash{}(field.get())); + case Which::Decimal128: + return ::combine_hashes(type_hash, std::hash{}(field.get())); + case Which::Decimal256: + return ::combine_hashes(type_hash, std::hash{}(field.get())); + case Which::AggregateFunctionState: { + auto const & agg = field.get(); + return ::combine_hashes(type_hash, ::combine_hashes(std::hash{}(agg.name), std::hash{}(agg.data))); + } + case Which::CustomType: + return ::combine_hashes(type_hash, std::hash{}(field.get().toString())); + } + } +}; +} diff --git a/tests/queries/0_stateless/02968_sumMap_with_nan.reference b/tests/queries/0_stateless/02968_sumMap_with_nan.reference new file mode 100644 index 00000000000..83a2d98375f --- /dev/null +++ b/tests/queries/0_stateless/02968_sumMap_with_nan.reference @@ -0,0 +1,2 @@ +([6.7],[3]) +([1,4,5,6.7,nan],[2.3,5,1,3,inf]) diff --git a/tests/queries/0_stateless/02968_sumMap_with_nan.sql b/tests/queries/0_stateless/02968_sumMap_with_nan.sql new file mode 100644 index 00000000000..330da94cfea --- /dev/null +++ b/tests/queries/0_stateless/02968_sumMap_with_nan.sql @@ -0,0 +1,4 @@ +SELECT sumMapFiltered([6.7])([x], [y]) +FROM values('x Float64, y Float64', (0, 1), (1, 2.3), (nan, inf), (6.7, 3), (4, 4), (5, 1)); + +SELECT sumMap([x],[y]) FROM values('x Float64, y Float64', (4, 1), (1, 2.3), (nan,inf), (6.7,3), (4,4), (5, 1)); From 47df16f8965a3946570e48054c5bbd5b46a9ee54 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 19 Jan 2024 14:28:18 +0100 Subject: [PATCH 078/264] Style is not happy with orphan headers --- src/Core/FieldHash.h | 147 ------------------------------------------- 1 file changed, 147 deletions(-) delete mode 100644 src/Core/FieldHash.h diff --git a/src/Core/FieldHash.h b/src/Core/FieldHash.h deleted file mode 100644 index e0d1b997065..00000000000 --- a/src/Core/FieldHash.h +++ /dev/null @@ -1,147 +0,0 @@ -#pragma once - -#include -#include - -#include -#include - -namespace -{ -static_assert(sizeof(size_t) == 4 || sizeof(size_t) == 8); -using size_t_equivalent = std::conditional_t; - -inline size_t hash64(UInt64 x) -{ - return static_cast(intHash64(x)); -} - -inline size_t hash32(UInt32 x) -{ - /// It's ok to just treat it as 64bits - return static_cast(hash64(x)); -} - -inline size_t hash_size_t(size_t x) -{ - if constexpr (sizeof(size_t) == 4) - return hash32(std::bit_cast(x)); - else - return hash64(std::bit_cast(x)); -} - -inline size_t combine_hashes(size_t h1, size_t h2) -{ - return hash_size_t(h1) ^ h2; -} -} - - -namespace std -{ -/// Extends std::hash to support DB::Field so it can be used in std::unordered_map and similar structures -template <> -struct hash -{ - size_t operator()(const DB::Field & field) const noexcept - { - using Which = DB::Field::Types::Which; - size_t type_hash = ::hash32(field.getType()); - switch (field.getType()) - { - case Which::Null: - return 0; - case Which::Bool: - [[fallthrough]]; - case Which::UInt64: - return ::combine_hashes(type_hash, ::hash64(field.get())); - case Which::IPv6: - [[fallthrough]]; - case Which::UUID: - [[fallthrough]]; - case Which::UInt128: { - UInt128 n = field.get(); - size_t h1 = ::hash64(n.items[0]); - size_t h2 = ::hash64(n.items[1]); - return ::combine_hashes(type_hash, ::combine_hashes(h1, h2)); - } - case Which::UInt256: { - UInt256 n = field.get(); - size_t h1 = ::hash64(n.items[0]); - size_t h2 = ::hash64(n.items[1]); - size_t h3 = ::hash64(n.items[2]); - size_t h4 = ::hash64(n.items[3]); - return ::combine_hashes(::combine_hashes(type_hash, ::combine_hashes(h1, h2)), ::combine_hashes(h3, h4)); - } - case Which::Int64: { - UInt64 n = std::bit_cast(&field.get()); - return ::combine_hashes(type_hash, n); - } - case Which::Int128: { - UInt128 n = std::bit_cast(field.get()); - size_t h1 = ::hash64(n.items[0]); - size_t h2 = ::hash64(n.items[1]); - return ::combine_hashes(type_hash, ::combine_hashes(h1, h2)); - } - case Which::Int256: { - UInt256 n = std::bit_cast(field.get()); - size_t h1 = ::hash64(n.items[0]); - size_t h2 = ::hash64(n.items[1]); - size_t h3 = ::hash64(n.items[2]); - size_t h4 = ::hash64(n.items[3]); - return ::combine_hashes(::combine_hashes(type_hash, ::combine_hashes(h1, h2)), ::combine_hashes(h3, h4)); - } - case Which::IPv4: { - UInt32 n = field.get(); - return ::combine_hashes(type_hash, ::hash32(n)); - } - case Which::Float64: - return ::combine_hashes(type_hash, std::hash{}(field.get())); - case Which::String: - return ::combine_hashes(type_hash, std::hash{}(field.get())); - case Which::Array: { - auto const & array = field.get(); - size_t res = type_hash; - for (const auto & e : array) - res = ::combine_hashes(res, std::hash{}(e)); - return res; - } - case Which::Tuple: { - auto const & tuple = field.get(); - size_t res = type_hash; - for (const auto & e : tuple) - res = ::combine_hashes(res, std::hash{}(e)); - return res; - } - case Which::Map: { - auto const & map = field.get(); - size_t res = type_hash; - for (const auto & e : map) - res = ::combine_hashes(res, std::hash{}(e)); - return res; - } - case Which::Object: { - auto const & object = field.get(); - size_t res = type_hash; - for (const auto & e : object) - res = ::combine_hashes(res, ::combine_hashes(std::hash{}(e.first), std::hash{}(e.second))); - return res; - } - case Which::Decimal32: - return ::combine_hashes(type_hash, std::hash{}(field.get())); - case Which::Decimal64: - return ::combine_hashes(type_hash, std::hash{}(field.get())); - case Which::Decimal128: - return ::combine_hashes(type_hash, std::hash{}(field.get())); - case Which::Decimal256: - return ::combine_hashes(type_hash, std::hash{}(field.get())); - case Which::AggregateFunctionState: { - auto const & agg = field.get(); - return ::combine_hashes(type_hash, ::combine_hashes(std::hash{}(agg.name), std::hash{}(agg.data))); - } - case Which::CustomType: - return ::combine_hashes(type_hash, std::hash{}(field.get().toString())); - } - } -}; -} From d8b5623c674383f67aea5aeb2be8fc0a8e443380 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 17 Jan 2024 16:38:51 +0000 Subject: [PATCH 079/264] Analyzer: Support GROUP BY injective function elimination --- .../OptimizeGroupByInjectiveFunctionsPass.cpp | 142 ++++++++++++++++++ .../OptimizeGroupByInjectiveFunctionsPass.h | 20 +++ src/Analyzer/QueryTreePassManager.cpp | 4 +- 3 files changed, 164 insertions(+), 2 deletions(-) create mode 100644 src/Analyzer/Passes/OptimizeGroupByInjectiveFunctionsPass.cpp create mode 100644 src/Analyzer/Passes/OptimizeGroupByInjectiveFunctionsPass.h diff --git a/src/Analyzer/Passes/OptimizeGroupByInjectiveFunctionsPass.cpp b/src/Analyzer/Passes/OptimizeGroupByInjectiveFunctionsPass.cpp new file mode 100644 index 00000000000..750b119ad3d --- /dev/null +++ b/src/Analyzer/Passes/OptimizeGroupByInjectiveFunctionsPass.cpp @@ -0,0 +1,142 @@ +#include +#include +#include "Analyzer/ConstantNode.h" +#include "Analyzer/FunctionNode.h" +#include "Analyzer/IQueryTreeNode.h" +#include "DataTypes/IDataType.h" +#include +#include "Functions/FunctionFactory.h" +#include "Interpreters/Context_fwd.h" +#include "Interpreters/ExternalDictionariesLoader.h" + +namespace DB +{ + +namespace +{ + +const std::unordered_set possibly_injective_function_names +{ + "dictGet", + "dictGetString", + "dictGetUInt8", + "dictGetUInt16", + "dictGetUInt32", + "dictGetUInt64", + "dictGetInt8", + "dictGetInt16", + "dictGetInt32", + "dictGetInt64", + "dictGetFloat32", + "dictGetFloat64", + "dictGetDate", + "dictGetDateTime" +}; + +class OptimizeGroupByInjectiveFunctionsVisitor : public InDepthQueryTreeVisitorWithContext +{ + using Base = InDepthQueryTreeVisitorWithContext; +public: + explicit OptimizeGroupByInjectiveFunctionsVisitor(ContextPtr context) + : Base(std::move(context)) + {} + + void enterImpl(QueryTreeNodePtr & node) + { + auto * query = node->as(); + if (!query) + return; + + if (!query->hasGroupBy()) + return; + + if (query->isGroupByWithCube() || query->isGroupByWithRollup()) + return; + + auto & group_by = query->getGroupBy().getNodes(); + if (query->isGroupByWithGroupingSets()) + { + for (auto & set : group_by) + { + auto & grouping_set = set->as()->getNodes(); + optimizeGroupingSet(grouping_set); + } + } + else + optimizeGroupingSet(group_by); + } + +private: + void optimizeGroupingSet(QueryTreeNodes & grouping_set) + { + auto context = getContext(); + const FunctionFactory & function_factory = FunctionFactory::instance(); + + QueryTreeNodes new_group_by_keys; + new_group_by_keys.reserve(grouping_set.size()); + for (auto & group_by_elem : grouping_set) + { + if (auto const * function_node = group_by_elem->as()) + { + bool can_be_eliminated = false; + if (possibly_injective_function_names.contains(function_node->getFunctionName())) + { + can_be_eliminated = canBeEliminated(function_node, context); + } + else + { + FunctionOverloadResolverPtr function_builder = UserDefinedExecutableFunctionFactory::instance().tryGet(function_node->getFunctionName(), context); + + // TODO: fix me + if (!function_builder) + function_builder = function_factory.get(function_node->getFunctionName(), context); + + can_be_eliminated = function_builder->isInjective({}); + } + + if (can_be_eliminated) + { + for (auto const & argument : function_node->getArguments()) + { + if (argument->getNodeType() != QueryTreeNodeType::CONSTANT) + new_group_by_keys.push_back(argument); + } + } + else + new_group_by_keys.push_back(group_by_elem); + } + else + new_group_by_keys.push_back(group_by_elem); + } + + grouping_set = std::move(new_group_by_keys); + } + + bool canBeEliminated(const FunctionNode * function_node, const ContextPtr & context) + { + auto const * dict_name_arg = function_node->getArguments().getNodes()[0]->as(); + if (!dict_name_arg || isString(dict_name_arg->getResultType())) + return false; + auto dict_name = dict_name_arg->getValue().safeGet(); + + const auto & dict_ptr = context->getExternalDictionariesLoader().getDictionary(dict_name, context); + + auto const * attr_arg = function_node->getArguments().getNodes()[0]->as(); + if (!attr_arg || isString(attr_arg->getResultType())) + return false; + auto attr_name = attr_arg->getValue().safeGet(); + + return dict_ptr->isInjective(attr_name); + } + +}; + +} + +void OptimizeGroupByInjectiveFunctionsPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) +{ + OptimizeGroupByInjectiveFunctionsVisitor visitor(std::move(context)); + visitor.visit(query_tree_node); +} + +} diff --git a/src/Analyzer/Passes/OptimizeGroupByInjectiveFunctionsPass.h b/src/Analyzer/Passes/OptimizeGroupByInjectiveFunctionsPass.h new file mode 100644 index 00000000000..22390451824 --- /dev/null +++ b/src/Analyzer/Passes/OptimizeGroupByInjectiveFunctionsPass.h @@ -0,0 +1,20 @@ +#pragma once + +#include + +namespace DB +{ + +/* Eliminates injective functions in GROUP BY section. + */ +class OptimizeGroupByInjectiveFunctionsPass final : public IQueryTreePass +{ +public: + String getName() override { return "OptimizeGroupByInjectiveFunctionsPass"; } + + String getDescription() override { return "Replaces injective functions by it's arguments in GROUP BY section."; } + + void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; +}; + +} diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index 58fd1d5937b..c7a2421b6d7 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -3,6 +3,7 @@ #include #include +#include "Analyzer/Passes/OptimizeGroupByInjectiveFunctionsPass.h" #include #include @@ -162,8 +163,6 @@ private: /** ClickHouse query tree pass manager. * - * TODO: Support setting optimize_substitute_columns. - * TODO: Support GROUP BY injective function elimination. * TODO: Support setting optimize_aggregators_of_group_by_keys. * TODO: Support setting optimize_monotonous_functions_in_order_by. * TODO: Add optimizations based on function semantics. Example: SELECT * FROM test_table WHERE id != id. (id is not nullable column). @@ -266,6 +265,7 @@ void addQueryTreePasses(QueryTreePassManager & manager) manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); + manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); From 773e5b0bbe698e3cbab89938bc998d2bfad49767 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 19 Jan 2024 14:52:04 +0000 Subject: [PATCH 080/264] Update reference file --- tests/queries/0_stateless/02303_query_kind.reference | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02303_query_kind.reference b/tests/queries/0_stateless/02303_query_kind.reference index 53a0df682b2..9f1c026f889 100644 --- a/tests/queries/0_stateless/02303_query_kind.reference +++ b/tests/queries/0_stateless/02303_query_kind.reference @@ -20,17 +20,17 @@ clickhouse-client --allow_experimental_analyzer=1 --query_kind initial_query -q Expression ((Project names + Projection)) Header: dummy String Aggregating - Header: toString(__table1.dummy) String + Header: __table1.dummy UInt8 Expression ((Before GROUP BY + Change column names to column identifiers)) - Header: toString(__table1.dummy) String + Header: __table1.dummy UInt8 ReadFromStorage (SystemOne) Header: dummy UInt8 clickhouse-local --allow_experimental_analyzer=1 --query_kind initial_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy Expression ((Project names + Projection)) Header: dummy String Aggregating - Header: toString(__table1.dummy) String + Header: __table1.dummy UInt8 Expression ((Before GROUP BY + Change column names to column identifiers)) - Header: toString(__table1.dummy) String + Header: __table1.dummy UInt8 ReadFromStorage (SystemOne) Header: dummy UInt8 From c51d1f04f6d135c63f5123d4aaef47cef5474525 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 19 Jan 2024 15:57:20 +0100 Subject: [PATCH 081/264] Add settings max_unexpected_write_error_retries for Azure Blob Storage --- src/Core/Settings.h | 1 + src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp | 10 +++++----- src/Disks/IO/WriteBufferFromAzureBlobStorage.h | 2 ++ .../AzureBlobStorage/AzureBlobStorageAuth.cpp | 3 ++- .../AzureBlobStorage/AzureObjectStorage.cpp | 1 + .../AzureBlobStorage/AzureObjectStorage.h | 5 ++++- 6 files changed, 15 insertions(+), 7 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 292e945a29c..59f32c60f63 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -86,6 +86,7 @@ class IColumn; M(UInt64, azure_max_single_part_upload_size, 100*1024*1024, "The maximum size of object to upload using singlepart upload to Azure blob storage.", 0) \ M(UInt64, s3_max_single_read_retries, 4, "The maximum number of retries during single S3 read.", 0) \ M(UInt64, azure_max_single_read_retries, 4, "The maximum number of retries during single Azure blob storage read.", 0) \ + M(UInt64, azure_max_unexpected_write_error_retries, 4, "The maximum number of retries in case of unexpected errors during Azure blob storage write", 0) \ M(UInt64, s3_max_unexpected_write_error_retries, 4, "The maximum number of retries in case of unexpected errors during S3 write.", 0) \ M(UInt64, s3_max_redirects, 10, "Max number of S3 redirects hops allowed.", 0) \ M(UInt64, s3_max_connections, 1024, "The maximum number of connections per server.", 0) \ diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 60bc04f5f95..b4665eb7346 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -18,17 +18,17 @@ namespace ProfileEvents namespace DB { -static constexpr auto DEFAULT_RETRY_NUM = 3; - WriteBufferFromAzureBlobStorage::WriteBufferFromAzureBlobStorage( std::shared_ptr blob_container_client_, const String & blob_path_, size_t max_single_part_upload_size_, + size_t max_unexpected_write_error_retries_, size_t buf_size_, const WriteSettings & write_settings_) : WriteBufferFromFileBase(buf_size_, nullptr, 0) , log(&Poco::Logger::get("WriteBufferFromAzureBlobStorage")) , max_single_part_upload_size(max_single_part_upload_size_) + , max_unexpected_write_error_retries(max_unexpected_write_error_retries_) , blob_path(blob_path_) , write_settings(write_settings_) , blob_container_client(blob_container_client_) @@ -77,13 +77,13 @@ void WriteBufferFromAzureBlobStorage::execWithRetry(std::function func, void WriteBufferFromAzureBlobStorage::finalizeImpl() { - execWithRetry([this](){ next(); }, DEFAULT_RETRY_NUM); + execWithRetry([this](){ next(); }, max_unexpected_write_error_retries); if (tmp_buffer_write_offset > 0) uploadBlock(tmp_buffer->data(), tmp_buffer_write_offset); auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path); - execWithRetry([&](){ block_blob_client.CommitBlockList(block_ids); }, DEFAULT_RETRY_NUM); + execWithRetry([&](){ block_blob_client.CommitBlockList(block_ids); }, max_unexpected_write_error_retries); LOG_TRACE(log, "Committed {} blocks for blob `{}`", block_ids.size(), blob_path); } @@ -94,7 +94,7 @@ void WriteBufferFromAzureBlobStorage::uploadBlock(const char * data, size_t size const std::string & block_id = block_ids.emplace_back(getRandomASCIIString(64)); Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast(data), size); - execWithRetry([&](){ block_blob_client.StageBlock(block_id, memory_stream); }, DEFAULT_RETRY_NUM, size); + execWithRetry([&](){ block_blob_client.StageBlock(block_id, memory_stream); }, max_unexpected_write_error_retries, size); tmp_buffer_write_offset = 0; LOG_TRACE(log, "Staged block (id: {}) of size {} (blob path: {}).", block_id, size, blob_path); diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h index f1be81922e1..7494130134b 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h @@ -30,6 +30,7 @@ public: AzureClientPtr blob_container_client_, const String & blob_path_, size_t max_single_part_upload_size_, + size_t max_unexpected_write_error_retries_, size_t buf_size_, const WriteSettings & write_settings_); @@ -48,6 +49,7 @@ private: Poco::Logger * log; const size_t max_single_part_upload_size; + const size_t max_unexpected_write_error_retries; const std::string blob_path; const WriteSettings write_settings; diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp index a5c8afe0264..a209049ceee 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp @@ -164,7 +164,8 @@ std::unique_ptr getAzureBlobStorageSettings(const Po config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024), config.getInt(config_prefix + ".max_single_read_retries", 3), config.getInt(config_prefix + ".max_single_download_retries", 3), - config.getInt(config_prefix + ".list_object_keys_size", 1000) + config.getInt(config_prefix + ".list_object_keys_size", 1000), + config.getUInt64(config_prefix + ".max_unexpected_write_error_retries", 4) ); } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 068e2aebab1..683bfeb74a7 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -268,6 +268,7 @@ std::unique_ptr AzureObjectStorage::writeObject( /// NO client.get(), object.remote_path, settings.get()->max_single_part_upload_size, + settings.get()->max_unexpected_write_error_retries, buf_size, patchSettings(write_settings)); } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 4718dce9bf9..2d505c6a022 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -23,12 +23,14 @@ struct AzureObjectStorageSettings uint64_t min_bytes_for_seek_, int max_single_read_retries_, int max_single_download_retries_, - int list_object_keys_size_) + int list_object_keys_size_, + size_t max_unexpected_write_error_retries_) : max_single_part_upload_size(max_single_part_upload_size_) , min_bytes_for_seek(min_bytes_for_seek_) , max_single_read_retries(max_single_read_retries_) , max_single_download_retries(max_single_download_retries_) , list_object_keys_size(list_object_keys_size_) + , max_unexpected_write_error_retries (max_unexpected_write_error_retries_) { } @@ -39,6 +41,7 @@ struct AzureObjectStorageSettings size_t max_single_read_retries = 3; size_t max_single_download_retries = 3; int list_object_keys_size = 1000; + size_t max_unexpected_write_error_retries = 4; }; using AzureClient = Azure::Storage::Blobs::BlobContainerClient; From 728276ab3f29c6fe187b30d2f6f20d9a9379aad4 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 17 Jan 2024 14:22:56 +0000 Subject: [PATCH 082/264] Fix crash when indexHint() is used --- src/Interpreters/ActionsDAG.cpp | 14 ++++++++++---- src/Storages/MergeTree/RPNBuilder.h | 6 ++++++ .../0_stateless/02967_index_hint_crash.reference | 0 .../0_stateless/02967_index_hint_crash.sql | 16 ++++++++++++++++ 4 files changed, 32 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02967_index_hint_crash.reference create mode 100644 tests/queries/0_stateless/02967_index_hint_crash.sql diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 31b7bdffb1f..7e86d87db14 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -2542,10 +2542,16 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( { if (const auto * index_hint = typeid_cast(adaptor->getFunction().get())) { - auto index_hint_filter_dag = buildFilterActionsDAG(index_hint->getActions()->getOutputs(), - node_name_to_input_node_column, - context, - false /*single_output_condition_node*/); + ActionsDAGPtr index_hint_filter_dag; + const auto & index_hint_args = index_hint->getActions()->getOutputs(); + + if (index_hint_args.empty()) + index_hint_filter_dag = std::make_shared(); + else + index_hint_filter_dag = buildFilterActionsDAG(index_hint_args, + node_name_to_input_node_column, + context, + false /*single_output_condition_node*/); auto index_hint_function_clone = std::make_shared(); index_hint_function_clone->setActions(std::move(index_hint_filter_dag)); diff --git a/src/Storages/MergeTree/RPNBuilder.h b/src/Storages/MergeTree/RPNBuilder.h index b0755ccd3ca..d750c02d3e1 100644 --- a/src/Storages/MergeTree/RPNBuilder.h +++ b/src/Storages/MergeTree/RPNBuilder.h @@ -229,6 +229,12 @@ private: rpn_elements.emplace_back(std::move(element)); } + if (arguments_size == 0 && function_node.getFunctionName() == "indexHint") + { + element.function = RPNElement::ALWAYS_TRUE; + rpn_elements.emplace_back(std::move(element)); + } + return; } } diff --git a/tests/queries/0_stateless/02967_index_hint_crash.reference b/tests/queries/0_stateless/02967_index_hint_crash.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02967_index_hint_crash.sql b/tests/queries/0_stateless/02967_index_hint_crash.sql new file mode 100644 index 00000000000..e33a4992c6c --- /dev/null +++ b/tests/queries/0_stateless/02967_index_hint_crash.sql @@ -0,0 +1,16 @@ +CREATE TABLE tab +( + `foo` Array(LowCardinality(String)), + INDEX idx foo TYPE bloom_filter GRANULARITY 1 +) +ENGINE = MergeTree +PRIMARY KEY tuple(); + +INSERT INTO tab SELECT if(number % 2, ['value'], []) +FROM system.numbers +LIMIT 10000; + +SELECT * +FROM tab +PREWHERE indexHint() +FORMAT Null; From d22fc3a224ac29857f3dc3eb60ff872221829006 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 19 Jan 2024 16:13:58 +0100 Subject: [PATCH 083/264] Updated to fetch default from settings --- .../ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp index a209049ceee..e0199fde98b 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp @@ -7,6 +7,7 @@ #include #include #include +#include using namespace Azure::Storage::Blobs; @@ -157,7 +158,7 @@ std::unique_ptr getAzureBlobContainerClient( } } -std::unique_ptr getAzureBlobStorageSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr /*context*/) +std::unique_ptr getAzureBlobStorageSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context) { return std::make_unique( config.getUInt64(config_prefix + ".max_single_part_upload_size", 100 * 1024 * 1024), @@ -165,7 +166,7 @@ std::unique_ptr getAzureBlobStorageSettings(const Po config.getInt(config_prefix + ".max_single_read_retries", 3), config.getInt(config_prefix + ".max_single_download_retries", 3), config.getInt(config_prefix + ".list_object_keys_size", 1000), - config.getUInt64(config_prefix + ".max_unexpected_write_error_retries", 4) + config.getUInt64(config_prefix + ".max_unexpected_write_error_retries", context->getSettings().azure_max_unexpected_write_error_retries) ); } From 0132455b393d4c54a5a03893b47829ec240f77e9 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 19 Jan 2024 15:28:02 +0000 Subject: [PATCH 084/264] Reduce Keeper memory usage --- src/Coordination/KeeperSnapshotManager.cpp | 38 +++--- src/Coordination/KeeperSnapshotManager.h | 3 +- src/Coordination/KeeperStorage.cpp | 71 +++++----- src/Coordination/KeeperStorage.h | 49 ++++--- src/Coordination/SnapshotableHashTable.h | 123 ++++++++++++++---- src/Coordination/ZooKeeperDataReader.cpp | 1 - src/Coordination/tests/gtest_coordination.cpp | 16 +-- utils/keeper-data-dumper/main.cpp | 2 +- 8 files changed, 204 insertions(+), 99 deletions(-) diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index fffa6eaa941..ee5935015e4 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -74,7 +74,10 @@ namespace /// Serialize ACL writeBinary(node.acl_id, out); - writeBinary(node.is_sequental, out); + /// Write is_sequential for backwards compatibility + if (version < SnapshotVersion::V6) + writeBinary(false, out); + /// Serialize stat writeBinary(node.stat.czxid, out); writeBinary(node.stat.mzxid, out); @@ -84,16 +87,15 @@ namespace writeBinary(node.stat.cversion, out); writeBinary(node.stat.aversion, out); writeBinary(node.stat.ephemeralOwner, out); - writeBinary(node.stat.dataLength, out); + if (version < SnapshotVersion::V6) + writeBinary(static_cast(node.getData().size()), out); writeBinary(node.stat.numChildren, out); writeBinary(node.stat.pzxid, out); writeBinary(node.seq_num, out); - if (version >= SnapshotVersion::V4) - { - writeBinary(node.size_bytes, out); - } + if (version >= SnapshotVersion::V4 && version <= SnapshotVersion::V5) + writeBinary(node.sizeInBytes(), out); } void readNode(KeeperStorage::Node & node, ReadBuffer & in, SnapshotVersion version, ACLMap & acl_map) @@ -129,7 +131,11 @@ namespace acl_map.addUsage(node.acl_id); - readBinary(node.is_sequental, in); + if (version < SnapshotVersion::V6) + { + bool is_sequential = false; + readBinary(is_sequential, in); + } /// Deserialize stat readBinary(node.stat.czxid, in); @@ -140,14 +146,19 @@ namespace readBinary(node.stat.cversion, in); readBinary(node.stat.aversion, in); readBinary(node.stat.ephemeralOwner, in); - readBinary(node.stat.dataLength, in); + if (version < SnapshotVersion::V6) + { + int32_t data_length = 0; + readBinary(data_length, in); + } readBinary(node.stat.numChildren, in); readBinary(node.stat.pzxid, in); readBinary(node.seq_num, in); - if (version >= SnapshotVersion::V4) + if (version >= SnapshotVersion::V4 && version <= SnapshotVersion::V5) { - readBinary(node.size_bytes, in); + uint64_t size_bytes = 0; + readBinary(size_bytes, in); } } @@ -354,7 +365,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial const auto is_node_empty = [](const auto & node) { - return node.getData().empty() && node.stat == Coordination::Stat{}; + return node.getData().empty() && node.stat == KeeperStorage::Node::Stat{}; }; for (size_t nodes_read = 0; nodes_read < snapshot_container_size; ++nodes_read) @@ -398,9 +409,6 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial "If you still want to ignore it, you can set 'keeper_server.ignore_system_path_on_startup' to true", error_msg); } - - // we always ignore the written size for this node - node.recalculateSize(); } storage.container.insertOrReplace(path, node); @@ -417,7 +425,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial { auto parent_path = parentNodePath(itr.key); storage.container.updateValue( - parent_path, [version, path = itr.key](KeeperStorage::Node & value) { value.addChild(getBaseNodeName(path), /*update_size*/ version < SnapshotVersion::V4); }); + parent_path, [path = itr.key](KeeperStorage::Node & value) { value.addChild(getBaseNodeName(path)); }); } } diff --git a/src/Coordination/KeeperSnapshotManager.h b/src/Coordination/KeeperSnapshotManager.h index 9bb287b9276..6096ba318da 100644 --- a/src/Coordination/KeeperSnapshotManager.h +++ b/src/Coordination/KeeperSnapshotManager.h @@ -24,9 +24,10 @@ enum SnapshotVersion : uint8_t V3 = 3, /// compress snapshots with ZSTD codec V4 = 4, /// add Node size to snapshots V5 = 5, /// add ZXID and digest to snapshots + V6 = 6, /// remove is_sequential, per node size, data length }; -static constexpr auto CURRENT_SNAPSHOT_VERSION = SnapshotVersion::V5; +static constexpr auto CURRENT_SNAPSHOT_VERSION = SnapshotVersion::V6; /// What is stored in binary snapshot struct SnapshotDeserializationResult diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 41e6f5b5e2b..c128d7c2f98 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -24,7 +24,6 @@ #include #include -#include #include #include #include @@ -167,7 +166,7 @@ KeeperStorage::ResponsesForSessions processWatchesImpl( } // When this function is updated, update CURRENT_DIGEST_VERSION!! -uint64_t calculateDigest(std::string_view path, std::string_view data, const Coordination::Stat & stat) +uint64_t calculateDigest(std::string_view path, std::string_view data, const KeeperStorage::Node::Stat & stat) { SipHash hash; @@ -184,7 +183,7 @@ uint64_t calculateDigest(std::string_view path, std::string_view data, const Coo hash.update(stat.cversion); hash.update(stat.aversion); hash.update(stat.ephemeralOwner); - hash.update(stat.dataLength); + hash.update(data.length()); hash.update(stat.numChildren); hash.update(stat.pzxid); @@ -193,36 +192,56 @@ uint64_t calculateDigest(std::string_view path, std::string_view data, const Coo } +void KeeperStorage::Node::setResponseStat(Coordination::Stat & response_stat) const +{ + response_stat.czxid = stat.czxid; + response_stat.mzxid = stat.mzxid; + response_stat.ctime = stat.ctime; + response_stat.mtime = stat.mtime; + response_stat.version = stat.version; + response_stat.cversion = stat.cversion; + response_stat.aversion = stat.aversion; + response_stat.ephemeralOwner = stat.ephemeralOwner; + response_stat.dataLength = static_cast(data.size()); + response_stat.numChildren = stat.numChildren; + response_stat.pzxid = stat.pzxid; + +} + +uint64_t KeeperStorage::Node::sizeInBytes() const +{ + return sizeof(Node) + children.size() * sizeof(StringRef) + data.size(); +} + void KeeperStorage::Node::setData(String new_data) { - size_bytes = size_bytes - data.size() + new_data.size(); data = std::move(new_data); } -void KeeperStorage::Node::addChild(StringRef child_path, bool update_size) +void KeeperStorage::Node::addChild(StringRef child_path) { - if (update_size) [[likely]] - size_bytes += sizeof child_path; children.insert(child_path); } void KeeperStorage::Node::removeChild(StringRef child_path) { - size_bytes -= sizeof child_path; children.erase(child_path); } void KeeperStorage::Node::invalidateDigestCache() const { - cached_digest.reset(); + has_cached_digest = false; } UInt64 KeeperStorage::Node::getDigest(const std::string_view path) const { - if (!cached_digest) + if (!has_cached_digest) + { cached_digest = calculateDigest(path, data, stat); + has_cached_digest = true; + } - return *cached_digest; + return cached_digest; }; void KeeperStorage::Node::shallowCopy(const KeeperStorage::Node & other) @@ -233,13 +252,6 @@ void KeeperStorage::Node::shallowCopy(const KeeperStorage::Node & other) cached_digest = other.cached_digest; } -void KeeperStorage::Node::recalculateSize() -{ - size_bytes = sizeof(Node); - size_bytes += children.size() * sizeof(decltype(children)::value_type); - size_bytes += data.size(); -} - KeeperStorage::KeeperStorage( int64_t tick_time_ms, const String & superdigest_, const KeeperContextPtr & keeper_context_, const bool initialize_system_nodes) : session_expiry_queue(tick_time_ms), keeper_context(keeper_context_), superdigest(superdigest_) @@ -650,7 +662,6 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) path, std::move(operation.data), operation.stat, - operation.is_sequental, std::move(operation.acls))) onStorageInconsistency(); @@ -729,8 +740,7 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) bool KeeperStorage::createNode( const std::string & path, String data, - const Coordination::Stat & stat, - bool is_sequental, + const KeeperStorage::Node::Stat & stat, Coordination::ACLs node_acls) { auto parent_path = parentNodePath(path); @@ -753,7 +763,6 @@ bool KeeperStorage::createNode( created_node.acl_id = acl_id; created_node.stat = stat; created_node.setData(std::move(data)); - created_node.is_sequental = is_sequental; auto [map_key, _] = container.insert(path, created_node); /// Take child path from key owned by map. auto child_path = getBaseNodeName(map_key->getKey()); @@ -1012,7 +1021,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr new_deltas.emplace_back(std::string{parent_path}, zxid, KeeperStorage::UpdateNodeDelta{std::move(parent_update)}); - Coordination::Stat stat; + KeeperStorage::Node::Stat stat; stat.czxid = zxid; stat.mzxid = zxid; stat.pzxid = zxid; @@ -1022,13 +1031,12 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr stat.version = 0; stat.aversion = 0; stat.cversion = 0; - stat.dataLength = static_cast(request.data.length()); stat.ephemeralOwner = request.is_ephemeral ? session_id : 0; new_deltas.emplace_back( std::move(path_created), zxid, - KeeperStorage::CreateNodeDelta{stat, request.is_sequential, std::move(node_acls), request.data}); + KeeperStorage::CreateNodeDelta{stat, std::move(node_acls), request.data}); digest = storage.calculateNodesDigest(digest, new_deltas); return new_deltas; @@ -1126,7 +1134,7 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce } else { - response.stat = node_it->value.stat; + node_it->value.setResponseStat(response.stat); response.data = node_it->value.getData(); response.error = Coordination::Error::ZOK; } @@ -1285,7 +1293,7 @@ struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestPr } else { - response.stat = node_it->value.stat; + node_it->value.setResponseStat(response.stat); response.error = Coordination::Error::ZOK; } @@ -1345,7 +1353,6 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce value.stat.version++; value.stat.mzxid = zxid; value.stat.mtime = time; - value.stat.dataLength = static_cast(data.length()); value.setData(data); }, request.version}); @@ -1384,7 +1391,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce if (node_it == container.end()) onStorageInconsistency(); - response.stat = node_it->value.stat; + node_it->value.setResponseStat(response.stat); response.error = Coordination::Error::ZOK; return response_ptr; @@ -1481,7 +1488,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc response.names.push_back(child.toString()); } - response.stat = node_it->value.stat; + node_it->value.setResponseStat(response.stat); response.error = Coordination::Error::ZOK; } @@ -1675,7 +1682,7 @@ struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestPr auto node_it = storage.container.find(request.path); if (node_it == storage.container.end()) onStorageInconsistency(); - response.stat = node_it->value.stat; + node_it->value.setResponseStat(response.stat); response.error = Coordination::Error::ZOK; return response_ptr; @@ -1729,7 +1736,7 @@ struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestPr } else { - response.stat = node_it->value.stat; + node_it->value.setResponseStat(response.stat); response.acl = storage.acl_map.convertNumber(node_it->value.acl_id); } diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index ec5df74efb6..01c1413a884 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -30,24 +30,47 @@ struct KeeperStorageSnapshot; class KeeperStorage { public: + /// Node should have as minimal size as possible to reduce memory footprint + /// of stored nodes + /// New fields should be added to the struct only if it's really necessary struct Node { - uint64_t acl_id = 0; /// 0 -- no ACL by default - bool is_sequental = false; - Coordination::Stat stat{}; - int32_t seq_num = 0; - uint64_t size_bytes; // save size to avoid calculate every time + /// to reduce size of the Node struct we use a custom Stat without dataLength + struct Stat + { + int64_t czxid{0}; + int64_t mzxid{0}; + int64_t ctime{0}; + int64_t mtime{0}; + int32_t version{0}; + int32_t cversion{0}; + int32_t aversion{0}; + int32_t numChildren{0}; /// NOLINT + int64_t ephemeralOwner{0}; /// NOLINT + int64_t pzxid{0}; - Node() : size_bytes(sizeof(Node)) { } + bool operator==(const Stat &) const = default; + }; + + uint64_t acl_id = 0; /// 0 -- no ACL by default + Stat stat{}; + int32_t seq_num = 0; + + /// we cannot use `std::optional because we want to + /// pack the boolean with seq_num above + mutable bool has_cached_digest = false; + mutable uint64_t cached_digest = 0; + + void setResponseStat(Coordination::Stat & response_stat) const; /// Object memory size - uint64_t sizeInBytes() const { return size_bytes; } + uint64_t sizeInBytes() const; void setData(String new_data); const auto & getData() const noexcept { return data; } - void addChild(StringRef child_path, bool update_size = true); + void addChild(StringRef child_path); void removeChild(StringRef child_path); @@ -63,13 +86,9 @@ public: // copy only necessary information for preprocessing and digest calculation // (e.g. we don't need to copy list of children) void shallowCopy(const Node & other); - - void recalculateSize(); - private: String data; ChildrenSet children{}; - mutable std::optional cached_digest; }; enum DigestVersion : uint8_t @@ -158,8 +177,7 @@ public: // - quickly commit the changes to the storage struct CreateNodeDelta { - Coordination::Stat stat; - bool is_sequental; + KeeperStorage::Node::Stat stat; Coordination::ACLs acls; String data; }; @@ -324,8 +342,7 @@ public: bool createNode( const std::string & path, String data, - const Coordination::Stat & stat, - bool is_sequental, + const KeeperStorage::Node::Stat & stat, Coordination::ACLs node_acls); // Remove node in the storage diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index 093126237ef..b12e797d84e 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -3,30 +3,93 @@ #include #include #include -#include #include -#include namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + template struct ListNode { StringRef key; V value; - /// Monotonically increasing version info for snapshot - size_t version{0}; - bool active_in_map{true}; - bool free_key{false}; + /// |* * ****** | + /// ^ ^ ^ + /// active_in_map free_key version + /// (1 byte) (1 byte) (6 bytes) + uint64_t node_metadata = 0; + + void setInactiveInMap() + { + node_metadata &= ~active_in_map_mask; + } + + void setActiveInMap() + { + node_metadata |= active_in_map_mask; + } + + bool isActiveInMap() + { + return node_metadata & active_in_map_mask; + } + + void setFreeKey() + { + node_metadata |= free_key_mask; + } + + bool getFreeKey() + { + return node_metadata & free_key_mask; + } + + uint64_t getVersion() + { + return node_metadata & version_mask; + } + + void setVersion(uint64_t version) + { + if (version > version_mask) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Snapshot version {} is larger than maximum allowed value {}", + version, + ListNode::version_mask); + + node_metadata &= ~version_mask; + node_metadata |= version; + } + + static constexpr uint64_t active_in_map_mask = static_cast(1) << 63; + static constexpr uint64_t free_key_mask = static_cast(1) << 62; + static constexpr uint64_t version_mask = ~(static_cast(3) << 62); }; template class SnapshotableHashTable { private: + struct GlobalArena + { + char * alloc(const size_t size) + { + return new char[size]; + } + + void free(const char * ptr, size_t /*size*/) + { + delete [] ptr; + } + }; using ListElem = ListNode; using List = std::list; @@ -39,7 +102,12 @@ private: /// Allows to avoid additional copies in updateValue function size_t current_version{0}; size_t snapshot_up_to_version{0}; - ArenaWithFreeLists arena; + + /// Arena used for keys + /// we don't use std::string because it uses 24 bytes (because of SSO) + /// we want to always allocate the key on heap and use StringRef to it + GlobalArena arena; + /// Collect invalid iterators to avoid traversing the whole list std::vector snapshot_invalid_iters; @@ -132,11 +200,13 @@ public: if (!it) { - ListElem elem{copyStringInArena(arena, key), value, current_version}; + ListElem elem{copyStringInArena(arena, key), value}; + elem.setVersion(current_version); auto itr = list.insert(list.end(), std::move(elem)); bool inserted; map.emplace(itr->key, it, inserted, hash_value); - assert(inserted); + itr->setActiveInMap(); + chassert(inserted); it->getMapped() = itr; updateDataSize(INSERT, key.size(), value.sizeInBytes(), 0); @@ -154,11 +224,13 @@ public: if (it == map.end()) { - ListElem elem{copyStringInArena(arena, key), value, current_version}; + ListElem elem{copyStringInArena(arena, key), value}; + elem.setVersion(current_version); auto itr = list.insert(list.end(), std::move(elem)); bool inserted; map.emplace(itr->key, it, inserted, hash_value); - assert(inserted); + itr->setActiveInMap(); + chassert(inserted); it->getMapped() = itr; } else @@ -166,8 +238,9 @@ public: auto list_itr = it->getMapped(); if (snapshot_mode) { - ListElem elem{list_itr->key, value, current_version}; - list_itr->active_in_map = false; + ListElem elem{list_itr->key, value}; + elem.setVersion(current_version); + list_itr->setInactiveInMap(); auto new_list_itr = list.insert(list.end(), std::move(elem)); it->getMapped() = new_list_itr; snapshot_invalid_iters.push_back(list_itr); @@ -190,9 +263,9 @@ public: uint64_t old_data_size = list_itr->value.sizeInBytes(); if (snapshot_mode) { - list_itr->active_in_map = false; + list_itr->setInactiveInMap(); snapshot_invalid_iters.push_back(list_itr); - list_itr->free_key = true; + list_itr->setFreeKey(); map.erase(it->getKey()); } else @@ -215,7 +288,7 @@ public: { size_t hash_value = map.hash(key); auto it = map.find(key, hash_value); - assert(it != map.end()); + chassert(it != map.end()); auto list_itr = it->getMapped(); uint64_t old_value_size = list_itr->value.sizeInBytes(); @@ -228,13 +301,14 @@ public: /// We in snapshot mode but updating some node which is already more /// fresh than snapshot distance. So it will not participate in /// snapshot and we don't need to copy it. - if (list_itr->version <= snapshot_up_to_version) + if (list_itr->getVersion() <= snapshot_up_to_version) { auto elem_copy = *(list_itr); - list_itr->active_in_map = false; + list_itr->setInactiveInMap(); snapshot_invalid_iters.push_back(list_itr); updater(elem_copy.value); - elem_copy.version = current_version; + + elem_copy.setVersion(current_version); auto itr = list.insert(list.end(), std::move(elem_copy)); it->getMapped() = itr; ret = itr; @@ -269,17 +343,17 @@ public: const V & getValue(StringRef key) const { auto it = map.find(key); - assert(it); + chassert(it); return it->getMapped()->value; } void clearOutdatedNodes() { - for (auto & itr: snapshot_invalid_iters) + for (auto & itr : snapshot_invalid_iters) { - assert(!itr->active_in_map); + chassert(!itr->isActiveInMap()); updateDataSize(CLEAR_OUTDATED_NODES, itr->key.size, itr->value.sizeInBytes(), 0); - if (itr->free_key) + if (itr->getFreeKey()) arena.free(const_cast(itr->key.data), itr->key.size); list.erase(itr); } @@ -327,13 +401,12 @@ public: approximate_data_size = 0; for (auto & node : list) { - node.value.recalculateSize(); approximate_data_size += node.key.size; approximate_data_size += node.value.sizeInBytes(); } } - uint64_t keyArenaSize() const { return arena.allocatedBytes(); } + uint64_t keyArenaSize() const { return 0; } iterator begin() { return list.begin(); } const_iterator begin() const { return list.cbegin(); } diff --git a/src/Coordination/ZooKeeperDataReader.cpp b/src/Coordination/ZooKeeperDataReader.cpp index 3c1550f08c8..b55ebef327f 100644 --- a/src/Coordination/ZooKeeperDataReader.cpp +++ b/src/Coordination/ZooKeeperDataReader.cpp @@ -120,7 +120,6 @@ int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, Poco::L Coordination::read(node.stat.pzxid, in); if (!path.empty()) { - node.stat.dataLength = static_cast(node.getData().length()); node.seq_num = node.stat.cversion; storage.container.insertOrReplace(path, node); diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index dd19f0b9967..f4e73149589 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1367,11 +1367,11 @@ TEST_P(CoordinationTest, SnapshotableHashMapTrySnapshot) auto itr = map_snp.begin(); EXPECT_EQ(itr->key, "/hello"); EXPECT_EQ(itr->value, 7); - EXPECT_EQ(itr->active_in_map, false); + EXPECT_EQ(itr->isActiveInMap(), false); itr = std::next(itr); EXPECT_EQ(itr->key, "/hello"); EXPECT_EQ(itr->value, 554); - EXPECT_EQ(itr->active_in_map, true); + EXPECT_EQ(itr->isActiveInMap(), true); itr = std::next(itr); EXPECT_EQ(itr, map_snp.end()); for (int i = 0; i < 5; ++i) @@ -1387,7 +1387,7 @@ TEST_P(CoordinationTest, SnapshotableHashMapTrySnapshot) { EXPECT_EQ(itr->key, "/hello" + std::to_string(i)); EXPECT_EQ(itr->value, i); - EXPECT_EQ(itr->active_in_map, true); + EXPECT_EQ(itr->isActiveInMap(), true); itr = std::next(itr); } @@ -1401,7 +1401,7 @@ TEST_P(CoordinationTest, SnapshotableHashMapTrySnapshot) { EXPECT_EQ(itr->key, "/hello" + std::to_string(i)); EXPECT_EQ(itr->value, i); - EXPECT_EQ(itr->active_in_map, i != 3 && i != 2); + EXPECT_EQ(itr->isActiveInMap(), i != 3 && i != 2); itr = std::next(itr); } map_snp.clearOutdatedNodes(); @@ -1411,19 +1411,19 @@ TEST_P(CoordinationTest, SnapshotableHashMapTrySnapshot) itr = map_snp.begin(); EXPECT_EQ(itr->key, "/hello"); EXPECT_EQ(itr->value, 554); - EXPECT_EQ(itr->active_in_map, true); + EXPECT_EQ(itr->isActiveInMap(), true); itr = std::next(itr); EXPECT_EQ(itr->key, "/hello0"); EXPECT_EQ(itr->value, 0); - EXPECT_EQ(itr->active_in_map, true); + EXPECT_EQ(itr->isActiveInMap(), true); itr = std::next(itr); EXPECT_EQ(itr->key, "/hello1"); EXPECT_EQ(itr->value, 1); - EXPECT_EQ(itr->active_in_map, true); + EXPECT_EQ(itr->isActiveInMap(), true); itr = std::next(itr); EXPECT_EQ(itr->key, "/hello4"); EXPECT_EQ(itr->value, 4); - EXPECT_EQ(itr->active_in_map, true); + EXPECT_EQ(itr->isActiveInMap(), true); itr = std::next(itr); EXPECT_EQ(itr, map_snp.end()); map_snp.disableSnapshotMode(); diff --git a/utils/keeper-data-dumper/main.cpp b/utils/keeper-data-dumper/main.cpp index aa8c0efbb26..e06b301edbf 100644 --- a/utils/keeper-data-dumper/main.cpp +++ b/utils/keeper-data-dumper/main.cpp @@ -31,7 +31,7 @@ void dumpMachine(std::shared_ptr machine) ", czxid: " << value.stat.czxid << ", mzxid: " << value.stat.mzxid << ", numChildren: " << value.stat.numChildren << - ", dataLength: " << value.stat.dataLength << + ", dataLength: " << value.getData().size() << "}" << std::endl; std::cout << "\tData: " << storage.container.getValue(key).getData() << std::endl; From 7635da91a83377f2f532880c31d9607adb1c5a26 Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Fri, 19 Jan 2024 21:55:41 +0100 Subject: [PATCH 085/264] Update rename.md Remove wrong note. RENAME is supported by most of database engines (including old Ordinary). --- docs/en/sql-reference/statements/rename.md | 4 ---- 1 file changed, 4 deletions(-) diff --git a/docs/en/sql-reference/statements/rename.md b/docs/en/sql-reference/statements/rename.md index bb62cc3af1c..667ccbc6c93 100644 --- a/docs/en/sql-reference/statements/rename.md +++ b/docs/en/sql-reference/statements/rename.md @@ -9,10 +9,6 @@ sidebar_label: RENAME Renames databases, tables, or dictionaries. Several entities can be renamed in a single query. Note that the `RENAME` query with several entities is non-atomic operation. To swap entities names atomically, use the [EXCHANGE](./exchange.md) statement. -:::note -The `RENAME` query is supported by the [Atomic](../../engines/database-engines/atomic.md) database engine only. -::: - **Syntax** ```sql From f3a1a2f3ad48d2d6fbf54300fd249b5f1fe2715e Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sat, 20 Jan 2024 15:02:13 +0100 Subject: [PATCH 086/264] Fix build --- src/Interpreters/Cache/FileSegment.cpp | 4 ++-- src/Interpreters/Cache/WriteBufferToFileSegment.cpp | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 20a0f32dbd5..f79add2e10a 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -347,7 +347,7 @@ void FileSegment::write(const char * from, size_t size, size_t offset) assertNotDetachedUnlocked(lock); } - const auto file_segment_path = getPathInLocalCache(); + const auto file_segment_path = getPath(); { if (download_state != State::DOWNLOADING) @@ -816,7 +816,7 @@ bool FileSegment::assertCorrectnessUnlocked(const FileSegmentGuard::Lock & lock) chassert(entry->offset == offset()); }; - const auto file_path = getPathInLocalCache(); + const auto file_path = getPath(); if (segment_kind != FileSegmentKind::Temporary) { std::lock_guard lk(write_mutex); diff --git a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp index bcf202e3a26..77d8f953903 100644 --- a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp +++ b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp @@ -58,8 +58,8 @@ void WriteBufferToFileSegment::nextImpl() reserve_stat_msg += fmt::format("{} hold {}, can release {}; ", toString(kind), ReadableSize(stat.non_releasable_size), ReadableSize(stat.releasable_size)); - if (std::filesystem::exists(file_segment->getPathInLocalCache())) - std::filesystem::remove(file_segment->getPathInLocalCache()); + if (std::filesystem::exists(file_segment->getPath())) + std::filesystem::remove(file_segment->getPath()); throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, "Failed to reserve {} bytes for {}: {}(segment info: {})", bytes_to_write, From 0aa6164294c9b9ff0906c049a67059c7aef3c111 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 21 Jan 2024 21:08:08 +0100 Subject: [PATCH 087/264] Fix test 02932_kill_query_sleep when running with query cache. --- tests/queries/0_stateless/02932_kill_query_sleep.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02932_kill_query_sleep.sh b/tests/queries/0_stateless/02932_kill_query_sleep.sh index 84e84204aa1..5ae47755722 100755 --- a/tests/queries/0_stateless/02932_kill_query_sleep.sh +++ b/tests/queries/0_stateless/02932_kill_query_sleep.sh @@ -10,7 +10,7 @@ function wait_query_started() local query_id="$1" timeout=60 start=$EPOCHSECONDS - while [[ $($CLICKHOUSE_CLIENT --query="SELECT count() FROM system.processes WHERE query_id='$query_id'") == 0 ]]; do + while [[ $($CLICKHOUSE_CLIENT --query="SELECT count() FROM system.processes WHERE query_id='$query_id' SETTINGS use_query_cache=0") == 0 ]]; do if ((EPOCHSECONDS-start > timeout )); then echo "Timeout while waiting for query $query_id to start" exit 1 @@ -26,7 +26,7 @@ function kill_query() $CLICKHOUSE_CLIENT --query "KILL QUERY WHERE query_id='$query_id'" >/dev/null timeout=60 start=$EPOCHSECONDS - while [[ $($CLICKHOUSE_CLIENT --query="SELECT count() FROM system.processes WHERE query_id='$query_id'") != 0 ]]; do + while [[ $($CLICKHOUSE_CLIENT --query="SELECT count() FROM system.processes WHERE query_id='$query_id' SETTINGS use_query_cache=0") != 0 ]]; do if ((EPOCHSECONDS-start > timeout )); then echo "Timeout while waiting for query $query_id to cancel" exit 1 @@ -49,4 +49,4 @@ echo "Cancelling query" kill_query "$sleep_query_id" $CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS;" -$CLICKHOUSE_CLIENT --query "SELECT exception FROM system.query_log WHERE query_id='$sleep_query_id' AND current_database = currentDatabase()" | grep -oF "QUERY_WAS_CANCELLED" +$CLICKHOUSE_CLIENT --query "SELECT exception FROM system.query_log WHERE query_id='$sleep_query_id' AND current_database = '$CLICKHOUSE_DATABASE'" | grep -oF "QUERY_WAS_CANCELLED" From 02b178cc9c9b38a30344e2ddd896c1eaf429c3de Mon Sep 17 00:00:00 2001 From: MochiXu Date: Mon, 22 Jan 2024 11:08:03 +0800 Subject: [PATCH 088/264] fix drop inverted index --- src/Storages/MergeTree/MutateTask.cpp | 22 ++++++++++++++-------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index e4070aa8262..8ed8b8bba4c 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -674,15 +674,21 @@ static NameToNameVector collectFilesForRenames( { if (command.type == MutationCommand::Type::DROP_INDEX) { - if (source_part->checksums.has(INDEX_FILE_PREFIX + command.column_name + ".idx2")) + const std::vector suffixes = {".idx2", ".idx", ".gin_dict", ".gin_post", ".gin_seg", ".gin_sid"}; + + for (const auto& suffix : suffixes) { - add_rename(INDEX_FILE_PREFIX + command.column_name + ".idx2", ""); - add_rename(INDEX_FILE_PREFIX + command.column_name + mrk_extension, ""); - } - else if (source_part->checksums.has(INDEX_FILE_PREFIX + command.column_name + ".idx")) - { - add_rename(INDEX_FILE_PREFIX + command.column_name + ".idx", ""); - add_rename(INDEX_FILE_PREFIX + command.column_name + mrk_extension, ""); + String filename = INDEX_FILE_PREFIX + command.column_name + suffix; + + if ((suffix == ".idx2" || suffix == ".idx") && source_part->checksums.has(filename)) + { + add_rename(filename, ""); + add_rename(INDEX_FILE_PREFIX + command.column_name + mrk_extension, ""); + } + else if (source_part->checksums.has(filename)) + { + add_rename(filename, ""); + } } } else if (command.type == MutationCommand::Type::DROP_PROJECTION) From fdf8008804054238affd29bd99028cf280340b51 Mon Sep 17 00:00:00 2001 From: skyoct Date: Mon, 22 Jan 2024 10:27:35 +0000 Subject: [PATCH 089/264] Fix redirect retry --- src/IO/ReadWriteBufferFromHTTP.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index 6dd6269e16f..ea18b369c52 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -216,6 +216,9 @@ void ReadWriteBufferFromHTTPBase::getHeadResponse(Poco::Net if (i == settings.http_max_tries - 1 || !isRetriableError(response.getStatus())) throw; + if (e.code() == ErrorCodes::TOO_MANY_REDIRECTS) + throw; + LOG_ERROR(log, "Failed to make HTTP_HEAD request to {}. Error: {}", uri.toString(), e.displayText()); } } @@ -545,6 +548,9 @@ bool ReadWriteBufferFromHTTPBase::nextImpl() if (e.code() == POCO_EMFILE) throw; + if (e.code() == ErrorCodes::TOO_MANY_REDIRECTS) + throw; + /** Retry request unconditionally if nothing has been read yet. * Otherwise if it is GET method retry with range header. */ From 274c128bd78f91cac46a3b9d89b4990c71e56608 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 22 Jan 2024 08:23:04 +0000 Subject: [PATCH 090/264] Free memory --- src/Coordination/SnapshotableHashTable.h | 12 +++++++----- src/Coordination/tests/gtest_coordination.cpp | 5 +---- 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index b12e797d84e..ac8d36745c2 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -1,7 +1,6 @@ #pragma once #include #include -#include #include #include @@ -60,10 +59,7 @@ struct ListNode { if (version > version_mask) throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Snapshot version {} is larger than maximum allowed value {}", - version, - ListNode::version_mask); + ErrorCodes::LOGICAL_ERROR, "Snapshot version {} is larger than maximum allowed value {}", version, version_mask); node_metadata &= ~version_mask; node_metadata |= version; @@ -193,6 +189,11 @@ public: using const_iterator = typename List::const_iterator; using ValueUpdater = std::function; + ~SnapshotableHashTable() + { + clear(); + } + std::pair insert(const std::string & key, const V & value) { size_t hash_value = map.hash(key); @@ -362,6 +363,7 @@ public: void clear() { + clearOutdatedNodes(); map.clear(); for (auto itr = list.begin(); itr != list.end(); ++itr) arena.free(const_cast(itr->key.data), itr->key.size); diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 8559accbaf4..c981085359e 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -45,10 +45,7 @@ struct ChangelogDirTest bool drop; explicit ChangelogDirTest(std::string path_, bool drop_ = true) : path(path_), drop(drop_) { - if (fs::exists(path)) - { - EXPECT_TRUE(false) << "Path " << path << " already exists, remove it to run test"; - } + EXPECT_FALSE(fs::exists(path)) << "Path " << path << " already exists, remove it to run test"; fs::create_directory(path); } From 3faf4087e3a724403b0b8e84cc00e158073023dc Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 22 Jan 2024 11:48:37 +0100 Subject: [PATCH 091/264] Fix --- src/Interpreters/Context.cpp | 6 ++++++ src/Interpreters/Context.h | 2 +- src/Interpreters/executeQuery.cpp | 2 +- 3 files changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 7e89c794712..fbc1b7086f9 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1644,6 +1644,12 @@ void Context::addQueryAccessInfo(const QualifiedProjectionName & qualified_proje "{}.{}", qualified_projection_name.storage_id.getFullTableName(), backQuoteIfNeed(qualified_projection_name.projection_name))); } +Context::QueryFactoriesInfo Context::getQueryFactoriesInfo() const +{ + std::lock_guard lock(query_factories_info.mutex); + return query_factories_info; +} + void Context::addQueryFactoriesInfo(QueryLogFactories factory_type, const String & created_object) const { if (isGlobalContext()) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 65566876a80..01ccbbdc8c0 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -727,7 +727,7 @@ public: TableFunction }; - const QueryFactoriesInfo & getQueryFactoriesInfo() const { return query_factories_info; } + QueryFactoriesInfo getQueryFactoriesInfo() const; void addQueryFactoriesInfo(QueryLogFactories factory_type, const String & created_object) const; /// For table functions s3/file/url/hdfs/input we can use structure from diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 4b5a6a84e17..533d58aaa8f 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -257,7 +257,7 @@ addStatusInfoToQueryLogElement(QueryLogElement & element, const QueryStatusInfo element.query_projections.insert(access_info.projections.begin(), access_info.projections.end()); element.query_views.insert(access_info.views.begin(), access_info.views.end()); - const auto & factories_info = context_ptr->getQueryFactoriesInfo(); + const auto factories_info = context_ptr->getQueryFactoriesInfo(); element.used_aggregate_functions = factories_info.aggregate_functions; element.used_aggregate_function_combinators = factories_info.aggregate_function_combinators; element.used_database_engines = factories_info.database_engines; From 4b4eac7c0604b1d5eac9a8a821f96d32474d1e5d Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 22 Jan 2024 13:29:53 +0100 Subject: [PATCH 092/264] Fix --- src/Interpreters/Context.cpp | 1 - src/Interpreters/Context.h | 19 ------------------- 2 files changed, 20 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index fbc1b7086f9..b118abd1286 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1646,7 +1646,6 @@ void Context::addQueryAccessInfo(const QualifiedProjectionName & qualified_proje Context::QueryFactoriesInfo Context::getQueryFactoriesInfo() const { - std::lock_guard lock(query_factories_info.mutex); return query_factories_info; } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 01ccbbdc8c0..b58997eb3a9 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -373,25 +373,6 @@ protected: QueryFactoriesInfo(QueryFactoriesInfo && rhs) = delete; - QueryFactoriesInfo & operator=(QueryFactoriesInfo rhs) - { - swap(rhs); - return *this; - } - - void swap(QueryFactoriesInfo & rhs) - { - std::swap(aggregate_functions, rhs.aggregate_functions); - std::swap(aggregate_function_combinators, rhs.aggregate_function_combinators); - std::swap(database_engines, rhs.database_engines); - std::swap(data_type_families, rhs.data_type_families); - std::swap(dictionaries, rhs.dictionaries); - std::swap(formats, rhs.formats); - std::swap(functions, rhs.functions); - std::swap(storages, rhs.storages); - std::swap(table_functions, rhs.table_functions); - } - std::unordered_set aggregate_functions; std::unordered_set aggregate_function_combinators; std::unordered_set database_engines; From 66cd61eb7e4dd572e458e096babdace3ae0c7374 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 22 Jan 2024 14:17:25 +0100 Subject: [PATCH 093/264] remove flaky test --- .../tests/gtest_generate_random_by_regexp.cpp | 35 ------------------- 1 file changed, 35 deletions(-) diff --git a/src/Common/tests/gtest_generate_random_by_regexp.cpp b/src/Common/tests/gtest_generate_random_by_regexp.cpp index 063257bdfd7..acf49cc30eb 100644 --- a/src/Common/tests/gtest_generate_random_by_regexp.cpp +++ b/src/Common/tests/gtest_generate_random_by_regexp.cpp @@ -64,38 +64,3 @@ TEST(GenerateRandomString, FullRange) std::cerr << " +1 "; std::cerr << "all possible letters, ok" << std::endl; } - -UInt64 elapsed(DB::ObjectStorageKeysGeneratorPtr generator) -{ - String path = "some_path"; - - Stopwatch watch; - - for (int i = 0; i < 100000; ++i) - { - [[ maybe_unused ]] auto result = generator->generate(path).serialize(); - } - - return watch.elapsedMicroseconds(); -} - -TEST(ObjectStorageKey, Performance) -{ - auto elapsed_old = elapsed(DB::createObjectStorageKeysGeneratorByPrefix( - "xx-xx-xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx/mergetree/")); - std::cerr << "old: " << elapsed_old << std::endl; - - auto elapsed_new = elapsed(DB::createObjectStorageKeysGeneratorByTemplate( - "xx-xx-xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx/mergetree/[a-z]{3}/[a-z]{29}")); - std::cerr << "new: " << elapsed_new << std::endl; - - if (elapsed_new > elapsed_old) - { - if (elapsed_new > elapsed_old) - std::cerr << "slow ratio: +" << float(elapsed_new) / elapsed_old << std::endl; - else - std::cerr << "fast ratio: " << float(elapsed_old) / elapsed_new << std::endl; - ASSERT_LT(elapsed_new, 1.5 * elapsed_old); - } - -} From b4dcd6755a8a2384e5937991e0656058aed4f95a Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 22 Jan 2024 13:41:58 +0000 Subject: [PATCH 094/264] Use ColumnConst instead of Field in IColumn::createWithOffsets --- src/Columns/ColumnConst.cpp | 22 +++++++++++++++++++ src/Columns/ColumnConst.h | 5 +++++ src/Columns/ColumnNullable.cpp | 16 ++++++-------- src/Columns/ColumnNullable.h | 2 +- src/Columns/ColumnObject.cpp | 3 ++- src/Columns/ColumnSparse.cpp | 3 ++- src/Columns/ColumnVector.cpp | 5 +++-- src/Columns/ColumnVector.h | 2 +- src/Columns/IColumn.cpp | 6 ++--- src/Columns/IColumn.h | 5 +++-- src/Functions/IFunction.cpp | 4 ++-- .../0_stateless/02941_variant_type_alters.sh | 1 + 12 files changed, 52 insertions(+), 22 deletions(-) diff --git a/src/Columns/ColumnConst.cpp b/src/Columns/ColumnConst.cpp index 9aa0f5cfa49..6e5a3c45c4e 100644 --- a/src/Columns/ColumnConst.cpp +++ b/src/Columns/ColumnConst.cpp @@ -159,4 +159,26 @@ void ColumnConst::compareColumn( std::fill(compare_results.begin(), compare_results.end(), res); } +ColumnConst::Ptr createColumnConst(const ColumnPtr & column, Field value) +{ + auto data = column->cloneEmpty(); + data->insert(value); + return ColumnConst::create(std::move(data), 1); +} + +ColumnConst::Ptr createColumnConst(const ColumnPtr & column, size_t const_value_index) +{ + auto data = column->cloneEmpty(); + data->insertFrom(*column, const_value_index); + return ColumnConst::create(std::move(data), 1); +} + +ColumnConst::Ptr createColumnConstWithDefaultValue(const ColumnPtr & column) +{ + auto data = column->cloneEmpty(); + data->insertDefault(); + return ColumnConst::create(std::move(data), 1); +} + + } diff --git a/src/Columns/ColumnConst.h b/src/Columns/ColumnConst.h index dc84e0c2402..3c646a62795 100644 --- a/src/Columns/ColumnConst.h +++ b/src/Columns/ColumnConst.h @@ -292,4 +292,9 @@ public: bool isCollationSupported() const override { return data->isCollationSupported(); } }; +ColumnConst::Ptr createColumnConst(const ColumnPtr & column, Field value); +ColumnConst::Ptr createColumnConst(const ColumnPtr & column, size_t const_value_index); +ColumnConst::Ptr createColumnConstWithDefaultValue(const ColumnPtr &column); + + } diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 25b0e35e15e..c0b13204b8e 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -827,24 +827,22 @@ void ColumnNullable::checkConsistency() const "Logical error: Sizes of nested column and null map of Nullable column are not equal"); } -ColumnPtr ColumnNullable::createWithOffsets(const IColumn::Offsets & offsets, const Field & default_field, size_t total_rows, size_t shift) const +ColumnPtr ColumnNullable::createWithOffsets(const IColumn::Offsets & offsets, const ColumnConst & column_with_default_value, size_t total_rows, size_t shift) const { ColumnPtr new_values; ColumnPtr new_null_map; - if (default_field.getType() == Field::Types::Null) + const ColumnNullable & nullable_column_with_default_value = assert_cast(column_with_default_value.getDataColumn()); + if (nullable_column_with_default_value.isNullAt(0)) { - auto default_column = nested_column->cloneEmpty(); - default_column->insertDefault(); - /// Value in main column, when null map is 1 is implementation defined. So, take any value. - new_values = nested_column->createWithOffsets(offsets, (*default_column)[0], total_rows, shift); - new_null_map = null_map->createWithOffsets(offsets, Field(1u), total_rows, shift); + new_values = nested_column->createWithOffsets(offsets, *createColumnConstWithDefaultValue(nested_column), total_rows, shift); + new_null_map = null_map->createWithOffsets(offsets, *createColumnConst(null_map, Field(1u)), total_rows, shift); } else { - new_values = nested_column->createWithOffsets(offsets, default_field, total_rows, shift); - new_null_map = null_map->createWithOffsets(offsets, Field(0u), total_rows, shift); + new_values = nested_column->createWithOffsets(offsets, *ColumnConst::create(nullable_column_with_default_value.getNestedColumnPtr(), 1), total_rows, shift); + new_null_map = null_map->createWithOffsets(offsets, *createColumnConst(null_map, Field(0u)), total_rows, shift); } return ColumnNullable::create(new_values, new_null_map); diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index 60c7750f8fc..3e04ba8a180 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -167,7 +167,7 @@ public: getIndicesOfNonDefaultRowsImpl(indices, from, limit); } - ColumnPtr createWithOffsets(const Offsets & offsets, const Field & default_field, size_t total_rows, size_t shift) const override; + ColumnPtr createWithOffsets(const Offsets & offsets, const ColumnConst & column_with_default_value, size_t total_rows, size_t shift) const override; bool isNullable() const override { return true; } bool isFixedAndContiguous() const override { return false; } diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index f7176568a1b..0ec9c616bab 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -475,7 +476,7 @@ void ColumnObject::Subcolumn::finalize() { auto values = part->index(*offsets, offsets->size()); values = castColumn({values, from_type, ""}, to_type); - part = values->createWithOffsets(offsets_data, to_type->getDefault(), part_size, /*shift=*/ 0); + part = values->createWithOffsets(offsets_data, *createColumnConstWithDefaultValue(result_column->getPtr()), part_size, /*shift=*/ 0); } } diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 02e6e9e56b4..eeeec912ce8 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -130,7 +131,7 @@ StringRef ColumnSparse::getDataAt(size_t n) const ColumnPtr ColumnSparse::convertToFullColumnIfSparse() const { - return values->createWithOffsets(getOffsetsData(), (*values)[0], _size, /*shift=*/ 1); + return values->createWithOffsets(getOffsetsData(), *createColumnConst(values, 0), _size, /*shift=*/ 1); } void ColumnSparse::insertSingleValue(const Inserter & inserter) diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index b1cf449dfde..3aadc530878 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -940,7 +941,7 @@ ColumnPtr ColumnVector::compress() const } template -ColumnPtr ColumnVector::createWithOffsets(const IColumn::Offsets & offsets, const Field & default_field, size_t total_rows, size_t shift) const +ColumnPtr ColumnVector::createWithOffsets(const IColumn::Offsets & offsets, const ColumnConst & column_with_default_value, size_t total_rows, size_t shift) const { if (offsets.size() + shift != size()) throw Exception(ErrorCodes::LOGICAL_ERROR, @@ -949,7 +950,7 @@ ColumnPtr ColumnVector::createWithOffsets(const IColumn::Offsets & offsets, c auto res = this->create(); auto & res_data = res->getData(); - T default_value = static_cast(default_field.safeGet()); + T default_value = assert_cast &>(column_with_default_value.getDataColumn()).getElement(0); res_data.resize_fill(total_rows, default_value); for (size_t i = 0; i < offsets.size(); ++i) res_data[offsets[i]] = data[i + shift]; diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index fab2d5f06aa..652cc1f5ff9 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -379,7 +379,7 @@ public: return this->template getIndicesOfNonDefaultRowsImpl(indices, from, limit); } - ColumnPtr createWithOffsets(const IColumn::Offsets & offsets, const Field & default_field, size_t total_rows, size_t shift) const override; + ColumnPtr createWithOffsets(const IColumn::Offsets & offsets, const ColumnConst & column_with_default_value, size_t total_rows, size_t shift) const override; ColumnPtr compress() const override; diff --git a/src/Columns/IColumn.cpp b/src/Columns/IColumn.cpp index 82dc82e0bd9..d7f83b822d2 100644 --- a/src/Columns/IColumn.cpp +++ b/src/Columns/IColumn.cpp @@ -35,7 +35,7 @@ void IColumn::insertFrom(const IColumn & src, size_t n) insert(src[n]); } -ColumnPtr IColumn::createWithOffsets(const Offsets & offsets, const Field & default_field, size_t total_rows, size_t shift) const +ColumnPtr IColumn::createWithOffsets(const Offsets & offsets, const ColumnConst & column_with_default_value, size_t total_rows, size_t shift) const { if (offsets.size() + shift != size()) throw Exception(ErrorCodes::LOGICAL_ERROR, @@ -51,14 +51,14 @@ ColumnPtr IColumn::createWithOffsets(const Offsets & offsets, const Field & defa current_offset = offsets[i]; if (offsets_diff > 1) - res->insertMany(default_field, offsets_diff - 1); + res->insertManyFrom(column_with_default_value.getDataColumn(), 0, offsets_diff - 1); res->insertFrom(*this, i + shift); } ssize_t offsets_diff = static_cast(total_rows) - current_offset; if (offsets_diff > 1) - res->insertMany(default_field, offsets_diff - 1); + res->insertManyFrom(column_with_default_value.getDataColumn(), 0, offsets_diff - 1); return res; } diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 3f866e6213d..1dcd3acdd19 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -34,6 +34,7 @@ class Arena; class ColumnGathererStream; class Field; class WeakHash32; +class ColumnConst; /* * Represents a set of equal ranges in previous column to perform sorting in current column. @@ -462,10 +463,10 @@ public: /// Returns column with @total_size elements. /// In result column values from current column are at positions from @offsets. - /// Other values are filled by @default_value. + /// Other values are filled by value from @column_with_default_value. /// @shift means how much rows to skip from the beginning of current column. /// Used to create full column from sparse. - [[nodiscard]] virtual Ptr createWithOffsets(const Offsets & offsets, const Field & default_field, size_t total_rows, size_t shift) const; + [[nodiscard]] virtual Ptr createWithOffsets(const Offsets & offsets, const ColumnConst & column_with_default_value, size_t total_rows, size_t shift) const; /// Compress column in memory to some representation that allows to decompress it back. /// Return itself if compression is not applicable for this column type. diff --git a/src/Functions/IFunction.cpp b/src/Functions/IFunction.cpp index a46f4d2a11d..d4c6b8f4ba6 100644 --- a/src/Functions/IFunction.cpp +++ b/src/Functions/IFunction.cpp @@ -313,7 +313,7 @@ ColumnPtr IExecutableFunction::execute(const ColumnsWithTypeAndName & arguments, { bool use_default_implementation_for_sparse_columns = useDefaultImplementationForSparseColumns(); /// DataTypeFunction does not support obtaining default (isDefaultAt()) - /// ColumnFunction does not support getting specific values + /// ColumnFunction does not support getting specific values. if (result_type->getTypeId() != TypeIndex::Function && use_default_implementation_for_sparse_columns) { size_t num_sparse_columns = 0; @@ -368,7 +368,7 @@ ColumnPtr IExecutableFunction::execute(const ColumnsWithTypeAndName & arguments, if (!result_type->canBeInsideSparseColumns() || !res->isDefaultAt(0) || res->getNumberOfDefaultRows() != 1) { const auto & offsets_data = assert_cast &>(*sparse_offsets).getData(); - return res->createWithOffsets(offsets_data, (*res)[0], input_rows_count, /*shift=*/ 1); + return res->createWithOffsets(offsets_data, *createColumnConst(res, 0), input_rows_count, /*shift=*/ 1); } return ColumnSparse::create(res, sparse_offsets, input_rows_count); diff --git a/tests/queries/0_stateless/02941_variant_type_alters.sh b/tests/queries/0_stateless/02941_variant_type_alters.sh index 7e2ecbd67aa..9b0d4febd65 100755 --- a/tests/queries/0_stateless/02941_variant_type_alters.sh +++ b/tests/queries/0_stateless/02941_variant_type_alters.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: long CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # reset --log_comment From 3d0e91508880c33907ccc107e15b4149fcfc0498 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 22 Jan 2024 14:44:07 +0100 Subject: [PATCH 095/264] Followup to 57853 --- .../Passes/RewriteSumFunctionWithSumAndCountPass.cpp | 11 +++-------- .../Passes/RewriteSumFunctionWithSumAndCountPass.h | 8 ++++++++ src/Interpreters/TreeOptimizer.cpp | 3 ++- 3 files changed, 13 insertions(+), 9 deletions(-) diff --git a/src/Analyzer/Passes/RewriteSumFunctionWithSumAndCountPass.cpp b/src/Analyzer/Passes/RewriteSumFunctionWithSumAndCountPass.cpp index 7887a1b7175..24ddb7522c8 100644 --- a/src/Analyzer/Passes/RewriteSumFunctionWithSumAndCountPass.cpp +++ b/src/Analyzer/Passes/RewriteSumFunctionWithSumAndCountPass.cpp @@ -10,14 +10,6 @@ namespace DB { -/** Rewrites `sum(column +/- literal)` into two individual functions - * `sum(column)` and `literal * count(column)`. - * sum(column + literal) -> sum(column) + literal * count(column) - * sum(literal + column) -> literal * count(column) + sum(column) - * sum(column - literal) -> sum(column) - literal * count(column) - * sum(literal - column) -> literal * count(column) - sum(column) - */ - namespace { @@ -29,6 +21,9 @@ public: void enterImpl(QueryTreeNodePtr & node) { + if (!getSettings().optimize_arithmetic_operations_in_aggregate_functions) + return; + static const std::unordered_set func_supported = { "plus", "minus" diff --git a/src/Analyzer/Passes/RewriteSumFunctionWithSumAndCountPass.h b/src/Analyzer/Passes/RewriteSumFunctionWithSumAndCountPass.h index cb76448b34a..bb1603472b0 100644 --- a/src/Analyzer/Passes/RewriteSumFunctionWithSumAndCountPass.h +++ b/src/Analyzer/Passes/RewriteSumFunctionWithSumAndCountPass.h @@ -5,6 +5,14 @@ namespace DB { +/** + * Rewrites `sum(column +/- literal)` into two individual functions + * `sum(column)` and `literal * count(column)`. + * sum(column + literal) -> sum(column) + literal * count(column) + * sum(literal + column) -> literal * count(column) + sum(column) + * sum(column - literal) -> sum(column) - literal * count(column) + * sum(literal - column) -> literal * count(column) - sum(column) + */ class RewriteSumFunctionWithSumAndCountPass final : public IQueryTreePass { public: diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index de3b72f4801..2aa52c33048 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -752,7 +752,8 @@ void TreeOptimizer::apply(ASTPtr & query, TreeRewriterResult & result, } /// Rewrite sum(column +/- literal) function with sum(column) +/- literal * count(column). - rewriteSumFunctionWithSumAndCount(query, tables_with_columns); + if (settings.optimize_arithmetic_operations_in_aggregate_functions) + rewriteSumFunctionWithSumAndCount(query, tables_with_columns); /// Rewrite date filters to avoid the calls of converters such as toYear, toYYYYMM, etc. optimizeDateFilters(select_query, tables_with_columns, context); From 1e875431db49658873e3794ab092400645508723 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 22 Jan 2024 14:59:53 +0100 Subject: [PATCH 096/264] Style --- src/Analyzer/Passes/RewriteSumFunctionWithSumAndCountPass.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/RewriteSumFunctionWithSumAndCountPass.h b/src/Analyzer/Passes/RewriteSumFunctionWithSumAndCountPass.h index bb1603472b0..e878a2c0e7a 100644 --- a/src/Analyzer/Passes/RewriteSumFunctionWithSumAndCountPass.h +++ b/src/Analyzer/Passes/RewriteSumFunctionWithSumAndCountPass.h @@ -5,7 +5,7 @@ namespace DB { -/** +/** * Rewrites `sum(column +/- literal)` into two individual functions * `sum(column)` and `literal * count(column)`. * sum(column + literal) -> sum(column) + literal * count(column) From 0606a772674fdecf08a9a904ef46293e8bba9acc Mon Sep 17 00:00:00 2001 From: mochi Date: Mon, 22 Jan 2024 22:02:50 +0800 Subject: [PATCH 097/264] Update src/Storages/MergeTree/MutateTask.cpp Co-authored-by: Dmitry Novik --- src/Storages/MergeTree/MutateTask.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 8ed8b8bba4c..fccee6bd887 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -676,7 +676,7 @@ static NameToNameVector collectFilesForRenames( { const std::vector suffixes = {".idx2", ".idx", ".gin_dict", ".gin_post", ".gin_seg", ".gin_sid"}; - for (const auto& suffix : suffixes) + for (const auto & suffix : suffixes) { String filename = INDEX_FILE_PREFIX + command.column_name + suffix; From 2881eb0f51513a41b9b38e71087179705d2492a6 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Mon, 22 Jan 2024 16:09:40 +0100 Subject: [PATCH 098/264] Update the tests --- .../02969_mysql_cast_type_aliases.reference | 30 +---------- .../02969_mysql_cast_type_aliases.sh | 52 ------------------- .../02969_mysql_cast_type_aliases.sql | 46 ++++++++++++++++ 3 files changed, 48 insertions(+), 80 deletions(-) mode change 100755 => 100644 tests/queries/0_stateless/02969_mysql_cast_type_aliases.reference delete mode 100755 tests/queries/0_stateless/02969_mysql_cast_type_aliases.sh create mode 100755 tests/queries/0_stateless/02969_mysql_cast_type_aliases.sql diff --git a/tests/queries/0_stateless/02969_mysql_cast_type_aliases.reference b/tests/queries/0_stateless/02969_mysql_cast_type_aliases.reference old mode 100755 new mode 100644 index cd9ed5eb39f..5555c918500 --- a/tests/queries/0_stateless/02969_mysql_cast_type_aliases.reference +++ b/tests/queries/0_stateless/02969_mysql_cast_type_aliases.reference @@ -1,54 +1,28 @@ -- Uppercase tests -mysql_type result native_type Binary(N) foo FixedString(3) -mysql_type result native_type Char 44 String -mysql_type result native_type Date 2021-02-03 Date -mysql_type result native_type DateTime 2021-02-03 12:01:02 DateTime -mysql_type result native_type Decimal 45 Decimal(10, 0) -mysql_type result native_type Decimal(M) 46 Decimal(4, 0) -mysql_type result native_type Decimal(M, D) 47.21 Decimal(4, 2) -mysql_type result native_type Double 48.11 Float64 -mysql_type result native_type -JSON {"foo":"bar"} Object('json') -mysql_type result native_type +JSON {"foo":"bar"} Object(\'json\') Real 49.22 Float32 -mysql_type result native_type Signed 50 Int64 -mysql_type result native_type Unsigned 52 UInt64 -mysql_type result native_type Year 2007 UInt16 -- Lowercase tests -mysql_type result native_type Binary(N) foo FixedString(3) -mysql_type result native_type Char 44 String -mysql_type result native_type Date 2021-02-03 Date -mysql_type result native_type DateTime 2021-02-03 12:01:02 DateTime -mysql_type result native_type Decimal 45 Decimal(10, 0) -mysql_type result native_type Decimal(M) 46 Decimal(4, 0) -mysql_type result native_type Decimal(M, D) 47.21 Decimal(4, 2) -mysql_type result native_type Double 48.11 Float64 -mysql_type result native_type -JSON {"foo":"bar"} Object('json') -mysql_type result native_type +JSON {"foo":"bar"} Object(\'json\') Real 49.22 Float32 -mysql_type result native_type Signed 50 Int64 -mysql_type result native_type Unsigned 52 UInt64 -mysql_type result native_type Year 2007 UInt16 diff --git a/tests/queries/0_stateless/02969_mysql_cast_type_aliases.sh b/tests/queries/0_stateless/02969_mysql_cast_type_aliases.sh deleted file mode 100755 index 35218bec549..00000000000 --- a/tests/queries/0_stateless/02969_mysql_cast_type_aliases.sh +++ /dev/null @@ -1,52 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-fasttest -# Tag no-fasttest: requires mysql client - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -# See https://dev.mysql.com/doc/refman/8.0/en/cast-functions.html#function_cast -# Tests are in order of the type appearance in the docs - -echo "-- Uppercase tests" -#### Not supported as it is translated to FixedString without arguments -# ${MYSQL_CLIENT} --execute "SELECT 'Binary' AS mysql_type, CAST('' AS BINARY) AS result, toTypeName(result) AS native_type;" -${MYSQL_CLIENT} --execute "SELECT 'Binary(N)' AS mysql_type, CAST('foo' AS BINARY(3)) AS result, toTypeName(result) AS native_type;" -${MYSQL_CLIENT} --execute "SELECT 'Char' AS mysql_type, CAST(44 AS CHAR) AS result, toTypeName(result) AS native_type;" -${MYSQL_CLIENT} --execute "SELECT 'Date' AS mysql_type, CAST('2021-02-03' AS DATE) AS result, toTypeName(result) AS native_type;" -${MYSQL_CLIENT} --execute "SELECT 'DateTime' AS mysql_type, CAST('2021-02-03 12:01:02' AS DATETIME) AS result, toTypeName(result) AS native_type;" -${MYSQL_CLIENT} --execute "SELECT 'Decimal' AS mysql_type, CAST(45.1 AS DECIMAL) AS result, toTypeName(result) AS native_type;" -${MYSQL_CLIENT} --execute "SELECT 'Decimal(M)' AS mysql_type, CAST(46.2 AS DECIMAL(4)) AS result, toTypeName(result) AS native_type;" -${MYSQL_CLIENT} --execute "SELECT 'Decimal(M, D)' AS mysql_type, CAST(47.21 AS DECIMAL(4, 2)) AS result, toTypeName(result) AS native_type;" -${MYSQL_CLIENT} --execute "SELECT 'Double' AS mysql_type, CAST(48.11 AS DOUBLE) AS result, toTypeName(result) AS native_type;" -${MYSQL_CLIENT} --execute "SET allow_experimental_object_type = 1; SELECT 'JSON' AS mysql_type, CAST('{\"foo\":\"bar\"}' AS JSON) AS result, toTypeName(result) AS native_type;" -${MYSQL_CLIENT} --execute "SELECT 'Real' AS mysql_type, CAST(49.22 AS REAL) AS result, toTypeName(result) AS native_type;" -${MYSQL_CLIENT} --execute "SELECT 'Signed' AS mysql_type, CAST(50 AS SIGNED) AS result, toTypeName(result) AS native_type;" -${MYSQL_CLIENT} --execute "SELECT 'Unsigned' AS mysql_type, CAST(52 AS UNSIGNED) AS result, toTypeName(result) AS native_type;" -#### Could be added as an alias, but SIGNED INTEGER in CAST context means UInt64, -#### while INTEGER SIGNED as a column definition means UInt32. -# ${MYSQL_CLIENT} --execute "SELECT 'Signed integer' AS mysql_type, CAST(51 AS SIGNED INTEGER) AS result, toTypeName(result) AS native_type;" -# ${MYSQL_CLIENT} --execute "SELECT 'Unsigned integer' AS mysql_type, CAST(53 AS UNSIGNED INTEGER) AS result, toTypeName(result) AS native_type;" -${MYSQL_CLIENT} --execute "SELECT 'Year' AS mysql_type, CAST(2007 AS YEAR) AS result, toTypeName(result) AS native_type;" -#### Currently, expects UInt64 as an argument -# ${MYSQL_CLIENT} --execute "SELECT 'Time' AS mysql_type, CAST('12:45' AS TIME) AS result, toTypeName(result) AS native_type;" - -echo "-- Lowercase tests" -# ${MYSQL_CLIENT} --execute "select 'Binary' as mysql_type, cast('' as binary) as result, toTypeName(result) as native_type;" -${MYSQL_CLIENT} --execute "select 'Binary(N)' as mysql_type, cast('foo' as binary(3)) as result, toTypeName(result) as native_type;" -${MYSQL_CLIENT} --execute "select 'Char' as mysql_type, cast(44 as char) as result, toTypeName(result) as native_type;" -${MYSQL_CLIENT} --execute "select 'Date' as mysql_type, cast('2021-02-03' as date) as result, toTypeName(result) as native_type;" -${MYSQL_CLIENT} --execute "select 'DateTime' as mysql_type, cast('2021-02-03 12:01:02' as datetime) as result, toTypeName(result) as native_type;" -${MYSQL_CLIENT} --execute "select 'Decimal' as mysql_type, cast(45.1 as decimal) as result, toTypeName(result) as native_type;" -${MYSQL_CLIENT} --execute "select 'Decimal(M)' as mysql_type, cast(46.2 as decimal(4)) as result, toTypeName(result) as native_type;" -${MYSQL_CLIENT} --execute "select 'Decimal(M, D)' as mysql_type, cast(47.21 as decimal(4, 2)) as result, toTypeName(result) as native_type;" -${MYSQL_CLIENT} --execute "select 'Double' as mysql_type, cast(48.11 as double) as result, toTypeName(result) as native_type;" -${MYSQL_CLIENT} --execute "set allow_experimental_object_type = 1; select 'JSON' as mysql_type, cast('{\"foo\":\"bar\"}' as json) as result, toTypeName(result) as native_type;" -${MYSQL_CLIENT} --execute "select 'Real' as mysql_type, cast(49.22 as real) as result, toTypeName(result) as native_type;" -${MYSQL_CLIENT} --execute "select 'Signed' as mysql_type, cast(50 as signed) as result, toTypeName(result) as native_type;" -${MYSQL_CLIENT} --execute "select 'Unsigned' as mysql_type, cast(52 as unsigned) as result, toTypeName(result) as native_type;" -# ${MYSQL_CLIENT} --execute "select 'Signed integer' as mysql_type, cast(51 as signed integer) as result, toTypeName(result) as native_type;" -# ${MYSQL_CLIENT} --execute "select 'Unsigned integer' as mysql_type, cast(53 as unsigned integer) as result, toTypeName(result) as native_type;" -${MYSQL_CLIENT} --execute "select 'Year' as mysql_type, cast(2007 as year) as result, toTypeName(result) as native_type;" -# ${MYSQL_CLIENT} --execute "select 'Time' as mysql_type, cast('12:45' as time) as result, toTypeName(result) as native_type;" diff --git a/tests/queries/0_stateless/02969_mysql_cast_type_aliases.sql b/tests/queries/0_stateless/02969_mysql_cast_type_aliases.sql new file mode 100755 index 00000000000..7b5735cdebc --- /dev/null +++ b/tests/queries/0_stateless/02969_mysql_cast_type_aliases.sql @@ -0,0 +1,46 @@ +-- See https://dev.mysql.com/doc/refman/8.0/en/cast-functions.html#function_cast +-- Tests are in order of the type appearance in the docs + +SET allow_experimental_object_type = 1; + +SELECT '-- Uppercase tests'; +-- Not supported as it is translated to FixedString without arguments +-- SELECT 'Binary' AS mysql_type, CAST('' AS BINARY) AS result, toTypeName(result) AS native_type; +SELECT 'Binary(N)' AS mysql_type, CAST('foo' AS BINARY(3)) AS result, toTypeName(result) AS native_type; +SELECT 'Char' AS mysql_type, CAST(44 AS CHAR) AS result, toTypeName(result) AS native_type; +SELECT 'Date' AS mysql_type, CAST('2021-02-03' AS DATE) AS result, toTypeName(result) AS native_type; +SELECT 'DateTime' AS mysql_type, CAST('2021-02-03 12:01:02' AS DATETIME) AS result, toTypeName(result) AS native_type; +SELECT 'Decimal' AS mysql_type, CAST(45.1 AS DECIMAL) AS result, toTypeName(result) AS native_type; +SELECT 'Decimal(M)' AS mysql_type, CAST(46.2 AS DECIMAL(4)) AS result, toTypeName(result) AS native_type; +SELECT 'Decimal(M, D)' AS mysql_type, CAST(47.21 AS DECIMAL(4, 2)) AS result, toTypeName(result) AS native_type; +SELECT 'Double' AS mysql_type, CAST(48.11 AS DOUBLE) AS result, toTypeName(result) AS native_type; +SELECT 'JSON' AS mysql_type, CAST('{\"foo\":\"bar\"}' AS JSON) AS result, toTypeName(result) AS native_type; +SELECT 'Real' AS mysql_type, CAST(49.22 AS REAL) AS result, toTypeName(result) AS native_type; +SELECT 'Signed' AS mysql_type, CAST(50 AS SIGNED) AS result, toTypeName(result) AS native_type; +SELECT 'Unsigned' AS mysql_type, CAST(52 AS UNSIGNED) AS result, toTypeName(result) AS native_type; +-- Could be added as an alias, but SIGNED INTEGER in CAST context means UInt64, +-- while INTEGER SIGNED as a column definition means UInt32. +-- SELECT 'Signed integer' AS mysql_type, CAST(51 AS SIGNED INTEGER) AS result, toTypeName(result) AS native_type; +-- SELECT 'Unsigned integer' AS mysql_type, CAST(53 AS UNSIGNED INTEGER) AS result, toTypeName(result) AS native_type; +SELECT 'Year' AS mysql_type, CAST(2007 AS YEAR) AS result, toTypeName(result) AS native_type; +-- Currently, expects UInt64 as an argument +-- SELECT 'Time' AS mysql_type, CAST('12:45' AS TIME) AS result, toTypeName(result) AS native_type; + +SELECT '-- Lowercase tests'; +-- select 'Binary' as mysql_type, cast('' as binary) as result, toTypeName(result) as native_type; +select 'Binary(N)' as mysql_type, cast('foo' as binary(3)) as result, toTypeName(result) as native_type; +select 'Char' as mysql_type, cast(44 as char) as result, toTypeName(result) as native_type; +select 'Date' as mysql_type, cast('2021-02-03' as date) as result, toTypeName(result) as native_type; +select 'DateTime' as mysql_type, cast('2021-02-03 12:01:02' as datetime) as result, toTypeName(result) as native_type; +select 'Decimal' as mysql_type, cast(45.1 as decimal) as result, toTypeName(result) as native_type; +select 'Decimal(M)' as mysql_type, cast(46.2 as decimal(4)) as result, toTypeName(result) as native_type; +select 'Decimal(M, D)' as mysql_type, cast(47.21 as decimal(4, 2)) as result, toTypeName(result) as native_type; +select 'Double' as mysql_type, cast(48.11 as double) as result, toTypeName(result) as native_type; +select 'JSON' as mysql_type, cast('{\"foo\":\"bar\"}' as json) as result, toTypeName(result) as native_type; +select 'Real' as mysql_type, cast(49.22 as real) as result, toTypeName(result) as native_type; +select 'Signed' as mysql_type, cast(50 as signed) as result, toTypeName(result) as native_type; +select 'Unsigned' as mysql_type, cast(52 as unsigned) as result, toTypeName(result) as native_type; +-- select 'Signed integer' as mysql_type, cast(51 as signed integer) as result, toTypeName(result) as native_type; +-- select 'Unsigned integer' as mysql_type, cast(53 as unsigned integer) as result, toTypeName(result) as native_type; +select 'Year' as mysql_type, cast(2007 as year) as result, toTypeName(result) as native_type; +-- select 'Time' as mysql_type, cast('12:45' as time) as result, toTypeName(result) as native_type; From 1abe1307db1c6e4feb5b7748bb44d71725e0ae3f Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 22 Jan 2024 15:15:30 +0000 Subject: [PATCH 099/264] Make test to check the feature --- .../OptimizeGroupByInjectiveFunctionsPass.cpp | 21 +++++++---------- ...BY_injective_elimination_dictGet.reference | 23 +++++++++++++++++++ ...GROUP_BY_injective_elimination_dictGet.sql | 10 +++++++- 3 files changed, 40 insertions(+), 14 deletions(-) diff --git a/src/Analyzer/Passes/OptimizeGroupByInjectiveFunctionsPass.cpp b/src/Analyzer/Passes/OptimizeGroupByInjectiveFunctionsPass.cpp index 750b119ad3d..d0e01f9290a 100644 --- a/src/Analyzer/Passes/OptimizeGroupByInjectiveFunctionsPass.cpp +++ b/src/Analyzer/Passes/OptimizeGroupByInjectiveFunctionsPass.cpp @@ -70,7 +70,6 @@ private: void optimizeGroupingSet(QueryTreeNodes & grouping_set) { auto context = getContext(); - const FunctionFactory & function_factory = FunctionFactory::instance(); QueryTreeNodes new_group_by_keys; new_group_by_keys.reserve(grouping_set.size()); @@ -85,13 +84,8 @@ private: } else { - FunctionOverloadResolverPtr function_builder = UserDefinedExecutableFunctionFactory::instance().tryGet(function_node->getFunctionName(), context); - - // TODO: fix me - if (!function_builder) - function_builder = function_factory.get(function_node->getFunctionName(), context); - - can_be_eliminated = function_builder->isInjective({}); + auto function = function_node->getFunctionOrThrow(); + can_be_eliminated = function->isInjective(function_node->getArgumentColumns()); } if (can_be_eliminated) @@ -114,17 +108,18 @@ private: bool canBeEliminated(const FunctionNode * function_node, const ContextPtr & context) { - auto const * dict_name_arg = function_node->getArguments().getNodes()[0]->as(); - if (!dict_name_arg || isString(dict_name_arg->getResultType())) + const auto & function_arguments = function_node->getArguments().getNodes(); + auto const * dict_name_arg = function_arguments[0]->as(); + if (!dict_name_arg || !isString(dict_name_arg->getResultType())) return false; auto dict_name = dict_name_arg->getValue().safeGet(); const auto & dict_ptr = context->getExternalDictionariesLoader().getDictionary(dict_name, context); - auto const * attr_arg = function_node->getArguments().getNodes()[0]->as(); - if (!attr_arg || isString(attr_arg->getResultType())) + auto const * attr_name_arg = function_arguments[1]->as(); + if (!attr_name_arg || !isString(attr_name_arg->getResultType())) return false; - auto attr_name = attr_arg->getValue().safeGet(); + auto attr_name = attr_name_arg->getValue().safeGet(); return dict_ptr->isInjective(attr_name); } diff --git a/tests/queries/0_stateless/01376_GROUP_BY_injective_elimination_dictGet.reference b/tests/queries/0_stateless/01376_GROUP_BY_injective_elimination_dictGet.reference index 9459d4ba2a0..6de0a5be0a5 100644 --- a/tests/queries/0_stateless/01376_GROUP_BY_injective_elimination_dictGet.reference +++ b/tests/queries/0_stateless/01376_GROUP_BY_injective_elimination_dictGet.reference @@ -1 +1,24 @@ 1.1 +SELECT dictGet(\'dictdb_01376.dict_exists\', \'value\', toUInt64(1)) AS val +FROM numbers(2) +GROUP BY toUInt64(1) +QUERY id: 0 + PROJECTION COLUMNS + val Float64 + PROJECTION + LIST id: 1, nodes: 1 + FUNCTION id: 2, function_name: dictGet, function_type: ordinary, result_type: Float64 + ARGUMENTS + LIST id: 3, nodes: 3 + CONSTANT id: 4, constant_value: \'dictdb_01376.dict_exists\', constant_value_type: String + CONSTANT id: 5, constant_value: \'value\', constant_value_type: String + COLUMN id: 6, column_name: number, result_type: UInt64, source_id: 7 + JOIN TREE + TABLE_FUNCTION id: 7, alias: __table1, table_function_name: numbers + ARGUMENTS + LIST id: 8, nodes: 1 + CONSTANT id: 9, constant_value: UInt64_2, constant_value_type: UInt8 + GROUP BY + LIST id: 10, nodes: 1 + COLUMN id: 6, column_name: number, result_type: UInt64, source_id: 7 + SETTINGS allow_experimental_analyzer=1 diff --git a/tests/queries/0_stateless/01376_GROUP_BY_injective_elimination_dictGet.sql b/tests/queries/0_stateless/01376_GROUP_BY_injective_elimination_dictGet.sql index 29ffcb46fbf..5a070b443aa 100644 --- a/tests/queries/0_stateless/01376_GROUP_BY_injective_elimination_dictGet.sql +++ b/tests/queries/0_stateless/01376_GROUP_BY_injective_elimination_dictGet.sql @@ -23,7 +23,7 @@ INSERT INTO dictdb_01376.table_for_dict VALUES (1, 1.1); CREATE DICTIONARY IF NOT EXISTS dictdb_01376.dict_exists ( key_column UInt64, - value Float64 DEFAULT 77.77 + value Float64 DEFAULT 77.77 INJECTIVE ) PRIMARY KEY key_column SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB 'dictdb_01376')) @@ -32,6 +32,14 @@ LAYOUT(FLAT()); SELECT dictGet('dictdb_01376.dict_exists', 'value', toUInt64(1)) as val FROM numbers(2) GROUP BY val; +EXPLAIN SYNTAX SELECT dictGet('dictdb_01376.dict_exists', 'value', toUInt64(1)) as val FROM numbers(2) GROUP BY val; + +EXPLAIN QUERY TREE +SELECT dictGet('dictdb_01376.dict_exists', 'value', number) as val +FROM numbers(2) +GROUP BY val +SETTINGS allow_experimental_analyzer = 1; + DROP DICTIONARY dictdb_01376.dict_exists; DROP TABLE dictdb_01376.table_for_dict; DROP DATABASE dictdb_01376; From f702afb72dfac417c51563bb1f854d917923422f Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 22 Jan 2024 15:26:25 +0000 Subject: [PATCH 100/264] Fix includes --- .../OptimizeGroupByInjectiveFunctionsPass.cpp | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/src/Analyzer/Passes/OptimizeGroupByInjectiveFunctionsPass.cpp b/src/Analyzer/Passes/OptimizeGroupByInjectiveFunctionsPass.cpp index d0e01f9290a..6dd36733edc 100644 --- a/src/Analyzer/Passes/OptimizeGroupByInjectiveFunctionsPass.cpp +++ b/src/Analyzer/Passes/OptimizeGroupByInjectiveFunctionsPass.cpp @@ -1,13 +1,10 @@ #include +#include +#include #include -#include "Analyzer/ConstantNode.h" -#include "Analyzer/FunctionNode.h" -#include "Analyzer/IQueryTreeNode.h" -#include "DataTypes/IDataType.h" -#include -#include "Functions/FunctionFactory.h" -#include "Interpreters/Context_fwd.h" -#include "Interpreters/ExternalDictionariesLoader.h" +#include +#include +#include namespace DB { From ed031f32afb7863f278f125fc99082ead3c1fa68 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 22 Jan 2024 16:06:29 +0000 Subject: [PATCH 101/264] address review comments Signed-off-by: Duc Canh Le --- src/Parsers/ParserCreateQuery.h | 13 ++++++++++--- .../0_stateless/02870_per_column_settings.sql | 8 -------- 2 files changed, 10 insertions(+), 11 deletions(-) diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index 06973573023..c9059324bbe 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -329,10 +329,17 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E primary_key_specifier = true; } + auto old_pos = pos; if (s_settings.ignore(pos, expected)) { + /// When the keyword `SETTINGS` appear here, it can be a column settings declaration or query settings + /// For example: + /// - Column settings: `ALTER TABLE xx MODIFY COLUMN yy SETTINGS (name = value)` + /// - Query settings: ` ALTER TABLE xx MODIFY COLUMN yy SETTINGS mutation_sync = 2` + /// So after parsing keyword `SETTINGS`, we check if it's followed by an `(` then it's the column + /// settings, otherwise it's the query settings and we need to move `pos` back to origin position. ParserToken parser_opening_bracket(TokenType::OpeningRoundBracket); - if (parser_opening_bracket.check(pos, expected)) + if (parser_opening_bracket.ignore(pos, expected)) { if (!settings_parser.parse(pos, settings, expected)) return false; @@ -340,8 +347,8 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E if (!parser_closing_bracket.ignore(pos, expected)) return false; } - /// This could be settings in alter query - /// E.g: ALTER TABLE alter_enum_array MODIFY COLUMN x String SETTINGS mutations_sync=2; + else + pos = old_pos; } node = column_declaration; diff --git a/tests/queries/0_stateless/02870_per_column_settings.sql b/tests/queries/0_stateless/02870_per_column_settings.sql index bd115885330..9f48c666ff9 100644 --- a/tests/queries/0_stateless/02870_per_column_settings.sql +++ b/tests/queries/0_stateless/02870_per_column_settings.sql @@ -68,11 +68,3 @@ ORDER BY id SETTINGS min_bytes_for_wide_part = 1; -- {serverError UNKNOWN_SETTING} SELECT '--- '; - --- Column-level settings are only supported for MergeTree* tables -CREATE TABLE tab -( - id UInt64 CODEC(ZSTD), - long_string String CODEC(ZSTD) SETTINGS (min_compress_block_size = 81920, max_compress_block_size = 163840), -) -ENGINE = TinyLog; -- {serverError ILLEGAL_COLUMN} From 65c5ea31a8216f61a46543d53b4e03344f28758c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 22 Jan 2024 17:09:34 +0100 Subject: [PATCH 102/264] Revert "Revert "Allow parallel replicas for JOIN with analyzer [part 1]."" --- .../ClusterProxy/executeQuery.cpp | 2 - src/Interpreters/ClusterProxy/executeQuery.h | 1 - src/Planner/Planner.cpp | 2 +- src/Processors/QueryPlan/ReadFromRemote.cpp | 2 - src/Processors/QueryPlan/ReadFromRemote.h | 2 - src/Storages/StorageMergeTree.cpp | 26 ++++--- src/Storages/StorageReplicatedMergeTree.cpp | 5 +- src/Storages/buildQueryTreeForShard.cpp | 30 +++++++- src/Storages/buildQueryTreeForShard.h | 2 + ..._parallel_replicas_join_subquery.reference | 28 +++++++- .../02731_parallel_replicas_join_subquery.sql | 68 +++++++++++++++++-- ...replicas_automatic_decision_join.reference | 2 - ...rallel_replicas_automatic_decision_join.sh | 2 +- 13 files changed, 139 insertions(+), 33 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index b3a48e3e611..5865e669e47 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -322,7 +322,6 @@ void executeQuery( void executeQueryWithParallelReplicas( QueryPlan & query_plan, - const StorageID & main_table, SelectStreamFactory & stream_factory, const ASTPtr & query_ast, ContextPtr context, @@ -414,7 +413,6 @@ void executeQueryWithParallelReplicas( std::move(coordinator), stream_factory.header, stream_factory.processed_stage, - main_table, new_context, getThrottler(new_context), std::move(scalars), diff --git a/src/Interpreters/ClusterProxy/executeQuery.h b/src/Interpreters/ClusterProxy/executeQuery.h index b5ee4a11df6..a19ece0bbdc 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.h +++ b/src/Interpreters/ClusterProxy/executeQuery.h @@ -70,7 +70,6 @@ void executeQuery( void executeQueryWithParallelReplicas( QueryPlan & query_plan, - const StorageID & main_table, SelectStreamFactory & stream_factory, const ASTPtr & query_ast, ContextPtr context, diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 5ee7bd3e0ba..f2def571325 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1391,7 +1391,7 @@ void Planner::buildPlanForQueryNode() } } - if (query_context->canUseTaskBasedParallelReplicas() || !settings.parallel_replicas_custom_key.value.empty()) + if (!settings.parallel_replicas_custom_key.value.empty()) { /// Check support for JOIN for parallel replicas with custom key if (planner_context->getTableExpressionNodeToData().size() > 1) diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 43dcca6d59b..4bbba4cfa30 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -357,7 +357,6 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( ParallelReplicasReadingCoordinatorPtr coordinator_, Block header_, QueryProcessingStage::Enum stage_, - StorageID main_table_, ContextMutablePtr context_, ThrottlerPtr throttler_, Scalars scalars_, @@ -369,7 +368,6 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( , query_ast(query_ast_) , coordinator(std::move(coordinator_)) , stage(std::move(stage_)) - , main_table(std::move(main_table_)) , context(context_) , throttler(throttler_) , scalars(scalars_) diff --git a/src/Processors/QueryPlan/ReadFromRemote.h b/src/Processors/QueryPlan/ReadFromRemote.h index f6fd7b3a630..82ef45d6bbf 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.h +++ b/src/Processors/QueryPlan/ReadFromRemote.h @@ -76,7 +76,6 @@ public: ParallelReplicasReadingCoordinatorPtr coordinator_, Block header_, QueryProcessingStage::Enum stage_, - StorageID main_table_, ContextMutablePtr context_, ThrottlerPtr throttler_, Scalars scalars_, @@ -99,7 +98,6 @@ private: ASTPtr query_ast; ParallelReplicasReadingCoordinatorPtr coordinator; QueryProcessingStage::Enum stage; - StorageID main_table; ContextMutablePtr context; ThrottlerPtr throttler; Scalars scalars; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 4761ccd8b58..45057813297 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -29,6 +29,7 @@ #include #include #include +#include #include #include #include @@ -39,6 +40,7 @@ #include #include #include +#include #include #include #include @@ -213,16 +215,25 @@ void StorageMergeTree::read( { if (local_context->canUseParallelReplicasOnInitiator() && local_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree) { - const auto table_id = getStorageID(); - const auto & modified_query_ast = ClusterProxy::rewriteSelectQuery( - local_context, query_info.query, - table_id.database_name, table_id.table_name, /*remote_table_function_ptr*/nullptr); - + ASTPtr modified_query_ast; Block header; if (local_context->getSettingsRef().allow_experimental_analyzer) - header = InterpreterSelectQueryAnalyzer::getSampleBlock(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()); + { + QueryTreeNodePtr modified_query_tree = query_info.query_tree->clone(); + rewriteJoinToGlobalJoin(modified_query_tree); + modified_query_tree = buildQueryTreeForShard(query_info, modified_query_tree); + header = InterpreterSelectQueryAnalyzer::getSampleBlock( + modified_query_tree, local_context, SelectQueryOptions(processed_stage).analyze()); + modified_query_ast = queryNodeToSelectQuery(modified_query_tree); + } else - header = InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); + { + const auto table_id = getStorageID(); + modified_query_ast = ClusterProxy::rewriteSelectQuery(local_context, query_info.query, + table_id.database_name, table_id.table_name, /*remote_table_function_ptr*/nullptr); + header + = InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); + } ClusterProxy::SelectStreamFactory select_stream_factory = ClusterProxy::SelectStreamFactory( @@ -233,7 +244,6 @@ void StorageMergeTree::read( ClusterProxy::executeQueryWithParallelReplicas( query_plan, - getStorageID(), select_stream_factory, modified_query_ast, local_context, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index f7e6783dbc2..715cbab9eea 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5385,7 +5385,9 @@ void StorageReplicatedMergeTree::readParallelReplicasImpl( if (local_context->getSettingsRef().allow_experimental_analyzer) { - auto modified_query_tree = buildQueryTreeForShard(query_info, query_info.query_tree); + QueryTreeNodePtr modified_query_tree = query_info.query_tree->clone(); + rewriteJoinToGlobalJoin(modified_query_tree); + modified_query_tree = buildQueryTreeForShard(query_info, modified_query_tree); header = InterpreterSelectQueryAnalyzer::getSampleBlock( modified_query_tree, local_context, SelectQueryOptions(processed_stage).analyze()); @@ -5408,7 +5410,6 @@ void StorageReplicatedMergeTree::readParallelReplicasImpl( ClusterProxy::executeQueryWithParallelReplicas( query_plan, - getStorageID(), select_stream_factory, modified_query_ast, local_context, diff --git a/src/Storages/buildQueryTreeForShard.cpp b/src/Storages/buildQueryTreeForShard.cpp index 00cc5e3ee58..5ea28d9e09c 100644 --- a/src/Storages/buildQueryTreeForShard.cpp +++ b/src/Storages/buildQueryTreeForShard.cpp @@ -373,11 +373,37 @@ QueryTreeNodePtr buildQueryTreeForShard(SelectQueryInfo & query_info, QueryTreeN removeGroupingFunctionSpecializations(query_tree_to_modify); - // std::cerr << "====================== build 1 \n" << query_tree_to_modify->dumpTree() << std::endl; createUniqueTableAliases(query_tree_to_modify, nullptr, planner_context->getQueryContext()); - // std::cerr << "====================== build 2 \n" << query_tree_to_modify->dumpTree() << std::endl; return query_tree_to_modify; } +class RewriteJoinToGlobalJoinVisitor : public InDepthQueryTreeVisitor +{ +public: + using Base = InDepthQueryTreeVisitor; + using Base::Base; + + void visitImpl(QueryTreeNodePtr & node) + { + if (auto * join_node = node->as()) + join_node->setLocality(JoinLocality::Global); + } + + static bool needChildVisit(QueryTreeNodePtr & parent, QueryTreeNodePtr & child) + { + auto * join_node = parent->as(); + if (join_node && join_node->getRightTableExpression() == child) + return false; + + return true; + } +}; + +void rewriteJoinToGlobalJoin(QueryTreeNodePtr query_tree_to_modify) +{ + RewriteJoinToGlobalJoinVisitor visitor; + visitor.visit(query_tree_to_modify); +} + } diff --git a/src/Storages/buildQueryTreeForShard.h b/src/Storages/buildQueryTreeForShard.h index 05d63faeb9f..eec5a0dc38a 100644 --- a/src/Storages/buildQueryTreeForShard.h +++ b/src/Storages/buildQueryTreeForShard.h @@ -12,4 +12,6 @@ using QueryTreeNodePtr = std::shared_ptr; QueryTreeNodePtr buildQueryTreeForShard(SelectQueryInfo & query_info, QueryTreeNodePtr query_tree_to_modify); +void rewriteJoinToGlobalJoin(QueryTreeNodePtr query_tree_to_modify); + } diff --git a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference index 608fac626fa..ec4928bc325 100644 --- a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference +++ b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference @@ -20,9 +20,21 @@ 23 Sx>b:^UG XpedE)Q: 7433019734386307503 29 2j&S)ba?XG QuQj 17163829389637435056 3 UlI+1 14144472852965836438 +0 PJFiUe#J2O _s\' 14427935816175499794 +1 >T%O ,z< 17537932797009027240 +12 D[6,P #}Lmb[ ZzU 6394957109822140795 +18 $_N- 24422838680427462 +2 bX?}ix [ Ny]2 G 16242612901291874718 +20 VE] Y 15120036904703536841 +22 Ti~3)N)< A!( 3 18361093572663329113 +23 Sx>b:^UG XpedE)Q: 7433019734386307503 +29 2j&S)ba?XG QuQj 17163829389637435056 +3 UlI+1 14144472852965836438 =============== QUERIES EXECUTED BY PARALLEL INNER QUERY ALONE =============== +0 3 SELECT `__table1`.`key` AS `key`, `__table1`.`value1` AS `value1`, `__table1`.`value2` AS `value2`, toUInt64(min(`__table1`.`time`)) AS `start_ts` FROM `default`.`join_inner_table` AS `__table1` PREWHERE (`__table1`.`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`__table1`.`number` > 1610517366120) GROUP BY `__table1`.`key`, `__table1`.`value1`, `__table1`.`value2` ORDER BY `__table1`.`key` ASC, `__table1`.`value1` ASC, `__table1`.`value2` ASC LIMIT _CAST(10, \'UInt64\') SETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer = 1 0 3 SELECT `key`, `value1`, `value2`, toUInt64(min(`time`)) AS `start_ts` FROM `default`.`join_inner_table` PREWHERE (`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`number` > toUInt64(\'1610517366120\')) GROUP BY `key`, `value1`, `value2` ORDER BY `key` ASC, `value1` ASC, `value2` ASC LIMIT 10 -1 1 -- Parallel inner query alone\nSELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\nFROM join_inner_table\nPREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\nGROUP BY key, value1, value2\nORDER BY key, value1, value2\nLIMIT 10\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1; +1 1 -- Parallel inner query alone\nSELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\nFROM join_inner_table\nPREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\nGROUP BY key, value1, value2\nORDER BY key, value1, value2\nLIMIT 10\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer=0; +1 1 -- Parallel inner query alone\nSELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\nFROM join_inner_table\nPREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\nGROUP BY key, value1, value2\nORDER BY key, value1, value2\nLIMIT 10\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer=1; =============== OUTER QUERY (NO PARALLEL) =============== >T%O ,z< 10 NQTpY# W\\Xx4 10 @@ -39,6 +51,16 @@ U c 10 UlI+1 10 bX?}ix [ Ny]2 G 10 tT%O ,z< 10 +NQTpY# W\\Xx4 10 +PJFiUe#J2O _s\' 10 +U c 10 +UlI+1 10 +bX?}ix [ Ny]2 G 10 +t 1610517366120) GROUP BY `__table1`.`key`, `__table1`.`value1`, `__table1`.`value2` +0 3 SELECT `__table2`.`value1` AS `value1`, `__table2`.`value2` AS `value2`, count() AS `count` FROM `default`.`join_outer_table` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` USING (`key`) GROUP BY `__table1`.`key`, `__table2`.`value1`, `__table2`.`value2` 0 3 SELECT `key`, `value1`, `value2` FROM `default`.`join_inner_table` PREWHERE (`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`number` > toUInt64(\'1610517366120\')) GROUP BY `key`, `value1`, `value2` -0 3 SELECT `value1`, `value2`, count() AS `count` FROM `default`.`join_outer_table` ALL INNER JOIN `_data_7105554115296635472_12427301373021079614` USING (`key`) GROUP BY `key`, `value1`, `value2` -1 1 -- Parallel full query\nSELECT\n value1,\n value2,\n avg(count) AS avg\nFROM\n (\n SELECT\n key,\n value1,\n value2,\n count() AS count\n FROM join_outer_table\n INNER JOIN\n (\n SELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\n FROM join_inner_table\n PREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\n GROUP BY key, value1, value2\n ) USING (key)\n GROUP BY key, value1, value2\n )\nGROUP BY value1, value2\nORDER BY value1, value2\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1; +0 3 SELECT `value1`, `value2`, count() AS `count` FROM `default`.`join_outer_table` ALL INNER JOIN `_data_` USING (`key`) GROUP BY `key`, `value1`, `value2` +1 1 -- Parallel full query\nSELECT\n value1,\n value2,\n avg(count) AS avg\nFROM\n (\n SELECT\n key,\n value1,\n value2,\n count() AS count\n FROM join_outer_table\n INNER JOIN\n (\n SELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\n FROM join_inner_table\n PREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\n GROUP BY key, value1, value2\n ) USING (key)\n GROUP BY key, value1, value2\n )\nGROUP BY value1, value2\nORDER BY value1, value2\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer=0; +1 1 -- Parallel full query\nSELECT\n value1,\n value2,\n avg(count) AS avg\nFROM\n (\n SELECT\n key,\n value1,\n value2,\n count() AS count\n FROM join_outer_table\n INNER JOIN\n (\n SELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\n FROM join_inner_table\n PREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\n GROUP BY key, value1, value2\n ) USING (key)\n GROUP BY key, value1, value2\n )\nGROUP BY value1, value2\nORDER BY value1, value2\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer=1; diff --git a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql index a117378b0bf..7693d0da295 100644 --- a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql +++ b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql @@ -21,7 +21,6 @@ SELECT * FROM generateRandom('number Int64, value1 String, value2 String, time Int64', 1, 10, 2) LIMIT 100; -SET allow_experimental_analyzer = 0; SET max_parallel_replicas = 3; SET prefer_localhost_replica = 1; SET cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; @@ -39,6 +38,18 @@ FROM join_inner_table GROUP BY key, value1, value2 ORDER BY key, value1, value2 LIMIT 10; +-- settings allow_experimental_analyzer=0; + +-- SELECT +-- key, +-- value1, +-- value2, +-- toUInt64(min(time)) AS start_ts +-- FROM join_inner_table +-- PREWHERE (id = '833c9e22-c245-4eb5-8745-117a9a1f26b1') AND (number > toUInt64('1610517366120')) +-- GROUP BY key, value1, value2 +-- ORDER BY key, value1, value2 +-- LIMIT 10 settings allow_experimental_analyzer=1; SELECT '=============== INNER QUERY (PARALLEL) ==============='; @@ -53,18 +64,31 @@ PREWHERE (id = '833c9e22-c245-4eb5-8745-117a9a1f26b1') AND (number > toUInt64('1 GROUP BY key, value1, value2 ORDER BY key, value1, value2 LIMIT 10 -SETTINGS allow_experimental_parallel_reading_from_replicas = 1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer=0; + +-- Parallel inner query alone +SELECT + key, + value1, + value2, + toUInt64(min(time)) AS start_ts +FROM join_inner_table +PREWHERE (id = '833c9e22-c245-4eb5-8745-117a9a1f26b1') AND (number > toUInt64('1610517366120')) +GROUP BY key, value1, value2 +ORDER BY key, value1, value2 +LIMIT 10 +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer=1; SELECT '=============== QUERIES EXECUTED BY PARALLEL INNER QUERY ALONE ==============='; SYSTEM FLUSH LOGS; -- There should be 4 queries. The main query as received by the initiator and the 3 equal queries sent to each replica -SELECT is_initial_query, count() as c, query, +SELECT is_initial_query, count() as c, replaceRegexpAll(query, '_data_(\d+)_(\d+)', '_data_') as query FROM system.query_log WHERE event_date >= yesterday() AND type = 'QueryFinish' - AND initial_query_id = + AND initial_query_id IN ( SELECT query_id FROM system.query_log @@ -160,18 +184,48 @@ FROM ) GROUP BY value1, value2 ORDER BY value1, value2 -SETTINGS allow_experimental_parallel_reading_from_replicas = 1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer=0; + +-- Parallel full query +SELECT + value1, + value2, + avg(count) AS avg +FROM + ( + SELECT + key, + value1, + value2, + count() AS count + FROM join_outer_table + INNER JOIN + ( + SELECT + key, + value1, + value2, + toUInt64(min(time)) AS start_ts + FROM join_inner_table + PREWHERE (id = '833c9e22-c245-4eb5-8745-117a9a1f26b1') AND (number > toUInt64('1610517366120')) + GROUP BY key, value1, value2 + ) USING (key) + GROUP BY key, value1, value2 + ) +GROUP BY value1, value2 +ORDER BY value1, value2 +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer=1; SYSTEM FLUSH LOGS; -- There should be 7 queries. The main query as received by the initiator, the 3 equal queries to execute the subquery -- in the inner join and the 3 queries executing the whole query (but replacing the subquery with a temp table) -SELECT is_initial_query, count() as c, query, +SELECT is_initial_query, count() as c, replaceRegexpAll(query, '_data_(\d+)_(\d+)', '_data_') as query FROM system.query_log WHERE event_date >= yesterday() AND type = 'QueryFinish' - AND initial_query_id = + AND initial_query_id IN ( SELECT query_id FROM system.query_log diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.reference b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.reference index 802d55f8ae3..521e3e2edbc 100644 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.reference +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.reference @@ -1,6 +1,4 @@ -02784_automatic_parallel_replicas_join-default_simple_join_10M_pure 0 estimated parallel replicas 02784_automatic_parallel_replicas_join-default_simple_join_10M_pure 1 estimated parallel replicas -02784_automatic_parallel_replicas_join-default_simple_join_5M_pure 0 estimated parallel replicas 02784_automatic_parallel_replicas_join-default_simple_join_5M_pure 2 estimated parallel replicas 02784_automatic_parallel_replicas_join-default_simple_join_1M_pure 1 estimated parallel replicas 02784_automatic_parallel_replicas_join-default_simple_join_1M_pure 10 estimated parallel replicas diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh index baeeb820da5..1a74c3230c6 100755 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh @@ -68,7 +68,7 @@ function run_query_with_pure_parallel_replicas () { --allow_experimental_parallel_reading_from_replicas 1 \ --parallel_replicas_for_non_replicated_merge_tree 1 \ --parallel_replicas_min_number_of_rows_per_replica "$2" \ - |& grep "It is enough work for" | awk '{ print substr($7, 2, length($7) - 2) "\t" $20 " estimated parallel replicas" }' + |& grep "It is enough work for" | awk '{ print substr($7, 2, length($7) - 2) "\t" $20 " estimated parallel replicas" }' | sort -n -k2 -b | grep -Pv "\t0 estimated parallel replicas" } query_id_base="02784_automatic_parallel_replicas_join-$CLICKHOUSE_DATABASE" From db56f11d50097c16e735582bb12e7644d5217762 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Mon, 22 Jan 2024 17:44:55 +0100 Subject: [PATCH 103/264] Fix style check --- tests/queries/0_stateless/02969_mysql_cast_type_aliases.sql | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100755 => 100644 tests/queries/0_stateless/02969_mysql_cast_type_aliases.sql diff --git a/tests/queries/0_stateless/02969_mysql_cast_type_aliases.sql b/tests/queries/0_stateless/02969_mysql_cast_type_aliases.sql old mode 100755 new mode 100644 From 1262945df779daadd90effd89747ccfe8789ed29 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 22 Jan 2024 16:16:27 +0000 Subject: [PATCH 104/264] Cosmetics --- src/Common/AsynchronousMetrics.cpp | 36 ++++++++++++------- src/Common/AsynchronousMetrics.h | 18 +++++----- .../KeeperAsynchronousMetrics.cpp | 2 +- src/Coordination/KeeperAsynchronousMetrics.h | 2 +- .../ServerAsynchronousMetrics.cpp | 12 +++---- src/Interpreters/ServerAsynchronousMetrics.h | 4 +-- 6 files changed, 42 insertions(+), 32 deletions(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index 31cf1962251..4fffedfb8b7 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -281,7 +281,7 @@ void AsynchronousMetrics::stop() try { { - std::lock_guard lock{mutex}; + std::lock_guard lock(thread_mutex); quit = true; } @@ -306,11 +306,14 @@ AsynchronousMetrics::~AsynchronousMetrics() AsynchronousMetricValues AsynchronousMetrics::getValues() const { - std::lock_guard lock{mutex}; + std::lock_guard lock(thread_mutex); return values; } -static auto get_next_update_time(std::chrono::seconds update_period) +namespace +{ + +auto get_next_update_time(std::chrono::seconds update_period) { using namespace std::chrono; @@ -334,6 +337,8 @@ static auto get_next_update_time(std::chrono::seconds update_period) return time_next; } +} + void AsynchronousMetrics::run() { setThreadName("AsyncMetrics"); @@ -344,9 +349,9 @@ void AsynchronousMetrics::run() { // Wait first, so that the first metric collection is also on even time. - std::unique_lock lock{mutex}; + std::unique_lock lock(thread_mutex); if (wait_cond.wait_until(lock, next_update_time, - [this] { return quit; })) + [this] TSA_REQUIRES(thread_mutex) { return quit; })) { break; } @@ -364,6 +369,9 @@ void AsynchronousMetrics::run() } #if USE_JEMALLOC +namespace +{ + uint64_t updateJemallocEpoch() { uint64_t value = 0; @@ -373,7 +381,7 @@ uint64_t updateJemallocEpoch() } template -static Value saveJemallocMetricImpl( +Value saveJemallocMetricImpl( AsynchronousMetricValues & values, const std::string & jemalloc_full_name, const std::string & clickhouse_full_name) @@ -386,7 +394,7 @@ static Value saveJemallocMetricImpl( } template -static Value saveJemallocMetric(AsynchronousMetricValues & values, +Value saveJemallocMetric(AsynchronousMetricValues & values, const std::string & metric_name) { return saveJemallocMetricImpl(values, @@ -395,13 +403,15 @@ static Value saveJemallocMetric(AsynchronousMetricValues & values, } template -static Value saveAllArenasMetric(AsynchronousMetricValues & values, +Value saveAllArenasMetric(AsynchronousMetricValues & values, const std::string & metric_name) { return saveJemallocMetricImpl(values, fmt::format("stats.arenas.{}.{}", MALLCTL_ARENAS_ALL, metric_name), fmt::format("jemalloc.arenas.all.{}", metric_name)); } + +} #endif @@ -554,14 +564,14 @@ void AsynchronousMetrics::update(TimePoint update_time) AsynchronousMetricValues new_values; auto current_time = std::chrono::system_clock::now(); - auto time_after_previous_update = current_time - previous_update_time; + auto time_since_previous_update = current_time - previous_update_time; previous_update_time = update_time; double update_interval = 0.; if (first_run) update_interval = update_period.count(); else - update_interval = std::chrono::duration_cast(time_after_previous_update).count() / 1e6; + update_interval = std::chrono::duration_cast(time_since_previous_update).count() / 1e6; new_values["AsynchronousMetricsUpdateInterval"] = { update_interval, "Metrics update interval" }; /// This is also a good indicator of system responsiveness. @@ -815,7 +825,7 @@ void AsynchronousMetrics::update(TimePoint update_time) if (-1 == hz) throw ErrnoException(ErrorCodes::CANNOT_SYSCONF, "Cannot call 'sysconf' to obtain system HZ"); - double multiplier = 1.0 / hz / (std::chrono::duration_cast(time_after_previous_update).count() / 1e9); + double multiplier = 1.0 / hz / (std::chrono::duration_cast(time_since_previous_update).count() / 1e9); size_t num_cpus = 0; ProcStatValuesOther current_other_values{}; @@ -1572,7 +1582,7 @@ void AsynchronousMetrics::update(TimePoint update_time) /// Add more metrics as you wish. - updateImpl(new_values, update_time, current_time); + updateImpl(update_time, current_time, first_run, new_values); new_values["AsynchronousMetricsCalculationTimeSpent"] = { watch.elapsedSeconds(), "Time in seconds spent for calculation of asynchronous metrics (this is the overhead of asynchronous metrics)." }; @@ -1581,7 +1591,7 @@ void AsynchronousMetrics::update(TimePoint update_time) first_run = false; // Finally, update the current metrics. - std::lock_guard lock(mutex); + std::lock_guard lock(thread_mutex); values = new_values; } diff --git a/src/Common/AsynchronousMetrics.h b/src/Common/AsynchronousMetrics.h index e3b5142553b..f9c55d7552e 100644 --- a/src/Common/AsynchronousMetrics.h +++ b/src/Common/AsynchronousMetrics.h @@ -78,22 +78,22 @@ protected: const Duration update_period; - /// Some values are incremental and we have to calculate the difference. - /// On first run we will only collect the values to subtract later. - bool first_run = true; - TimePoint previous_update_time; - Poco::Logger * log; private: - virtual void updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time) = 0; + virtual void updateImpl(TimePoint update_time, TimePoint current_time, bool first_run, AsynchronousMetricValues & new_values) = 0; virtual void logImpl(AsynchronousMetricValues &) {} ProtocolServerMetricsFunc protocol_server_metrics_func; - mutable std::mutex mutex; + mutable std::mutex thread_mutex; std::condition_variable wait_cond; - bool quit {false}; - AsynchronousMetricValues values; + bool quit TSA_GUARDED_BY(thread_mutex) = false; + AsynchronousMetricValues values TSA_GUARDED_BY(thread_mutex); + + /// Some values are incremental and we have to calculate the difference. + /// On first run we will only collect the values to subtract later. + bool first_run = true; + TimePoint previous_update_time; #if defined(OS_LINUX) || defined(OS_FREEBSD) MemoryStatisticsOS memory_stat; diff --git a/src/Coordination/KeeperAsynchronousMetrics.cpp b/src/Coordination/KeeperAsynchronousMetrics.cpp index e3e91e6bd07..3eb086bb2de 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.cpp +++ b/src/Coordination/KeeperAsynchronousMetrics.cpp @@ -121,7 +121,7 @@ KeeperAsynchronousMetrics::~KeeperAsynchronousMetrics() stop(); } -void KeeperAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values, TimePoint /*update_time*/, TimePoint /*current_time*/) +void KeeperAsynchronousMetrics::updateImpl(TimePoint /*update_time*/, TimePoint /*current_time*/, bool /*first_run*/, AsynchronousMetricValues & new_values) { #if USE_NURAFT { diff --git a/src/Coordination/KeeperAsynchronousMetrics.h b/src/Coordination/KeeperAsynchronousMetrics.h index 457a7112507..557d3e5e5ee 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.h +++ b/src/Coordination/KeeperAsynchronousMetrics.h @@ -19,7 +19,7 @@ public: private: ContextPtr context; - void updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time) override; + void updateImpl(TimePoint update_time, TimePoint current_time, bool first_run, AsynchronousMetricValues & new_values) override; }; diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 31d4a4e51a4..3a32734723d 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -69,7 +69,7 @@ ServerAsynchronousMetrics::~ServerAsynchronousMetrics() stop(); } -void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time) +void ServerAsynchronousMetrics::updateImpl(TimePoint update_time, TimePoint current_time, bool first_run, AsynchronousMetricValues & new_values) { if (auto mark_cache = getContext()->getMarkCache()) { @@ -377,7 +377,7 @@ void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values } #endif - updateHeavyMetricsIfNeeded(current_time, update_time, new_values); + updateHeavyMetricsIfNeeded(current_time, update_time, first_run, new_values); } void ServerAsynchronousMetrics::logImpl(AsynchronousMetricValues & new_values) @@ -421,10 +421,10 @@ void ServerAsynchronousMetrics::updateDetachedPartsStats() detached_parts_stats = current_values; } -void ServerAsynchronousMetrics::updateHeavyMetricsIfNeeded(TimePoint current_time, TimePoint update_time, AsynchronousMetricValues & new_values) +void ServerAsynchronousMetrics::updateHeavyMetricsIfNeeded(TimePoint current_time, TimePoint update_time, bool first_run, AsynchronousMetricValues & new_values) { - const auto time_after_previous_update = current_time - heavy_metric_previous_update_time; - const bool update_heavy_metric = time_after_previous_update >= heavy_metric_update_period || first_run; + const auto time_since_previous_update = current_time - heavy_metric_previous_update_time; + const bool update_heavy_metric = (time_since_previous_update >= heavy_metric_update_period) || first_run; Stopwatch watch; if (update_heavy_metric) @@ -433,7 +433,7 @@ void ServerAsynchronousMetrics::updateHeavyMetricsIfNeeded(TimePoint current_tim if (first_run) heavy_update_interval = heavy_metric_update_period.count(); else - heavy_update_interval = std::chrono::duration_cast(time_after_previous_update).count() / 1e6; + heavy_update_interval = std::chrono::duration_cast(time_since_previous_update).count() / 1e6; /// Test shows that listing 100000 entries consuming around 0.15 sec. updateDetachedPartsStats(); diff --git a/src/Interpreters/ServerAsynchronousMetrics.h b/src/Interpreters/ServerAsynchronousMetrics.h index a579d12de2c..1bd6bce42d2 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.h +++ b/src/Interpreters/ServerAsynchronousMetrics.h @@ -18,7 +18,7 @@ public: ~ServerAsynchronousMetrics() override; private: - void updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time) override; + void updateImpl(TimePoint update_time, TimePoint current_time, bool first_run, AsynchronousMetricValues & new_values) override; void logImpl(AsynchronousMetricValues & new_values) override; const Duration heavy_metric_update_period; @@ -34,7 +34,7 @@ private: DetachedPartsStats detached_parts_stats{}; void updateDetachedPartsStats(); - void updateHeavyMetricsIfNeeded(TimePoint current_time, TimePoint update_time, AsynchronousMetricValues & new_values); + void updateHeavyMetricsIfNeeded(TimePoint current_time, TimePoint update_time, bool first_run, AsynchronousMetricValues & new_values); }; } From 748371d1271edd191502646f7ae49c120e1ba89b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 22 Jan 2024 16:26:30 +0000 Subject: [PATCH 105/264] Refactor locking in AsynchronousMetrics --- src/Common/AsynchronousMetrics.cpp | 13 ++++--- src/Common/AsynchronousMetrics.h | 59 ++++++++++++++++-------------- 2 files changed, 38 insertions(+), 34 deletions(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index 4fffedfb8b7..d80896e13c4 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -98,7 +98,7 @@ AsynchronousMetrics::AsynchronousMetrics( } #if defined(OS_LINUX) -void AsynchronousMetrics::openSensors() +void AsynchronousMetrics::openSensors() TSA_REQUIRES(data_mutex) { LOG_TRACE(log, "Scanning /sys/class/thermal"); @@ -136,7 +136,7 @@ void AsynchronousMetrics::openSensors() } } -void AsynchronousMetrics::openBlockDevices() +void AsynchronousMetrics::openBlockDevices() TSA_REQUIRES(data_mutex) { LOG_TRACE(log, "Scanning /sys/block"); @@ -163,7 +163,7 @@ void AsynchronousMetrics::openBlockDevices() } } -void AsynchronousMetrics::openEDAC() +void AsynchronousMetrics::openEDAC() TSA_REQUIRES(data_mutex) { LOG_TRACE(log, "Scanning /sys/devices/system/edac"); @@ -194,7 +194,7 @@ void AsynchronousMetrics::openEDAC() } } -void AsynchronousMetrics::openSensorsChips() +void AsynchronousMetrics::openSensorsChips() TSA_REQUIRES(data_mutex) { LOG_TRACE(log, "Scanning /sys/class/hwmon"); @@ -306,7 +306,7 @@ AsynchronousMetrics::~AsynchronousMetrics() AsynchronousMetricValues AsynchronousMetrics::getValues() const { - std::lock_guard lock(thread_mutex); + std::lock_guard lock(data_mutex); return values; } @@ -563,6 +563,8 @@ void AsynchronousMetrics::update(TimePoint update_time) AsynchronousMetricValues new_values; + std::lock_guard lock(data_mutex); + auto current_time = std::chrono::system_clock::now(); auto time_since_previous_update = current_time - previous_update_time; previous_update_time = update_time; @@ -1591,7 +1593,6 @@ void AsynchronousMetrics::update(TimePoint update_time) first_run = false; // Finally, update the current metrics. - std::lock_guard lock(thread_mutex); values = new_values; } diff --git a/src/Common/AsynchronousMetrics.h b/src/Common/AsynchronousMetrics.h index f9c55d7552e..8daf3b4ded1 100644 --- a/src/Common/AsynchronousMetrics.h +++ b/src/Common/AsynchronousMetrics.h @@ -85,46 +85,51 @@ private: ProtocolServerMetricsFunc protocol_server_metrics_func; + std::unique_ptr thread; + mutable std::mutex thread_mutex; std::condition_variable wait_cond; bool quit TSA_GUARDED_BY(thread_mutex) = false; - AsynchronousMetricValues values TSA_GUARDED_BY(thread_mutex); + + mutable std::mutex data_mutex; /// Some values are incremental and we have to calculate the difference. /// On first run we will only collect the values to subtract later. - bool first_run = true; - TimePoint previous_update_time; + bool first_run TSA_GUARDED_BY(data_mutex) = true; + TimePoint previous_update_time TSA_GUARDED_BY(data_mutex); + + AsynchronousMetricValues values TSA_GUARDED_BY(data_mutex); #if defined(OS_LINUX) || defined(OS_FREEBSD) - MemoryStatisticsOS memory_stat; + MemoryStatisticsOS memory_stat TSA_GUARDED_BY(data_mutex); #endif #if defined(OS_LINUX) - std::optional meminfo; - std::optional loadavg; - std::optional proc_stat; - std::optional cpuinfo; - std::optional file_nr; - std::optional uptime; - std::optional net_dev; + std::optional meminfo TSA_GUARDED_BY(data_mutex); + std::optional loadavg TSA_GUARDED_BY(data_mutex); + std::optional proc_stat TSA_GUARDED_BY(data_mutex); + std::optional cpuinfo TSA_GUARDED_BY(data_mutex); + std::optional file_nr TSA_GUARDED_BY(data_mutex); + std::optional uptime TSA_GUARDED_BY(data_mutex); + std::optional net_dev TSA_GUARDED_BY(data_mutex); - std::optional cgroupmem_limit_in_bytes; - std::optional cgroupmem_usage_in_bytes; - std::optional cgroupcpu_cfs_period; - std::optional cgroupcpu_cfs_quota; - std::optional cgroupcpu_max; + std::optional cgroupmem_limit_in_bytes TSA_GUARDED_BY(data_mutex); + std::optional cgroupmem_usage_in_bytes TSA_GUARDED_BY(data_mutex); + std::optional cgroupcpu_cfs_period TSA_GUARDED_BY(data_mutex); + std::optional cgroupcpu_cfs_quota TSA_GUARDED_BY(data_mutex); + std::optional cgroupcpu_max TSA_GUARDED_BY(data_mutex); - std::vector> thermal; + std::vector> thermal TSA_GUARDED_BY(data_mutex); std::unordered_map>> hwmon_devices; + std::unique_ptr>> hwmon_devices TSA_GUARDED_BY(data_mutex); std::vector /* correctable errors */, - std::unique_ptr /* uncorrectable errors */>> edac; + std::unique_ptr /* uncorrectable errors */>> edac TSA_GUARDED_BY(data_mutex); - std::unordered_map> block_devs; + std::unordered_map> block_devs TSA_GUARDED_BY(data_mutex); /// TODO: socket statistics. @@ -154,9 +159,9 @@ private: ProcStatValuesOther operator-(const ProcStatValuesOther & other) const; }; - ProcStatValuesCPU proc_stat_values_all_cpus{}; - ProcStatValuesOther proc_stat_values_other{}; - std::vector proc_stat_values_per_cpu; + ProcStatValuesCPU proc_stat_values_all_cpus TSA_GUARDED_BY(data_mutex) {}; + ProcStatValuesOther proc_stat_values_other TSA_GUARDED_BY(data_mutex) {}; + std::vector proc_stat_values_per_cpu TSA_GUARDED_BY(data_mutex); /// https://www.kernel.org/doc/Documentation/block/stat.txt struct BlockDeviceStatValues @@ -181,7 +186,7 @@ private: BlockDeviceStatValues operator-(const BlockDeviceStatValues & other) const; }; - std::unordered_map block_device_stats; + std::unordered_map block_device_stats TSA_GUARDED_BY(data_mutex); struct NetworkInterfaceStatValues { @@ -197,9 +202,9 @@ private: NetworkInterfaceStatValues operator-(const NetworkInterfaceStatValues & other) const; }; - std::unordered_map network_interface_stats; + std::unordered_map network_interface_stats TSA_GUARDED_BY(data_mutex); - Stopwatch block_devices_rescan_delay; + Stopwatch block_devices_rescan_delay TSA_GUARDED_BY(data_mutex); void openSensors(); void openBlockDevices(); @@ -207,8 +212,6 @@ private: void openEDAC(); #endif - std::unique_ptr thread; - void run(); void update(TimePoint update_time); }; From 704e5ac88355456ededa543c17884d367ec6c8da Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 22 Jan 2024 17:52:13 +0100 Subject: [PATCH 106/264] Fix `test_parallel_replicas_invisible_parts` --- .../integration/test_parallel_replicas_invisible_parts/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_parallel_replicas_invisible_parts/test.py b/tests/integration/test_parallel_replicas_invisible_parts/test.py index 1a95d2a468d..cab3fb46fe9 100644 --- a/tests/integration/test_parallel_replicas_invisible_parts/test.py +++ b/tests/integration/test_parallel_replicas_invisible_parts/test.py @@ -56,6 +56,8 @@ def _create_tables(table_name, table_size, index_granularity): """ ) + nodes[0].query(f"SYSTEM SYNC REPLICA ON CLUSTER {cluster_name} {table_name}") + nodes[0].query(f"SYSTEM STOP FETCHES ON CLUSTER {cluster_name} {table_name}") for node in nodes: From 906947787141857d4a0025790157731c8e418818 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 22 Jan 2024 17:57:04 +0100 Subject: [PATCH 107/264] Increase max_bytes_before_external_group_by for 00165_jit_aggregate_functions --- tests/queries/1_stateful/00165_jit_aggregate_functions.sql | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/1_stateful/00165_jit_aggregate_functions.sql b/tests/queries/1_stateful/00165_jit_aggregate_functions.sql index 157d5892ad8..6017fc57c52 100644 --- a/tests/queries/1_stateful/00165_jit_aggregate_functions.sql +++ b/tests/queries/1_stateful/00165_jit_aggregate_functions.sql @@ -1,6 +1,8 @@ SET compile_aggregate_expressions = 1; SET min_count_to_compile_aggregate_expression = 0; -SET max_bytes_before_external_group_by='200M'; -- might be randomized to 1 leading to timeout +-- The test uses many aggregations. A low max_bytes_before_external_group_by value will lead to high disk usage +-- which in CI leads to timeouts +SET max_bytes_before_external_group_by=0; SELECT 'Aggregation using JIT compilation'; From 74e012049bad67b1c70d2f3aa9eb32d9fc96a1ca Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 22 Jan 2024 18:03:59 +0100 Subject: [PATCH 108/264] Adapt 00082_quantiles too --- tests/queries/1_stateful/00082_quantiles.sql | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/1_stateful/00082_quantiles.sql b/tests/queries/1_stateful/00082_quantiles.sql index 3c42b43f3f9..6405a27a050 100644 --- a/tests/queries/1_stateful/00082_quantiles.sql +++ b/tests/queries/1_stateful/00082_quantiles.sql @@ -1,3 +1,6 @@ +-- The test uses quite a bit of memory. A low max_bytes_before_external_group_by value will lead to high disk usage +-- which in CI leads to timeouts +SET max_bytes_before_external_group_by=0; SELECT CounterID AS k, quantileExact(0.5)(ResolutionWidth) FROM test.hits GROUP BY k ORDER BY count() DESC, CounterID LIMIT 10; SELECT CounterID AS k, quantilesExact(0.1, 0.5, 0.9, 0.99, 0.999)(ResolutionWidth) FROM test.hits GROUP BY k ORDER BY count() DESC, CounterID LIMIT 10; From 90a0ea36b3ac08d2efa20fe5b23199dacac6b3cb Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 22 Aug 2023 21:29:05 +0000 Subject: [PATCH 109/264] Add statement 'SYSTEM RELOAD ASYNCHRONOUS METRICS' --- .../system-tables/asynchronous_metric_log.md | 2 +- programs/server/Server.cpp | 1 + src/Access/Common/AccessType.h | 1 + src/Common/AsynchronousMetrics.cpp | 6 +-- src/Common/AsynchronousMetrics.h | 13 ++++-- .../KeeperAsynchronousMetrics.cpp | 2 +- src/Coordination/KeeperAsynchronousMetrics.h | 2 +- src/Interpreters/Context.cpp | 13 ++++++ src/Interpreters/Context.h | 4 ++ src/Interpreters/InterpreterSystemQuery.cpp | 13 ++++++ .../ServerAsynchronousMetrics.cpp | 10 ++-- src/Interpreters/ServerAsynchronousMetrics.h | 4 +- src/Parsers/ASTSystemQuery.h | 1 + .../__init__.py | 0 .../configs/default.xml | 6 +++ .../test_system_reload_async_metrics/test.py | 46 +++++++++++++++++++ .../01271_show_privileges.reference | 1 + ...stem_reload_asynchronous_metrics.reference | 0 ...970_system_reload_asynchronous_metrics.sql | 1 + 19 files changed, 108 insertions(+), 18 deletions(-) create mode 100644 tests/integration/test_system_reload_async_metrics/__init__.py create mode 100644 tests/integration/test_system_reload_async_metrics/configs/default.xml create mode 100644 tests/integration/test_system_reload_async_metrics/test.py create mode 100644 tests/queries/0_stateless/02970_system_reload_asynchronous_metrics.reference create mode 100644 tests/queries/0_stateless/02970_system_reload_asynchronous_metrics.sql diff --git a/docs/en/operations/system-tables/asynchronous_metric_log.md b/docs/en/operations/system-tables/asynchronous_metric_log.md index 65b2e349707..e63ab65ba07 100644 --- a/docs/en/operations/system-tables/asynchronous_metric_log.md +++ b/docs/en/operations/system-tables/asynchronous_metric_log.md @@ -10,7 +10,7 @@ Columns: - `hostname` ([LowCardinality(String)](../../sql-reference/data-types/string.md)) — Hostname of the server executing the query. - `event_date` ([Date](../../sql-reference/data-types/date.md)) — Event date. - `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Event time. -- `name` ([String](../../sql-reference/data-types/string.md)) — Metric name. +- `metric` ([String](../../sql-reference/data-types/string.md)) — Metric name. - `value` ([Float64](../../sql-reference/data-types/float.md)) — Metric value. **Example** diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 8281afbf320..6365f990e9f 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1888,6 +1888,7 @@ try /// Must be done after initialization of `servers`, because async_metrics will access `servers` variable from its thread. async_metrics.start(); + global_context->setAsynchronousMetrics(&async_metrics); main_config_reloader->start(); access_control.startPeriodicReloading(); diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 6bbb9acc0c1..b305b6fca86 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -170,6 +170,7 @@ enum class AccessType M(SYSTEM_RELOAD_MODEL, "SYSTEM RELOAD MODELS, RELOAD MODEL, RELOAD MODELS", GLOBAL, SYSTEM_RELOAD) \ M(SYSTEM_RELOAD_FUNCTION, "SYSTEM RELOAD FUNCTIONS, RELOAD FUNCTION, RELOAD FUNCTIONS", GLOBAL, SYSTEM_RELOAD) \ M(SYSTEM_RELOAD_EMBEDDED_DICTIONARIES, "RELOAD EMBEDDED DICTIONARIES", GLOBAL, SYSTEM_RELOAD) /* implicitly enabled by the grant SYSTEM_RELOAD_DICTIONARY ON *.* */\ + M(SYSTEM_RELOAD_ASYNCHRONOUS_METRICS, "RELOAD ASYNCHRONOUS METRICS", GLOBAL, SYSTEM_RELOAD) \ M(SYSTEM_RELOAD, "", GROUP, SYSTEM) \ M(SYSTEM_RESTART_DISK, "SYSTEM RESTART DISK", GLOBAL, SYSTEM) \ M(SYSTEM_MERGES, "SYSTEM STOP MERGES, SYSTEM START MERGES, STOP MERGES, START MERGES", TABLE, SYSTEM) \ diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index d80896e13c4..b328f8a09f4 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -272,7 +272,7 @@ void AsynchronousMetrics::start() { /// Update once right now, to make metrics available just after server start /// (without waiting for asynchronous_metrics_update_period_s). - update(std::chrono::system_clock::now()); + update(std::chrono::system_clock::now(), false); thread = std::make_unique([this] { run(); }); } @@ -557,7 +557,7 @@ AsynchronousMetrics::NetworkInterfaceStatValues::operator-(const AsynchronousMet #endif -void AsynchronousMetrics::update(TimePoint update_time) +void AsynchronousMetrics::update(TimePoint update_time, bool force_update) { Stopwatch watch; @@ -1584,7 +1584,7 @@ void AsynchronousMetrics::update(TimePoint update_time) /// Add more metrics as you wish. - updateImpl(update_time, current_time, first_run, new_values); + updateImpl(update_time, current_time, force_update, first_run, new_values); new_values["AsynchronousMetricsCalculationTimeSpent"] = { watch.elapsedSeconds(), "Time in seconds spent for calculation of asynchronous metrics (this is the overhead of asynchronous metrics)." }; diff --git a/src/Common/AsynchronousMetrics.h b/src/Common/AsynchronousMetrics.h index 8daf3b4ded1..b9a5862dbff 100644 --- a/src/Common/AsynchronousMetrics.h +++ b/src/Common/AsynchronousMetrics.h @@ -56,8 +56,13 @@ struct ProtocolServerMetrics */ class AsynchronousMetrics { +protected: + using Duration = std::chrono::seconds; + using TimePoint = std::chrono::system_clock::time_point; + public: using ProtocolServerMetricsFunc = std::function()>; + AsynchronousMetrics( int update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_); @@ -69,18 +74,17 @@ public: void stop(); + void update(TimePoint update_time, bool force_update = false); + /// Returns copy of all values. AsynchronousMetricValues getValues() const; protected: - using Duration = std::chrono::seconds; - using TimePoint = std::chrono::system_clock::time_point; - const Duration update_period; Poco::Logger * log; private: - virtual void updateImpl(TimePoint update_time, TimePoint current_time, bool first_run, AsynchronousMetricValues & new_values) = 0; + virtual void updateImpl(TimePoint update_time, TimePoint current_time, bool force_update, bool first_run, AsynchronousMetricValues & new_values) = 0; virtual void logImpl(AsynchronousMetricValues &) {} ProtocolServerMetricsFunc protocol_server_metrics_func; @@ -213,7 +217,6 @@ private: #endif void run(); - void update(TimePoint update_time); }; } diff --git a/src/Coordination/KeeperAsynchronousMetrics.cpp b/src/Coordination/KeeperAsynchronousMetrics.cpp index 3eb086bb2de..8f6e1dec6c1 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.cpp +++ b/src/Coordination/KeeperAsynchronousMetrics.cpp @@ -121,7 +121,7 @@ KeeperAsynchronousMetrics::~KeeperAsynchronousMetrics() stop(); } -void KeeperAsynchronousMetrics::updateImpl(TimePoint /*update_time*/, TimePoint /*current_time*/, bool /*first_run*/, AsynchronousMetricValues & new_values) +void KeeperAsynchronousMetrics::updateImpl(TimePoint /*update_time*/, TimePoint /*current_time*/, bool /*force_update*/, bool /*first_run*/, AsynchronousMetricValues & new_values) { #if USE_NURAFT { diff --git a/src/Coordination/KeeperAsynchronousMetrics.h b/src/Coordination/KeeperAsynchronousMetrics.h index 557d3e5e5ee..33e8d6818d7 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.h +++ b/src/Coordination/KeeperAsynchronousMetrics.h @@ -19,7 +19,7 @@ public: private: ContextPtr context; - void updateImpl(TimePoint update_time, TimePoint current_time, bool first_run, AsynchronousMetricValues & new_values) override; + void updateImpl(TimePoint update_time, TimePoint current_time, bool force_update, bool first_run, AsynchronousMetricValues & new_values) override; }; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 7e89c794712..6799c3acfb5 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -290,6 +290,7 @@ struct ContextSharedPart : boost::noncopyable mutable QueryCachePtr query_cache TSA_GUARDED_BY(mutex); /// Cache of query results. mutable MarkCachePtr index_mark_cache TSA_GUARDED_BY(mutex); /// Cache of marks in compressed files of MergeTree indices. mutable MMappedFileCachePtr mmap_cache TSA_GUARDED_BY(mutex); /// Cache of mmapped files to avoid frequent open/map/unmap/close and to reuse from several threads. + AsynchronousMetrics * asynchronous_metrics TSA_GUARDED_BY(mutex) = nullptr; /// Points to asynchronous metrics ProcessList process_list; /// Executing queries at the moment. SessionTracker session_tracker; GlobalOvercommitTracker global_overcommit_tracker; @@ -2860,6 +2861,18 @@ void Context::clearCaches() const /// Intentionally not clearing the query cache which is transactionally inconsistent by design. } +void Context::setAsynchronousMetrics(AsynchronousMetrics * asynchronous_metrics_) +{ + std::lock_guard lock(shared->mutex); + shared->asynchronous_metrics = asynchronous_metrics_; +} + +AsynchronousMetrics * Context::getAsynchronousMetrics() const +{ + SharedLockGuard lock(shared->mutex); + return shared->asynchronous_metrics; +} + ThreadPool & Context::getPrefetchThreadpool() const { callOnce(shared->prefetch_threadpool_initialized, [&] { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 65566876a80..7d6441a9ddc 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -70,6 +70,7 @@ class IUserDefinedSQLObjectsStorage; class InterserverCredentials; using InterserverCredentialsPtr = std::shared_ptr; class InterserverIOHandler; +class AsynchronousMetrics; class BackgroundSchedulePool; class MergeList; class MovesList; @@ -1014,6 +1015,9 @@ public: /// ----------------------------------------------------------------------------------------------------- + void setAsynchronousMetrics(AsynchronousMetrics * asynchronous_metrics_); + AsynchronousMetrics * getAsynchronousMetrics() const; + ThreadPool & getPrefetchThreadpool() const; /// Note: prefetchThreadpool is different from threadpoolReader diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 1712c9608bf..1411e7e017b 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -561,6 +561,14 @@ BlockIO InterpreterSystemQuery::execute() getContext()->checkAccess(AccessType::SYSTEM_RELOAD_USERS); system_context->getAccessControl().reload(AccessControl::ReloadMode::ALL); break; + case Type::RELOAD_ASYNCHRONOUS_METRICS: + { + getContext()->checkAccess(AccessType::SYSTEM_RELOAD_ASYNCHRONOUS_METRICS); + auto * asynchronous_metrics = system_context->getAsynchronousMetrics(); + if (asynchronous_metrics) + asynchronous_metrics->update(std::chrono::system_clock::now(), /*force_update*/ true); + break; + } case Type::STOP_MERGES: startStopAction(ActionLocks::PartsMerge, false); break; @@ -1225,6 +1233,11 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() required_access.emplace_back(AccessType::SYSTEM_RELOAD_USERS); break; } + case Type::RELOAD_ASYNCHRONOUS_METRICS: + { + required_access.emplace_back(AccessType::SYSTEM_RELOAD_ASYNCHRONOUS_METRICS); + break; + } case Type::STOP_MERGES: case Type::START_MERGES: { diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 3a32734723d..bdf314f35b9 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -69,7 +69,7 @@ ServerAsynchronousMetrics::~ServerAsynchronousMetrics() stop(); } -void ServerAsynchronousMetrics::updateImpl(TimePoint update_time, TimePoint current_time, bool first_run, AsynchronousMetricValues & new_values) +void ServerAsynchronousMetrics::updateImpl(TimePoint update_time, TimePoint current_time, bool force_update, bool first_run, AsynchronousMetricValues & new_values) { if (auto mark_cache = getContext()->getMarkCache()) { @@ -377,7 +377,7 @@ void ServerAsynchronousMetrics::updateImpl(TimePoint update_time, TimePoint curr } #endif - updateHeavyMetricsIfNeeded(current_time, update_time, first_run, new_values); + updateHeavyMetricsIfNeeded(current_time, update_time, force_update, first_run, new_values); } void ServerAsynchronousMetrics::logImpl(AsynchronousMetricValues & new_values) @@ -421,13 +421,13 @@ void ServerAsynchronousMetrics::updateDetachedPartsStats() detached_parts_stats = current_values; } -void ServerAsynchronousMetrics::updateHeavyMetricsIfNeeded(TimePoint current_time, TimePoint update_time, bool first_run, AsynchronousMetricValues & new_values) +void ServerAsynchronousMetrics::updateHeavyMetricsIfNeeded(TimePoint current_time, TimePoint update_time, bool force_update, bool first_run, AsynchronousMetricValues & new_values) { const auto time_since_previous_update = current_time - heavy_metric_previous_update_time; - const bool update_heavy_metric = (time_since_previous_update >= heavy_metric_update_period) || first_run; + const bool update_heavy_metrics = (time_since_previous_update >= heavy_metric_update_period) || force_update || first_run; Stopwatch watch; - if (update_heavy_metric) + if (update_heavy_metrics) { heavy_metric_previous_update_time = update_time; if (first_run) diff --git a/src/Interpreters/ServerAsynchronousMetrics.h b/src/Interpreters/ServerAsynchronousMetrics.h index 1bd6bce42d2..b0cf8efbfd7 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.h +++ b/src/Interpreters/ServerAsynchronousMetrics.h @@ -18,7 +18,7 @@ public: ~ServerAsynchronousMetrics() override; private: - void updateImpl(TimePoint update_time, TimePoint current_time, bool first_run, AsynchronousMetricValues & new_values) override; + void updateImpl(TimePoint update_time, TimePoint current_time, bool force_update, bool first_run, AsynchronousMetricValues & new_values) override; void logImpl(AsynchronousMetricValues & new_values) override; const Duration heavy_metric_update_period; @@ -34,7 +34,7 @@ private: DetachedPartsStats detached_parts_stats{}; void updateDetachedPartsStats(); - void updateHeavyMetricsIfNeeded(TimePoint current_time, TimePoint update_time, bool first_run, AsynchronousMetricValues & new_values); + void updateHeavyMetricsIfNeeded(TimePoint current_time, TimePoint update_time, bool force_update, bool first_run, AsynchronousMetricValues & new_values); }; } diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 027bb1ec56f..a74275f1638 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -67,6 +67,7 @@ public: RELOAD_EMBEDDED_DICTIONARIES, RELOAD_CONFIG, RELOAD_USERS, + RELOAD_ASYNCHRONOUS_METRICS, RESTART_DISK, STOP_MERGES, START_MERGES, diff --git a/tests/integration/test_system_reload_async_metrics/__init__.py b/tests/integration/test_system_reload_async_metrics/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_system_reload_async_metrics/configs/default.xml b/tests/integration/test_system_reload_async_metrics/configs/default.xml new file mode 100644 index 00000000000..eb168c1967d --- /dev/null +++ b/tests/integration/test_system_reload_async_metrics/configs/default.xml @@ -0,0 +1,6 @@ + + + 60000 + 60000 + + diff --git a/tests/integration/test_system_reload_async_metrics/test.py b/tests/integration/test_system_reload_async_metrics/test.py new file mode 100644 index 00000000000..00616f4293f --- /dev/null +++ b/tests/integration/test_system_reload_async_metrics/test.py @@ -0,0 +1,46 @@ +import os +import pytest +import shutil +import time +from helpers.cluster import ClickHouseCluster + +# Tests that SYSTEM RELOAD ASYNCHRONOUS METRICS works. + +# Config default.xml sets a large refresh interval of asynchronous metrics, so that the periodic updates don't interfere with the manual +# update below. +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node", + main_configs=["configs/default.xml"], + stay_alive=True, +) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +CONFIG_DIR = os.path.join(SCRIPT_DIR, "configs") + + +def test_query_cache_size_is_runtime_configurable(start_cluster): + node.query("SYSTEM DROP QUERY CACHE") + + res1 = node.query( + "SELECT value FROM system.asynchronous_metrics WHERE metric = 'NumberOfTables'" + ) + + node.query("CREATE TABLE tab (col UInt64) ENGINE MergeTree ORDER BY tuple()") # do anything dumb + + node.query("SYSTEM RELOAD ASYNCHRONOUS METRICS") + + res2 = node.query( + "SELECT value FROM system.asynchronous_metrics WHERE metric = 'NumberOfTables'" + ) + assert int(res1.rstrip()) + 1 == int(res2.rstrip()) diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index f9f5c2bd3df..6a7e4748130 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -120,6 +120,7 @@ SYSTEM RELOAD DICTIONARY ['SYSTEM RELOAD DICTIONARIES','RELOAD DICTIONARY','RELO SYSTEM RELOAD MODEL ['SYSTEM RELOAD MODELS','RELOAD MODEL','RELOAD MODELS'] GLOBAL SYSTEM RELOAD SYSTEM RELOAD FUNCTION ['SYSTEM RELOAD FUNCTIONS','RELOAD FUNCTION','RELOAD FUNCTIONS'] GLOBAL SYSTEM RELOAD SYSTEM RELOAD EMBEDDED DICTIONARIES ['RELOAD EMBEDDED DICTIONARIES'] GLOBAL SYSTEM RELOAD +SYSTEM RELOAD ASYNCHRONOUS METRICS ['RELOAD ASYNCHRONOUS METRICS'] GLOBAL SYSTEM RELOAD SYSTEM RELOAD [] \N SYSTEM SYSTEM RESTART DISK ['SYSTEM RESTART DISK'] GLOBAL SYSTEM SYSTEM MERGES ['SYSTEM STOP MERGES','SYSTEM START MERGES','STOP MERGES','START MERGES'] TABLE SYSTEM diff --git a/tests/queries/0_stateless/02970_system_reload_asynchronous_metrics.reference b/tests/queries/0_stateless/02970_system_reload_asynchronous_metrics.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02970_system_reload_asynchronous_metrics.sql b/tests/queries/0_stateless/02970_system_reload_asynchronous_metrics.sql new file mode 100644 index 00000000000..44901079838 --- /dev/null +++ b/tests/queries/0_stateless/02970_system_reload_asynchronous_metrics.sql @@ -0,0 +1 @@ +SYSTEM RELOAD ASYNCHRONOUS METRICS; From c0868eedbd863f6555b2566f79fe8cc18b96f2b5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 22 Jan 2024 17:19:10 +0000 Subject: [PATCH 110/264] Remove obsolete test --- .../02970_system_reload_asynchronous_metrics.reference | 0 .../0_stateless/02970_system_reload_asynchronous_metrics.sql | 1 - 2 files changed, 1 deletion(-) delete mode 100644 tests/queries/0_stateless/02970_system_reload_asynchronous_metrics.reference delete mode 100644 tests/queries/0_stateless/02970_system_reload_asynchronous_metrics.sql diff --git a/tests/queries/0_stateless/02970_system_reload_asynchronous_metrics.reference b/tests/queries/0_stateless/02970_system_reload_asynchronous_metrics.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02970_system_reload_asynchronous_metrics.sql b/tests/queries/0_stateless/02970_system_reload_asynchronous_metrics.sql deleted file mode 100644 index 44901079838..00000000000 --- a/tests/queries/0_stateless/02970_system_reload_asynchronous_metrics.sql +++ /dev/null @@ -1 +0,0 @@ -SYSTEM RELOAD ASYNCHRONOUS METRICS; From a23d71b7bbd141808566578be8d1c3119b69367c Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 22 Jan 2024 18:19:58 +0100 Subject: [PATCH 111/264] Update run.sh --- docker/test/stateless/run.sh | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 08ee52e4f1b..27bd263f349 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -249,10 +249,13 @@ stop_logs_replication successfuly_saved=0 for table in query_log zookeeper_log trace_log transactions_info_log do - clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst || successfuly_saved=$((successfuly_saved+$?)) + clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst + successfuly_saved=$? if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then - clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst || successfuly_saved=$((successfuly_saved+$?)) - clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst || successfuly_saved=$((successfuly_saved+$?)) + clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst + successfuly_saved=$(($successfuly_saved | $?)) + clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst + successfuly_saved=$(($successfuly_saved | $?)) fi done From 1aa8e62bb78bdeb41479b48d345e83bf317cb961 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 22 Jan 2024 18:24:48 +0100 Subject: [PATCH 112/264] Fix --- src/Storages/DataLakes/IStorageDataLake.h | 45 ++++++++----- .../DataLakes/Iceberg/StorageIceberg.cpp | 36 +++++++++-- .../DataLakes/Iceberg/StorageIceberg.h | 1 + src/Storages/StorageS3.h | 1 + src/TableFunctions/ITableFunctionDataLake.h | 2 +- tests/integration/helpers/s3_tools.py | 10 +++ tests/integration/test_storage_delta/test.py | 60 ++++++++++++++++- .../integration/test_storage_iceberg/test.py | 64 ++++++++++++++++++- 8 files changed, 190 insertions(+), 29 deletions(-) diff --git a/src/Storages/DataLakes/IStorageDataLake.h b/src/Storages/DataLakes/IStorageDataLake.h index 77a22cd00fc..0e21b3d65ac 100644 --- a/src/Storages/DataLakes/IStorageDataLake.h +++ b/src/Storages/DataLakes/IStorageDataLake.h @@ -22,15 +22,15 @@ public: using Configuration = typename Storage::Configuration; template - explicit IStorageDataLake(const Configuration & configuration_, ContextPtr context_, Args && ...args) - : Storage(getConfigurationForDataRead(configuration_, context_), context_, std::forward(args)...) + explicit IStorageDataLake(const Configuration & configuration_, ContextPtr context_, bool attach, Args && ...args) + : Storage(getConfigurationForDataRead(configuration_, context_, {}, attach), context_, std::forward(args)...) , base_configuration(configuration_) , log(&Poco::Logger::get(getName())) {} // NOLINT(clang-analyzer-optin.cplusplus.VirtualCall) template - static StoragePtr create(const Configuration & configuration_, ContextPtr context_, Args && ...args) + static StoragePtr create(const Configuration & configuration_, ContextPtr context_, bool attach, Args && ...args) { - return std::make_shared>(configuration_, context_, std::forward(args)...); + return std::make_shared>(configuration_, context_, attach, std::forward(args)...); } String getName() const override { return name; } @@ -64,24 +64,34 @@ public: private: static Configuration getConfigurationForDataRead( - const Configuration & base_configuration, ContextPtr local_context, const Strings & keys = {}) + const Configuration & base_configuration, ContextPtr local_context, const Strings & keys = {}, bool attach = false) { auto configuration{base_configuration}; configuration.update(local_context); configuration.static_configuration = true; - if (keys.empty()) - configuration.keys = getDataFiles(configuration, local_context); - else - configuration.keys = keys; + try + { + if (keys.empty()) + configuration.keys = getDataFiles(configuration, local_context); + else + configuration.keys = keys; - LOG_TRACE( - &Poco::Logger::get("DataLake"), - "New configuration path: {}, keys: {}", - configuration.getPath(), fmt::join(configuration.keys, ", ")); + LOG_TRACE( + &Poco::Logger::get("DataLake"), + "New configuration path: {}, keys: {}", + configuration.getPath(), fmt::join(configuration.keys, ", ")); - configuration.connect(local_context); - return configuration; + configuration.connect(local_context); + return configuration; + } + catch (...) + { + if (!attach) + throw; + configuration.is_broken = true; + return configuration; + } } static Strings getDataFiles(const Configuration & configuration, ContextPtr local_context) @@ -94,10 +104,11 @@ private: const bool updated = base_configuration.update(local_context); auto new_keys = getDataFiles(base_configuration, local_context); - if (!updated && new_keys == Storage::getConfiguration().keys) + if (!updated && !base_configuration.is_broken && new_keys == Storage::getConfiguration().keys) return; Storage::useConfiguration(getConfigurationForDataRead(base_configuration, local_context, new_keys)); + base_configuration.is_broken = false; } Configuration base_configuration; @@ -115,7 +126,7 @@ static StoragePtr createDataLakeStorage(const StorageFactory::Arguments & args) if (configuration.format == "auto") configuration.format = "Parquet"; - return DataLake::create(configuration, args.getContext(), args.table_id, args.columns, args.constraints, + return DataLake::create(configuration, args.getContext(), args.attach, args.table_id, args.columns, args.constraints, args.comment, getFormatSettings(args.getContext())); } diff --git a/src/Storages/DataLakes/Iceberg/StorageIceberg.cpp b/src/Storages/DataLakes/Iceberg/StorageIceberg.cpp index 20ac77976cb..2f6b9a5a694 100644 --- a/src/Storages/DataLakes/Iceberg/StorageIceberg.cpp +++ b/src/Storages/DataLakes/Iceberg/StorageIceberg.cpp @@ -8,6 +8,7 @@ namespace DB StoragePtr StorageIceberg::create( const DB::StorageIceberg::Configuration & base_configuration, DB::ContextPtr context_, + bool attach, const DB::StorageID & table_id_, const DB::ColumnsDescription & columns_, const DB::ConstraintsDescription & constraints_, @@ -16,10 +17,30 @@ StoragePtr StorageIceberg::create( { auto configuration{base_configuration}; configuration.update(context_); - auto metadata = parseIcebergMetadata(configuration, context_); - auto schema_from_metadata = metadata->getTableSchema(); - configuration.keys = metadata->getDataFiles(); - return std::make_shared(std::move(metadata), configuration, context_, table_id_, columns_.empty() ? ColumnsDescription(schema_from_metadata) : columns_, constraints_, comment, format_settings_); + std::unique_ptr metadata; + NamesAndTypesList schema_from_metadata; + try + { + metadata = parseIcebergMetadata(configuration, context_); + schema_from_metadata = metadata->getTableSchema(); + configuration.keys = metadata->getDataFiles(); + } + catch (...) + { + if (!attach) + throw; + configuration.is_broken = true; + } + + return std::make_shared( + std::move(metadata), + configuration, + context_, + table_id_, + columns_.empty() ? ColumnsDescription(schema_from_metadata) : columns_, + constraints_, + comment, + format_settings_); } StorageIceberg::StorageIceberg( @@ -52,8 +73,12 @@ void StorageIceberg::updateConfigurationImpl(ContextPtr local_context) { const bool updated = base_configuration.update(local_context); auto new_metadata = parseIcebergMetadata(base_configuration, local_context); + + if (!current_metadata) + current_metadata = parseIcebergMetadata(base_configuration, local_context); + /// Check if nothing was changed. - if (updated && new_metadata->getVersion() == current_metadata->getVersion()) + if (!updated && !base_configuration.is_broken && new_metadata->getVersion() == current_metadata->getVersion()) return; if (new_metadata->getVersion() != current_metadata->getVersion()) @@ -63,6 +88,7 @@ void StorageIceberg::updateConfigurationImpl(ContextPtr local_context) /// If metadata wasn't changed, we won't list data files again. updated_configuration.keys = current_metadata->getDataFiles(); StorageS3::useConfiguration(updated_configuration); + base_configuration.is_broken = false; } } diff --git a/src/Storages/DataLakes/Iceberg/StorageIceberg.h b/src/Storages/DataLakes/Iceberg/StorageIceberg.h index a18865b5a54..4e63da5508a 100644 --- a/src/Storages/DataLakes/Iceberg/StorageIceberg.h +++ b/src/Storages/DataLakes/Iceberg/StorageIceberg.h @@ -30,6 +30,7 @@ public: static StoragePtr create(const Configuration & base_configuration, ContextPtr context_, + bool attach, const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index b90a0d394cb..f33c13ece86 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -304,6 +304,7 @@ public: std::shared_ptr client; std::vector keys; + bool is_broken = false; }; StorageS3( diff --git a/src/TableFunctions/ITableFunctionDataLake.h b/src/TableFunctions/ITableFunctionDataLake.h index d8524963776..3e4d6f8201d 100644 --- a/src/TableFunctions/ITableFunctionDataLake.h +++ b/src/TableFunctions/ITableFunctionDataLake.h @@ -34,7 +34,7 @@ protected: columns = parseColumnsListFromString(TableFunction::configuration.structure, context); StoragePtr storage = Storage::create( - TableFunction::configuration, context, StorageID(TableFunction::getDatabaseName(), table_name), + TableFunction::configuration, context, true, StorageID(TableFunction::getDatabaseName(), table_name), columns, ConstraintsDescription{}, String{}, std::nullopt); storage->startup(); diff --git a/tests/integration/helpers/s3_tools.py b/tests/integration/helpers/s3_tools.py index 777b3394dc1..c4f4c3f5aaa 100644 --- a/tests/integration/helpers/s3_tools.py +++ b/tests/integration/helpers/s3_tools.py @@ -36,6 +36,16 @@ def get_file_contents(minio_client, bucket, s3_path): return data_str.decode() +def list_s3_objects(minio_client, bucket, prefix=""): + prefix_len = len(prefix) + return [ + obj.object_name[prefix_len:] + for obj in minio_client.list_objects( + bucket, prefix=prefix, recursive=True + ) + ] + + # Creates S3 bucket for tests and allows anonymous read-write access to it. def prepare_s3_bucket(started_cluster): # Allows read-write access for bucket without authorization. diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index 621d2b89fc5..8f7349ad709 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -26,8 +26,14 @@ from pyspark.sql.functions import current_timestamp from datetime import datetime from pyspark.sql.functions import monotonically_increasing_id, row_number from pyspark.sql.window import Window +from minio.deleteobjects import DeleteObject -from helpers.s3_tools import prepare_s3_bucket, upload_directory, get_file_contents +from helpers.s3_tools import ( + prepare_s3_bucket, + upload_directory, + get_file_contents, + list_s3_objects, +) SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -55,6 +61,7 @@ def started_cluster(): main_configs=["configs/config.d/named_collections.xml"], user_configs=["configs/users.d/users.xml"], with_minio=True, + stay_alive=True, ) logging.info("Starting cluster...") @@ -111,12 +118,12 @@ def get_delta_metadata(delta_metadata_file): return combined_json -def create_delta_table(node, table_name): +def create_delta_table(node, table_name, bucket="root"): node.query( f""" DROP TABLE IF EXISTS {table_name}; CREATE TABLE {table_name} - ENGINE=DeltaLake(s3, filename = '{table_name}/')""" + ENGINE=DeltaLake(s3, filename = '{table_name}/', url = 'http://minio1:9001/{bucket}/')""" ) @@ -401,3 +408,50 @@ def test_types(started_cluster): ["e", "Nullable(Bool)"], ] ) + + +def test_restart_broken(started_cluster): + instance = started_cluster.instances["node1"] + spark = started_cluster.spark_session + minio_client = started_cluster.minio_client + bucket = "broken" + TABLE_NAME = "test_restart_broken" + + if not minio_client.bucket_exists(bucket): + minio_client.make_bucket(bucket) + + parquet_data_path = create_initial_data_file( + started_cluster, + instance, + "SELECT number, toString(number) FROM numbers(100)", + TABLE_NAME, + ) + + write_delta_from_file(spark, parquet_data_path, f"/{TABLE_NAME}") + upload_directory(minio_client, bucket, f"/{TABLE_NAME}", "") + create_delta_table(instance, TABLE_NAME, bucket=bucket) + assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 + + s3_objects = list_s3_objects(minio_client, bucket, prefix="") + assert ( + len( + list( + minio_client.remove_objects( + bucket, + [DeleteObject(obj) for obj in s3_objects], + ) + ) + ) + == 0 + ) + minio_client.remove_bucket(bucket) + + instance.restart_clickhouse() + + assert "NoSuchBucket" in instance.query_and_get_error(f"SELECT count() FROM {TABLE_NAME}") + + minio_client.make_bucket(bucket) + + upload_directory(minio_client, bucket, f"/{TABLE_NAME}", "") + + assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index 9a75dc50d61..ee7c20025f4 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -27,8 +27,14 @@ from datetime import datetime from pyspark.sql.functions import monotonically_increasing_id, row_number from pyspark.sql.window import Window from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2 +from minio.deleteobjects import DeleteObject -from helpers.s3_tools import prepare_s3_bucket, upload_directory, get_file_contents +from helpers.s3_tools import ( + prepare_s3_bucket, + upload_directory, + get_file_contents, + list_s3_objects, +) SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -61,6 +67,7 @@ def started_cluster(): main_configs=["configs/config.d/named_collections.xml"], user_configs=["configs/users.d/users.xml"], with_minio=True, + stay_alive=True, ) logging.info("Starting cluster...") @@ -135,12 +142,12 @@ def generate_data(spark, start, end): return df -def create_iceberg_table(node, table_name, format="Parquet"): +def create_iceberg_table(node, table_name, format="Parquet", bucket="root"): node.query( f""" DROP TABLE IF EXISTS {table_name}; CREATE TABLE {table_name} - ENGINE=Iceberg(s3, filename = 'iceberg_data/default/{table_name}/', format={format})""" + ENGINE=Iceberg(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" ) @@ -551,3 +558,54 @@ def test_metadata_file_format_with_uuid(started_cluster, format_version): create_iceberg_table(instance, TABLE_NAME) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 500 + + +def test_restart_broken(started_cluster): + instance = started_cluster.instances["node1"] + spark = started_cluster.spark_session + minio_client = started_cluster.minio_client + bucket = "broken" + TABLE_NAME = "test_restart_broken" + + if not minio_client.bucket_exists(bucket): + minio_client.make_bucket(bucket) + + parquet_data_path = create_initial_data_file( + started_cluster, + instance, + "SELECT number, toString(number) FROM numbers(100)", + TABLE_NAME, + ) + + write_iceberg_from_file(spark, parquet_data_path, TABLE_NAME, format_version="1") + files = upload_directory( + minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + ) + create_iceberg_table(instance, TABLE_NAME, bucket=bucket) + assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 + + s3_objects = list_s3_objects(minio_client, bucket, prefix="") + assert ( + len( + list( + minio_client.remove_objects( + bucket, + [DeleteObject(obj) for obj in s3_objects], + ) + ) + ) + == 0 + ) + minio_client.remove_bucket(bucket) + + instance.restart_clickhouse() + + assert "NoSuchBucket" in instance.query_and_get_error(f"SELECT count() FROM {TABLE_NAME}") + + minio_client.make_bucket(bucket) + + files = upload_directory( + minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + ) + + assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 From d739a7230a7af01bde51971872076ba1026ca141 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 22 Jan 2024 17:37:16 +0000 Subject: [PATCH 113/264] Fix style --- tests/integration/test_system_reload_async_metrics/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_system_reload_async_metrics/test.py b/tests/integration/test_system_reload_async_metrics/test.py index 00616f4293f..941fb90f464 100644 --- a/tests/integration/test_system_reload_async_metrics/test.py +++ b/tests/integration/test_system_reload_async_metrics/test.py @@ -36,7 +36,9 @@ def test_query_cache_size_is_runtime_configurable(start_cluster): "SELECT value FROM system.asynchronous_metrics WHERE metric = 'NumberOfTables'" ) - node.query("CREATE TABLE tab (col UInt64) ENGINE MergeTree ORDER BY tuple()") # do anything dumb + node.query( + "CREATE TABLE tab (col UInt64) ENGINE MergeTree ORDER BY tuple()" + ) # do anything dumb node.query("SYSTEM RELOAD ASYNCHRONOUS METRICS") From 48f0580512fd4abfaf52c187dcb30dee7639f66f Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 22 Jan 2024 17:39:44 +0000 Subject: [PATCH 114/264] Automatic style fix --- tests/integration/helpers/s3_tools.py | 4 +--- tests/integration/test_storage_delta/test.py | 4 +++- tests/integration/test_storage_iceberg/test.py | 4 +++- 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/tests/integration/helpers/s3_tools.py b/tests/integration/helpers/s3_tools.py index c4f4c3f5aaa..0c3538c3c39 100644 --- a/tests/integration/helpers/s3_tools.py +++ b/tests/integration/helpers/s3_tools.py @@ -40,9 +40,7 @@ def list_s3_objects(minio_client, bucket, prefix=""): prefix_len = len(prefix) return [ obj.object_name[prefix_len:] - for obj in minio_client.list_objects( - bucket, prefix=prefix, recursive=True - ) + for obj in minio_client.list_objects(bucket, prefix=prefix, recursive=True) ] diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index 8f7349ad709..c76a5251d8d 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -448,7 +448,9 @@ def test_restart_broken(started_cluster): instance.restart_clickhouse() - assert "NoSuchBucket" in instance.query_and_get_error(f"SELECT count() FROM {TABLE_NAME}") + assert "NoSuchBucket" in instance.query_and_get_error( + f"SELECT count() FROM {TABLE_NAME}" + ) minio_client.make_bucket(bucket) diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index ee7c20025f4..c4b48ce884c 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -600,7 +600,9 @@ def test_restart_broken(started_cluster): instance.restart_clickhouse() - assert "NoSuchBucket" in instance.query_and_get_error(f"SELECT count() FROM {TABLE_NAME}") + assert "NoSuchBucket" in instance.query_and_get_error( + f"SELECT count() FROM {TABLE_NAME}" + ) minio_client.make_bucket(bucket) From f5ffe4e47304ca662fe604e9a2585a338a9715c0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 22 Jan 2024 18:58:06 +0100 Subject: [PATCH 115/264] Try getting a better error --- src/Functions/array/FunctionArrayMapped.h | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Functions/array/FunctionArrayMapped.h b/src/Functions/array/FunctionArrayMapped.h index 9773673c63c..49ed9d495e2 100644 --- a/src/Functions/array/FunctionArrayMapped.h +++ b/src/Functions/array/FunctionArrayMapped.h @@ -335,7 +335,11 @@ public: && column_array->getOffsets() != typeid_cast(*offsets_column).getData()) throw Exception( ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, - "Arrays passed to {} must have equal size", getName()); + "Arrays passed to {} must have equal size. Argument {} has size {}, but expected {}", + getName(), + i, + column_array->getOffsets().size(), + typeid_cast(*offsets_column).getData().size()); } const auto * column_tuple = checkAndGetColumn(&column_array->getData()); From f1220c56379beb07bb1bfdc37bc5e625fead228e Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Mon, 22 Jan 2024 19:19:22 +0000 Subject: [PATCH 116/264] CI: hot fix for reuse --- tests/ci/ci_config.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 89133f6cb79..ece7f2f7bae 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -534,7 +534,7 @@ class CiConfig: @classmethod def is_docs_job(cls, job: str) -> bool: - return job != JobNames.DOCS_CHECK + return job == JobNames.DOCS_CHECK def validate(self) -> None: errors = [] From bfb822b77550cc8aa0064f991db5fe9a567d62da Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 22 Jan 2024 20:32:47 +0100 Subject: [PATCH 117/264] Try fix style check --- docker/test/stateless/run.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 27bd263f349..638a2408748 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -253,9 +253,9 @@ do successfuly_saved=$? if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst - successfuly_saved=$(($successfuly_saved | $?)) + successfuly_saved=$((successfuly_saved | $?)) clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst - successfuly_saved=$(($successfuly_saved | $?)) + successfuly_saved=$((successfuly_saved | $?)) fi done From 1d6d1182fd2f91cab01f2d07bacb41e05d9b43a6 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 22 Jan 2024 19:38:26 +0000 Subject: [PATCH 118/264] Fix server shutdown due to exception while loading metadata --- programs/server/Server.cpp | 9 +++++++++ src/Common/AsyncLoader.cpp | 16 ++++++++++------ src/Common/AsyncLoader.h | 3 +++ src/Databases/DatabaseOrdinary.cpp | 5 ++++- src/Interpreters/loadMetadata.cpp | 4 ++-- 5 files changed, 28 insertions(+), 9 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 8281afbf320..70dcff56428 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1740,6 +1740,15 @@ try LOG_INFO(log, "Loading metadata from {}", path_str); LoadTaskPtrs load_metadata_tasks; + + // Make sure that if exception is thrown during startup async, new async loading jobs are not going to be called. + // This is important for the case when exception is thrown from loading of metadata with `async_load_databases = false` + // to avoid simultaneously running table startups and destructing databases. + SCOPE_EXIT_SAFE( + LOG_INFO(log, "Stopping AsyncLoader."); + global_context->getAsyncLoader().stopAndDoNotWait(); // Note that currently running jobs will proceed + ); + try { auto & database_catalog = DatabaseCatalog::instance(); diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 69363a7c54f..13463844cf8 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -13,6 +13,7 @@ #include #include #include +#include "base/types.h" namespace ProfileEvents @@ -38,7 +39,7 @@ void logAboutProgress(Poco::Logger * log, size_t processed, size_t total, Atomic { if (total && (processed % PRINT_MESSAGE_EACH_N_OBJECTS == 0 || watch.compareAndRestart(PRINT_MESSAGE_EACH_N_SECONDS))) { - LOG_INFO(log, "Processed: {}%", processed * 100.0 / total); + LOG_INFO(log, "Processed: {}%", static_cast(processed * 1000.0 / total) * 0.1); watch.restart(); } } @@ -241,14 +242,17 @@ void AsyncLoader::wait() void AsyncLoader::stop() { - { - std::unique_lock lock{mutex}; - is_running = false; - // NOTE: there is no need to notify because workers never wait - } + stopAndDoNotWait(); wait(); } +void AsyncLoader::stopAndDoNotWait() +{ + std::unique_lock lock{mutex}; + is_running = false; + // NOTE: there is no need to notify because workers never wait +} + void AsyncLoader::schedule(LoadTask & task) { chassert(this == &task.loader); diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index b02bc2ac06a..0d91f5fcb05 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -348,6 +348,9 @@ public: // - or canceled using ~Task() or remove() later. void stop(); + // Do not run new jobs + void stopAndDoNotWait(); + // Schedule all jobs of given `task` and their dependencies (even if they are not in task). // All dependencies of a scheduled job inherit its pool if it has higher priority. This way higher priority job // never waits for (blocked by) lower priority jobs. No priority inversion is possible. diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index ba1b2cdacad..14b21e5e7c7 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -215,8 +215,11 @@ LoadTaskPtr DatabaseOrdinary::startupTableAsync( logAboutProgress(log, ++tables_started, total_tables_to_startup, startup_watch); } else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Table {}.{} doesn't exist during startup", + { + // This might happen if synchronous metadata loading failed and server is going to shutdown. + throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist during startup", backQuote(name.database), backQuote(name.table)); + } }); return startup_table[name.table] = makeLoadTask(async_loader, {job}); diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index b2fd43c178c..a2d2c56c710 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -257,10 +257,10 @@ LoadTaskPtrs loadMetadata(ContextMutablePtr context, const String & default_data } else { + // NOTE: some tables can still be started up in the "loading" phase if they are required by dependencies during loading of other tables LOG_INFO(log, "Start synchronous loading of databases"); - - // Note that wait implicitly calls schedule waitLoad(TablesLoaderForegroundPoolId, load_tasks); // First prioritize, schedule and wait all the load table tasks + LOG_INFO(log, "Start synchronous startup of databases"); waitLoad(TablesLoaderForegroundPoolId, startup_tasks); // Only then prioritize, schedule and wait all the startup tasks return {}; } From fad5aeede3f5eb4121acd8cc723311940f910f65 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 22 Jan 2024 19:43:25 +0000 Subject: [PATCH 119/264] fix --- src/Common/AsyncLoader.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 13463844cf8..62f97c18c1d 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -13,7 +13,6 @@ #include #include #include -#include "base/types.h" namespace ProfileEvents From f1ab22d4c3ef76ddf2d3e5cea020b242b61d9afe Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 22 Jan 2024 19:54:50 +0000 Subject: [PATCH 120/264] fix style --- src/Databases/DatabaseOrdinary.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 14b21e5e7c7..8194f04446e 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -37,6 +37,7 @@ namespace DB namespace ErrorCodes { + extern const int UNKNOWN_TABLE; extern const int LOGICAL_ERROR; extern const int UNKNOWN_DATABASE_ENGINE; } From 24b8bbe9fad03a6591fa6c3871927b0ac8af2070 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 22 Jan 2024 17:00:04 -0300 Subject: [PATCH 121/264] Allow to attach partition from table with different partition expression when destination partition expression doesn't re-partition (#39507) * temp commit * temp commit * draft impl for feedback * fix weird style changes * fix weird style changes * fix weird style changes * fix weird style changes * fix weird style changes * aa * aa * Add integ tests and remove partition key restriction * fix small incosistency in partition id * style fix * style fix * style fix * use existing DataPartStorageBuilder instead of new one * Refactor part clone to make it more readable and maintainable * Add MergeTreeDataPartCloner docs * define ErrorCodes::BAD_ARGUMENTS * Rebase * camel case methods * address some comments * yet another rebase? * Move from integ tests to stateless tests * address more comments * add finalize on min_max_idx files * Add sync option to DistinctPartitionExpCloner * just a temp test * revert temp change * Use echoOn to distinguish test queries * remove comment * fix build issue during rebase * atempt to fix build after rebase * finally fix build * clear minmaxidx hyperrectangle before loading it * Fix error on min_max files deletion where it was being assumed that partition expression contained all columns * get it to the state it was previously * add missing include * getting functional? * refactoring and renaming * some more refactoring * extern bad arguments * try to fix style * improvements and docs * remove duplicate includes * fix crash * make tests more stable by ordering * rebase once again.. * fix * make ci happy? * fix rebase issues * docs * rebase, but prolly needs to be improved * refactor out from nasty inheritance to static methods * fix style * work around optional * refactor & integrate some changes * update column_type * add tests by dencrane * set utc * fix ref file * fix tests * use MergeTree instead of SummingMergeTree * mark MergeTreeDataPart::getBlock as const * address a few comments * compute module function name size at compile time * simplify branching in getPartitionAstFieldsCount * remove column_indexes argument * merge getBlock with buildBlock * add some const specifiers * small adjustments * remove no longer needed isNull check * use std::min and max to update global min max idx * add some assertions * forward declare some symbols * fix grammar * forward decl * try to fix build.. * remove IFunction forward decl * Revert "use std::min and max to update global min max idx" This reverts commit b2fe79dda777783da78c07f3ee6c609672dd0895. * Revert "remove no longer needed isNull check" This reverts commit 129db2610f48df002e318ed690ce8f1b349465b4. * Revert "Revert "remove no longer needed isNull check"" This reverts commit 9416087dd8f897d9fd191a8c4308de3c40b28edf. * Revert "Revert "use std::min and max to update global min max idx"" This reverts commit 20246d44167221316c3a7d1e846a02c881e6b576. * remove some comments * partial use of MonotonicityCheckMatcher * ranges * remove KeyDescriptionMonotonicityChecker * remove duplication of applyfunction * move functions to anonymous namespace * move functions to cpp * Relax partition compatibility requirements by accepting subset, add tests from partitioned to unpartitioned * updte reference file * Support for partition by a, b, c to partition by a, b * refactoring part 1 * refactoring part 2, use hyperrectangle, still not complete * refactoring part 3, build hyperrectangle with intersection of source & destination min max columns * Support attaching to table with partition expression of multiple expressions * add tests * rename method * remove some code duplication * draft impl for replicatedmergetree, need to dive deeper * ship ref file * fix impl for replicatedmergetree.. * forbid attach empty partition replicatedmergetree * Add replicated merge tree integration tests * add test missing files * fix black * do not check for monotonicity of empty partition * add empty tests & fix replicated * remove no longer needed buildBlockWithMinMaxINdexes * remove column logic in buildHyperrectangle * simplify implementation by using existing methods * further simplify implementation * move all MergeTreeDataPartClone private methods to .cpp file * decrease decomposition * use different namespaces * reduce code duplication * fix style * address a few comments * add chassert to assert arguments size on MonotonicityCheckVisitor * remove deleteMinMaxFiles method * remove useless checks from sanitycheck * add tests for attach partition (not id) * Remove sanityCheckASTPartition and bring back conditional getPartitionIDFromQuery * remove empty block comment * small fixes * fix formatting * add missing include * remove duplicate iuncludes * trigger ci * reduce some code duplication * use updated partition id on replicatedmergetree * fix build * fix build * small refactor * do not use insert increment on fetch part * remove duplicate includes * add one more integ test * black * black * rely on partition exp instead of partition id on replicated part fetch to decide if it is a different partition exp * add one more integ test * add order by clause * fix black --------- Co-authored-by: Alexey Milovidov --- .../statements/alter/partition.md | 2 +- src/Interpreters/MonotonicityCheckVisitor.h | 102 +++- src/Interpreters/applyFunction.cpp | 43 ++ src/Interpreters/applyFunction.h | 16 + src/Parsers/queryToString.cpp | 5 + src/Parsers/queryToString.h | 1 + src/Storages/MergeTree/IMergeTreeDataPart.cpp | 37 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 1 + src/Storages/MergeTree/KeyCondition.cpp | 83 +-- src/Storages/MergeTree/MergeTreeData.cpp | 350 +++++-------- src/Storages/MergeTree/MergeTreeData.h | 18 + .../MergeTree/MergeTreeDataPartCloner.cpp | 320 ++++++++++++ .../MergeTree/MergeTreeDataPartCloner.h | 43 ++ src/Storages/MergeTree/MergeTreePartition.cpp | 39 ++ src/Storages/MergeTree/MergeTreePartition.h | 10 +- ...ergeTreePartitionCompatibilityVerifier.cpp | 91 ++++ .../MergeTreePartitionCompatibilityVerifier.h | 30 ++ ...TreePartitionGlobalMinMaxIdxCalculator.cpp | 25 + ...geTreePartitionGlobalMinMaxIdxCalculator.h | 24 + src/Storages/StorageMergeTree.cpp | 93 +++- src/Storages/StorageReplicatedMergeTree.cpp | 135 ++++- .../__init__.py | 0 .../configs/remote_servers.xml | 17 + .../test.py | 214 ++++++++ ...artition_different_partition_exp.reference | 467 +++++++++++++++++ ...tach_partition_different_partition_exp.sql | 485 ++++++++++++++++++ 26 files changed, 2310 insertions(+), 341 deletions(-) create mode 100644 src/Interpreters/applyFunction.cpp create mode 100644 src/Interpreters/applyFunction.h create mode 100644 src/Storages/MergeTree/MergeTreeDataPartCloner.cpp create mode 100644 src/Storages/MergeTree/MergeTreeDataPartCloner.h create mode 100644 src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.cpp create mode 100644 src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.h create mode 100644 src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.cpp create mode 100644 src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.h create mode 100644 tests/integration/test_attach_partition_distinct_expression_replicated/__init__.py create mode 100644 tests/integration/test_attach_partition_distinct_expression_replicated/configs/remote_servers.xml create mode 100644 tests/integration/test_attach_partition_distinct_expression_replicated/test.py create mode 100644 tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.reference create mode 100644 tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.sql diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index 114b8d5ffe3..5659a0565c5 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -112,7 +112,7 @@ Note that: For the query to run successfully, the following conditions must be met: - Both tables must have the same structure. -- Both tables must have the same partition key, the same order by key and the same primary key. +- Both tables must have the same order by key and the same primary key. - Both tables must have the same indices and projections. - Both tables must have the same storage policy. diff --git a/src/Interpreters/MonotonicityCheckVisitor.h b/src/Interpreters/MonotonicityCheckVisitor.h index cc386825024..4e71bd56851 100644 --- a/src/Interpreters/MonotonicityCheckVisitor.h +++ b/src/Interpreters/MonotonicityCheckVisitor.h @@ -1,13 +1,17 @@ #pragma once #include +#include #include +#include #include #include -#include #include +#include +#include #include #include +#include #include #include #include @@ -33,6 +37,8 @@ public: ASTIdentifier * identifier = nullptr; DataTypePtr arg_data_type = {}; + Range range = Range::createWholeUniverse(); + void reject() { monotonicity.is_monotonic = false; } bool isRejected() const { return !monotonicity.is_monotonic; } @@ -97,13 +103,30 @@ public: if (data.isRejected()) return; - /// TODO: monotonicity for functions of several arguments - if (!ast_function.arguments || ast_function.arguments->children.size() != 1) + /// Monotonicity check only works for functions that contain at most two arguments and one of them must be a constant. + if (!ast_function.arguments) { data.reject(); return; } + auto arguments_size = ast_function.arguments->children.size(); + + if (arguments_size == 0 || arguments_size > 2) + { + data.reject(); + return; + } + else if (arguments_size == 2) + { + /// If the function has two arguments, then one of them must be a constant. + if (!ast_function.arguments->children[0]->as() && !ast_function.arguments->children[1]->as()) + { + data.reject(); + return; + } + } + if (!data.canOptimize(ast_function)) { data.reject(); @@ -124,14 +147,33 @@ public: return; } - ColumnsWithTypeAndName args; - args.emplace_back(data.arg_data_type, "tmp"); - auto function_base = function->build(args); + auto function_arguments = getFunctionArguments(ast_function, data); + + auto function_base = function->build(function_arguments); if (function_base && function_base->hasInformationAboutMonotonicity()) { bool is_positive = data.monotonicity.is_positive; - data.monotonicity = function_base->getMonotonicityForRange(*data.arg_data_type, Field(), Field()); + data.monotonicity = function_base->getMonotonicityForRange(*data.arg_data_type, data.range.left, data.range.right); + + auto & key_range = data.range; + + /// If we apply function to open interval, we can get empty intervals in result. + /// E.g. for ('2020-01-03', '2020-01-20') after applying 'toYYYYMM' we will get ('202001', '202001'). + /// To avoid this we make range left and right included. + /// Any function that treats NULL specially is not monotonic. + /// Thus we can safely use isNull() as an -Inf/+Inf indicator here. + if (!key_range.left.isNull()) + { + key_range.left = applyFunction(function_base, data.arg_data_type, key_range.left); + key_range.left_included = true; + } + + if (!key_range.right.isNull()) + { + key_range.right = applyFunction(function_base, data.arg_data_type, key_range.right); + key_range.right_included = true; + } if (!is_positive) data.monotonicity.is_positive = !data.monotonicity.is_positive; @@ -143,13 +185,53 @@ public: static bool needChildVisit(const ASTPtr & parent, const ASTPtr &) { - /// Currently we check monotonicity only for single-argument functions. - /// Although, multi-argument functions with all but one constant arguments can also be monotonic. + /// Multi-argument functions with all but one constant arguments can be monotonic. if (const auto * func = typeid_cast(parent.get())) - return func->arguments->children.size() < 2; + return func->arguments->children.size() <= 2; return true; } + + static ColumnWithTypeAndName extractLiteralColumnAndTypeFromAstLiteral(const ASTLiteral * literal) + { + ColumnWithTypeAndName result; + + result.type = applyVisitor(FieldToDataType(), literal->value); + result.column = result.type->createColumnConst(0, literal->value); + + return result; + } + + static ColumnsWithTypeAndName getFunctionArguments(const ASTFunction & ast_function, const Data & data) + { + ColumnsWithTypeAndName args; + + auto arguments_size = ast_function.arguments->children.size(); + + chassert(arguments_size == 1 || arguments_size == 2); + + if (arguments_size == 2) + { + if (ast_function.arguments->children[0]->as()) + { + const auto * literal = ast_function.arguments->children[0]->as(); + args.push_back(extractLiteralColumnAndTypeFromAstLiteral(literal)); + args.emplace_back(data.arg_data_type, "tmp"); + } + else + { + const auto * literal = ast_function.arguments->children[1]->as(); + args.emplace_back(data.arg_data_type, "tmp"); + args.push_back(extractLiteralColumnAndTypeFromAstLiteral(literal)); + } + } + else + { + args.emplace_back(data.arg_data_type, "tmp"); + } + + return args; + } }; using MonotonicityCheckVisitor = ConstInDepthNodeVisitor; diff --git a/src/Interpreters/applyFunction.cpp b/src/Interpreters/applyFunction.cpp new file mode 100644 index 00000000000..a53f14f0381 --- /dev/null +++ b/src/Interpreters/applyFunction.cpp @@ -0,0 +1,43 @@ +#include + +#include +#include + +namespace DB +{ + +static Field applyFunctionForField(const FunctionBasePtr & func, const DataTypePtr & arg_type, const Field & arg_value) +{ + ColumnsWithTypeAndName columns{ + {arg_type->createColumnConst(1, arg_value), arg_type, "x"}, + }; + + auto col = func->execute(columns, func->getResultType(), 1); + return (*col)[0]; +} + +FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & current_type, const FieldRef & field) +{ + /// Fallback for fields without block reference. + if (field.isExplicit()) + return applyFunctionForField(func, current_type, field); + + String result_name = "_" + func->getName() + "_" + toString(field.column_idx); + const auto & columns = field.columns; + size_t result_idx = columns->size(); + + for (size_t i = 0; i < result_idx; ++i) + if ((*columns)[i].name == result_name) + result_idx = i; + + if (result_idx == columns->size()) + { + ColumnsWithTypeAndName args{(*columns)[field.column_idx]}; + field.columns->emplace_back(ColumnWithTypeAndName{nullptr, func->getResultType(), result_name}); + (*columns)[result_idx].column = func->execute(args, (*columns)[result_idx].type, columns->front().column->size()); + } + + return {field.columns, field.row_idx, result_idx}; +} + +} diff --git a/src/Interpreters/applyFunction.h b/src/Interpreters/applyFunction.h new file mode 100644 index 00000000000..9b8ae43a53c --- /dev/null +++ b/src/Interpreters/applyFunction.h @@ -0,0 +1,16 @@ +#pragma once + +#include + +namespace DB +{ +struct FieldRef; + +class IFunctionBase; +class IDataType; + +using DataTypePtr = std::shared_ptr; +using FunctionBasePtr = std::shared_ptr; + +FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & current_type, const FieldRef & field); +} diff --git a/src/Parsers/queryToString.cpp b/src/Parsers/queryToString.cpp index 9721aa1f128..4a1903393f6 100644 --- a/src/Parsers/queryToString.cpp +++ b/src/Parsers/queryToString.cpp @@ -3,6 +3,11 @@ namespace DB { + String queryToStringNullable(const ASTPtr & query) + { + return query ? queryToString(query) : ""; + } + String queryToString(const ASTPtr & query) { return queryToString(*query); diff --git a/src/Parsers/queryToString.h b/src/Parsers/queryToString.h index 873de218293..3acd560b1e2 100644 --- a/src/Parsers/queryToString.h +++ b/src/Parsers/queryToString.h @@ -6,4 +6,5 @@ namespace DB { String queryToString(const ASTPtr & query); String queryToString(const IAST & query); + String queryToStringNullable(const ASTPtr & query); } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 87f23b0da2a..f3057a8254f 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -81,6 +81,7 @@ void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const Par auto minmax_column_types = data.getMinMaxColumnsTypes(partition_key); size_t minmax_idx_size = minmax_column_types.size(); + hyperrectangle.clear(); hyperrectangle.reserve(minmax_idx_size); for (size_t i = 0; i < minmax_idx_size; ++i) { @@ -104,6 +105,39 @@ void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const Par initialized = true; } +Block IMergeTreeDataPart::MinMaxIndex::getBlock(const MergeTreeData & data) const +{ + if (!initialized) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to get block from uninitialized MinMax index."); + + Block block; + + const auto metadata_snapshot = data.getInMemoryMetadataPtr(); + const auto & partition_key = metadata_snapshot->getPartitionKey(); + + const auto minmax_column_names = data.getMinMaxColumnsNames(partition_key); + const auto minmax_column_types = data.getMinMaxColumnsTypes(partition_key); + const auto minmax_idx_size = minmax_column_types.size(); + + for (size_t i = 0; i < minmax_idx_size; ++i) + { + const auto & data_type = minmax_column_types[i]; + const auto & column_name = minmax_column_names[i]; + + const auto column = data_type->createColumn(); + + const auto min_val = hyperrectangle.at(i).left; + const auto max_val = hyperrectangle.at(i).right; + + column->insert(min_val); + column->insert(max_val); + + block.insert(ColumnWithTypeAndName(column->getPtr(), data_type, column_name)); + } + + return block; +} + IMergeTreeDataPart::MinMaxIndex::WrittenFiles IMergeTreeDataPart::MinMaxIndex::store( const MergeTreeData & data, IDataPartStorage & part_storage, Checksums & out_checksums) const { @@ -185,8 +219,7 @@ void IMergeTreeDataPart::MinMaxIndex::merge(const MinMaxIndex & other) if (!initialized) { - hyperrectangle = other.hyperrectangle; - initialized = true; + *this = other; } else { diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 640a1f1d0a3..29f0f54d419 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -336,6 +336,7 @@ public: } void load(const MergeTreeData & data, const PartMetadataManagerPtr & manager); + Block getBlock(const MergeTreeData & data) const; using WrittenFiles = std::vector>; diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index d5922ae1bc2..e5bcb11091f 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1,36 +1,37 @@ -#include -#include -#include +#include +#include #include #include #include #include +#include #include -#include #include -#include -#include -#include -#include -#include -#include -#include +#include #include +#include #include -#include -#include -#include -#include -#include -#include +#include +#include +#include +#include +#include #include -#include +#include +#include +#include +#include +#include #include #include #include -#include -#include +#include +#include +#include #include +#include +#include +#include #include #include @@ -836,21 +837,6 @@ bool KeyCondition::getConstant(const ASTPtr & expr, Block & block_with_constants return node.tryGetConstant(out_value, out_type); } - -static Field applyFunctionForField( - const FunctionBasePtr & func, - const DataTypePtr & arg_type, - const Field & arg_value) -{ - ColumnsWithTypeAndName columns - { - { arg_type->createColumnConst(1, arg_value), arg_type, "x" }, - }; - - auto col = func->execute(columns, func->getResultType(), 1); - return (*col)[0]; -} - /// The case when arguments may have types different than in the primary key. static std::pair applyFunctionForFieldOfUnknownType( const FunctionBasePtr & func, @@ -890,33 +876,6 @@ static std::pair applyBinaryFunctionForFieldOfUnknownType( return {std::move(result), std::move(return_type)}; } - -static FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & current_type, const FieldRef & field) -{ - /// Fallback for fields without block reference. - if (field.isExplicit()) - return applyFunctionForField(func, current_type, field); - - String result_name = "_" + func->getName() + "_" + toString(field.column_idx); - const auto & columns = field.columns; - size_t result_idx = columns->size(); - - for (size_t i = 0; i < result_idx; ++i) - { - if ((*columns)[i].name == result_name) - result_idx = i; - } - - if (result_idx == columns->size()) - { - ColumnsWithTypeAndName args{(*columns)[field.column_idx]}; - field.columns->emplace_back(ColumnWithTypeAndName {nullptr, func->getResultType(), result_name}); - (*columns)[result_idx].column = func->execute(args, (*columns)[result_idx].type, columns->front().column->size()); - } - - return {field.columns, field.row_idx, result_idx}; -} - /** When table's key has expression with these functions from a column, * and when a column in a query is compared with a constant, such as: * CREATE TABLE (x String) ORDER BY toDate(x) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 61332a4ff38..c3e348a549a 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8,21 +8,6 @@ #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include #include #include @@ -43,19 +28,20 @@ #include #include #include -#include -#include #include #include #include #include #include #include +#include +#include #include +#include #include -#include -#include #include +#include +#include #include #include #include @@ -64,26 +50,41 @@ #include #include #include -#include #include #include #include #include #include #include +#include #include #include +#include #include #include #include #include #include +#include #include #include -#include #include #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include @@ -197,6 +198,50 @@ namespace ErrorCodes extern const int LIMIT_EXCEEDED; } +static size_t getPartitionAstFieldsCount(const ASTPartition & partition_ast, ASTPtr partition_value_ast) +{ + if (partition_ast.fields_count.has_value()) + return *partition_ast.fields_count; + + if (partition_value_ast->as()) + return 1; + + const auto * tuple_ast = partition_value_ast->as(); + + if (!tuple_ast) + { + throw Exception( + ErrorCodes::INVALID_PARTITION_VALUE, "Expected literal or tuple for partition key, got {}", partition_value_ast->getID()); + } + + if (tuple_ast->name != "tuple") + { + if (!isFunctionCast(tuple_ast)) + throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name); + + if (tuple_ast->arguments->as()->children.empty()) + throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name); + + auto first_arg = tuple_ast->arguments->as()->children.at(0); + if (const auto * inner_tuple = first_arg->as(); inner_tuple && inner_tuple->name == "tuple") + { + const auto * arguments_ast = tuple_ast->arguments->as(); + return arguments_ast ? arguments_ast->children.size() : 0; + } + else if (const auto * inner_literal_tuple = first_arg->as(); inner_literal_tuple) + { + return inner_literal_tuple->value.getType() == Field::Types::Tuple ? inner_literal_tuple->value.safeGet().size() : 1; + } + + throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name); + } + else + { + const auto * arguments_ast = tuple_ast->arguments->as(); + return arguments_ast ? arguments_ast->children.size() : 0; + } +} + static void checkSuspiciousIndices(const ASTFunction * index_function) { std::unordered_set unique_index_expression_hashes; @@ -4854,7 +4899,7 @@ void MergeTreeData::removePartContributionToColumnAndSecondaryIndexSizes(const D } void MergeTreeData::checkAlterPartitionIsPossible( - const PartitionCommands & commands, const StorageMetadataPtr & /*metadata_snapshot*/, const Settings & settings, ContextPtr local_context) const + const PartitionCommands & commands, const StorageMetadataPtr & /*metadata_snapshot*/, const Settings & settings, ContextPtr) const { for (const auto & command : commands) { @@ -4882,7 +4927,15 @@ void MergeTreeData::checkAlterPartitionIsPossible( throw DB::Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only support DROP/DETACH PARTITION ALL currently"); } else - getPartitionIDFromQuery(command.partition, local_context); + { + // The below `getPartitionIDFromQuery` call will not work for attach / replace because it assumes the partition expressions + // are the same and deliberately uses this storage. Later on, `MergeTreeData::replaceFrom` is called, and it makes the right + // call to `getPartitionIDFromQuery` using source storage. + // Note: `PartitionCommand::REPLACE_PARTITION` is used both for `REPLACE PARTITION` and `ATTACH PARTITION FROM` queries. + // But not for `ATTACH PARTITION` queries. + if (command.type != PartitionCommand::REPLACE_PARTITION) + getPartitionIDFromQuery(command.partition, getContext()); + } } } } @@ -5616,69 +5669,8 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc MergeTreePartInfo::validatePartitionID(partition_ast.id->clone(), format_version); return partition_ast.id->as()->value.safeGet(); } - size_t partition_ast_fields_count = 0; ASTPtr partition_value_ast = partition_ast.value->clone(); - if (!partition_ast.fields_count.has_value()) - { - if (partition_value_ast->as()) - { - partition_ast_fields_count = 1; - } - else if (const auto * tuple_ast = partition_value_ast->as()) - { - if (tuple_ast->name != "tuple") - { - if (isFunctionCast(tuple_ast)) - { - if (tuple_ast->arguments->as()->children.empty()) - { - throw Exception( - ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name); - } - auto first_arg = tuple_ast->arguments->as()->children.at(0); - if (const auto * inner_tuple = first_arg->as(); inner_tuple && inner_tuple->name == "tuple") - { - const auto * arguments_ast = tuple_ast->arguments->as(); - if (arguments_ast) - partition_ast_fields_count = arguments_ast->children.size(); - else - partition_ast_fields_count = 0; - } - else if (const auto * inner_literal_tuple = first_arg->as(); inner_literal_tuple) - { - if (inner_literal_tuple->value.getType() == Field::Types::Tuple) - partition_ast_fields_count = inner_literal_tuple->value.safeGet().size(); - else - partition_ast_fields_count = 1; - } - else - { - throw Exception( - ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name); - } - } - else - throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name); - } - else - { - const auto * arguments_ast = tuple_ast->arguments->as(); - if (arguments_ast) - partition_ast_fields_count = arguments_ast->children.size(); - else - partition_ast_fields_count = 0; - } - } - else - { - throw Exception( - ErrorCodes::INVALID_PARTITION_VALUE, "Expected literal or tuple for partition key, got {}", partition_value_ast->getID()); - } - } - else - { - partition_ast_fields_count = *partition_ast.fields_count; - } + auto partition_ast_fields_count = getPartitionAstFieldsCount(partition_ast, partition_value_ast); if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { @@ -7014,23 +7006,35 @@ MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(IStorage & sour if (my_snapshot->getColumns().getAllPhysical().sizeOfDifference(src_snapshot->getColumns().getAllPhysical())) throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Tables have different structure"); - auto query_to_string = [] (const ASTPtr & ast) - { - return ast ? queryToString(ast) : ""; - }; - - if (query_to_string(my_snapshot->getSortingKeyAST()) != query_to_string(src_snapshot->getSortingKeyAST())) + if (queryToStringNullable(my_snapshot->getSortingKeyAST()) != queryToStringNullable(src_snapshot->getSortingKeyAST())) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different ordering"); - if (query_to_string(my_snapshot->getPartitionKeyAST()) != query_to_string(src_snapshot->getPartitionKeyAST())) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different partition key"); - if (format_version != src_data->format_version) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different format_version"); - if (query_to_string(my_snapshot->getPrimaryKeyAST()) != query_to_string(src_snapshot->getPrimaryKeyAST())) + if (queryToStringNullable(my_snapshot->getPrimaryKeyAST()) != queryToStringNullable(src_snapshot->getPrimaryKeyAST())) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different primary key"); + const auto is_a_subset_of = [](const auto & lhs, const auto & rhs) + { + if (lhs.size() > rhs.size()) + return false; + + const auto rhs_set = NameSet(rhs.begin(), rhs.end()); + for (const auto & lhs_element : lhs) + if (!rhs_set.contains(lhs_element)) + return false; + + return true; + }; + + if (!is_a_subset_of(my_snapshot->getColumnsRequiredForPartitionKey(), src_snapshot->getColumnsRequiredForPartitionKey())) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Destination table partition expression columns must be a subset of source table partition expression columns"); + } + const auto check_definitions = [](const auto & my_descriptions, const auto & src_descriptions) { if (my_descriptions.size() != src_descriptions.size()) @@ -7071,128 +7075,56 @@ std::pair MergeTreeData::cloneAn const ReadSettings & read_settings, const WriteSettings & write_settings) { - /// Check that the storage policy contains the disk where the src_part is located. - bool does_storage_policy_allow_same_disk = false; - for (const DiskPtr & disk : getStoragePolicy()->getDisks()) - { - if (disk->getName() == src_part->getDataPartStorage().getDiskName()) - { - does_storage_policy_allow_same_disk = true; - break; - } - } - if (!does_storage_policy_allow_same_disk) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Could not clone and load part {} because disk does not belong to storage policy", - quoteString(src_part->getDataPartStorage().getFullPath())); + return MergeTreeDataPartCloner::clone( + this, src_part, metadata_snapshot, dst_part_info, tmp_part_prefix, require_part_metadata, params, read_settings, write_settings); +} - String dst_part_name = src_part->getNewName(dst_part_info); - String tmp_dst_part_name = tmp_part_prefix + dst_part_name; - auto temporary_directory_lock = getTemporaryPartDirectoryHolder(tmp_dst_part_name); +std::pair MergeTreeData::cloneAndLoadPartOnSameDiskWithDifferentPartitionKey( + const MergeTreeData::DataPartPtr & src_part, + const MergeTreePartition & new_partition, + const String & partition_id, + const IMergeTreeDataPart::MinMaxIndex & min_max_index, + const String & tmp_part_prefix, + const StorageMetadataPtr & my_metadata_snapshot, + const IDataPartStorage::ClonePartParams & clone_params, + ContextPtr local_context, + Int64 min_block, + Int64 max_block +) +{ + MergeTreePartInfo dst_part_info(partition_id, min_block, max_block, src_part->info.level); - /// Why it is needed if we only hardlink files? - auto reservation = src_part->getDataPartStorage().reserve(src_part->getBytesOnDisk()); - auto src_part_storage = src_part->getDataPartStoragePtr(); + return MergeTreeDataPartCloner::cloneWithDistinctPartitionExpression( + this, + src_part, + my_metadata_snapshot, + dst_part_info, + tmp_part_prefix, + local_context->getReadSettings(), + local_context->getWriteSettings(), + new_partition, + min_max_index, + false, + clone_params); +} - scope_guard src_flushed_tmp_dir_lock; - MergeTreeData::MutableDataPartPtr src_flushed_tmp_part; +std::pair MergeTreeData::createPartitionAndMinMaxIndexFromSourcePart( + const MergeTreeData::DataPartPtr & src_part, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr local_context) +{ + const auto & src_data = src_part->storage; - /// If source part is in memory, flush it to disk and clone it already in on-disk format - /// Protect tmp dir from removing by cleanup thread with src_flushed_tmp_dir_lock - /// Construct src_flushed_tmp_part in order to delete part with its directory at destructor - if (auto src_part_in_memory = asInMemoryPart(src_part)) - { - auto flushed_part_path = *src_part_in_memory->getRelativePathForPrefix(tmp_part_prefix); + auto metadata_manager = std::make_shared(src_part.get()); + IMergeTreeDataPart::MinMaxIndex min_max_index; - auto tmp_src_part_file_name = fs::path(tmp_dst_part_name).filename(); - src_flushed_tmp_dir_lock = src_part->storage.getTemporaryPartDirectoryHolder(tmp_src_part_file_name); + min_max_index.load(src_data, metadata_manager); - auto flushed_part_storage = src_part_in_memory->flushToDisk(flushed_part_path, metadata_snapshot); + MergeTreePartition new_partition; - src_flushed_tmp_part = MergeTreeDataPartBuilder(*this, src_part->name, flushed_part_storage) - .withPartInfo(src_part->info) - .withPartFormatFromDisk() - .build(); + new_partition.create(metadata_snapshot, min_max_index.getBlock(src_data), 0u, local_context); - src_flushed_tmp_part->is_temp = true; - src_part_storage = flushed_part_storage; - } - - String with_copy; - if (params.copy_instead_of_hardlink) - with_copy = " (copying data)"; - - auto dst_part_storage = src_part_storage->freeze( - relative_data_path, - tmp_dst_part_name, - read_settings, - write_settings, - /* save_metadata_callback= */ {}, - params); - - if (params.metadata_version_to_write.has_value()) - { - chassert(!params.keep_metadata_version); - auto out_metadata = dst_part_storage->writeFile(IMergeTreeDataPart::METADATA_VERSION_FILE_NAME, 4096, getContext()->getWriteSettings()); - writeText(metadata_snapshot->getMetadataVersion(), *out_metadata); - out_metadata->finalize(); - if (getSettings()->fsync_after_insert) - out_metadata->sync(); - } - - LOG_DEBUG(log, "Clone{} part {} to {}{}", - src_flushed_tmp_part ? " flushed" : "", - src_part_storage->getFullPath(), - std::string(fs::path(dst_part_storage->getFullRootPath()) / tmp_dst_part_name), - with_copy); - - auto dst_data_part = MergeTreeDataPartBuilder(*this, dst_part_name, dst_part_storage) - .withPartFormatFromDisk() - .build(); - - if (!params.copy_instead_of_hardlink && params.hardlinked_files) - { - params.hardlinked_files->source_part_name = src_part->name; - params.hardlinked_files->source_table_shared_id = src_part->storage.getTableSharedID(); - - for (auto it = src_part->getDataPartStorage().iterate(); it->isValid(); it->next()) - { - if (!params.files_to_copy_instead_of_hardlinks.contains(it->name()) - && it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED - && it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME) - { - params.hardlinked_files->hardlinks_from_source_part.insert(it->name()); - } - } - - auto projections = src_part->getProjectionParts(); - for (const auto & [name, projection_part] : projections) - { - const auto & projection_storage = projection_part->getDataPartStorage(); - for (auto it = projection_storage.iterate(); it->isValid(); it->next()) - { - auto file_name_with_projection_prefix = fs::path(projection_storage.getPartDirectory()) / it->name(); - if (!params.files_to_copy_instead_of_hardlinks.contains(file_name_with_projection_prefix) - && it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED - && it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME) - { - params.hardlinked_files->hardlinks_from_source_part.insert(file_name_with_projection_prefix); - } - } - } - } - - /// We should write version metadata on part creation to distinguish it from parts that were created without transaction. - TransactionID tid = params.txn ? params.txn->tid : Tx::PrehistoricTID; - dst_data_part->version.setCreationTID(tid, nullptr); - dst_data_part->storeVersionMetadata(); - - dst_data_part->is_temp = true; - - dst_data_part->loadColumnsChecksumsIndexes(require_part_metadata, true); - dst_data_part->modification_time = dst_part_storage->getLastModified().epochTime(); - return std::make_pair(dst_data_part, std::move(temporary_directory_lock)); + return {new_partition, min_max_index}; } String MergeTreeData::getFullPathOnDisk(const DiskPtr & disk) const diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index f0dbaf0e307..9c433e11b84 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -231,6 +231,7 @@ public: } }; + using DataParts = std::set; using MutableDataParts = std::set; using DataPartsVector = std::vector; @@ -848,6 +849,23 @@ public: const ReadSettings & read_settings, const WriteSettings & write_settings); + std::pair cloneAndLoadPartOnSameDiskWithDifferentPartitionKey( + const MergeTreeData::DataPartPtr & src_part, + const MergeTreePartition & new_partition, + const String & partition_id, + const IMergeTreeDataPart::MinMaxIndex & min_max_index, + const String & tmp_part_prefix, + const StorageMetadataPtr & my_metadata_snapshot, + const IDataPartStorage::ClonePartParams & clone_params, + ContextPtr local_context, + Int64 min_block, + Int64 max_block); + + static std::pair createPartitionAndMinMaxIndexFromSourcePart( + const MergeTreeData::DataPartPtr & src_part, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr local_context); + virtual std::vector getMutationsStatus() const = 0; /// Returns true if table can create new parts with adaptive granularity diff --git a/src/Storages/MergeTree/MergeTreeDataPartCloner.cpp b/src/Storages/MergeTree/MergeTreeDataPartCloner.cpp new file mode 100644 index 00000000000..78cb9aa0624 --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeDataPartCloner.cpp @@ -0,0 +1,320 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +} + +static Poco::Logger * log = &Poco::Logger::get("MergeTreeDataPartCloner"); + +namespace DistinctPartitionExpression +{ +std::unique_ptr updatePartitionFile( + const MergeTreeData & merge_tree_data, + const MergeTreePartition & partition, + const MergeTreeData::MutableDataPartPtr & dst_part, + IDataPartStorage & storage) +{ + storage.removeFile("partition.dat"); + // Leverage already implemented MergeTreePartition::store to create & store partition.dat. + // Checksum is re-calculated later. + return partition.store(merge_tree_data, storage, dst_part->checksums); +} + +IMergeTreeDataPart::MinMaxIndex::WrittenFiles updateMinMaxFiles( + const MergeTreeData & merge_tree_data, + const MergeTreeData::MutableDataPartPtr & dst_part, + IDataPartStorage & storage, + const StorageMetadataPtr & metadata_snapshot) +{ + for (const auto & column_name : MergeTreeData::getMinMaxColumnsNames(metadata_snapshot->partition_key)) + { + auto file = "minmax_" + escapeForFileName(column_name) + ".idx"; + storage.removeFile(file); + } + + return dst_part->minmax_idx->store(merge_tree_data, storage, dst_part->checksums); +} + +void finalizeNewFiles(const std::vector> & files, bool sync_new_files) +{ + for (const auto & file : files) + { + file->finalize(); + if (sync_new_files) + file->sync(); + } +} + +void updateNewPartFiles( + const MergeTreeData & merge_tree_data, + const MergeTreeData::MutableDataPartPtr & dst_part, + const MergeTreePartition & new_partition, + const IMergeTreeDataPart::MinMaxIndex & new_min_max_index, + const StorageMetadataPtr & src_metadata_snapshot, + bool sync_new_files) +{ + auto & storage = dst_part->getDataPartStorage(); + + *dst_part->minmax_idx = new_min_max_index; + + auto partition_file = updatePartitionFile(merge_tree_data, new_partition, dst_part, storage); + + auto min_max_files = updateMinMaxFiles(merge_tree_data, dst_part, storage, src_metadata_snapshot); + + IMergeTreeDataPart::MinMaxIndex::WrittenFiles written_files; + + if (partition_file) + written_files.emplace_back(std::move(partition_file)); + + written_files.insert(written_files.end(), std::make_move_iterator(min_max_files.begin()), std::make_move_iterator(min_max_files.end())); + + finalizeNewFiles(written_files, sync_new_files); + + // MergeTreeDataPartCloner::finalize_part calls IMergeTreeDataPart::loadColumnsChecksumsIndexes, which will re-create + // the checksum file if it doesn't exist. Relying on that is cumbersome, but this refactoring is simply a code extraction + // with small improvements. It can be further improved in the future. + storage.removeFile("checksums.txt"); +} +} + +namespace +{ +bool doesStoragePolicyAllowSameDisk(MergeTreeData * merge_tree_data, const MergeTreeData::DataPartPtr & src_part) +{ + for (const DiskPtr & disk : merge_tree_data->getStoragePolicy()->getDisks()) + if (disk->getName() == src_part->getDataPartStorage().getDiskName()) + return true; + return false; +} + +DataPartStoragePtr flushPartStorageToDiskIfInMemory( + MergeTreeData * merge_tree_data, + const MergeTreeData::DataPartPtr & src_part, + const StorageMetadataPtr & metadata_snapshot, + const String & tmp_part_prefix, + const String & tmp_dst_part_name, + scope_guard & src_flushed_tmp_dir_lock, + MergeTreeData::MutableDataPartPtr src_flushed_tmp_part) +{ + if (auto src_part_in_memory = asInMemoryPart(src_part)) + { + auto flushed_part_path = src_part_in_memory->getRelativePathForPrefix(tmp_part_prefix); + auto tmp_src_part_file_name = fs::path(tmp_dst_part_name).filename(); + + src_flushed_tmp_dir_lock = src_part->storage.getTemporaryPartDirectoryHolder(tmp_src_part_file_name); + + auto flushed_part_storage = src_part_in_memory->flushToDisk(*flushed_part_path, metadata_snapshot); + + src_flushed_tmp_part = MergeTreeDataPartBuilder(*merge_tree_data, src_part->name, flushed_part_storage) + .withPartInfo(src_part->info) + .withPartFormatFromDisk() + .build(); + + src_flushed_tmp_part->is_temp = true; + + return flushed_part_storage; + } + + return src_part->getDataPartStoragePtr(); +} + +std::shared_ptr hardlinkAllFiles( + MergeTreeData * merge_tree_data, + const DB::ReadSettings & read_settings, + const DB::WriteSettings & write_settings, + const DataPartStoragePtr & storage, + const String & path, + const DB::IDataPartStorage::ClonePartParams & params) +{ + return storage->freeze( + merge_tree_data->getRelativeDataPath(), + path, + read_settings, + write_settings, + /*save_metadata_callback=*/{}, + params); +} + +std::pair cloneSourcePart( + MergeTreeData * merge_tree_data, + const MergeTreeData::DataPartPtr & src_part, + const StorageMetadataPtr & metadata_snapshot, + const MergeTreePartInfo & dst_part_info, + const String & tmp_part_prefix, + const ReadSettings & read_settings, + const WriteSettings & write_settings, + const DB::IDataPartStorage::ClonePartParams & params) +{ + const auto dst_part_name = src_part->getNewName(dst_part_info); + + const auto tmp_dst_part_name = tmp_part_prefix + dst_part_name; + + auto temporary_directory_lock = merge_tree_data->getTemporaryPartDirectoryHolder(tmp_dst_part_name); + + src_part->getDataPartStorage().reserve(src_part->getBytesOnDisk()); + + scope_guard src_flushed_tmp_dir_lock; + MergeTreeData::MutableDataPartPtr src_flushed_tmp_part; + + auto src_part_storage = flushPartStorageToDiskIfInMemory( + merge_tree_data, src_part, metadata_snapshot, tmp_part_prefix, tmp_dst_part_name, src_flushed_tmp_dir_lock, src_flushed_tmp_part); + + auto dst_part_storage = hardlinkAllFiles(merge_tree_data, read_settings, write_settings, src_part_storage, tmp_dst_part_name, params); + + if (params.metadata_version_to_write.has_value()) + { + chassert(!params.keep_metadata_version); + auto out_metadata = dst_part_storage->writeFile( + IMergeTreeDataPart::METADATA_VERSION_FILE_NAME, 4096, merge_tree_data->getContext()->getWriteSettings()); + writeText(metadata_snapshot->getMetadataVersion(), *out_metadata); + out_metadata->finalize(); + if (merge_tree_data->getSettings()->fsync_after_insert) + out_metadata->sync(); + } + + LOG_DEBUG( + log, + "Clone {} part {} to {}{}", + src_flushed_tmp_part ? "flushed" : "", + src_part_storage->getFullPath(), + std::string(fs::path(dst_part_storage->getFullRootPath()) / tmp_dst_part_name), + false); + + + auto part = MergeTreeDataPartBuilder(*merge_tree_data, dst_part_name, dst_part_storage).withPartFormatFromDisk().build(); + + return std::make_pair(part, std::move(temporary_directory_lock)); +} + +void handleHardLinkedParameterFiles(const MergeTreeData::DataPartPtr & src_part, const DB::IDataPartStorage::ClonePartParams & params) +{ + const auto & hardlinked_files = params.hardlinked_files; + + hardlinked_files->source_part_name = src_part->name; + hardlinked_files->source_table_shared_id = src_part->storage.getTableSharedID(); + + for (auto it = src_part->getDataPartStorage().iterate(); it->isValid(); it->next()) + { + if (!params.files_to_copy_instead_of_hardlinks.contains(it->name()) + && it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED + && it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME) + { + hardlinked_files->hardlinks_from_source_part.insert(it->name()); + } + } +} + +void handleProjections(const MergeTreeData::DataPartPtr & src_part, const DB::IDataPartStorage::ClonePartParams & params) +{ + auto projections = src_part->getProjectionParts(); + for (const auto & [name, projection_part] : projections) + { + const auto & projection_storage = projection_part->getDataPartStorage(); + for (auto it = projection_storage.iterate(); it->isValid(); it->next()) + { + auto file_name_with_projection_prefix = fs::path(projection_storage.getPartDirectory()) / it->name(); + if (!params.files_to_copy_instead_of_hardlinks.contains(file_name_with_projection_prefix) + && it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED + && it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME) + { + params.hardlinked_files->hardlinks_from_source_part.insert(file_name_with_projection_prefix); + } + } + } +} + +MergeTreeData::MutableDataPartPtr finalizePart( + const MergeTreeData::MutableDataPartPtr & dst_part, const DB::IDataPartStorage::ClonePartParams & params, bool require_part_metadata) +{ + /// We should write version metadata on part creation to distinguish it from parts that were created without transaction. + TransactionID tid = params.txn ? params.txn->tid : Tx::PrehistoricTID; + dst_part->version.setCreationTID(tid, nullptr); + dst_part->storeVersionMetadata(); + + dst_part->is_temp = true; + + dst_part->loadColumnsChecksumsIndexes(require_part_metadata, true); + + dst_part->modification_time = dst_part->getDataPartStorage().getLastModified().epochTime(); + + return dst_part; +} + +std::pair cloneAndHandleHardlinksAndProjections( + MergeTreeData * merge_tree_data, + const DataPartPtr & src_part, + const StorageMetadataPtr & metadata_snapshot, + const MergeTreePartInfo & dst_part_info, + const String & tmp_part_prefix, + const ReadSettings & read_settings, + const WriteSettings & write_settings, + const IDataPartStorage::ClonePartParams & params) +{ + if (!doesStoragePolicyAllowSameDisk(merge_tree_data, src_part)) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Could not clone and load part {} because disk does not belong to storage policy", + quoteString(src_part->getDataPartStorage().getFullPath())); + + auto [destination_part, temporary_directory_lock] = cloneSourcePart( + merge_tree_data, src_part, metadata_snapshot, dst_part_info, tmp_part_prefix, read_settings, write_settings, params); + + if (!params.copy_instead_of_hardlink && params.hardlinked_files) + { + handleHardLinkedParameterFiles(src_part, params); + handleProjections(src_part, params); + } + + return std::make_pair(destination_part, std::move(temporary_directory_lock)); +} +} + +std::pair MergeTreeDataPartCloner::clone( + MergeTreeData * merge_tree_data, + const DataPartPtr & src_part, + const StorageMetadataPtr & metadata_snapshot, + const MergeTreePartInfo & dst_part_info, + const String & tmp_part_prefix, + bool require_part_metadata, + const IDataPartStorage::ClonePartParams & params, + const ReadSettings & read_settings, + const WriteSettings & write_settings) +{ + auto [destination_part, temporary_directory_lock] = cloneAndHandleHardlinksAndProjections( + merge_tree_data, src_part, metadata_snapshot, dst_part_info, tmp_part_prefix, read_settings, write_settings, params); + + return std::make_pair(finalizePart(destination_part, params, require_part_metadata), std::move(temporary_directory_lock)); +} + +std::pair MergeTreeDataPartCloner::cloneWithDistinctPartitionExpression( + MergeTreeData * merge_tree_data, + const DataPartPtr & src_part, + const StorageMetadataPtr & metadata_snapshot, + const MergeTreePartInfo & dst_part_info, + const String & tmp_part_prefix, + const ReadSettings & read_settings, + const WriteSettings & write_settings, + const MergeTreePartition & new_partition, + const IMergeTreeDataPart::MinMaxIndex & new_min_max_index, + bool sync_new_files, + const IDataPartStorage::ClonePartParams & params) +{ + auto [destination_part, temporary_directory_lock] = cloneAndHandleHardlinksAndProjections( + merge_tree_data, src_part, metadata_snapshot, dst_part_info, tmp_part_prefix, read_settings, write_settings, params); + + DistinctPartitionExpression::updateNewPartFiles( + *merge_tree_data, destination_part, new_partition, new_min_max_index, src_part->storage.getInMemoryMetadataPtr(), sync_new_files); + + return std::make_pair(finalizePart(destination_part, params, false), std::move(temporary_directory_lock)); +} + +} diff --git a/src/Storages/MergeTree/MergeTreeDataPartCloner.h b/src/Storages/MergeTree/MergeTreeDataPartCloner.h new file mode 100644 index 00000000000..53585f20b7f --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeDataPartCloner.h @@ -0,0 +1,43 @@ +#pragma once + +namespace DB +{ + +struct StorageInMemoryMetadata; +using StorageMetadataPtr = std::shared_ptr; +struct MergeTreePartition; +class IMergeTreeDataPart; + +class MergeTreeDataPartCloner +{ +public: + using DataPart = IMergeTreeDataPart; + using MutableDataPartPtr = std::shared_ptr; + using DataPartPtr = std::shared_ptr; + + static std::pair clone( + MergeTreeData * merge_tree_data, + const DataPartPtr & src_part, + const StorageMetadataPtr & metadata_snapshot, + const MergeTreePartInfo & dst_part_info, + const String & tmp_part_prefix, + bool require_part_metadata, + const IDataPartStorage::ClonePartParams & params, + const ReadSettings & read_settings, + const WriteSettings & write_settings); + + static std::pair cloneWithDistinctPartitionExpression( + MergeTreeData * merge_tree_data, + const DataPartPtr & src_part, + const StorageMetadataPtr & metadata_snapshot, + const MergeTreePartInfo & dst_part_info, + const String & tmp_part_prefix, + const ReadSettings & read_settings, + const WriteSettings & write_settings, + const MergeTreePartition & new_partition, + const IMergeTreeDataPart::MinMaxIndex & new_min_max_index, + bool sync_new_files, + const IDataPartStorage::ClonePartParams & params); +}; + +} diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index ddeaf69136a..76ef3be25b3 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -467,6 +467,45 @@ void MergeTreePartition::create(const StorageMetadataPtr & metadata_snapshot, Bl } } +void MergeTreePartition::createAndValidateMinMaxPartitionIds( + const StorageMetadataPtr & metadata_snapshot, Block block_with_min_max_partition_ids, ContextPtr context) +{ + if (!metadata_snapshot->hasPartitionKey()) + return; + + auto partition_key_names_and_types = executePartitionByExpression(metadata_snapshot, block_with_min_max_partition_ids, context); + value.resize(partition_key_names_and_types.size()); + + /// Executing partition_by expression adds new columns to passed block according to partition functions. + /// The block is passed by reference and is used afterwards. `moduloLegacy` needs to be substituted back + /// with just `modulo`, because it was a temporary substitution. + static constexpr std::string_view modulo_legacy_function_name = "moduloLegacy"; + + size_t i = 0; + for (const auto & element : partition_key_names_and_types) + { + auto & partition_column = block_with_min_max_partition_ids.getByName(element.name); + + if (element.name.starts_with(modulo_legacy_function_name)) + partition_column.name.replace(0, modulo_legacy_function_name.size(), "modulo"); + + Field extracted_min_partition_id_field; + Field extracted_max_partition_id_field; + + partition_column.column->get(0, extracted_min_partition_id_field); + partition_column.column->get(1, extracted_max_partition_id_field); + + if (extracted_min_partition_id_field != extracted_max_partition_id_field) + { + throw Exception( + ErrorCodes::INVALID_PARTITION_VALUE, + "Can not create the partition. A partition can not contain values that have different partition ids"); + } + + partition_column.column->get(0u, value[i++]); + } +} + NamesAndTypesList MergeTreePartition::executePartitionByExpression(const StorageMetadataPtr & metadata_snapshot, Block & block, ContextPtr context) { auto adjusted_partition_key = adjustPartitionKey(metadata_snapshot, context); diff --git a/src/Storages/MergeTree/MergeTreePartition.h b/src/Storages/MergeTree/MergeTreePartition.h index 78b141f26ec..fd7ae02cde4 100644 --- a/src/Storages/MergeTree/MergeTreePartition.h +++ b/src/Storages/MergeTree/MergeTreePartition.h @@ -1,11 +1,12 @@ #pragma once -#include +#include #include #include #include #include -#include +#include +#include namespace DB { @@ -51,6 +52,11 @@ public: void create(const StorageMetadataPtr & metadata_snapshot, Block block, size_t row, ContextPtr context); + /// Copy of MergeTreePartition::create, but also validates if min max partition keys are equal. If they are different, + /// it means the partition can't be created because the data doesn't belong to the same partition. + void createAndValidateMinMaxPartitionIds( + const StorageMetadataPtr & metadata_snapshot, Block block_with_min_max_partition_ids, ContextPtr context); + static void appendFiles(const MergeTreeData & storage, Strings & files); /// Adjust partition key and execute its expression on block. Return sample block according to used expression. diff --git a/src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.cpp b/src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.cpp new file mode 100644 index 00000000000..21bcdb84a96 --- /dev/null +++ b/src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.cpp @@ -0,0 +1,91 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +} + +namespace +{ +bool isDestinationPartitionExpressionMonotonicallyIncreasing( + const std::vector & hyperrectangle, const MergeTreeData & destination_storage) +{ + auto destination_table_metadata = destination_storage.getInMemoryMetadataPtr(); + + auto key_description = destination_table_metadata->getPartitionKey(); + auto definition_ast = key_description.definition_ast->clone(); + + auto table_identifier = std::make_shared(destination_storage.getStorageID().getTableName()); + auto table_with_columns + = TableWithColumnNamesAndTypes{DatabaseAndTableWithAlias(table_identifier), destination_table_metadata->getColumns().getOrdinary()}; + + auto expression_list = extractKeyExpressionList(definition_ast); + + MonotonicityCheckVisitor::Data data{{table_with_columns}, destination_storage.getContext(), /*group_by_function_hashes*/ {}}; + + for (auto i = 0u; i < expression_list->children.size(); i++) + { + data.range = hyperrectangle[i]; + + MonotonicityCheckVisitor(data).visit(expression_list->children[i]); + + if (!data.monotonicity.is_monotonic || !data.monotonicity.is_positive) + return false; + } + + return true; +} + +bool isExpressionDirectSubsetOf(const ASTPtr source, const ASTPtr destination) +{ + auto source_expression_list = extractKeyExpressionList(source); + auto destination_expression_list = extractKeyExpressionList(destination); + + std::unordered_set source_columns; + + for (auto i = 0u; i < source_expression_list->children.size(); ++i) + source_columns.insert(source_expression_list->children[i]->getColumnName()); + + for (auto i = 0u; i < destination_expression_list->children.size(); ++i) + if (!source_columns.contains(destination_expression_list->children[i]->getColumnName())) + return false; + + return true; +} +} + +void MergeTreePartitionCompatibilityVerifier::verify( + const MergeTreeData & source_storage, const MergeTreeData & destination_storage, const DataPartsVector & source_parts) +{ + const auto source_metadata = source_storage.getInMemoryMetadataPtr(); + const auto destination_metadata = destination_storage.getInMemoryMetadataPtr(); + + const auto source_partition_key_ast = source_metadata->getPartitionKeyAST(); + const auto destination_partition_key_ast = destination_metadata->getPartitionKeyAST(); + + // If destination partition expression columns are a subset of source partition expression columns, + // there is no need to check for monotonicity. + if (isExpressionDirectSubsetOf(source_partition_key_ast, destination_partition_key_ast)) + return; + + const auto src_global_min_max_indexes = MergeTreePartitionGlobalMinMaxIdxCalculator::calculate(source_parts, destination_storage); + + assert(!src_global_min_max_indexes.hyperrectangle.empty()); + + if (!isDestinationPartitionExpressionMonotonicallyIncreasing(src_global_min_max_indexes.hyperrectangle, destination_storage)) + throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Destination table partition expression is not monotonically increasing"); + + MergeTreePartition().createAndValidateMinMaxPartitionIds( + destination_storage.getInMemoryMetadataPtr(), + src_global_min_max_indexes.getBlock(destination_storage), + destination_storage.getContext()); +} + +} diff --git a/src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.h b/src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.h new file mode 100644 index 00000000000..1682add3ebd --- /dev/null +++ b/src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.h @@ -0,0 +1,30 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/* + * Verifies that source and destination partitions are compatible. + * To be compatible, one of the following criteria must be met: + * 1. Destination partition expression columns are a subset of source partition columns; or + * 2. Destination partition expression is monotonic on the source global min_max idx Range AND the computer partition id for + * the source global min_max idx range is the same. + * + * If not, an exception is thrown. + * */ + +class MergeTreePartitionCompatibilityVerifier +{ +public: + using DataPart = IMergeTreeDataPart; + using DataPartPtr = std::shared_ptr; + using DataPartsVector = std::vector; + + static void + verify(const MergeTreeData & source_storage, const MergeTreeData & destination_storage, const DataPartsVector & source_parts); +}; + +} diff --git a/src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.cpp b/src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.cpp new file mode 100644 index 00000000000..0871efadf0c --- /dev/null +++ b/src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.cpp @@ -0,0 +1,25 @@ +#include + +namespace DB +{ + +IMergeTreeDataPart::MinMaxIndex +MergeTreePartitionGlobalMinMaxIdxCalculator::calculate(const DataPartsVector & parts, const MergeTreeData & storage) +{ + IMergeTreeDataPart::MinMaxIndex global_min_max_indexes; + + for (const auto & part : parts) + { + auto metadata_manager = std::make_shared(part.get()); + + auto local_min_max_index = MergeTreeData::DataPart::MinMaxIndex(); + + local_min_max_index.load(storage, metadata_manager); + + global_min_max_indexes.merge(local_min_max_index); + } + + return global_min_max_indexes; +} + +} diff --git a/src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.h b/src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.h new file mode 100644 index 00000000000..4f271177246 --- /dev/null +++ b/src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.h @@ -0,0 +1,24 @@ +#pragma once + +#include + +#include +#include + +namespace DB +{ + +/* + * Calculates global min max indexes for a given set of parts on given storage. + * */ +class MergeTreePartitionGlobalMinMaxIdxCalculator +{ + using DataPart = IMergeTreeDataPart; + using DataPartPtr = std::shared_ptr; + using DataPartsVector = std::vector; + +public: + static IMergeTreeDataPart::MinMaxIndex calculate(const DataPartsVector & parts, const MergeTreeData & storage); +}; + +} diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 4761ccd8b58..fd5354a00a9 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -5,9 +5,9 @@ #include #include -#include #include #include +#include #include "Common/Exception.h" #include #include @@ -20,25 +20,30 @@ #include #include #include +#include #include -#include #include #include #include #include #include -#include #include +#include +#include #include #include #include -#include -#include -#include -#include -#include #include +#include +#include +#include +#include +#include +#include #include +#include +#include +#include #include #include #include @@ -2039,41 +2044,73 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con ProfileEventsScope profile_events_scope; MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table, source_metadata_snapshot, my_metadata_snapshot); - String partition_id = getPartitionIDFromQuery(partition, local_context); + String partition_id = src_data.getPartitionIDFromQuery(partition, local_context); DataPartsVector src_parts = src_data.getVisibleDataPartsVectorInPartition(local_context, partition_id); + + bool attach_empty_partition = !replace && src_parts.empty(); + if (attach_empty_partition) + return; + MutableDataPartsVector dst_parts; std::vector dst_parts_locks; static const String TMP_PREFIX = "tmp_replace_from_"; - for (const DataPartPtr & src_part : src_parts) + const auto my_partition_expression = my_metadata_snapshot->getPartitionKeyAST(); + const auto src_partition_expression = source_metadata_snapshot->getPartitionKeyAST(); + const auto is_partition_exp_different = queryToStringNullable(my_partition_expression) != queryToStringNullable(src_partition_expression); + + if (is_partition_exp_different && !src_parts.empty()) + MergeTreePartitionCompatibilityVerifier::verify(src_data, /* destination_storage */ *this, src_parts); + + for (DataPartPtr & src_part : src_parts) { if (!canReplacePartition(src_part)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot replace partition '{}' because part '{}' has inconsistent granularity with table", partition_id, src_part->name); - /// This will generate unique name in scope of current server process. - Int64 temp_index = insert_increment.get(); - MergeTreePartInfo dst_part_info(partition_id, temp_index, temp_index, src_part->info.level); - IDataPartStorage::ClonePartParams clone_params{.txn = local_context->getCurrentTransaction()}; - auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk( - src_part, - TMP_PREFIX, - dst_part_info, - my_metadata_snapshot, - clone_params, - local_context->getReadSettings(), - local_context->getWriteSettings()); - dst_parts.emplace_back(std::move(dst_part)); - dst_parts_locks.emplace_back(std::move(part_lock)); - } + /// This will generate unique name in scope of current server process. + auto index = insert_increment.get(); - /// ATTACH empty part set - if (!replace && dst_parts.empty()) - return; + if (is_partition_exp_different) + { + auto [new_partition, new_min_max_index] = createPartitionAndMinMaxIndexFromSourcePart( + src_part, my_metadata_snapshot, local_context); + + auto [dst_part, part_lock] = cloneAndLoadPartOnSameDiskWithDifferentPartitionKey( + src_part, + new_partition, + new_partition.getID(*this), + new_min_max_index, + TMP_PREFIX, + my_metadata_snapshot, + clone_params, + local_context, + index, + index); + + dst_parts.emplace_back(std::move(dst_part)); + dst_parts_locks.emplace_back(std::move(part_lock)); + } + else + { + MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); + + auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk( + src_part, + TMP_PREFIX, + dst_part_info, + my_metadata_snapshot, + clone_params, + local_context->getReadSettings(), + local_context->getWriteSettings()); + dst_parts.emplace_back(std::move(dst_part)); + dst_parts_locks.emplace_back(std::move(part_lock)); + } + } MergeTreePartInfo drop_range; if (replace) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index f7e6783dbc2..512811e39d7 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -26,22 +26,21 @@ #include -#include #include #include #include #include #include -#include #include #include -#include #include #include #include #include #include +#include #include +#include #include #include #include @@ -53,9 +52,11 @@ #include #include #include +#include #include #include #include +#include #include #include @@ -2713,16 +2714,48 @@ bool StorageReplicatedMergeTree::executeReplaceRange(LogEntry & entry) .copy_instead_of_hardlink = storage_settings_ptr->always_use_copy_instead_of_hardlinks || ((our_zero_copy_enabled || source_zero_copy_enabled) && part_desc->src_table_part->isStoredOnRemoteDiskWithZeroCopySupport()), .metadata_version_to_write = metadata_snapshot->getMetadataVersion() }; - auto [res_part, temporary_part_lock] = cloneAndLoadDataPartOnSameDisk( - part_desc->src_table_part, - TMP_PREFIX + "clone_", - part_desc->new_part_info, - metadata_snapshot, - clone_params, - getContext()->getReadSettings(), - getContext()->getWriteSettings()); - part_desc->res_part = std::move(res_part); - part_desc->temporary_part_lock = std::move(temporary_part_lock); + + const auto my_partition_expression = metadata_snapshot->getPartitionKeyAST(); + const auto src_partition_expression = source_table->getInMemoryMetadataPtr()->getPartitionKeyAST(); + + const auto is_partition_exp_different = queryToStringNullable(my_partition_expression) != queryToStringNullable(src_partition_expression); + + if (is_partition_exp_different) + { + auto [new_partition, new_min_max_index] = createPartitionAndMinMaxIndexFromSourcePart( + part_desc->src_table_part, metadata_snapshot, getContext()); + + auto partition_id = new_partition.getID(*this); + + auto [res_part, temporary_part_lock] = cloneAndLoadPartOnSameDiskWithDifferentPartitionKey( + part_desc->src_table_part, + new_partition, + partition_id, + new_min_max_index, + TMP_PREFIX + "clone_", + metadata_snapshot, + clone_params, + getContext(), + part_desc->new_part_info.min_block, + part_desc->new_part_info.max_block); + + part_desc->res_part = std::move(res_part); + part_desc->temporary_part_lock = std::move(temporary_part_lock); + } + else + { + auto [res_part, temporary_part_lock] = cloneAndLoadDataPartOnSameDisk( + part_desc->src_table_part, + TMP_PREFIX + "clone_", + part_desc->new_part_info, + metadata_snapshot, + clone_params, + getContext()->getReadSettings(), + getContext()->getWriteSettings()); + + part_desc->res_part = std::move(res_part); + part_desc->temporary_part_lock = std::move(temporary_part_lock); + } } else if (!part_desc->replica.empty()) { @@ -7852,11 +7885,22 @@ void StorageReplicatedMergeTree::replacePartitionFrom( ProfileEventsScope profile_events_scope; MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table, source_metadata_snapshot, metadata_snapshot); - String partition_id = getPartitionIDFromQuery(partition, query_context); + String partition_id = src_data.getPartitionIDFromQuery(partition, query_context); /// NOTE: Some covered parts may be missing in src_all_parts if corresponding log entries are not executed yet. DataPartsVector src_all_parts = src_data.getVisibleDataPartsVectorInPartition(query_context, partition_id); + bool attach_empty_partition = !replace && src_all_parts.empty(); + if (attach_empty_partition) + return; + + const auto my_partition_expression = metadata_snapshot->getPartitionKeyAST(); + const auto src_partition_expression = source_metadata_snapshot->getPartitionKeyAST(); + const auto is_partition_exp_different = queryToStringNullable(my_partition_expression) != queryToStringNullable(src_partition_expression); + + if (is_partition_exp_different && !src_all_parts.empty()) + MergeTreePartitionCompatibilityVerifier::verify(src_data, /* destination_storage */ *this, src_all_parts); + LOG_DEBUG(log, "Cloning {} parts", src_all_parts.size()); static const String TMP_PREFIX = "tmp_replace_from_"; @@ -7911,6 +7955,18 @@ void StorageReplicatedMergeTree::replacePartitionFrom( "Cannot replace partition '{}' because part '{}" "' has inconsistent granularity with table", partition_id, src_part->name); + IMergeTreeDataPart::MinMaxIndex min_max_index = *src_part->minmax_idx; + MergeTreePartition merge_tree_partition = src_part->partition; + + if (is_partition_exp_different) + { + auto [new_partition, new_min_max_index] = createPartitionAndMinMaxIndexFromSourcePart(src_part, metadata_snapshot, query_context); + + merge_tree_partition = new_partition; + min_max_index = new_min_max_index; + partition_id = merge_tree_partition.getID(*this); + } + String hash_hex = src_part->checksums.getTotalChecksumHex(); const bool is_duplicated_part = replaced_parts.contains(hash_hex); replaced_parts.insert(hash_hex); @@ -7929,27 +7985,52 @@ void StorageReplicatedMergeTree::replacePartitionFrom( continue; } - UInt64 index = lock->getNumber(); - MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); - bool zero_copy_enabled = storage_settings_ptr->allow_remote_fs_zero_copy_replication || dynamic_cast(source_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; + + UInt64 index = lock->getNumber(); + IDataPartStorage::ClonePartParams clone_params { .copy_instead_of_hardlink = storage_settings_ptr->always_use_copy_instead_of_hardlinks || (zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport()), .metadata_version_to_write = metadata_snapshot->getMetadataVersion() }; - auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk( - src_part, - TMP_PREFIX, - dst_part_info, - metadata_snapshot, - clone_params, - query_context->getReadSettings(), - query_context->getWriteSettings()); + + if (is_partition_exp_different) + { + auto [dst_part, part_lock] = cloneAndLoadPartOnSameDiskWithDifferentPartitionKey( + src_part, + merge_tree_partition, + partition_id, + min_max_index, + TMP_PREFIX, + metadata_snapshot, + clone_params, + query_context, + index, + index); + + dst_parts.emplace_back(dst_part); + dst_parts_locks.emplace_back(std::move(part_lock)); + } + else + { + MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); + + auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk( + src_part, + TMP_PREFIX, + dst_part_info, + metadata_snapshot, + clone_params, + query_context->getReadSettings(), + query_context->getWriteSettings()); + + dst_parts.emplace_back(dst_part); + dst_parts_locks.emplace_back(std::move(part_lock)); + } + src_parts.emplace_back(src_part); - dst_parts.emplace_back(dst_part); - dst_parts_locks.emplace_back(std::move(part_lock)); ephemeral_locks.emplace_back(std::move(*lock)); block_id_paths.emplace_back(block_id_path); part_checksums.emplace_back(hash_hex); diff --git a/tests/integration/test_attach_partition_distinct_expression_replicated/__init__.py b/tests/integration/test_attach_partition_distinct_expression_replicated/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_attach_partition_distinct_expression_replicated/configs/remote_servers.xml b/tests/integration/test_attach_partition_distinct_expression_replicated/configs/remote_servers.xml new file mode 100644 index 00000000000..b40730e9f7d --- /dev/null +++ b/tests/integration/test_attach_partition_distinct_expression_replicated/configs/remote_servers.xml @@ -0,0 +1,17 @@ + + + + + true + + replica1 + 9000 + + + replica2 + 9000 + + + + + diff --git a/tests/integration/test_attach_partition_distinct_expression_replicated/test.py b/tests/integration/test_attach_partition_distinct_expression_replicated/test.py new file mode 100644 index 00000000000..1d8ac4e9e37 --- /dev/null +++ b/tests/integration/test_attach_partition_distinct_expression_replicated/test.py @@ -0,0 +1,214 @@ +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry + +cluster = ClickHouseCluster(__file__) + +replica1 = cluster.add_instance( + "replica1", with_zookeeper=True, main_configs=["configs/remote_servers.xml"] +) +replica2 = cluster.add_instance( + "replica2", with_zookeeper=True, main_configs=["configs/remote_servers.xml"] +) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + except Exception as ex: + print(ex) + finally: + cluster.shutdown() + + +def cleanup(nodes): + for node in nodes: + node.query("DROP TABLE IF EXISTS source SYNC") + node.query("DROP TABLE IF EXISTS destination SYNC") + + +def create_table(node, table_name, replicated): + replica = node.name + engine = ( + f"ReplicatedMergeTree('/clickhouse/tables/1/{table_name}', '{replica}')" + if replicated + else "MergeTree()" + ) + partition_expression = ( + "toYYYYMMDD(timestamp)" if table_name == "source" else "toYYYYMM(timestamp)" + ) + node.query_with_retry( + """ + CREATE TABLE {table_name}(timestamp DateTime) + ENGINE = {engine} + ORDER BY tuple() PARTITION BY {partition_expression} + SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, max_cleanup_delay_period=1; + """.format( + table_name=table_name, + engine=engine, + partition_expression=partition_expression, + ) + ) + + +def test_both_replicated(start_cluster): + for node in [replica1, replica2]: + create_table(node, "source", True) + create_table(node, "destination", True) + + replica1.query("INSERT INTO source VALUES ('2010-03-02 02:01:01')") + replica1.query("SYSTEM SYNC REPLICA source") + replica1.query("SYSTEM SYNC REPLICA destination") + replica1.query( + f"ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source" + ) + + assert_eq_with_retry( + replica1, f"SELECT * FROM destination", "2010-03-02 02:01:01\n" + ) + assert_eq_with_retry( + replica1, + f"SELECT * FROM destination", + replica2.query(f"SELECT * FROM destination"), + ) + + cleanup([replica1, replica2]) + + +def test_only_destination_replicated(start_cluster): + create_table(replica1, "source", False) + create_table(replica1, "destination", True) + create_table(replica2, "destination", True) + + replica1.query("INSERT INTO source VALUES ('2010-03-02 02:01:01')") + replica1.query("SYSTEM SYNC REPLICA destination") + replica1.query( + f"ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source" + ) + + assert_eq_with_retry( + replica1, f"SELECT * FROM destination", "2010-03-02 02:01:01\n" + ) + assert_eq_with_retry( + replica1, + f"SELECT * FROM destination", + replica2.query(f"SELECT * FROM destination"), + ) + + cleanup([replica1, replica2]) + + +def test_both_replicated_partitioned_to_unpartitioned(start_cluster): + def create_tables(nodes): + for node in nodes: + source_engine = ( + f"ReplicatedMergeTree('/clickhouse/tables/1/source', '{node.name}')" + ) + node.query( + """ + CREATE TABLE source(timestamp DateTime) + ENGINE = {engine} + ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp) + SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, max_cleanup_delay_period=1; + """.format( + engine=source_engine, + ) + ) + + destination_engine = f"ReplicatedMergeTree('/clickhouse/tables/1/destination', '{node.name}')" + node.query( + """ + CREATE TABLE destination(timestamp DateTime) + ENGINE = {engine} + ORDER BY tuple() PARTITION BY tuple() + SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, max_cleanup_delay_period=1; + """.format( + engine=destination_engine, + ) + ) + + create_tables([replica1, replica2]) + + replica1.query("INSERT INTO source VALUES ('2010-03-02 02:01:01')") + replica1.query("INSERT INTO source VALUES ('2010-03-03 02:01:01')") + replica1.query("SYSTEM SYNC REPLICA source") + replica1.query("SYSTEM SYNC REPLICA destination") + + replica1.query( + f"ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source" + ) + replica1.query( + f"ALTER TABLE destination ATTACH PARTITION ID '20100303' FROM source" + ) + + assert_eq_with_retry( + replica1, + f"SELECT * FROM destination ORDER BY timestamp", + "2010-03-02 02:01:01\n2010-03-03 02:01:01\n", + ) + assert_eq_with_retry( + replica1, + f"SELECT * FROM destination ORDER BY timestamp", + replica2.query(f"SELECT * FROM destination ORDER BY timestamp"), + ) + + cleanup([replica1, replica2]) + + +def test_both_replicated_different_exp_same_id(start_cluster): + def create_tables(nodes): + for node in nodes: + source_engine = ( + f"ReplicatedMergeTree('/clickhouse/tables/1/source', '{node.name}')" + ) + node.query( + """ + CREATE TABLE source(a UInt16,b UInt16,c UInt16,extra UInt64,Path String,Time DateTime,Value Float64,Timestamp Int64,sign Int8) + ENGINE = {engine} + ORDER BY tuple() PARTITION BY a % 3 + SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, max_cleanup_delay_period=1; + """.format( + engine=source_engine, + ) + ) + + destination_engine = f"ReplicatedMergeTree('/clickhouse/tables/1/destination', '{node.name}')" + node.query( + """ + CREATE TABLE destination(a UInt16,b UInt16,c UInt16,extra UInt64,Path String,Time DateTime,Value Float64,Timestamp Int64,sign Int8) + ENGINE = {engine} + ORDER BY tuple() PARTITION BY a + SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, max_cleanup_delay_period=1; + """.format( + engine=destination_engine, + ) + ) + + create_tables([replica1, replica2]) + + replica1.query( + "INSERT INTO source (a, b, c, extra, sign) VALUES (1, 5, 9, 1000, 1)" + ) + replica1.query( + "INSERT INTO source (a, b, c, extra, sign) VALUES (2, 6, 10, 1000, 1)" + ) + replica1.query("SYSTEM SYNC REPLICA source") + replica1.query("SYSTEM SYNC REPLICA destination") + + replica1.query(f"ALTER TABLE destination ATTACH PARTITION 1 FROM source") + replica1.query(f"ALTER TABLE destination ATTACH PARTITION 2 FROM source") + + assert_eq_with_retry( + replica1, + f"SELECT * FROM destination ORDER BY a", + "1\t5\t9\t1000\t\t1970-01-01 00:00:00\t0\t0\t1\n2\t6\t10\t1000\t\t1970-01-01 00:00:00\t0\t0\t1\n", + ) + assert_eq_with_retry( + replica1, + f"SELECT * FROM destination ORDER BY a", + replica2.query(f"SELECT * FROM destination ORDER BY a"), + ) + + cleanup([replica1, replica2]) diff --git a/tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.reference b/tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.reference new file mode 100644 index 00000000000..f1d036b08bf --- /dev/null +++ b/tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.reference @@ -0,0 +1,467 @@ +-- { echoOn } +-- Should be allowed since destination partition expr is monotonically increasing and compatible +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); +ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +201003 +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION '20100302' FROM source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +201003 +-- Should be allowed since destination partition expr is monotonically increasing and compatible. Note that even though +-- the destination partition expression is more granular, the data would still fall in the same partition. Thus, it is valid +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); +ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +20100302 +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION '201003' FROM source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +20100302 +-- Should be allowed since destination partition expr is monotonically increasing and compatible for those specific values +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY intDiv(A, 6); +CREATE TABLE destination (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY A; +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01', 1), ('2010-03-02 02:01:03', 1); +ALTER TABLE destination ATTACH PARTITION ID '0' FROM source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 1 +2010-03-02 02:01:03 1 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 1 +2010-03-02 02:01:03 1 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +1 +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION 0 FROM source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 1 +2010-03-02 02:01:03 1 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 1 +2010-03-02 02:01:03 1 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +1 +-- Should be allowed because dst partition exp is monot inc and data is not split +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY cityHash64(category); +CREATE TABLE destination (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(category); +INSERT INTO TABLE source VALUES ('spaghetti', 'food'), ('mop', 'general'); +INSERT INTO TABLE source VALUES ('rice', 'food'); +ALTER TABLE destination ATTACH PARTITION ID '17908065610379824077' from source; +SELECT * FROM source ORDER BY productName; +mop general +rice food +spaghetti food +SELECT * FROM destination ORDER BY productName; +rice food +spaghetti food +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +59532f3c39a412a413f0f014c7750a9d +59532f3c39a412a413f0f014c7750a9d +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION '17908065610379824077' from source; +SELECT * FROM source ORDER BY productName; +mop general +rice food +spaghetti food +SELECT * FROM destination ORDER BY productName; +rice food +spaghetti food +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +59532f3c39a412a413f0f014c7750a9d +59532f3c39a412a413f0f014c7750a9d +-- Should be allowed, extra test case to validate https://github.com/ClickHouse/ClickHouse/pull/39507#issuecomment-1747574133 + +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (timestamp Int64) engine=MergeTree ORDER BY (timestamp) PARTITION BY intDiv(timestamp, 86400000); +CREATE TABLE destination (timestamp Int64) engine=MergeTree ORDER BY (timestamp) PARTITION BY toYear(toDateTime(intDiv(timestamp, 1000))); +INSERT INTO TABLE source VALUES (1267495261123); +ALTER TABLE destination ATTACH PARTITION ID '14670' FROM source; +SELECT * FROM source ORDER BY timestamp; +1267495261123 +SELECT * FROM destination ORDER BY timestamp; +1267495261123 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +2010 +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION '14670' from source; +SELECT * FROM source ORDER BY timestamp; +1267495261123 +SELECT * FROM destination ORDER BY timestamp; +1267495261123 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +2010 +-- Should be allowed, extra test case to validate https://github.com/ClickHouse/ClickHouse/pull/39507#issuecomment-1747511726 + +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (timestamp DateTime('UTC'), key Int64, f Float64) engine=MergeTree ORDER BY (key, timestamp) PARTITION BY toYear(timestamp); +CREATE TABLE destination (timestamp DateTime('UTC'), key Int64, f Float64) engine=MergeTree ORDER BY (key, timestamp) PARTITION BY (intDiv(toUInt32(timestamp),86400)); +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01',1,1),('2010-03-02 02:01:01',1,1),('2011-02-02 02:01:03',1,1); +ALTER TABLE destination ATTACH PARTITION ID '2010' FROM source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 1 1 +2010-03-02 02:01:01 1 1 +2011-02-02 02:01:03 1 1 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 1 1 +2010-03-02 02:01:01 1 1 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +14670 +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION '2010' from source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 1 1 +2010-03-02 02:01:01 1 1 +2011-02-02 02:01:03 1 1 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 1 1 +2010-03-02 02:01:01 1 1 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +14670 +-- Should be allowed, partitioned table to unpartitioned. Since the destination is unpartitioned, parts would ultimately +-- fall into the same partition. +-- Destination partition by expression is omitted, which causes StorageMetadata::getPartitionKeyAST() to be nullptr. +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple(); +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); +ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +all +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION '201003' from source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +all +-- Same as above, but destination partition by expression is explicitly defined. Test case required to validate that +-- partition by tuple() is accepted. +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY tuple(); +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); +ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +all +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION '201003' from source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +all +-- Should be allowed because the destination partition expression columns are a subset of the source partition expression columns +-- Columns in this case refer to the expression elements, not to the actual table columns +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY (a, b, c); +CREATE TABLE destination (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY (a, b); +INSERT INTO TABLE source VALUES (1, 2, 3), (1, 2, 4); +ALTER TABLE destination ATTACH PARTITION ID '1-2-3' FROM source; +SELECT * FROM source ORDER BY (a, b, c); +1 2 3 +1 2 4 +SELECT * FROM destination ORDER BY (a, b, c); +1 2 3 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +1-2 +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION (1, 2, 3) from source; +SELECT * FROM source ORDER BY (a, b, c); +1 2 3 +1 2 4 +SELECT * FROM destination ORDER BY (a, b, c); +1 2 3 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +1-2 +-- Should be allowed because the destination partition expression columns are a subset of the source partition expression columns +-- Columns in this case refer to the expression elements, not to the actual table columns +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY (a, b, c); +CREATE TABLE destination (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY a; +INSERT INTO TABLE source VALUES (1, 2, 3), (1, 2, 4); +ALTER TABLE destination ATTACH PARTITION ID '1-2-3' FROM source; +SELECT * FROM source ORDER BY (a, b, c); +1 2 3 +1 2 4 +SELECT * FROM destination ORDER BY (a, b, c); +1 2 3 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +1 +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION (1, 2, 3) from source; +SELECT * FROM source ORDER BY (a, b, c); +1 2 3 +1 2 4 +SELECT * FROM destination ORDER BY (a, b, c); +1 2 3 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +1 +-- Should be allowed. Special test case, tricky to explain. First column of source partition expression is +-- timestamp, while first column of destination partition expression is `A`. One of the previous implementations +-- would not match the columns, which could lead to `timestamp` min max being used to calculate monotonicity of `A`. +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (`timestamp` DateTime, `A` Int64) ENGINE = MergeTree PARTITION BY tuple(toYYYYMM(timestamp), intDiv(A, 6)) ORDER BY timestamp; +CREATE TABLE destination (`timestamp` DateTime, `A` Int64) ENGINE = MergeTree PARTITION BY A ORDER BY timestamp; +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01', 5); +ALTER TABLE destination ATTACH PARTITION ID '201003-0' FROM source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 5 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 5 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +5 +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION (201003, 0) from source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 5 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 5 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +5 +-- Should be allowed. Destination partition expression contains multiple expressions, but all of them are monotonically +-- increasing in the source partition min max indexes. +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(A, B) ORDER BY tuple(); +CREATE TABLE destination (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(intDiv(A, 2), intDiv(B, 2)) ORDER BY tuple(); +INSERT INTO TABLE source VALUES (6, 12); +ALTER TABLE destination ATTACH PARTITION ID '6-12' FROM source; +SELECT * FROM source ORDER BY A; +6 12 +SELECT * FROM destination ORDER BY A; +6 12 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +3-6 +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION (6, 12) from source; +SELECT * FROM source ORDER BY A; +6 12 +SELECT * FROM destination ORDER BY A; +6 12 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +3-6 +-- Should be allowed. The same scenario as above, but partition expressions inverted. +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(intDiv(A, 2), intDiv(B, 2)) ORDER BY tuple(); +CREATE TABLE destination (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(A, B) ORDER BY tuple(); +INSERT INTO TABLE source VALUES (6, 12); +ALTER TABLE destination ATTACH PARTITION ID '3-6' FROM source; +SELECT * FROM source ORDER BY A; +6 12 +SELECT * FROM destination ORDER BY A; +6 12 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +6-12 +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION (3, 6) from source; +SELECT * FROM source ORDER BY A; +6 12 +SELECT * FROM destination ORDER BY A; +6 12 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +6-12 +-- Should be allowed, it is a local operation, no different than regular attach. Replicated to replicated. +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE + source(timestamp DateTime) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/source_replicated_to_replicated_distinct_expression', '1') + PARTITION BY toYYYYMMDD(timestamp) + ORDER BY tuple(); +CREATE TABLE + destination(timestamp DateTime) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/destination_replicated_to_replicated_distinct_expression', '1') + PARTITION BY toYYYYMM(timestamp) + ORDER BY tuple(); +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); +ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +201003 +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION '20100302' from source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +201003 +-- Should be allowed, it is a local operation, no different than regular attach. Non replicated to replicated +DROP TABLE IF EXISTS source SYNC; +DROP TABLE IF EXISTS destination SYNC; +CREATE TABLE source(timestamp DateTime) ENGINE = MergeTree() PARTITION BY toYYYYMMDD(timestamp) ORDER BY tuple(); +CREATE TABLE + destination(timestamp DateTime) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/destination_non_replicated_to_replicated_distinct_expression', '1') + PARTITION BY toYYYYMM(timestamp) + ORDER BY tuple(); +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); +ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +201003 +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION '20100302' from source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +201003 +-- Should not be allowed because data would be split into two different partitions +DROP TABLE IF EXISTS source SYNC; +DROP TABLE IF EXISTS destination SYNC; +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-03 02:01:03'); +ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; -- { serverError 248 } +ALTER TABLE destination ATTACH PARTITION '201003' from source; -- { serverError 248 } +-- Should not be allowed because data would be split into two different partitions +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY intDiv(A, 6); +CREATE TABLE destination (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY A; +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01', 1), ('2010-03-02 02:01:03', 2); +ALTER TABLE destination ATTACH PARTITION ID '0' FROM source; -- { serverError 248 } +ALTER TABLE destination ATTACH PARTITION 0 FROM source; -- { serverError 248 } +-- Should not be allowed because dst partition exp takes more than two arguments, so it's not considered monotonically inc +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(category); +CREATE TABLE destination (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY substring(category, 1, 2); +INSERT INTO TABLE source VALUES ('spaghetti', 'food'), ('mop', 'general'); +INSERT INTO TABLE source VALUES ('rice', 'food'); +ALTER TABLE destination ATTACH PARTITION ID '4590ba78048910b74a47d5bfb308abed' from source; -- { serverError 36 } +ALTER TABLE destination ATTACH PARTITION 'food' from source; -- { serverError 36 } +-- Should not be allowed because dst partition exp depends on a different set of columns +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(category); +CREATE TABLE destination (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(productName); +INSERT INTO TABLE source VALUES ('spaghetti', 'food'), ('mop', 'general'); +INSERT INTO TABLE source VALUES ('rice', 'food'); +ALTER TABLE destination ATTACH PARTITION ID '4590ba78048910b74a47d5bfb308abed' from source; -- { serverError 36 } +ALTER TABLE destination ATTACH PARTITION 'food' from source; -- { serverError 36 } +-- Should not be allowed because dst partition exp is not monotonically increasing +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (productName String) engine=MergeTree ORDER BY tuple() PARTITION BY left(productName, 2); +CREATE TABLE destination (productName String) engine=MergeTree ORDER BY tuple() PARTITION BY cityHash64(productName); +INSERT INTO TABLE source VALUES ('bread'), ('mop'); +INSERT INTO TABLE source VALUES ('broccoli'); +ALTER TABLE destination ATTACH PARTITION ID '4589453b7ee96ce9de1265bd57674496' from source; -- { serverError 36 } +ALTER TABLE destination ATTACH PARTITION 'br' from source; -- { serverError 36 } +-- Empty/ non-existent partition, same partition expression. Nothing should happen +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +ALTER TABLE destination ATTACH PARTITION ID '1' FROM source; +ALTER TABLE destination ATTACH PARTITION 1 FROM source; +SELECT * FROM destination; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +-- Empty/ non-existent partition, different partition expression. Nothing should happen +-- https://github.com/ClickHouse/ClickHouse/pull/39507#discussion_r1399839045 +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +ALTER TABLE destination ATTACH PARTITION ID '1' FROM source; +ALTER TABLE destination ATTACH PARTITION 1 FROM source; +SELECT * FROM destination; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +-- Replace instead of attach. Empty/ non-existent partition, same partition expression. Nothing should happen +-- https://github.com/ClickHouse/ClickHouse/pull/39507#discussion_r1399839045 +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +ALTER TABLE destination REPLACE PARTITION '1' FROM source; +SELECT * FROM destination; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +-- Replace instead of attach. Empty/ non-existent partition to non-empty partition, same partition id. +-- https://github.com/ClickHouse/ClickHouse/pull/39507#discussion_r1399839045 +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (A Int) engine=MergeTree ORDER BY tuple() PARTITION BY A; +CREATE TABLE destination (A Int) engine=MergeTree ORDER BY tuple() PARTITION BY A; +INSERT INTO TABLE destination VALUES (1); +ALTER TABLE destination REPLACE PARTITION '1' FROM source; +SELECT * FROM destination; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; diff --git a/tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.sql b/tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.sql new file mode 100644 index 00000000000..9547d6ae249 --- /dev/null +++ b/tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.sql @@ -0,0 +1,485 @@ +-- { echoOn } +-- Should be allowed since destination partition expr is monotonically increasing and compatible +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); + +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); + +ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION '20100302' FROM source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Should be allowed since destination partition expr is monotonically increasing and compatible. Note that even though +-- the destination partition expression is more granular, the data would still fall in the same partition. Thus, it is valid +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); + +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); + +ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION '201003' FROM source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Should be allowed since destination partition expr is monotonically increasing and compatible for those specific values +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY intDiv(A, 6); + +CREATE TABLE destination (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY A; + +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01', 1), ('2010-03-02 02:01:03', 1); + +ALTER TABLE destination ATTACH PARTITION ID '0' FROM source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION 0 FROM source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Should be allowed because dst partition exp is monot inc and data is not split +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY cityHash64(category); +CREATE TABLE destination (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(category); + +INSERT INTO TABLE source VALUES ('spaghetti', 'food'), ('mop', 'general'); +INSERT INTO TABLE source VALUES ('rice', 'food'); + +ALTER TABLE destination ATTACH PARTITION ID '17908065610379824077' from source; + +SELECT * FROM source ORDER BY productName; +SELECT * FROM destination ORDER BY productName; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION '17908065610379824077' from source; + +SELECT * FROM source ORDER BY productName; +SELECT * FROM destination ORDER BY productName; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Should be allowed, extra test case to validate https://github.com/ClickHouse/ClickHouse/pull/39507#issuecomment-1747574133 + +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (timestamp Int64) engine=MergeTree ORDER BY (timestamp) PARTITION BY intDiv(timestamp, 86400000); +CREATE TABLE destination (timestamp Int64) engine=MergeTree ORDER BY (timestamp) PARTITION BY toYear(toDateTime(intDiv(timestamp, 1000))); + +INSERT INTO TABLE source VALUES (1267495261123); + +ALTER TABLE destination ATTACH PARTITION ID '14670' FROM source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION '14670' from source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Should be allowed, extra test case to validate https://github.com/ClickHouse/ClickHouse/pull/39507#issuecomment-1747511726 + +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (timestamp DateTime('UTC'), key Int64, f Float64) engine=MergeTree ORDER BY (key, timestamp) PARTITION BY toYear(timestamp); +CREATE TABLE destination (timestamp DateTime('UTC'), key Int64, f Float64) engine=MergeTree ORDER BY (key, timestamp) PARTITION BY (intDiv(toUInt32(timestamp),86400)); + +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01',1,1),('2010-03-02 02:01:01',1,1),('2011-02-02 02:01:03',1,1); + +ALTER TABLE destination ATTACH PARTITION ID '2010' FROM source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION '2010' from source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Should be allowed, partitioned table to unpartitioned. Since the destination is unpartitioned, parts would ultimately +-- fall into the same partition. +-- Destination partition by expression is omitted, which causes StorageMetadata::getPartitionKeyAST() to be nullptr. +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple(); + +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); + +ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION '201003' from source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Same as above, but destination partition by expression is explicitly defined. Test case required to validate that +-- partition by tuple() is accepted. +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY tuple(); + +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); + +ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION '201003' from source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Should be allowed because the destination partition expression columns are a subset of the source partition expression columns +-- Columns in this case refer to the expression elements, not to the actual table columns +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY (a, b, c); +CREATE TABLE destination (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY (a, b); + +INSERT INTO TABLE source VALUES (1, 2, 3), (1, 2, 4); + +ALTER TABLE destination ATTACH PARTITION ID '1-2-3' FROM source; + +SELECT * FROM source ORDER BY (a, b, c); +SELECT * FROM destination ORDER BY (a, b, c); +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION (1, 2, 3) from source; + +SELECT * FROM source ORDER BY (a, b, c); +SELECT * FROM destination ORDER BY (a, b, c); +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Should be allowed because the destination partition expression columns are a subset of the source partition expression columns +-- Columns in this case refer to the expression elements, not to the actual table columns +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY (a, b, c); +CREATE TABLE destination (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY a; + +INSERT INTO TABLE source VALUES (1, 2, 3), (1, 2, 4); + +ALTER TABLE destination ATTACH PARTITION ID '1-2-3' FROM source; + +SELECT * FROM source ORDER BY (a, b, c); +SELECT * FROM destination ORDER BY (a, b, c); +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION (1, 2, 3) from source; + +SELECT * FROM source ORDER BY (a, b, c); +SELECT * FROM destination ORDER BY (a, b, c); +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Should be allowed. Special test case, tricky to explain. First column of source partition expression is +-- timestamp, while first column of destination partition expression is `A`. One of the previous implementations +-- would not match the columns, which could lead to `timestamp` min max being used to calculate monotonicity of `A`. +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (`timestamp` DateTime, `A` Int64) ENGINE = MergeTree PARTITION BY tuple(toYYYYMM(timestamp), intDiv(A, 6)) ORDER BY timestamp; +CREATE TABLE destination (`timestamp` DateTime, `A` Int64) ENGINE = MergeTree PARTITION BY A ORDER BY timestamp; + +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01', 5); + +ALTER TABLE destination ATTACH PARTITION ID '201003-0' FROM source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION (201003, 0) from source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Should be allowed. Destination partition expression contains multiple expressions, but all of them are monotonically +-- increasing in the source partition min max indexes. +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(A, B) ORDER BY tuple(); +CREATE TABLE destination (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(intDiv(A, 2), intDiv(B, 2)) ORDER BY tuple(); + +INSERT INTO TABLE source VALUES (6, 12); + +ALTER TABLE destination ATTACH PARTITION ID '6-12' FROM source; + +SELECT * FROM source ORDER BY A; +SELECT * FROM destination ORDER BY A; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION (6, 12) from source; + +SELECT * FROM source ORDER BY A; +SELECT * FROM destination ORDER BY A; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Should be allowed. The same scenario as above, but partition expressions inverted. +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(intDiv(A, 2), intDiv(B, 2)) ORDER BY tuple(); +CREATE TABLE destination (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(A, B) ORDER BY tuple(); + +INSERT INTO TABLE source VALUES (6, 12); + +ALTER TABLE destination ATTACH PARTITION ID '3-6' FROM source; + +SELECT * FROM source ORDER BY A; +SELECT * FROM destination ORDER BY A; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION (3, 6) from source; + +SELECT * FROM source ORDER BY A; +SELECT * FROM destination ORDER BY A; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Should be allowed, it is a local operation, no different than regular attach. Replicated to replicated. +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE + source(timestamp DateTime) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/source_replicated_to_replicated_distinct_expression', '1') + PARTITION BY toYYYYMMDD(timestamp) + ORDER BY tuple(); + +CREATE TABLE + destination(timestamp DateTime) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/destination_replicated_to_replicated_distinct_expression', '1') + PARTITION BY toYYYYMM(timestamp) + ORDER BY tuple(); + +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); + +ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION '20100302' from source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Should be allowed, it is a local operation, no different than regular attach. Non replicated to replicated +DROP TABLE IF EXISTS source SYNC; +DROP TABLE IF EXISTS destination SYNC; +CREATE TABLE source(timestamp DateTime) ENGINE = MergeTree() PARTITION BY toYYYYMMDD(timestamp) ORDER BY tuple(); + +CREATE TABLE + destination(timestamp DateTime) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/destination_non_replicated_to_replicated_distinct_expression', '1') + PARTITION BY toYYYYMM(timestamp) + ORDER BY tuple(); + +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); + +ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION '20100302' from source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Should not be allowed because data would be split into two different partitions +DROP TABLE IF EXISTS source SYNC; +DROP TABLE IF EXISTS destination SYNC; + +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); + +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-03 02:01:03'); + +ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; -- { serverError 248 } +ALTER TABLE destination ATTACH PARTITION '201003' from source; -- { serverError 248 } + +-- Should not be allowed because data would be split into two different partitions +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY intDiv(A, 6); + +CREATE TABLE destination (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY A; + +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01', 1), ('2010-03-02 02:01:03', 2); + +ALTER TABLE destination ATTACH PARTITION ID '0' FROM source; -- { serverError 248 } +ALTER TABLE destination ATTACH PARTITION 0 FROM source; -- { serverError 248 } + +-- Should not be allowed because dst partition exp takes more than two arguments, so it's not considered monotonically inc +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(category); +CREATE TABLE destination (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY substring(category, 1, 2); + +INSERT INTO TABLE source VALUES ('spaghetti', 'food'), ('mop', 'general'); +INSERT INTO TABLE source VALUES ('rice', 'food'); + +ALTER TABLE destination ATTACH PARTITION ID '4590ba78048910b74a47d5bfb308abed' from source; -- { serverError 36 } +ALTER TABLE destination ATTACH PARTITION 'food' from source; -- { serverError 36 } + +-- Should not be allowed because dst partition exp depends on a different set of columns +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(category); +CREATE TABLE destination (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(productName); + +INSERT INTO TABLE source VALUES ('spaghetti', 'food'), ('mop', 'general'); +INSERT INTO TABLE source VALUES ('rice', 'food'); + +ALTER TABLE destination ATTACH PARTITION ID '4590ba78048910b74a47d5bfb308abed' from source; -- { serverError 36 } +ALTER TABLE destination ATTACH PARTITION 'food' from source; -- { serverError 36 } + +-- Should not be allowed because dst partition exp is not monotonically increasing +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (productName String) engine=MergeTree ORDER BY tuple() PARTITION BY left(productName, 2); +CREATE TABLE destination (productName String) engine=MergeTree ORDER BY tuple() PARTITION BY cityHash64(productName); + +INSERT INTO TABLE source VALUES ('bread'), ('mop'); +INSERT INTO TABLE source VALUES ('broccoli'); + +ALTER TABLE destination ATTACH PARTITION ID '4589453b7ee96ce9de1265bd57674496' from source; -- { serverError 36 } +ALTER TABLE destination ATTACH PARTITION 'br' from source; -- { serverError 36 } + +-- Empty/ non-existent partition, same partition expression. Nothing should happen +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); + +ALTER TABLE destination ATTACH PARTITION ID '1' FROM source; +ALTER TABLE destination ATTACH PARTITION 1 FROM source; + +SELECT * FROM destination; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Empty/ non-existent partition, different partition expression. Nothing should happen +-- https://github.com/ClickHouse/ClickHouse/pull/39507#discussion_r1399839045 +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); + +ALTER TABLE destination ATTACH PARTITION ID '1' FROM source; +ALTER TABLE destination ATTACH PARTITION 1 FROM source; + +SELECT * FROM destination; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Replace instead of attach. Empty/ non-existent partition, same partition expression. Nothing should happen +-- https://github.com/ClickHouse/ClickHouse/pull/39507#discussion_r1399839045 +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); + +ALTER TABLE destination REPLACE PARTITION '1' FROM source; + +SELECT * FROM destination; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Replace instead of attach. Empty/ non-existent partition to non-empty partition, same partition id. +-- https://github.com/ClickHouse/ClickHouse/pull/39507#discussion_r1399839045 +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (A Int) engine=MergeTree ORDER BY tuple() PARTITION BY A; +CREATE TABLE destination (A Int) engine=MergeTree ORDER BY tuple() PARTITION BY A; + +INSERT INTO TABLE destination VALUES (1); + +ALTER TABLE destination REPLACE PARTITION '1' FROM source; + +SELECT * FROM destination; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; From c59f3e164ffedf1a7561b0fd0a65c2555685ca91 Mon Sep 17 00:00:00 2001 From: Bhavna Jindal Date: Thu, 18 Jan 2024 12:08:29 -0800 Subject: [PATCH 122/264] Return baseline component as fourth array --- .../sql-reference/functions/time-series-functions.md | 8 ++++++-- src/Functions/seriesDecomposeSTL.cpp | 10 +++++++++- .../0_stateless/02813_seriesDecomposeSTL.reference | 8 ++++---- 3 files changed, 19 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/functions/time-series-functions.md b/docs/en/sql-reference/functions/time-series-functions.md index 144d832b36a..2e42aa884b4 100644 --- a/docs/en/sql-reference/functions/time-series-functions.md +++ b/docs/en/sql-reference/functions/time-series-functions.md @@ -77,8 +77,8 @@ The number of data points in `series` should be at least twice the value of `per **Returned value** -- An array of three arrays where the first array include seasonal components, the second array - trend, -and the third array - residue component. +- An array of four arrays where the first array include seasonal components, the second array - trend, +the third array - residue component, and the fourth array - baseline component. Type: [Array](../../sql-reference/data-types/array.md). @@ -107,6 +107,10 @@ Result: [ 0, 0.0000019073486, -0.0000019073486, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, -0.0000019073486, 0, 0 + ], + [ + 10.1, 20.449999, 40.340004, 10.100001, 20.45, 40.34, 10.100001, 20.45, 40.34, 10.1, 20.45, 40.34, + 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.100002, 20.45, 40.34 ]] │ └────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` diff --git a/src/Functions/seriesDecomposeSTL.cpp b/src/Functions/seriesDecomposeSTL.cpp index 21e36761213..0c44afa32a6 100644 --- a/src/Functions/seriesDecomposeSTL.cpp +++ b/src/Functions/seriesDecomposeSTL.cpp @@ -128,6 +128,10 @@ public: res_data.insert(residue.begin(), residue.end()); res_col_offsets_data.push_back(res_data.size()); + // Create Baseline = seasonal + trend + std::transform(seasonal.begin(), seasonal.end(), trend.begin(), std::back_inserter(res_data), std::plus()); + res_col_offsets_data.push_back(res_data.size()); + root_offsets_data.push_back(res_col_offsets->size()); prev_src_offset = curr_offset; @@ -201,7 +205,7 @@ The number of data points in `series` should be at least twice the value of `per **Returned value** -- An array of three arrays where the first array include seasonal components, the second array - trend, and the third array - residue component. +- An array of four arrays where the first array include seasonal components, the second array - trend, the third array - residue component, and the fourth array - baseline component. Type: [Array](../../sql-reference/data-types/array.md). @@ -230,6 +234,10 @@ Result: [ 0, 0.0000019073486, -0.0000019073486, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, -0.0000019073486, 0, 0 + ], + [ + 10.1, 20.449999, 40.340004, 10.100001, 20.45, 40.34, 10.100001, 20.45, 40.34, 10.1, 20.45, 40.34, + 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.100002, 20.45, 40.34 ]] │ └────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ```)", diff --git a/tests/queries/0_stateless/02813_seriesDecomposeSTL.reference b/tests/queries/0_stateless/02813_seriesDecomposeSTL.reference index dc30e7f8371..28dae705335 100644 --- a/tests/queries/0_stateless/02813_seriesDecomposeSTL.reference +++ b/tests/queries/0_stateless/02813_seriesDecomposeSTL.reference @@ -1,4 +1,4 @@ -[[-13.529999,-3.1799996,16.71,-13.53,-3.1799996,16.71,-13.53,-3.1799996,16.71,-13.530001,-3.18,16.710001,-13.530001,-3.1800003,16.710001,-13.530001,-3.1800003,16.710001,-13.530001,-3.1799994,16.71,-13.529999,-3.1799994,16.709997],[23.63,23.63,23.630003,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.63,23.630001,23.630001,23.63,23.630001,23.630001,23.63,23.630001,23.630001,23.630001,23.630001,23.630001,23.630003],[0,0.0000019073486,-0.0000019073486,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,-0.0000019073486,0,0]] -[[4.04452e-8,-1.7846537e-8,-5.9488454e-9,0,0,0,0,0,0,-1.9868216e-8,-9.5297715e-8,2.2540547e-9,3.4229203e-8,8.573613e-8],[1.9999999,2,2,2,2,2,2,2,2,2,2,2,1.9999996,1.9999996],[1.1920929e-7,0,0,0,0,0,0,0,0,0,0,0,3.5762787e-7,2.3841858e-7]] -[[-13.529999,-3.1799996,16.71,-13.53,-3.1799996,16.71,-13.53,-3.1799996,16.71,-13.530001,-3.18,16.710001,-13.530001,-3.1800003,16.710001,-13.530001,-3.1800003,16.710001,-13.530001,-3.1799994,16.71,-13.529999,-3.1799994,16.709997],[23.63,23.63,23.630003,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.63,23.630001,23.630001,23.63,23.630001,23.630001,23.63,23.630001,23.630001,23.630001,23.630001,23.630001,23.630003],[0,0.0000019073486,-0.0000019073486,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,-0.0000019073486,0,0]] -[[53.946846,-4.8119445,43.525013,-23.71359,-42.472305,-51.636955,-50.458298,-51.982674,37.62072,-15.9006605,56.65076,-5.809669,57.143845,-2.0370207,54.050922,-4.897961,43.954018,-23.808758,-42.651337,-51.86827,-50.709732,-52.18156,37.734905,-15.853402,56.91643,-5.8815174,57.253094,-2.012879,54.157806,-4.9817176,44.384747,-23.902956,-42.830154,-52.10025,-50.96271,-52.3829,37.84573,-15.81032,57.177113,-5.958963,57.356136,-1.9952412,54.27533,-5.066312,44.878296,-23.956438,-42.993656,-52.337124,-51.208073,-52.615646,37.91102,-15.8062525,57.49891,-6.056076,57.45604,-1.9797823,54.39525,-5.1483474,45.374573],[88.028534,88.95315,89.87776,90.802376,91.64913,92.49588,93.342636,94.19737,95.0521,95.90684,96.712975,97.51912,98.32526,98.36342,98.40158,98.43974,98.36777,98.29579,98.223816,98.536446,98.849075,99.161705,99.7552,100.348694,100.94219,101.53184,102.12149,102.711136,103.79921,104.88729,105.975365,107.50462,109.033875,110.56313,111.79767,113.032196,114.26673,115.02128,115.775826,116.53037,117.15541,117.78044,118.40548,118.86489,119.3243,119.783714,120.04031,120.29691,120.55351,120.78621,121.01891,121.25161,121.533585,121.81555,122.09753,122.41821,122.7389,123.059586,123.39267],[-2.97538,2.8587952,-23.402771,0.91121674,4.8231735,9.141075,8.115662,10.785301,0.32717896,5.99382,-12.363731,5.29055,0.53089905,-2.3264008,-3.4524994,1.4582214,-2.321785,2.51297,5.4275208,3.3318253,5.8606567,0.019859314,-4.4901123,-12.495293,-5.8586197,-1.650322,-11.374588,4.3017426,4.042984,1.094429,9.639885,3.3983307,-3.20372,-5.462883,-5.834961,-6.649292,-1.1124649,3.7890396,16.047066,-2.5714111,8.488449,-2.785202,2.319191,-0.79857635,13.797401,-5.827278,-6.0466614,-5.9597855,-7.3454437,-3.1705627,6.0700684,3.5546417,1.9675064,-0.7594757,2.446434,0.5615692,0.86585236,-3.9112396,1.2327576]] +[[-13.529999,-3.1799996,16.71,-13.53,-3.1799996,16.71,-13.53,-3.1799996,16.71,-13.530001,-3.18,16.710001,-13.530001,-3.1800003,16.710001,-13.530001,-3.1800003,16.710001,-13.530001,-3.1799994,16.71,-13.529999,-3.1799994,16.709997],[23.63,23.63,23.630003,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.63,23.630001,23.630001,23.63,23.630001,23.630001,23.63,23.630001,23.630001,23.630001,23.630001,23.630001,23.630003],[0,0.0000019073486,-0.0000019073486,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,-0.0000019073486,0,0],[10.1,20.449999,40.340004,10.100001,20.45,40.34,10.100001,20.45,40.34,10.1,20.45,40.34,10.1,20.45,40.34,10.1,20.45,40.34,10.1,20.45,40.34,10.100002,20.45,40.34]] +[[4.04452e-8,-1.7846537e-8,-5.9488454e-9,0,0,0,0,0,0,-1.9868216e-8,-9.5297715e-8,2.2540547e-9,3.4229203e-8,8.573613e-8],[1.9999999,2,2,2,2,2,2,2,2,2,2,2,1.9999996,1.9999996],[1.1920929e-7,0,0,0,0,0,0,0,0,0,0,0,3.5762787e-7,2.3841858e-7],[1.9999999,2,2,2,2,2,2,2,2,2,1.9999999,2,1.9999996,1.9999998]] +[[-13.529999,-3.1799996,16.71,-13.53,-3.1799996,16.71,-13.53,-3.1799996,16.71,-13.530001,-3.18,16.710001,-13.530001,-3.1800003,16.710001,-13.530001,-3.1800003,16.710001,-13.530001,-3.1799994,16.71,-13.529999,-3.1799994,16.709997],[23.63,23.63,23.630003,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.630001,23.63,23.630001,23.630001,23.63,23.630001,23.630001,23.63,23.630001,23.630001,23.630001,23.630001,23.630001,23.630003],[0,0.0000019073486,-0.0000019073486,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,-0.0000019073486,0,0],[10.1,20.449999,40.340004,10.100001,20.45,40.34,10.100001,20.45,40.34,10.1,20.45,40.34,10.1,20.45,40.34,10.1,20.45,40.34,10.1,20.45,40.34,10.100002,20.45,40.34]] +[[53.946846,-4.8119445,43.525013,-23.71359,-42.472305,-51.636955,-50.458298,-51.982674,37.62072,-15.9006605,56.65076,-5.809669,57.143845,-2.0370207,54.050922,-4.897961,43.954018,-23.808758,-42.651337,-51.86827,-50.709732,-52.18156,37.734905,-15.853402,56.91643,-5.8815174,57.253094,-2.012879,54.157806,-4.9817176,44.384747,-23.902956,-42.830154,-52.10025,-50.96271,-52.3829,37.84573,-15.81032,57.177113,-5.958963,57.356136,-1.9952412,54.27533,-5.066312,44.878296,-23.956438,-42.993656,-52.337124,-51.208073,-52.615646,37.91102,-15.8062525,57.49891,-6.056076,57.45604,-1.9797823,54.39525,-5.1483474,45.374573],[88.028534,88.95315,89.87776,90.802376,91.64913,92.49588,93.342636,94.19737,95.0521,95.90684,96.712975,97.51912,98.32526,98.36342,98.40158,98.43974,98.36777,98.29579,98.223816,98.536446,98.849075,99.161705,99.7552,100.348694,100.94219,101.53184,102.12149,102.711136,103.79921,104.88729,105.975365,107.50462,109.033875,110.56313,111.79767,113.032196,114.26673,115.02128,115.775826,116.53037,117.15541,117.78044,118.40548,118.86489,119.3243,119.783714,120.04031,120.29691,120.55351,120.78621,121.01891,121.25161,121.533585,121.81555,122.09753,122.41821,122.7389,123.059586,123.39267],[-2.97538,2.8587952,-23.402771,0.91121674,4.8231735,9.141075,8.115662,10.785301,0.32717896,5.99382,-12.363731,5.29055,0.53089905,-2.3264008,-3.4524994,1.4582214,-2.321785,2.51297,5.4275208,3.3318253,5.8606567,0.019859314,-4.4901123,-12.495293,-5.8586197,-1.650322,-11.374588,4.3017426,4.042984,1.094429,9.639885,3.3983307,-3.20372,-5.462883,-5.834961,-6.649292,-1.1124649,3.7890396,16.047066,-2.5714111,8.488449,-2.785202,2.319191,-0.79857635,13.797401,-5.827278,-6.0466614,-5.9597855,-7.3454437,-3.1705627,6.0700684,3.5546417,1.9675064,-0.7594757,2.446434,0.5615692,0.86585236,-3.9112396,1.2327576],[141.97537,84.141205,133.40277,67.08878,49.176826,40.858925,42.88434,42.2147,132.67282,80.00618,153.36374,91.70945,155.4691,96.3264,152.4525,93.54178,142.32178,74.48703,55.57248,46.668175,48.139343,46.980145,137.49011,84.49529,157.85863,95.65032,159.37459,100.69826,157.95702,99.90557,150.3601,83.60167,66.20372,58.462883,60.834957,60.649296,152.11246,99.21096,172.95294,110.57141,174.51155,115.7852,172.68082,113.79858,164.2026,95.82728,77.04666,67.95979,69.34544,68.17056,158.92993,105.44536,179.0325,115.759476,179.55356,120.43843,177.13416,117.91124,168.76724]] From f935493f284e1acd94eacdf0c50f91de688d817e Mon Sep 17 00:00:00 2001 From: Bhavna Jindal Date: Fri, 19 Jan 2024 09:05:04 -0800 Subject: [PATCH 123/264] fix trailing whitespaces --- docs/en/sql-reference/functions/time-series-functions.md | 2 +- src/Functions/seriesDecomposeSTL.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/time-series-functions.md b/docs/en/sql-reference/functions/time-series-functions.md index 2e42aa884b4..21e66302ad2 100644 --- a/docs/en/sql-reference/functions/time-series-functions.md +++ b/docs/en/sql-reference/functions/time-series-functions.md @@ -78,7 +78,7 @@ The number of data points in `series` should be at least twice the value of `per **Returned value** - An array of four arrays where the first array include seasonal components, the second array - trend, -the third array - residue component, and the fourth array - baseline component. +the third array - residue component, and the fourth array - baseline(seasoanl + trend) component. Type: [Array](../../sql-reference/data-types/array.md). diff --git a/src/Functions/seriesDecomposeSTL.cpp b/src/Functions/seriesDecomposeSTL.cpp index 0c44afa32a6..9a6a229e282 100644 --- a/src/Functions/seriesDecomposeSTL.cpp +++ b/src/Functions/seriesDecomposeSTL.cpp @@ -128,7 +128,7 @@ public: res_data.insert(residue.begin(), residue.end()); res_col_offsets_data.push_back(res_data.size()); - // Create Baseline = seasonal + trend + // Create Baseline = seasonal + trend std::transform(seasonal.begin(), seasonal.end(), trend.begin(), std::back_inserter(res_data), std::plus()); res_col_offsets_data.push_back(res_data.size()); @@ -205,7 +205,7 @@ The number of data points in `series` should be at least twice the value of `per **Returned value** -- An array of four arrays where the first array include seasonal components, the second array - trend, the third array - residue component, and the fourth array - baseline component. +- An array of four arrays where the first array include seasonal components, the second array - trend, the third array - residue component, and the fourth array - baseline(seasoanl + trend) component. Type: [Array](../../sql-reference/data-types/array.md). From d9edd5a7f36491a8d86705e6c7221c1a74cd6ef5 Mon Sep 17 00:00:00 2001 From: Bhavna Jindal Date: Fri, 19 Jan 2024 10:20:01 -0800 Subject: [PATCH 124/264] fix spell check --- docs/en/sql-reference/functions/time-series-functions.md | 2 +- src/Functions/seriesDecomposeSTL.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/time-series-functions.md b/docs/en/sql-reference/functions/time-series-functions.md index 21e66302ad2..016c3410944 100644 --- a/docs/en/sql-reference/functions/time-series-functions.md +++ b/docs/en/sql-reference/functions/time-series-functions.md @@ -78,7 +78,7 @@ The number of data points in `series` should be at least twice the value of `per **Returned value** - An array of four arrays where the first array include seasonal components, the second array - trend, -the third array - residue component, and the fourth array - baseline(seasoanl + trend) component. +the third array - residue component, and the fourth array - baseline(seasonal + trend) component. Type: [Array](../../sql-reference/data-types/array.md). diff --git a/src/Functions/seriesDecomposeSTL.cpp b/src/Functions/seriesDecomposeSTL.cpp index 9a6a229e282..4376691868b 100644 --- a/src/Functions/seriesDecomposeSTL.cpp +++ b/src/Functions/seriesDecomposeSTL.cpp @@ -205,7 +205,7 @@ The number of data points in `series` should be at least twice the value of `per **Returned value** -- An array of four arrays where the first array include seasonal components, the second array - trend, the third array - residue component, and the fourth array - baseline(seasoanl + trend) component. +- An array of four arrays where the first array include seasonal components, the second array - trend, the third array - residue component, and the fourth array - baseline(seasonal + trend) component. Type: [Array](../../sql-reference/data-types/array.md). From 084ee74b6898214024feafcfa292ff8419bc0050 Mon Sep 17 00:00:00 2001 From: Bhavna Jindal Date: Mon, 22 Jan 2024 07:31:33 -0800 Subject: [PATCH 125/264] minor fix --- src/Functions/seriesDecomposeSTL.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/seriesDecomposeSTL.cpp b/src/Functions/seriesDecomposeSTL.cpp index 4376691868b..e9276c4aefb 100644 --- a/src/Functions/seriesDecomposeSTL.cpp +++ b/src/Functions/seriesDecomposeSTL.cpp @@ -129,7 +129,7 @@ public: res_col_offsets_data.push_back(res_data.size()); // Create Baseline = seasonal + trend - std::transform(seasonal.begin(), seasonal.end(), trend.begin(), std::back_inserter(res_data), std::plus()); + std::transform(seasonal.begin(), seasonal.end(), trend.begin(), std::back_inserter(res_data), std::plus<>()); res_col_offsets_data.push_back(res_data.size()); root_offsets_data.push_back(res_col_offsets->size()); From b1619218255f75dfa94ee8494757ddfd9795fcf9 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 22 Jan 2024 22:33:23 +0100 Subject: [PATCH 126/264] impl --- .../ParallelReplicasReadingCoordinator.cpp | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 980b128ec75..484a0b37644 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -56,6 +56,32 @@ takeFromRange(const MarkRange & range, size_t min_number_of_marks, size_t & curr current_marks_amount += range_we_take.getNumberOfMarks(); return range_we_take.getNumberOfMarks(); } + +void sortResponseRanges(RangesInDataPartsDescription & result) +{ + std::ranges::sort(result, [](const auto & lhs, const auto & rhs) { return lhs.info < rhs.info; }); + + RangesInDataPartsDescription new_result; + + /// Aggregate ranges for each part within a single entry + for (auto & ranges_in_part : result) + { + if (new_result.empty() || new_result.back().info != ranges_in_part.info) + new_result.push_back(RangesInDataPartDescription{.info = ranges_in_part.info}); + + new_result.back().ranges.insert( + new_result.back().ranges.end(), + std::make_move_iterator(ranges_in_part.ranges.begin()), + std::make_move_iterator(ranges_in_part.ranges.end())); + ranges_in_part.ranges.clear(); + } + + /// Sort ranges for each part + for (auto & ranges_in_part : new_result) + std::sort(ranges_in_part.ranges.begin(), ranges_in_part.ranges.end()); + + result = std::move(new_result); +} } namespace ProfileEvents @@ -775,6 +801,8 @@ ParallelReadResponse DefaultCoordinator::handleRequest(ParallelReadRequest reque } } + sortResponseRanges(response.description); + LOG_DEBUG( log, "Going to respond to replica {} with {}; mine_marks={}, stolen_by_hash={}, stolen_rest={}", From 099200c74bd7cfe6f9bf282e9d0f265a13502934 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 22 Jan 2024 22:50:48 +0100 Subject: [PATCH 127/264] Raise timeout in 02294_decimal_second_errors --- tests/queries/0_stateless/02294_decimal_second_errors.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02294_decimal_second_errors.sql b/tests/queries/0_stateless/02294_decimal_second_errors.sql index b6059dc3d48..52d2279be41 100644 --- a/tests/queries/0_stateless/02294_decimal_second_errors.sql +++ b/tests/queries/0_stateless/02294_decimal_second_errors.sql @@ -4,7 +4,7 @@ SELECT 1 SETTINGS max_execution_time=-Infinity; -- { clientError 72 }; -- Ok values SELECT 1 SETTINGS max_execution_time=-0.5; -SELECT 1 SETTINGS max_execution_time=0.5; +SELECT 1 SETTINGS max_execution_time=5.5; SELECT 1 SETTINGS max_execution_time=-1; SELECT 1 SETTINGS max_execution_time=0.0; SELECT 1 SETTINGS max_execution_time=-0.0; From 3355ed7139a5d05fedd9fdab8df2254bc440db9d Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 23 Jan 2024 00:28:15 +0100 Subject: [PATCH 128/264] Add `[[nodiscard]]` to a couple of methods --- src/Columns/IColumn.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index f012eeca61f..1ccdf255bf4 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -222,11 +222,11 @@ public: /// Deserializes a value that was serialized using IColumn::serializeValueIntoArena method. /// Returns pointer to the position after the read data. - virtual const char * deserializeAndInsertFromArena(const char * pos) = 0; + [[nodiscard]] virtual const char * deserializeAndInsertFromArena(const char * pos) = 0; /// Skip previously serialized value that was serialized using IColumn::serializeValueIntoArena method. /// Returns a pointer to the position after the deserialized data. - virtual const char * skipSerializedInArena(const char *) const = 0; + [[nodiscard]] virtual const char * skipSerializedInArena(const char *) const = 0; /// Update state of hash function with value of n-th element. /// On subsequent calls of this method for sequence of column values of arbitrary types, From a1de16d660fc38a658b158ab09ac2a53013684e0 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 22 Jan 2024 23:46:54 +0000 Subject: [PATCH 129/264] wait for currently running jobs to finish --- src/Common/AsyncLoader.cpp | 13 +++++-------- src/Common/AsyncLoader.h | 3 --- 2 files changed, 5 insertions(+), 11 deletions(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 62f97c18c1d..3b7eac3e0d4 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -241,17 +241,14 @@ void AsyncLoader::wait() void AsyncLoader::stop() { - stopAndDoNotWait(); + { + std::unique_lock lock{mutex}; + is_running = false; + // NOTE: there is no need to notify because workers never wait + } wait(); } -void AsyncLoader::stopAndDoNotWait() -{ - std::unique_lock lock{mutex}; - is_running = false; - // NOTE: there is no need to notify because workers never wait -} - void AsyncLoader::schedule(LoadTask & task) { chassert(this == &task.loader); diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 0d91f5fcb05..b02bc2ac06a 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -348,9 +348,6 @@ public: // - or canceled using ~Task() or remove() later. void stop(); - // Do not run new jobs - void stopAndDoNotWait(); - // Schedule all jobs of given `task` and their dependencies (even if they are not in task). // All dependencies of a scheduled job inherit its pool if it has higher priority. This way higher priority job // never waits for (blocked by) lower priority jobs. No priority inversion is possible. From 7b99d9a176f546d1d2a19e6625c9014168bfebaf Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 22 Jan 2024 23:50:53 +0000 Subject: [PATCH 130/264] wait for currently running jobs to finish #2 --- programs/server/Server.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 70dcff56428..6220009cd21 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1746,7 +1746,10 @@ try // to avoid simultaneously running table startups and destructing databases. SCOPE_EXIT_SAFE( LOG_INFO(log, "Stopping AsyncLoader."); - global_context->getAsyncLoader().stopAndDoNotWait(); // Note that currently running jobs will proceed + + // Waits for all currently running jobs to finish and do not run any other pending jobs. + // Pending jobs will be canceled and destructed later by `load_metadata_tasks` dtor. + global_context->getAsyncLoader().stop(); ); try From 72466557a8c85adf0d07318233d8adb8f183e7b8 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 23 Jan 2024 01:27:39 +0000 Subject: [PATCH 131/264] update test Signed-off-by: Duc Canh Le --- .../0_stateless/02870_per_column_settings.reference | 1 - tests/queries/0_stateless/02870_per_column_settings.sql | 7 +++---- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02870_per_column_settings.reference b/tests/queries/0_stateless/02870_per_column_settings.reference index a57407e4f0e..144c8c5ee2e 100644 --- a/tests/queries/0_stateless/02870_per_column_settings.reference +++ b/tests/queries/0_stateless/02870_per_column_settings.reference @@ -16,4 +16,3 @@ CREATE TABLE default.tab\n(\n `id` UInt64,\n `long_string` String SETTINGS (8,8) 8 (9,9) 9 --- ---- diff --git a/tests/queries/0_stateless/02870_per_column_settings.sql b/tests/queries/0_stateless/02870_per_column_settings.sql index 9f48c666ff9..345cf5cc744 100644 --- a/tests/queries/0_stateless/02870_per_column_settings.sql +++ b/tests/queries/0_stateless/02870_per_column_settings.sql @@ -1,5 +1,6 @@ --- Tags: no-random-merge-tree-settings - +-- Tags: no-random-merge-tree-settings, no-replicated-database +-- Tag no-replicated-database: Old syntax is not allowed +-- The test use replicated table to test serialize and deserialize column with settings declaration on zookeeper -- Tests column-level settings for MergeTree* tables DROP TABLE IF EXISTS tab; @@ -66,5 +67,3 @@ CREATE TABLE tab ENGINE = MergeTree ORDER BY id SETTINGS min_bytes_for_wide_part = 1; -- {serverError UNKNOWN_SETTING} - -SELECT '--- '; From 6db59de888269f7e616ab995451188bd865a7528 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 23 Jan 2024 05:54:25 +0000 Subject: [PATCH 132/264] use LOGICAL_ERROR to see if it breaks --- src/Databases/DatabaseOrdinary.cpp | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 8194f04446e..ba1b2cdacad 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -37,7 +37,6 @@ namespace DB namespace ErrorCodes { - extern const int UNKNOWN_TABLE; extern const int LOGICAL_ERROR; extern const int UNKNOWN_DATABASE_ENGINE; } @@ -216,11 +215,8 @@ LoadTaskPtr DatabaseOrdinary::startupTableAsync( logAboutProgress(log, ++tables_started, total_tables_to_startup, startup_watch); } else - { - // This might happen if synchronous metadata loading failed and server is going to shutdown. - throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist during startup", + throw Exception(ErrorCodes::LOGICAL_ERROR, "Table {}.{} doesn't exist during startup", backQuote(name.database), backQuote(name.table)); - } }); return startup_table[name.table] = makeLoadTask(async_loader, {job}); From e72b44a0bfabfda81ad8a6718babc265cf3db26f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 23 Jan 2024 05:58:24 +0000 Subject: [PATCH 133/264] Fix style, pt. II --- tests/integration/test_system_reload_async_metrics/test.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_system_reload_async_metrics/test.py b/tests/integration/test_system_reload_async_metrics/test.py index 941fb90f464..343f2aa0ae0 100644 --- a/tests/integration/test_system_reload_async_metrics/test.py +++ b/tests/integration/test_system_reload_async_metrics/test.py @@ -36,9 +36,10 @@ def test_query_cache_size_is_runtime_configurable(start_cluster): "SELECT value FROM system.asynchronous_metrics WHERE metric = 'NumberOfTables'" ) + # create table and test that the table creation is reflected in the asynchronous metrics node.query( "CREATE TABLE tab (col UInt64) ENGINE MergeTree ORDER BY tuple()" - ) # do anything dumb + ) node.query("SYSTEM RELOAD ASYNCHRONOUS METRICS") From ebdad85c4f7bb2a5f59e778474f2c3b32b1290b1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 23 Jan 2024 06:11:47 +0000 Subject: [PATCH 134/264] Sort integer aliases --- src/DataTypes/DataTypesNumber.cpp | 33 +++++++++++++++---------------- 1 file changed, 16 insertions(+), 17 deletions(-) diff --git a/src/DataTypes/DataTypesNumber.cpp b/src/DataTypes/DataTypesNumber.cpp index e472021e756..008fa287064 100644 --- a/src/DataTypes/DataTypesNumber.cpp +++ b/src/DataTypes/DataTypesNumber.cpp @@ -66,32 +66,27 @@ void registerDataTypeNumbers(DataTypeFactory & factory) /// These synonyms are added for compatibility. factory.registerAlias("TINYINT", "Int8", DataTypeFactory::CaseInsensitive); - factory.registerAlias("INT1", "Int8", DataTypeFactory::CaseInsensitive); /// MySQL - factory.registerAlias("BYTE", "Int8", DataTypeFactory::CaseInsensitive); /// MS Access - factory.registerAlias("SMALLINT", "Int16", DataTypeFactory::CaseInsensitive); - factory.registerAlias("INT", "Int32", DataTypeFactory::CaseInsensitive); - factory.registerAlias("INTEGER", "Int32", DataTypeFactory::CaseInsensitive); - factory.registerAlias("BIGINT", "Int64", DataTypeFactory::CaseInsensitive); - factory.registerAlias("FLOAT", "Float32", DataTypeFactory::CaseInsensitive); - factory.registerAlias("REAL", "Float32", DataTypeFactory::CaseInsensitive); - factory.registerAlias("SINGLE", "Float32", DataTypeFactory::CaseInsensitive); /// MS Access - factory.registerAlias("DOUBLE", "Float64", DataTypeFactory::CaseInsensitive); - factory.registerAlias("MEDIUMINT", "Int32", DataTypeFactory::CaseInsensitive); /// MySQL - - factory.registerAlias("DOUBLE PRECISION", "Float64", DataTypeFactory::CaseInsensitive); - - /// MySQL + factory.registerAlias("INT1", "Int8", DataTypeFactory::CaseInsensitive); + factory.registerAlias("BYTE", "Int8", DataTypeFactory::CaseInsensitive); factory.registerAlias("TINYINT SIGNED", "Int8", DataTypeFactory::CaseInsensitive); factory.registerAlias("INT1 SIGNED", "Int8", DataTypeFactory::CaseInsensitive); + factory.registerAlias("SMALLINT", "Int16", DataTypeFactory::CaseInsensitive); factory.registerAlias("SMALLINT SIGNED", "Int16", DataTypeFactory::CaseInsensitive); + factory.registerAlias("INT", "Int32", DataTypeFactory::CaseInsensitive); + factory.registerAlias("INTEGER", "Int32", DataTypeFactory::CaseInsensitive); + factory.registerAlias("MEDIUMINT", "Int32", DataTypeFactory::CaseInsensitive); factory.registerAlias("MEDIUMINT SIGNED", "Int32", DataTypeFactory::CaseInsensitive); factory.registerAlias("INT SIGNED", "Int32", DataTypeFactory::CaseInsensitive); factory.registerAlias("INTEGER SIGNED", "Int32", DataTypeFactory::CaseInsensitive); + factory.registerAlias("BIGINT", "Int64", DataTypeFactory::CaseInsensitive); factory.registerAlias("SIGNED", "Int64", DataTypeFactory::CaseInsensitive); factory.registerAlias("BIGINT SIGNED", "Int64", DataTypeFactory::CaseInsensitive); + factory.registerAlias("TIME", "Int64", DataTypeFactory::CaseInsensitive); + factory.registerAlias("TINYINT UNSIGNED", "UInt8", DataTypeFactory::CaseInsensitive); factory.registerAlias("INT1 UNSIGNED", "UInt8", DataTypeFactory::CaseInsensitive); factory.registerAlias("SMALLINT UNSIGNED", "UInt16", DataTypeFactory::CaseInsensitive); + factory.registerAlias("YEAR", "UInt16", DataTypeFactory::CaseInsensitive); factory.registerAlias("MEDIUMINT UNSIGNED", "UInt32", DataTypeFactory::CaseInsensitive); factory.registerAlias("INT UNSIGNED", "UInt32", DataTypeFactory::CaseInsensitive); factory.registerAlias("INTEGER UNSIGNED", "UInt32", DataTypeFactory::CaseInsensitive); @@ -99,8 +94,12 @@ void registerDataTypeNumbers(DataTypeFactory & factory) factory.registerAlias("BIGINT UNSIGNED", "UInt64", DataTypeFactory::CaseInsensitive); factory.registerAlias("BIT", "UInt64", DataTypeFactory::CaseInsensitive); factory.registerAlias("SET", "UInt64", DataTypeFactory::CaseInsensitive); - factory.registerAlias("YEAR", "UInt16", DataTypeFactory::CaseInsensitive); - factory.registerAlias("TIME", "Int64", DataTypeFactory::CaseInsensitive); + + factory.registerAlias("FLOAT", "Float32", DataTypeFactory::CaseInsensitive); + factory.registerAlias("REAL", "Float32", DataTypeFactory::CaseInsensitive); + factory.registerAlias("SINGLE", "Float32", DataTypeFactory::CaseInsensitive); + factory.registerAlias("DOUBLE", "Float64", DataTypeFactory::CaseInsensitive); + factory.registerAlias("DOUBLE PRECISION", "Float64", DataTypeFactory::CaseInsensitive); } } From 5a420cd9b11e3027bad18b77f7cfbfd0bb7a62cc Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 23 Jan 2024 06:17:09 +0000 Subject: [PATCH 135/264] Update docs --- docs/en/sql-reference/data-types/float.md | 4 ++-- docs/en/sql-reference/data-types/int-uint.md | 16 ++++++++++++---- 2 files changed, 14 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/data-types/float.md b/docs/en/sql-reference/data-types/float.md index f1b99153b41..be7b2a7fcd8 100644 --- a/docs/en/sql-reference/data-types/float.md +++ b/docs/en/sql-reference/data-types/float.md @@ -35,8 +35,8 @@ Types are equivalent to types of C: Aliases: -- `Float32` — `FLOAT`. -- `Float64` — `DOUBLE`. +- `Float32` — `FLOAT`, `REAL`, `SINGLE`. +- `Float64` — `DOUBLE`, `DOUBLE PRECISION`. When creating tables, numeric parameters for floating point numbers can be set (e.g. `FLOAT(12)`, `FLOAT(15, 22)`, `DOUBLE(12)`, `DOUBLE(4, 18)`), but ClickHouse ignores them. diff --git a/docs/en/sql-reference/data-types/int-uint.md b/docs/en/sql-reference/data-types/int-uint.md index b551143d92f..520454a859f 100644 --- a/docs/en/sql-reference/data-types/int-uint.md +++ b/docs/en/sql-reference/data-types/int-uint.md @@ -21,10 +21,10 @@ When creating tables, numeric parameters for integer numbers can be set (e.g. `T Aliases: -- `Int8` — `TINYINT`, `BOOL`, `BOOLEAN`, `INT1`. -- `Int16` — `SMALLINT`, `INT2`. -- `Int32` — `INT`, `INT4`, `INTEGER`. -- `Int64` — `BIGINT`. +- `Int8` — `TINYINT`, `INT1`, `BYTE`, `TINYINT SIGNED`, `INT1 SIGNED`. +- `Int16` — `SMALLINT`, `SMALLINT SIGNED`. +- `Int32` — `INT`, `INTEGER`, `MEDIUMINT`, `MEDIUMINT SIGNED`, `INT SIGNED`, `INTEGER SIGNED`. +- `Int64` — `BIGINT`, `SIGNED`, `BIGINT SIGNED`, `TIME`. ## UInt Ranges @@ -34,3 +34,11 @@ Aliases: - `UInt64` — \[0 : 18446744073709551615\] - `UInt128` — \[0 : 340282366920938463463374607431768211455\] - `UInt256` — \[0 : 115792089237316195423570985008687907853269984665640564039457584007913129639935\] + +Aliases: + +- `UInt8` — `TINYINT UNSIGNED`, `INT1 UNSIGNED`. +- `UInt16` — `SMALLINT UNSIGNED`. +- `UInt32` — `MEDIUMINT UNSIGNED`, `INT UNSIGNED`, `INTEGER UNSIGNED` +- `UInt64` — `UNSIGNED`, `BIGINT UNSIGNED`, `BIT`, `SET` + From 8fc918b2722b13858a00bf62e5bb21f0b750f451 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 23 Jan 2024 06:23:34 +0000 Subject: [PATCH 136/264] Fix style, pt. III --- tests/integration/test_system_reload_async_metrics/test.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/integration/test_system_reload_async_metrics/test.py b/tests/integration/test_system_reload_async_metrics/test.py index 343f2aa0ae0..7d1f8e995f6 100644 --- a/tests/integration/test_system_reload_async_metrics/test.py +++ b/tests/integration/test_system_reload_async_metrics/test.py @@ -37,9 +37,7 @@ def test_query_cache_size_is_runtime_configurable(start_cluster): ) # create table and test that the table creation is reflected in the asynchronous metrics - node.query( - "CREATE TABLE tab (col UInt64) ENGINE MergeTree ORDER BY tuple()" - ) + node.query("CREATE TABLE tab (col UInt64) ENGINE MergeTree ORDER BY tuple()") node.query("SYSTEM RELOAD ASYNCHRONOUS METRICS") From ff644831e2447d402e8446d906dea668a36d8944 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 23 Jan 2024 08:02:30 +0000 Subject: [PATCH 137/264] Avoid election timeouts during startup --- src/Coordination/KeeperServer.cpp | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 0ca88fafba7..2400e5d418c 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -208,6 +208,11 @@ struct KeeperServer::KeeperRaftServer : public nuraft::raft_server return sm_commit_exec_in_progress_; } + void setServingRequest(bool value) + { + serving_req_ = value; + } + using nuraft::raft_server::raft_server; // peers are initially marked as responding because at least one cycle @@ -687,6 +692,14 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ if (req.get_type() != nuraft::msg_type::append_entries_request) break; + if (req.log_entries().empty()) + break; + + /// comitting/preprocessing of local logs can take some time + /// and we don't want election to start during that time so we + /// set serving requests to avoid elections on timeout + raft_instance->setServingRequest(true); + SCOPE_EXIT(raft_instance->setServingRequest(false)); /// maybe we got snapshot installed if (state_machine->last_commit_index() >= last_log_idx_on_disk && !raft_instance->isCommitInProgress()) preprocess_logs(); From 9ce3f8a7aa5c0fb6e4da9003049c360ced3c9a78 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 23 Jan 2024 09:18:51 +0100 Subject: [PATCH 138/264] typo --- src/Coordination/KeeperServer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 2400e5d418c..45619ab38a1 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -695,7 +695,7 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ if (req.log_entries().empty()) break; - /// comitting/preprocessing of local logs can take some time + /// committing/preprocessing of local logs can take some time /// and we don't want election to start during that time so we /// set serving requests to avoid elections on timeout raft_instance->setServingRequest(true); From 35a3879c1f04984e04645a0eb3fed6f348719c21 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 23 Jan 2024 10:17:19 +0100 Subject: [PATCH 139/264] Add missing setting max_estimated_execution_time in SettingsChangesHistory --- src/Core/SettingsChangesHistory.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 9c8c46d5ce6..1f73b569f86 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -97,7 +97,8 @@ static std::map sett {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, {"max_rows_in_set_to_optimize_join", 100000, 0, "Disable join optimization as it prevents from read in order optimization"}, {"output_format_pretty_color", true, "auto", "Setting is changed to allow also for auto value, disabling ANSI escapes if output is not a tty"}, - {"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"}}}, + {"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"}, + {"max_estimated_execution_time", 0, 0, "Seperate max_execution_time and max_estimated_execution_time"}}}, {"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}, {"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, From 415589d4fc5d5ed963ccb6da81bab2ff7a81acc6 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 23 Jan 2024 10:32:28 +0100 Subject: [PATCH 140/264] Fix typo --- src/Core/SettingsChangesHistory.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 1f73b569f86..ef37aa5fb47 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -98,7 +98,7 @@ static std::map sett {"max_rows_in_set_to_optimize_join", 100000, 0, "Disable join optimization as it prevents from read in order optimization"}, {"output_format_pretty_color", true, "auto", "Setting is changed to allow also for auto value, disabling ANSI escapes if output is not a tty"}, {"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"}, - {"max_estimated_execution_time", 0, 0, "Seperate max_execution_time and max_estimated_execution_time"}}}, + {"max_estimated_execution_time", 0, 0, "Separate max_execution_time and max_estimated_execution_time"}}}, {"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}, {"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, From 78df07199bc57c8dac9a56fb8092eb1256ad8b56 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 23 Jan 2024 07:16:38 +0000 Subject: [PATCH 141/264] More consistent tests for inverted index --- ...> 02346_inverted_index_bug47393.reference} | 0 .../02346_inverted_index_bug47393.sql | 25 +++++++++++++++++++ ...> 02346_inverted_index_bug52019.reference} | 0 ....sql => 02346_inverted_index_bug52019.sql} | 17 ++++++++++--- ...46_inverted_index_detach_attach.reference} | 0 ...=> 02346_inverted_index_detach_attach.sql} | 6 ++--- ...nverted_index_experimental_flag.reference} | 0 ...2346_inverted_index_experimental_flag.sql} | 3 +++ ..._inverted_index_match_predicate.reference} | 0 ... 02346_inverted_index_match_predicate.sql} | 2 ++ .../02346_inverted_index_mutation.sql | 25 ------------------- ... => 02346_inverted_index_search.reference} | 0 ...ch.sql => 02346_inverted_index_search.sql} | 0 13 files changed, 47 insertions(+), 31 deletions(-) rename tests/queries/0_stateless/{02346_inverted_index_mutation.reference => 02346_inverted_index_bug47393.reference} (100%) create mode 100644 tests/queries/0_stateless/02346_inverted_index_bug47393.sql rename tests/queries/0_stateless/{02696_inverted_idx_checksums.reference => 02346_inverted_index_bug52019.reference} (100%) rename tests/queries/0_stateless/{02862_index_inverted_incorrect_args.sql => 02346_inverted_index_bug52019.sql} (62%) rename tests/queries/0_stateless/{02862_index_inverted_incorrect_args.reference => 02346_inverted_index_detach_attach.reference} (100%) rename tests/queries/0_stateless/{02696_inverted_idx_checksums.sql => 02346_inverted_index_detach_attach.sql} (75%) rename tests/queries/0_stateless/{02895_forbid_create_inverted_index.reference => 02346_inverted_index_experimental_flag.reference} (100%) rename tests/queries/0_stateless/{02895_forbid_create_inverted_index.sql => 02346_inverted_index_experimental_flag.sql} (72%) rename tests/queries/0_stateless/{02951_inverted_index_support_match.reference => 02346_inverted_index_match_predicate.reference} (100%) rename tests/queries/0_stateless/{02951_inverted_index_support_match.sql => 02346_inverted_index_match_predicate.sql} (97%) delete mode 100644 tests/queries/0_stateless/02346_inverted_index_mutation.sql rename tests/queries/0_stateless/{02346_full_text_search.reference => 02346_inverted_index_search.reference} (100%) rename tests/queries/0_stateless/{02346_full_text_search.sql => 02346_inverted_index_search.sql} (100%) diff --git a/tests/queries/0_stateless/02346_inverted_index_mutation.reference b/tests/queries/0_stateless/02346_inverted_index_bug47393.reference similarity index 100% rename from tests/queries/0_stateless/02346_inverted_index_mutation.reference rename to tests/queries/0_stateless/02346_inverted_index_bug47393.reference diff --git a/tests/queries/0_stateless/02346_inverted_index_bug47393.sql b/tests/queries/0_stateless/02346_inverted_index_bug47393.sql new file mode 100644 index 00000000000..166e051b120 --- /dev/null +++ b/tests/queries/0_stateless/02346_inverted_index_bug47393.sql @@ -0,0 +1,25 @@ +SET allow_experimental_inverted_index = 1; + +DROP TABLE IF EXISTS tab; +CREATE TABLE tab +( + id UInt64, + str String, + INDEX idx str TYPE inverted(3) GRANULARITY 1 +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS min_rows_for_wide_part = 1, min_bytes_for_wide_part = 1; + +INSERT INTO tab (str) VALUES ('I am inverted'); + +SELECT data_version FROM system.parts WHERE database = currentDatabase() AND table = 'tab' AND active = 1; + +-- update column synchronously +ALTER TABLE tab UPDATE str = 'I am not inverted' WHERE 1 SETTINGS mutations_sync=1; + +SELECT data_version FROM system.parts WHERE database = currentDatabase() AND table = 'tab' AND active = 1; + +SELECT str FROM tab WHERE str LIKE '%inverted%' SETTINGS force_data_skipping_indices = 'idx'; + +DROP TABLE tab; diff --git a/tests/queries/0_stateless/02696_inverted_idx_checksums.reference b/tests/queries/0_stateless/02346_inverted_index_bug52019.reference similarity index 100% rename from tests/queries/0_stateless/02696_inverted_idx_checksums.reference rename to tests/queries/0_stateless/02346_inverted_index_bug52019.reference diff --git a/tests/queries/0_stateless/02862_index_inverted_incorrect_args.sql b/tests/queries/0_stateless/02346_inverted_index_bug52019.sql similarity index 62% rename from tests/queries/0_stateless/02862_index_inverted_incorrect_args.sql rename to tests/queries/0_stateless/02346_inverted_index_bug52019.sql index 7ba122a7155..c61e17d9cea 100644 --- a/tests/queries/0_stateless/02862_index_inverted_incorrect_args.sql +++ b/tests/queries/0_stateless/02346_inverted_index_bug52019.sql @@ -1,9 +1,20 @@ --- https://github.com/ClickHouse/ClickHouse/issues/52019 -DROP TABLE IF EXISTS tab; +-- Test for Bug 52019: Undefined behavior + SET allow_experimental_inverted_index=1; -CREATE TABLE tab (`k` UInt64, `s` Map(String, String), INDEX af mapKeys(s) TYPE inverted(2) GRANULARITY 1) ENGINE = MergeTree ORDER BY k SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi'; + +DROP TABLE IF EXISTS tab; + +CREATE TABLE tab ( + k UInt64, + s Map(String, String), + INDEX idx mapKeys(s) TYPE inverted(2) GRANULARITY 1) +ENGINE = MergeTree +ORDER BY k +SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi'; + INSERT INTO tab (k) VALUES (0); SELECT * FROM tab PREWHERE (s[NULL]) = 'Click a03' SETTINGS allow_experimental_analyzer=1; SELECT * FROM tab PREWHERE (s[1]) = 'Click a03' SETTINGS allow_experimental_analyzer=1; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT * FROM tab PREWHERE (s['foo']) = 'Click a03' SETTINGS allow_experimental_analyzer=1; + DROP TABLE tab; diff --git a/tests/queries/0_stateless/02862_index_inverted_incorrect_args.reference b/tests/queries/0_stateless/02346_inverted_index_detach_attach.reference similarity index 100% rename from tests/queries/0_stateless/02862_index_inverted_incorrect_args.reference rename to tests/queries/0_stateless/02346_inverted_index_detach_attach.reference diff --git a/tests/queries/0_stateless/02696_inverted_idx_checksums.sql b/tests/queries/0_stateless/02346_inverted_index_detach_attach.sql similarity index 75% rename from tests/queries/0_stateless/02696_inverted_idx_checksums.sql rename to tests/queries/0_stateless/02346_inverted_index_detach_attach.sql index 92ffa7a6196..762d78922fe 100644 --- a/tests/queries/0_stateless/02696_inverted_idx_checksums.sql +++ b/tests/queries/0_stateless/02346_inverted_index_detach_attach.sql @@ -2,8 +2,8 @@ SET allow_experimental_inverted_index = 1; CREATE TABLE t ( - `key` UInt64, - `str` String, + key UInt64, + str String, INDEX inv_idx str TYPE inverted(0) GRANULARITY 1 ) ENGINE = MergeTree @@ -13,4 +13,4 @@ INSERT INTO t VALUES (1, 'Hello World'); ALTER TABLE t DETACH PART 'all_1_1_0'; -ALTER TABLE t ATTACH PART 'all_1_1_0'; \ No newline at end of file +ALTER TABLE t ATTACH PART 'all_1_1_0'; diff --git a/tests/queries/0_stateless/02895_forbid_create_inverted_index.reference b/tests/queries/0_stateless/02346_inverted_index_experimental_flag.reference similarity index 100% rename from tests/queries/0_stateless/02895_forbid_create_inverted_index.reference rename to tests/queries/0_stateless/02346_inverted_index_experimental_flag.reference diff --git a/tests/queries/0_stateless/02895_forbid_create_inverted_index.sql b/tests/queries/0_stateless/02346_inverted_index_experimental_flag.sql similarity index 72% rename from tests/queries/0_stateless/02895_forbid_create_inverted_index.sql rename to tests/queries/0_stateless/02346_inverted_index_experimental_flag.sql index dc92d9198fb..bf89265372e 100644 --- a/tests/queries/0_stateless/02895_forbid_create_inverted_index.sql +++ b/tests/queries/0_stateless/02346_inverted_index_experimental_flag.sql @@ -1,4 +1,7 @@ +-- Tests that the inverted index can only be supported when allow_experimental_inverted_index = 1. + SET allow_experimental_inverted_index = 0; + DROP TABLE IF EXISTS tab; CREATE TABLE tab ( diff --git a/tests/queries/0_stateless/02951_inverted_index_support_match.reference b/tests/queries/0_stateless/02346_inverted_index_match_predicate.reference similarity index 100% rename from tests/queries/0_stateless/02951_inverted_index_support_match.reference rename to tests/queries/0_stateless/02346_inverted_index_match_predicate.reference diff --git a/tests/queries/0_stateless/02951_inverted_index_support_match.sql b/tests/queries/0_stateless/02346_inverted_index_match_predicate.sql similarity index 97% rename from tests/queries/0_stateless/02951_inverted_index_support_match.sql rename to tests/queries/0_stateless/02346_inverted_index_match_predicate.sql index 9ebf10412d9..99405c0acf2 100644 --- a/tests/queries/0_stateless/02951_inverted_index_support_match.sql +++ b/tests/queries/0_stateless/02346_inverted_index_match_predicate.sql @@ -1,3 +1,5 @@ +-- Tests that match() utilizes the inverted index + SET allow_experimental_inverted_index = true; DROP TABLE IF EXISTS tab; diff --git a/tests/queries/0_stateless/02346_inverted_index_mutation.sql b/tests/queries/0_stateless/02346_inverted_index_mutation.sql deleted file mode 100644 index 83b73807cd7..00000000000 --- a/tests/queries/0_stateless/02346_inverted_index_mutation.sql +++ /dev/null @@ -1,25 +0,0 @@ -SET allow_experimental_inverted_index=1; - -DROP TABLE IF EXISTS t; -CREATE TABLE t -( - `timestamp` UInt64, - `s` String, - INDEX idx s TYPE inverted(3) GRANULARITY 1 -) -ENGINE = MergeTree -ORDER BY tuple() -SETTINGS min_rows_for_wide_part = 1, min_bytes_for_wide_part = 1; - -INSERT INTO t (s) VALUES ('I am inverted'); - -SELECT data_version FROM system.parts WHERE database=currentDatabase() AND table='t' AND active=1; - --- do update column synchronously -ALTER TABLE t UPDATE s='I am not inverted' WHERE 1 SETTINGS mutations_sync=1; - -SELECT data_version FROM system.parts WHERE database=currentDatabase() AND table='t' AND active=1; - -SELECT s FROM t WHERE s LIKE '%inverted%' SETTINGS force_data_skipping_indices='idx'; - -DROP TABLE t; diff --git a/tests/queries/0_stateless/02346_full_text_search.reference b/tests/queries/0_stateless/02346_inverted_index_search.reference similarity index 100% rename from tests/queries/0_stateless/02346_full_text_search.reference rename to tests/queries/0_stateless/02346_inverted_index_search.reference diff --git a/tests/queries/0_stateless/02346_full_text_search.sql b/tests/queries/0_stateless/02346_inverted_index_search.sql similarity index 100% rename from tests/queries/0_stateless/02346_full_text_search.sql rename to tests/queries/0_stateless/02346_inverted_index_search.sql From bfec324b2818a3764c09347508125051273dac25 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 23 Jan 2024 10:06:25 +0000 Subject: [PATCH 142/264] Some fixups + test --- src/Storages/MergeTree/MutateTask.cpp | 18 ++++++++++------- .../02346_inverted_index_bug59039.reference | 0 .../02346_inverted_index_bug59039.sql | 20 +++++++++++++++++++ 3 files changed, 31 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/02346_inverted_index_bug59039.reference create mode 100644 tests/queries/0_stateless/02346_inverted_index_bug59039.sql diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index fccee6bd887..48aad368dd4 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -674,21 +674,25 @@ static NameToNameVector collectFilesForRenames( { if (command.type == MutationCommand::Type::DROP_INDEX) { - const std::vector suffixes = {".idx2", ".idx", ".gin_dict", ".gin_post", ".gin_seg", ".gin_sid"}; + static const std::array suffixes = {".idx2", ".idx"}; + static const std::array gin_suffixes = {".gin_dict", ".gin_post", ".gin_seg", ".gin_sid"}; /// .gin_* is inverted index for (const auto & suffix : suffixes) { - String filename = INDEX_FILE_PREFIX + command.column_name + suffix; + const String filename = INDEX_FILE_PREFIX + command.column_name + suffix; + const String filename_mrk = INDEX_FILE_PREFIX + command.column_name + mrk_extension; - if ((suffix == ".idx2" || suffix == ".idx") && source_part->checksums.has(filename)) + if (source_part->checksums.has(filename)) { add_rename(filename, ""); - add_rename(INDEX_FILE_PREFIX + command.column_name + mrk_extension, ""); + add_rename(filename_mrk, ""); } - else if (source_part->checksums.has(filename)) - { + } + for (const auto & gin_suffix : gin_suffixes) + { + const String filename = INDEX_FILE_PREFIX + command.column_name + gin_suffix; + if (source_part->checksums.has(filename)) add_rename(filename, ""); - } } } else if (command.type == MutationCommand::Type::DROP_PROJECTION) diff --git a/tests/queries/0_stateless/02346_inverted_index_bug59039.reference b/tests/queries/0_stateless/02346_inverted_index_bug59039.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02346_inverted_index_bug59039.sql b/tests/queries/0_stateless/02346_inverted_index_bug59039.sql new file mode 100644 index 00000000000..0ef0cb0c733 --- /dev/null +++ b/tests/queries/0_stateless/02346_inverted_index_bug59039.sql @@ -0,0 +1,20 @@ +-- This is supposed to test that DROP INDEX removes all index related files. Can't test this directly but at least run the statement and +-- check that no bad things happen. + +SET allow_experimental_inverted_index = 1; + +DROP TABLE IF EXISTS tab; + +CREATE TABLE tab +( + id UInt64, + doc String, + INDEX text_idx doc TYPE inverted +) +ENGINE = MergeTree +ORDER BY id +SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi', min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0; + +ALTER TABLE tab DROP INDEX text_idx; + +DROP TABLE tab; From bca73d4b0a5f6d25e4b26a3c36caecf9aed797d7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 23 Jan 2024 11:51:21 +0100 Subject: [PATCH 143/264] Fix --- src/Storages/DataLakes/IStorageDataLake.h | 5 ++--- src/Storages/DataLakes/Iceberg/StorageIceberg.cpp | 14 ++++---------- src/Storages/StorageS3.h | 1 - src/TableFunctions/ITableFunctionDataLake.h | 2 +- 4 files changed, 7 insertions(+), 15 deletions(-) diff --git a/src/Storages/DataLakes/IStorageDataLake.h b/src/Storages/DataLakes/IStorageDataLake.h index 0e21b3d65ac..d87b1272ba6 100644 --- a/src/Storages/DataLakes/IStorageDataLake.h +++ b/src/Storages/DataLakes/IStorageDataLake.h @@ -89,7 +89,7 @@ private: { if (!attach) throw; - configuration.is_broken = true; + tryLogCurrentException(__PRETTY_FUNCTION__); return configuration; } } @@ -104,11 +104,10 @@ private: const bool updated = base_configuration.update(local_context); auto new_keys = getDataFiles(base_configuration, local_context); - if (!updated && !base_configuration.is_broken && new_keys == Storage::getConfiguration().keys) + if (!updated && new_keys == Storage::getConfiguration().keys) return; Storage::useConfiguration(getConfigurationForDataRead(base_configuration, local_context, new_keys)); - base_configuration.is_broken = false; } Configuration base_configuration; diff --git a/src/Storages/DataLakes/Iceberg/StorageIceberg.cpp b/src/Storages/DataLakes/Iceberg/StorageIceberg.cpp index 2f6b9a5a694..8a1a2cdbd8f 100644 --- a/src/Storages/DataLakes/Iceberg/StorageIceberg.cpp +++ b/src/Storages/DataLakes/Iceberg/StorageIceberg.cpp @@ -29,7 +29,7 @@ StoragePtr StorageIceberg::create( { if (!attach) throw; - configuration.is_broken = true; + tryLogCurrentException(__PRETTY_FUNCTION__); } return std::make_shared( @@ -74,21 +74,15 @@ void StorageIceberg::updateConfigurationImpl(ContextPtr local_context) const bool updated = base_configuration.update(local_context); auto new_metadata = parseIcebergMetadata(base_configuration, local_context); - if (!current_metadata) - current_metadata = parseIcebergMetadata(base_configuration, local_context); - - /// Check if nothing was changed. - if (!updated && !base_configuration.is_broken && new_metadata->getVersion() == current_metadata->getVersion()) - return; - - if (new_metadata->getVersion() != current_metadata->getVersion()) + if (!current_metadata || new_metadata->getVersion() != current_metadata->getVersion()) current_metadata = std::move(new_metadata); + else if (!updated) + return; auto updated_configuration{base_configuration}; /// If metadata wasn't changed, we won't list data files again. updated_configuration.keys = current_metadata->getDataFiles(); StorageS3::useConfiguration(updated_configuration); - base_configuration.is_broken = false; } } diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index f33c13ece86..b90a0d394cb 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -304,7 +304,6 @@ public: std::shared_ptr client; std::vector keys; - bool is_broken = false; }; StorageS3( diff --git a/src/TableFunctions/ITableFunctionDataLake.h b/src/TableFunctions/ITableFunctionDataLake.h index 3e4d6f8201d..961e5683fe2 100644 --- a/src/TableFunctions/ITableFunctionDataLake.h +++ b/src/TableFunctions/ITableFunctionDataLake.h @@ -34,7 +34,7 @@ protected: columns = parseColumnsListFromString(TableFunction::configuration.structure, context); StoragePtr storage = Storage::create( - TableFunction::configuration, context, true, StorageID(TableFunction::getDatabaseName(), table_name), + TableFunction::configuration, context, false, StorageID(TableFunction::getDatabaseName(), table_name), columns, ConstraintsDescription{}, String{}, std::nullopt); storage->startup(); From a4c6f87fb91c81d770316ba9054cd8b193021ce2 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 23 Jan 2024 10:52:52 +0000 Subject: [PATCH 144/264] Further reduce runtime of norm_distance.xml --- tests/performance/norm_distance.xml | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/performance/norm_distance.xml b/tests/performance/norm_distance.xml index 2bfcf2c9157..1e879607dac 100644 --- a/tests/performance/norm_distance.xml +++ b/tests/performance/norm_distance.xml @@ -4,8 +4,9 @@ element_type - UInt8 - Int16 + + + Int32 Int64 Float32 @@ -46,7 +47,7 @@ rand(n*10+140), rand(n*10+141), rand(n*10+142), rand(n*10+143), rand(n*10+144), rand(n*10+145), rand(n*10+146), rand(n*10+147), rand(n*10+148), rand(n*10+149) ] AS v FROM system.numbers - LIMIT 8000000 + LIMIT 5000000 ); From 97520799830548396be5b9cb175c48b6a5d521fa Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 23 Jan 2024 10:33:03 +0000 Subject: [PATCH 145/264] Cosmetics --- src/Common/AsynchronousMetrics.cpp | 2 +- tests/integration/test_system_reload_async_metrics/test.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index b328f8a09f4..ec94d17f590 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -272,7 +272,7 @@ void AsynchronousMetrics::start() { /// Update once right now, to make metrics available just after server start /// (without waiting for asynchronous_metrics_update_period_s). - update(std::chrono::system_clock::now(), false); + update(std::chrono::system_clock::now()); thread = std::make_unique([this] { run(); }); } diff --git a/tests/integration/test_system_reload_async_metrics/test.py b/tests/integration/test_system_reload_async_metrics/test.py index 7d1f8e995f6..f0572cd2db6 100644 --- a/tests/integration/test_system_reload_async_metrics/test.py +++ b/tests/integration/test_system_reload_async_metrics/test.py @@ -29,7 +29,7 @@ SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) CONFIG_DIR = os.path.join(SCRIPT_DIR, "configs") -def test_query_cache_size_is_runtime_configurable(start_cluster): +def test_system_reload_async_metrics(start_cluster): node.query("SYSTEM DROP QUERY CACHE") res1 = node.query( From 7a95eb8bd1df2cb469ed5b9c9c2a8707a8abf674 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 23 Jan 2024 11:59:41 +0100 Subject: [PATCH 146/264] Review fixes --- src/Analyzer/Utils.cpp | 20 +++++++++---------- src/Analyzer/Utils.h | 8 ++++---- .../InterpreterSelectQueryAnalyzer.cpp | 6 +----- 3 files changed, 15 insertions(+), 19 deletions(-) diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index 9c3a7c2fb55..da6ab844dd3 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -326,17 +326,17 @@ void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_q } } -QueryTreeNodes extractTrueTableExpressions(const QueryTreeNodePtr & tree) +QueryTreeNodes extractAllTableReferences(const QueryTreeNodePtr & tree) { QueryTreeNodes result; - std::deque nodes_to_process; + QueryTreeNodes nodes_to_process; nodes_to_process.push_back(tree); while (!nodes_to_process.empty()) { - auto node_to_process = std::move(nodes_to_process.front()); - nodes_to_process.pop_front(); + auto node_to_process = std::move(nodes_to_process.back()); + nodes_to_process.pop_back(); auto node_type = node_to_process->getNodeType(); @@ -349,31 +349,31 @@ QueryTreeNodes extractTrueTableExpressions(const QueryTreeNodePtr & tree) } case QueryTreeNodeType::QUERY: { - nodes_to_process.push_front(node_to_process->as()->getJoinTree()); + nodes_to_process.push_back(node_to_process->as()->getJoinTree()); break; } case QueryTreeNodeType::UNION: { for (const auto & union_node : node_to_process->as()->getQueries().getNodes()) - nodes_to_process.push_front(union_node); + nodes_to_process.push_back(union_node); break; } case QueryTreeNodeType::TABLE_FUNCTION: { for (const auto & argument_node : node_to_process->as()->getArgumentsNode()->getChildren()) - nodes_to_process.push_front(argument_node); + nodes_to_process.push_back(argument_node); break; } case QueryTreeNodeType::ARRAY_JOIN: { - nodes_to_process.push_front(node_to_process->as()->getTableExpression()); + nodes_to_process.push_back(node_to_process->as()->getTableExpression()); break; } case QueryTreeNodeType::JOIN: { auto & join_node = node_to_process->as(); - nodes_to_process.push_front(join_node.getRightTableExpression()); - nodes_to_process.push_front(join_node.getLeftTableExpression()); + nodes_to_process.push_back(join_node.getRightTableExpression()); + nodes_to_process.push_back(join_node.getLeftTableExpression()); break; } default: diff --git a/src/Analyzer/Utils.h b/src/Analyzer/Utils.h index 04a5e460974..121384bdc7c 100644 --- a/src/Analyzer/Utils.h +++ b/src/Analyzer/Utils.h @@ -50,13 +50,13 @@ std::optional tryExtractConstantFromConditionNode(const QueryTreeNodePtr & */ void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_query_ast, const QueryTreeNodePtr & table_expression, const IQueryTreeNode::ConvertToASTOptions & convert_to_ast_options); -/// Extract table expressions from tree -QueryTreeNodes extractTrueTableExpressions(const QueryTreeNodePtr & tree); +/// Extract all TableNodes from the query tree. +QueryTreeNodes extractAllTableReferences(const QueryTreeNodePtr & tree); -/// Extract table, table function, query, union from join tree +/// Extract table, table function, query, union from join tree. QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node, bool add_array_join = false); -/// Extract left table expression from join tree +/// Extract left table expression from join tree. QueryTreeNodePtr extractLeftTableExpression(const QueryTreeNodePtr & join_tree_node); /** Build table expressions stack that consists from table, table function, query, union, join, array join from join tree. diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index 8e6d48102f3..4897101d80b 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -1,4 +1,3 @@ -#include #include #include @@ -75,7 +74,7 @@ ContextMutablePtr buildContext(const ContextPtr & context, const SelectQueryOpti void replaceStorageInQueryTree(QueryTreeNodePtr & query_tree, const ContextPtr & context, const StoragePtr & storage) { - auto nodes = extractTrueTableExpressions(query_tree); + auto nodes = extractAllTableReferences(query_tree); IQueryTreeNode::ReplacementMap replacement_map; for (auto & node : nodes) @@ -91,9 +90,6 @@ void replaceStorageInQueryTree(QueryTreeNodePtr & query_tree, const ContextPtr & if (auto table_expression_modifiers = table_node.getTableExpressionModifiers()) replacement_table_expression->setTableExpressionModifiers(*table_expression_modifiers); - if (node->hasAlias()) - replacement_table_expression->setAlias(node->getAlias() + "_replacement"); - replacement_map.emplace(node.get(), std::move(replacement_table_expression)); } query_tree = query_tree->cloneAndReplace(replacement_map); From c079512741d22946eb30ecdbbdef9d278f18b636 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 23 Jan 2024 11:20:18 +0000 Subject: [PATCH 147/264] Add 02901_parallel_replicas_rollup to analyzer_tech_debt --- tests/analyzer_tech_debt.txt | 3 ++- .../0_stateless/02901_parallel_replicas_rollup.reference | 2 ++ tests/queries/0_stateless/02901_parallel_replicas_rollup.sh | 5 +++++ 3 files changed, 9 insertions(+), 1 deletion(-) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 4643d109c3d..a84f912f371 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -27,8 +27,9 @@ 00917_multiple_joins_denny_crane 02725_agg_projection_resprect_PK 02763_row_policy_storage_merge_alias -02784_parallel_replicas_automatic_decision_join 02818_parameterized_view_with_cte_multiple_usage +# Check after constants refactoring +02901_parallel_replicas_rollup # Flaky. Please don't delete them without fixing them: 01287_max_execution_speed 02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET diff --git a/tests/queries/0_stateless/02901_parallel_replicas_rollup.reference b/tests/queries/0_stateless/02901_parallel_replicas_rollup.reference index 29004fde9d4..c7f78cd74c2 100644 --- a/tests/queries/0_stateless/02901_parallel_replicas_rollup.reference +++ b/tests/queries/0_stateless/02901_parallel_replicas_rollup.reference @@ -1,5 +1,7 @@ 1 02901_parallel_replicas_rollup-default Used parallel replicas: true +Distributed query with analyzer +1 0 0 0 6 2019 0 0 2 2019 1 0 2 diff --git a/tests/queries/0_stateless/02901_parallel_replicas_rollup.sh b/tests/queries/0_stateless/02901_parallel_replicas_rollup.sh index 029b4d07ee2..def813c17b4 100755 --- a/tests/queries/0_stateless/02901_parallel_replicas_rollup.sh +++ b/tests/queries/0_stateless/02901_parallel_replicas_rollup.sh @@ -39,6 +39,11 @@ $CLICKHOUSE_CLIENT \ ORDER BY max((SELECT 1 WHERE 0)); "; were_parallel_replicas_used $query_id + +# It was a bug in analyzer distributed header. +echo "Distributed query with analyzer" +$CLICKHOUSE_CLIENT --query "SELECT 1 FROM remote('127.0.0.{2,3}', currentDatabase(), nested) GROUP BY 1 WITH ROLLUP ORDER BY max((SELECT 1 WHERE 0))" + $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS nested" From a4082f5b971b6da51bcdc65fca26b21b4b429c84 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 23 Jan 2024 13:28:25 +0100 Subject: [PATCH 148/264] Add a test --- tests/integration/test_storage_delta/test.py | 54 ++++++++++++++++++++ 1 file changed, 54 insertions(+) diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index c76a5251d8d..413245fdaa3 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -457,3 +457,57 @@ def test_restart_broken(started_cluster): upload_directory(minio_client, bucket, f"/{TABLE_NAME}", "") assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 + + +def test_restart_broken_table_function(started_cluster): + instance = started_cluster.instances["node1"] + spark = started_cluster.spark_session + minio_client = started_cluster.minio_client + bucket = "broken" + TABLE_NAME = "test_restart_broken" + + if not minio_client.bucket_exists(bucket): + minio_client.make_bucket(bucket) + + parquet_data_path = create_initial_data_file( + started_cluster, + instance, + "SELECT number, toString(number) FROM numbers(100)", + TABLE_NAME, + ) + + write_delta_from_file(spark, parquet_data_path, f"/{TABLE_NAME}") + upload_directory(minio_client, bucket, f"/{TABLE_NAME}", "") + instance.query( + f""" + DROP TABLE IF EXISTS {TABLE_NAME}; + CREATE TABLE {TABLE_NAME} + AS deltaLake(s3, filename = '{TABLE_NAME}/', url = 'http://minio1:9001/{bucket}/')""" + ) + assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 + + s3_objects = list_s3_objects(minio_client, bucket, prefix="") + assert ( + len( + list( + minio_client.remove_objects( + bucket, + [DeleteObject(obj) for obj in s3_objects], + ) + ) + ) + == 0 + ) + minio_client.remove_bucket(bucket) + + instance.restart_clickhouse() + + assert "NoSuchBucket" in instance.query_and_get_error( + f"SELECT count() FROM {TABLE_NAME}" + ) + + minio_client.make_bucket(bucket) + + upload_directory(minio_client, bucket, f"/{TABLE_NAME}", "") + + assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 From e9ea1307e085f425b041c049a66edcf5fd76e609 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 23 Jan 2024 13:03:09 +0000 Subject: [PATCH 149/264] Make optimization more general --- .../OptimizeGroupByInjectiveFunctionsPass.cpp | 54 +++---- src/Analyzer/Passes/QueryAnalysisPass.cpp | 7 +- src/Core/Settings.h | 1 + ...er_eliminate_injective_functions.reference | 142 ++++++++++++++++++ ...analyzer_eliminate_injective_functions.sql | 31 ++++ 5 files changed, 201 insertions(+), 34 deletions(-) create mode 100644 tests/queries/0_stateless/02969_analyzer_eliminate_injective_functions.reference create mode 100644 tests/queries/0_stateless/02969_analyzer_eliminate_injective_functions.sql diff --git a/src/Analyzer/Passes/OptimizeGroupByInjectiveFunctionsPass.cpp b/src/Analyzer/Passes/OptimizeGroupByInjectiveFunctionsPass.cpp index 6dd36733edc..864752cdbeb 100644 --- a/src/Analyzer/Passes/OptimizeGroupByInjectiveFunctionsPass.cpp +++ b/src/Analyzer/Passes/OptimizeGroupByInjectiveFunctionsPass.cpp @@ -40,6 +40,9 @@ public: void enterImpl(QueryTreeNodePtr & node) { + if (!getSettings().optimize_injective_functions_in_group_by) + return; + auto * query = node->as(); if (!query) return; @@ -72,55 +75,42 @@ private: new_group_by_keys.reserve(grouping_set.size()); for (auto & group_by_elem : grouping_set) { - if (auto const * function_node = group_by_elem->as()) + std::queue nodes_to_process; + nodes_to_process.push(group_by_elem); + + while (!nodes_to_process.empty()) { - bool can_be_eliminated = false; - if (possibly_injective_function_names.contains(function_node->getFunctionName())) + auto node_to_process = nodes_to_process.front(); + nodes_to_process.pop(); + + auto const * function_node = node_to_process->as(); + if (!function_node) { - can_be_eliminated = canBeEliminated(function_node, context); - } - else - { - auto function = function_node->getFunctionOrThrow(); - can_be_eliminated = function->isInjective(function_node->getArgumentColumns()); + // Constant aggregation keys are removed in PlannerExpressionAnalysis.cpp + new_group_by_keys.push_back(node_to_process); + continue; } + // Aggregate functions are not allowed in GROUP BY clause + auto function = function_node->getFunctionOrThrow(); + bool can_be_eliminated = function->isInjective(function_node->getArgumentColumns()); + if (can_be_eliminated) { for (auto const & argument : function_node->getArguments()) { + // We can skip constants here because aggregation key is already not a constant. if (argument->getNodeType() != QueryTreeNodeType::CONSTANT) - new_group_by_keys.push_back(argument); + nodes_to_process.push(argument); } } else - new_group_by_keys.push_back(group_by_elem); + new_group_by_keys.push_back(node_to_process); } - else - new_group_by_keys.push_back(group_by_elem); } grouping_set = std::move(new_group_by_keys); } - - bool canBeEliminated(const FunctionNode * function_node, const ContextPtr & context) - { - const auto & function_arguments = function_node->getArguments().getNodes(); - auto const * dict_name_arg = function_arguments[0]->as(); - if (!dict_name_arg || !isString(dict_name_arg->getResultType())) - return false; - auto dict_name = dict_name_arg->getValue().safeGet(); - - const auto & dict_ptr = context->getExternalDictionariesLoader().getDictionary(dict_name, context); - - auto const * attr_name_arg = function_arguments[1]->as(); - if (!attr_name_arg || !isString(attr_name_arg->getResultType())) - return false; - auto attr_name = attr_name_arg->getValue().safeGet(); - - return dict_ptr->isInjective(attr_name); - } - }; } diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index c71eb9e3aca..840b4dbb96e 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -2315,11 +2315,15 @@ std::pair QueryAnalyzer::recursivelyCollectMaxOrdinaryExpressions( */ void QueryAnalyzer::expandGroupByAll(QueryNode & query_tree_node_typed) { + if (!query_tree_node_typed.isGroupByAll()) + return; + auto & group_by_nodes = query_tree_node_typed.getGroupBy().getNodes(); auto & projection_list = query_tree_node_typed.getProjection(); for (auto & node : projection_list.getNodes()) recursivelyCollectMaxOrdinaryExpressions(node, group_by_nodes); + query_tree_node_typed.setIsGroupByAll(false); } void QueryAnalyzer::expandOrderByAll(QueryNode & query_tree_node_typed) @@ -7380,8 +7384,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier node->removeAlias(); } - if (query_node_typed.isGroupByAll()) - expandGroupByAll(query_node_typed); + expandGroupByAll(query_node_typed); validateFilters(query_node); validateAggregates(query_node, { .group_by_use_nulls = scope.group_by_use_nulls }); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 292e945a29c..def1a1a80d5 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -699,6 +699,7 @@ class IColumn; M(SetOperationMode, intersect_default_mode, SetOperationMode::ALL, "Set default mode in INTERSECT query. Possible values: empty string, 'ALL', 'DISTINCT'. If empty, query without mode will throw exception.", 0) \ M(SetOperationMode, except_default_mode, SetOperationMode::ALL, "Set default mode in EXCEPT query. Possible values: empty string, 'ALL', 'DISTINCT'. If empty, query without mode will throw exception.", 0) \ M(Bool, optimize_aggregators_of_group_by_keys, true, "Eliminates min/max/any/anyLast aggregators of GROUP BY keys in SELECT section", 0) \ + M(Bool, optimize_injective_functions_in_group_by, true, "Replaces injective functions by it's arguments in GROUP BY section", 0) \ M(Bool, optimize_group_by_function_keys, true, "Eliminates functions of other keys in GROUP BY section", 0) \ M(Bool, optimize_group_by_constant_keys, true, "Optimize GROUP BY when all keys in block are constant", 0) \ M(Bool, legacy_column_name_of_tuple_literal, false, "List all names of element of large tuple literals in their column names instead of hash. This settings exists only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher.", 0) \ diff --git a/tests/queries/0_stateless/02969_analyzer_eliminate_injective_functions.reference b/tests/queries/0_stateless/02969_analyzer_eliminate_injective_functions.reference new file mode 100644 index 00000000000..72d83e5cf6a --- /dev/null +++ b/tests/queries/0_stateless/02969_analyzer_eliminate_injective_functions.reference @@ -0,0 +1,142 @@ +QUERY id: 0 + PROJECTION COLUMNS + val String + count() UInt64 + PROJECTION + LIST id: 1, nodes: 2 + FUNCTION id: 2, function_name: toString, function_type: ordinary, result_type: String + ARGUMENTS + LIST id: 3, nodes: 1 + FUNCTION id: 4, function_name: toString, function_type: ordinary, result_type: String + ARGUMENTS + LIST id: 5, nodes: 1 + FUNCTION id: 6, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9 + CONSTANT id: 10, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 11, function_name: count, function_type: aggregate, result_type: UInt64 + JOIN TREE + TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers + ARGUMENTS + LIST id: 12, nodes: 1 + CONSTANT id: 13, constant_value: UInt64_2, constant_value_type: UInt8 + GROUP BY + LIST id: 14, nodes: 1 + FUNCTION id: 6, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9 + CONSTANT id: 10, constant_value: UInt64_1, constant_value_type: UInt8 + ORDER BY + LIST id: 15, nodes: 1 + SORT id: 16, sort_direction: ASCENDING, with_fill: 0 + EXPRESSION + FUNCTION id: 2, function_name: toString, function_type: ordinary, result_type: String + ARGUMENTS + LIST id: 3, nodes: 1 + FUNCTION id: 4, function_name: toString, function_type: ordinary, result_type: String + ARGUMENTS + LIST id: 5, nodes: 1 + FUNCTION id: 6, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9 + CONSTANT id: 10, constant_value: UInt64_1, constant_value_type: UInt8 +1 1 +2 1 +QUERY id: 0 + PROJECTION COLUMNS + val String + count() UInt64 + PROJECTION + LIST id: 1, nodes: 2 + FUNCTION id: 2, function_name: toString, function_type: ordinary, result_type: String + ARGUMENTS + LIST id: 3, nodes: 1 + FUNCTION id: 4, function_name: toString, function_type: ordinary, result_type: String + ARGUMENTS + LIST id: 5, nodes: 1 + FUNCTION id: 6, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9 + CONSTANT id: 10, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 11, function_name: count, function_type: aggregate, result_type: UInt64 + JOIN TREE + TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers + ARGUMENTS + LIST id: 12, nodes: 1 + CONSTANT id: 13, constant_value: UInt64_2, constant_value_type: UInt8 + GROUP BY + LIST id: 14, nodes: 1 + FUNCTION id: 6, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9 + CONSTANT id: 10, constant_value: UInt64_1, constant_value_type: UInt8 + ORDER BY + LIST id: 15, nodes: 1 + SORT id: 16, sort_direction: ASCENDING, with_fill: 0 + EXPRESSION + FUNCTION id: 2, function_name: toString, function_type: ordinary, result_type: String + ARGUMENTS + LIST id: 3, nodes: 1 + FUNCTION id: 4, function_name: toString, function_type: ordinary, result_type: String + ARGUMENTS + LIST id: 5, nodes: 1 + FUNCTION id: 6, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9 + CONSTANT id: 10, constant_value: UInt64_1, constant_value_type: UInt8 +CHECK WITH TOTALS +QUERY id: 0, is_group_by_with_totals: 1 + PROJECTION COLUMNS + val String + count() UInt64 + PROJECTION + LIST id: 1, nodes: 2 + FUNCTION id: 2, function_name: toString, function_type: ordinary, result_type: String + ARGUMENTS + LIST id: 3, nodes: 1 + FUNCTION id: 4, function_name: toString, function_type: ordinary, result_type: String + ARGUMENTS + LIST id: 5, nodes: 1 + FUNCTION id: 6, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9 + CONSTANT id: 10, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 11, function_name: count, function_type: aggregate, result_type: UInt64 + JOIN TREE + TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers + ARGUMENTS + LIST id: 12, nodes: 1 + CONSTANT id: 13, constant_value: UInt64_2, constant_value_type: UInt8 + GROUP BY + LIST id: 14, nodes: 1 + FUNCTION id: 6, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9 + CONSTANT id: 10, constant_value: UInt64_1, constant_value_type: UInt8 + ORDER BY + LIST id: 15, nodes: 1 + SORT id: 16, sort_direction: ASCENDING, with_fill: 0 + EXPRESSION + FUNCTION id: 2, function_name: toString, function_type: ordinary, result_type: String + ARGUMENTS + LIST id: 3, nodes: 1 + FUNCTION id: 4, function_name: toString, function_type: ordinary, result_type: String + ARGUMENTS + LIST id: 5, nodes: 1 + FUNCTION id: 6, function_name: plus, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9 + CONSTANT id: 10, constant_value: UInt64_1, constant_value_type: UInt8 +1 1 +2 1 + +0 2 diff --git a/tests/queries/0_stateless/02969_analyzer_eliminate_injective_functions.sql b/tests/queries/0_stateless/02969_analyzer_eliminate_injective_functions.sql new file mode 100644 index 00000000000..15f2550a63e --- /dev/null +++ b/tests/queries/0_stateless/02969_analyzer_eliminate_injective_functions.sql @@ -0,0 +1,31 @@ +set allow_experimental_analyzer = 1; + +EXPLAIN QUERY TREE +SELECT toString(toString(number + 1)) as val, count() +FROM numbers(2) +GROUP BY val +ORDER BY val; + +SELECT toString(toString(number + 1)) as val, count() +FROM numbers(2) +GROUP BY ALL +ORDER BY val; + +EXPLAIN QUERY TREE +SELECT toString(toString(number + 1)) as val, count() +FROM numbers(2) +GROUP BY ALL +ORDER BY val; + +SELECT 'CHECK WITH TOTALS'; + +EXPLAIN QUERY TREE +SELECT toString(toString(number + 1)) as val, count() +FROM numbers(2) +GROUP BY val WITH TOTALS +ORDER BY val; + +SELECT toString(toString(number + 1)) as val, count() +FROM numbers(2) +GROUP BY val WITH TOTALS +ORDER BY val; From c7be7363cf31397f9c57570d269ad9a606a11856 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 23 Jan 2024 14:45:46 +0100 Subject: [PATCH 150/264] fix --- src/AggregateFunctions/AggregateFunctionGroupArray.cpp | 2 +- src/AggregateFunctions/AggregateFunctionTopK.cpp | 10 ++-------- src/AggregateFunctions/KeyHolderHelpers.h | 2 +- src/Functions/array/arrayIntersect.cpp | 2 +- 4 files changed, 5 insertions(+), 11 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp index 6c6397e35d5..f27c3c21a73 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp @@ -440,7 +440,7 @@ struct GroupArrayNodeGeneral : public GroupArrayNodeBase return node; } - void insertInto(IColumn & column) { column.deserializeAndInsertFromArena(data()); } + void insertInto(IColumn & column) { std::ignore = column.deserializeAndInsertFromArena(data()); } }; template diff --git a/src/AggregateFunctions/AggregateFunctionTopK.cpp b/src/AggregateFunctions/AggregateFunctionTopK.cpp index 745fa9a6f23..660f136e4dc 100644 --- a/src/AggregateFunctions/AggregateFunctionTopK.cpp +++ b/src/AggregateFunctions/AggregateFunctionTopK.cpp @@ -19,6 +19,7 @@ #include #include +#include namespace DB @@ -153,8 +154,6 @@ private: UInt64 threshold; UInt64 reserved; - static void deserializeAndInsert(StringRef str, IColumn & data_to); - public: AggregateFunctionTopKGeneric( UInt64 threshold_, UInt64 load_factor, const DataTypes & argument_types_, const Array & params) @@ -251,12 +250,7 @@ public: offsets_to.push_back(offsets_to.back() + result_vec.size()); for (auto & elem : result_vec) - { - if constexpr (is_plain_column) - data_to.insertData(elem.key.data, elem.key.size); - else - data_to.deserializeAndInsertFromArena(elem.key.data); - } + deserializeAndInsert(elem.key, data_to); } }; diff --git a/src/AggregateFunctions/KeyHolderHelpers.h b/src/AggregateFunctions/KeyHolderHelpers.h index 812e681e409..b48c396046b 100644 --- a/src/AggregateFunctions/KeyHolderHelpers.h +++ b/src/AggregateFunctions/KeyHolderHelpers.h @@ -29,7 +29,7 @@ static void deserializeAndInsert(StringRef str, IColumn & data_to) if constexpr (is_plain_column) data_to.insertData(str.data, str.size); else - data_to.deserializeAndInsertFromArena(str.data); + std::ignore = data_to.deserializeAndInsertFromArena(str.data); } } diff --git a/src/Functions/array/arrayIntersect.cpp b/src/Functions/array/arrayIntersect.cpp index ee84e3138e8..ffc7480219e 100644 --- a/src/Functions/array/arrayIntersect.cpp +++ b/src/Functions/array/arrayIntersect.cpp @@ -623,7 +623,7 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable } else { - result_data.deserializeAndInsertFromArena(pair->getKey().data); + std::ignore = result_data.deserializeAndInsertFromArena(pair->getKey().data); } if (all_nullable) null_map.push_back(0); From 291427d931625558ac17645959cfb1d478de628e Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 23 Jan 2024 14:51:55 +0100 Subject: [PATCH 151/264] Update stateless/run.sh --- docker/test/stateless/run.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 638a2408748..b7c04a6fabd 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -247,7 +247,7 @@ stop_logs_replication # Try to get logs while server is running successfuly_saved=0 -for table in query_log zookeeper_log trace_log transactions_info_log +for table in query_log zookeeper_log trace_log transactions_info_log metric_log do clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst successfuly_saved=$? @@ -288,7 +288,7 @@ if [ $successfuly_saved -ne 0 ]; then # directly # - even though ci auto-compress some files (but not *.tsv) it does this only # for files >64MB, we want this files to be compressed explicitly - for table in query_log zookeeper_log trace_log transactions_info_log + for table in query_log zookeeper_log trace_log transactions_info_log metric_log do clickhouse-local "$data_path_config" --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst ||: if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then From a1c83e2f51117a69d484c7ae7884c3bc5dd98129 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 23 Jan 2024 15:26:04 +0100 Subject: [PATCH 152/264] Revert "Allow to attach partition from table with different partition expression when destination partition expression doesn't re-partition" --- .../statements/alter/partition.md | 2 +- src/Interpreters/MonotonicityCheckVisitor.h | 102 +--- src/Interpreters/applyFunction.cpp | 43 -- src/Interpreters/applyFunction.h | 16 - src/Parsers/queryToString.cpp | 5 - src/Parsers/queryToString.h | 1 - src/Storages/MergeTree/IMergeTreeDataPart.cpp | 37 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 1 - src/Storages/MergeTree/KeyCondition.cpp | 85 ++- src/Storages/MergeTree/MergeTreeData.cpp | 350 ++++++++----- src/Storages/MergeTree/MergeTreeData.h | 18 - .../MergeTree/MergeTreeDataPartCloner.cpp | 320 ------------ .../MergeTree/MergeTreeDataPartCloner.h | 43 -- src/Storages/MergeTree/MergeTreePartition.cpp | 39 -- src/Storages/MergeTree/MergeTreePartition.h | 10 +- ...ergeTreePartitionCompatibilityVerifier.cpp | 91 ---- .../MergeTreePartitionCompatibilityVerifier.h | 30 -- ...TreePartitionGlobalMinMaxIdxCalculator.cpp | 25 - ...geTreePartitionGlobalMinMaxIdxCalculator.h | 24 - src/Storages/StorageMergeTree.cpp | 93 +--- src/Storages/StorageReplicatedMergeTree.cpp | 135 +---- .../__init__.py | 0 .../configs/remote_servers.xml | 17 - .../test.py | 214 -------- ...artition_different_partition_exp.reference | 467 ----------------- ...tach_partition_different_partition_exp.sql | 485 ------------------ 26 files changed, 342 insertions(+), 2311 deletions(-) delete mode 100644 src/Interpreters/applyFunction.cpp delete mode 100644 src/Interpreters/applyFunction.h delete mode 100644 src/Storages/MergeTree/MergeTreeDataPartCloner.cpp delete mode 100644 src/Storages/MergeTree/MergeTreeDataPartCloner.h delete mode 100644 src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.cpp delete mode 100644 src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.h delete mode 100644 src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.cpp delete mode 100644 src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.h delete mode 100644 tests/integration/test_attach_partition_distinct_expression_replicated/__init__.py delete mode 100644 tests/integration/test_attach_partition_distinct_expression_replicated/configs/remote_servers.xml delete mode 100644 tests/integration/test_attach_partition_distinct_expression_replicated/test.py delete mode 100644 tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.reference delete mode 100644 tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.sql diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index 5659a0565c5..114b8d5ffe3 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -112,7 +112,7 @@ Note that: For the query to run successfully, the following conditions must be met: - Both tables must have the same structure. -- Both tables must have the same order by key and the same primary key. +- Both tables must have the same partition key, the same order by key and the same primary key. - Both tables must have the same indices and projections. - Both tables must have the same storage policy. diff --git a/src/Interpreters/MonotonicityCheckVisitor.h b/src/Interpreters/MonotonicityCheckVisitor.h index 4e71bd56851..cc386825024 100644 --- a/src/Interpreters/MonotonicityCheckVisitor.h +++ b/src/Interpreters/MonotonicityCheckVisitor.h @@ -1,17 +1,13 @@ #pragma once #include -#include #include -#include #include #include -#include #include -#include +#include #include #include -#include #include #include #include @@ -37,8 +33,6 @@ public: ASTIdentifier * identifier = nullptr; DataTypePtr arg_data_type = {}; - Range range = Range::createWholeUniverse(); - void reject() { monotonicity.is_monotonic = false; } bool isRejected() const { return !monotonicity.is_monotonic; } @@ -103,30 +97,13 @@ public: if (data.isRejected()) return; - /// Monotonicity check only works for functions that contain at most two arguments and one of them must be a constant. - if (!ast_function.arguments) + /// TODO: monotonicity for functions of several arguments + if (!ast_function.arguments || ast_function.arguments->children.size() != 1) { data.reject(); return; } - auto arguments_size = ast_function.arguments->children.size(); - - if (arguments_size == 0 || arguments_size > 2) - { - data.reject(); - return; - } - else if (arguments_size == 2) - { - /// If the function has two arguments, then one of them must be a constant. - if (!ast_function.arguments->children[0]->as() && !ast_function.arguments->children[1]->as()) - { - data.reject(); - return; - } - } - if (!data.canOptimize(ast_function)) { data.reject(); @@ -147,33 +124,14 @@ public: return; } - auto function_arguments = getFunctionArguments(ast_function, data); - - auto function_base = function->build(function_arguments); + ColumnsWithTypeAndName args; + args.emplace_back(data.arg_data_type, "tmp"); + auto function_base = function->build(args); if (function_base && function_base->hasInformationAboutMonotonicity()) { bool is_positive = data.monotonicity.is_positive; - data.monotonicity = function_base->getMonotonicityForRange(*data.arg_data_type, data.range.left, data.range.right); - - auto & key_range = data.range; - - /// If we apply function to open interval, we can get empty intervals in result. - /// E.g. for ('2020-01-03', '2020-01-20') after applying 'toYYYYMM' we will get ('202001', '202001'). - /// To avoid this we make range left and right included. - /// Any function that treats NULL specially is not monotonic. - /// Thus we can safely use isNull() as an -Inf/+Inf indicator here. - if (!key_range.left.isNull()) - { - key_range.left = applyFunction(function_base, data.arg_data_type, key_range.left); - key_range.left_included = true; - } - - if (!key_range.right.isNull()) - { - key_range.right = applyFunction(function_base, data.arg_data_type, key_range.right); - key_range.right_included = true; - } + data.monotonicity = function_base->getMonotonicityForRange(*data.arg_data_type, Field(), Field()); if (!is_positive) data.monotonicity.is_positive = !data.monotonicity.is_positive; @@ -185,53 +143,13 @@ public: static bool needChildVisit(const ASTPtr & parent, const ASTPtr &) { - /// Multi-argument functions with all but one constant arguments can be monotonic. + /// Currently we check monotonicity only for single-argument functions. + /// Although, multi-argument functions with all but one constant arguments can also be monotonic. if (const auto * func = typeid_cast(parent.get())) - return func->arguments->children.size() <= 2; + return func->arguments->children.size() < 2; return true; } - - static ColumnWithTypeAndName extractLiteralColumnAndTypeFromAstLiteral(const ASTLiteral * literal) - { - ColumnWithTypeAndName result; - - result.type = applyVisitor(FieldToDataType(), literal->value); - result.column = result.type->createColumnConst(0, literal->value); - - return result; - } - - static ColumnsWithTypeAndName getFunctionArguments(const ASTFunction & ast_function, const Data & data) - { - ColumnsWithTypeAndName args; - - auto arguments_size = ast_function.arguments->children.size(); - - chassert(arguments_size == 1 || arguments_size == 2); - - if (arguments_size == 2) - { - if (ast_function.arguments->children[0]->as()) - { - const auto * literal = ast_function.arguments->children[0]->as(); - args.push_back(extractLiteralColumnAndTypeFromAstLiteral(literal)); - args.emplace_back(data.arg_data_type, "tmp"); - } - else - { - const auto * literal = ast_function.arguments->children[1]->as(); - args.emplace_back(data.arg_data_type, "tmp"); - args.push_back(extractLiteralColumnAndTypeFromAstLiteral(literal)); - } - } - else - { - args.emplace_back(data.arg_data_type, "tmp"); - } - - return args; - } }; using MonotonicityCheckVisitor = ConstInDepthNodeVisitor; diff --git a/src/Interpreters/applyFunction.cpp b/src/Interpreters/applyFunction.cpp deleted file mode 100644 index a53f14f0381..00000000000 --- a/src/Interpreters/applyFunction.cpp +++ /dev/null @@ -1,43 +0,0 @@ -#include - -#include -#include - -namespace DB -{ - -static Field applyFunctionForField(const FunctionBasePtr & func, const DataTypePtr & arg_type, const Field & arg_value) -{ - ColumnsWithTypeAndName columns{ - {arg_type->createColumnConst(1, arg_value), arg_type, "x"}, - }; - - auto col = func->execute(columns, func->getResultType(), 1); - return (*col)[0]; -} - -FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & current_type, const FieldRef & field) -{ - /// Fallback for fields without block reference. - if (field.isExplicit()) - return applyFunctionForField(func, current_type, field); - - String result_name = "_" + func->getName() + "_" + toString(field.column_idx); - const auto & columns = field.columns; - size_t result_idx = columns->size(); - - for (size_t i = 0; i < result_idx; ++i) - if ((*columns)[i].name == result_name) - result_idx = i; - - if (result_idx == columns->size()) - { - ColumnsWithTypeAndName args{(*columns)[field.column_idx]}; - field.columns->emplace_back(ColumnWithTypeAndName{nullptr, func->getResultType(), result_name}); - (*columns)[result_idx].column = func->execute(args, (*columns)[result_idx].type, columns->front().column->size()); - } - - return {field.columns, field.row_idx, result_idx}; -} - -} diff --git a/src/Interpreters/applyFunction.h b/src/Interpreters/applyFunction.h deleted file mode 100644 index 9b8ae43a53c..00000000000 --- a/src/Interpreters/applyFunction.h +++ /dev/null @@ -1,16 +0,0 @@ -#pragma once - -#include - -namespace DB -{ -struct FieldRef; - -class IFunctionBase; -class IDataType; - -using DataTypePtr = std::shared_ptr; -using FunctionBasePtr = std::shared_ptr; - -FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & current_type, const FieldRef & field); -} diff --git a/src/Parsers/queryToString.cpp b/src/Parsers/queryToString.cpp index 4a1903393f6..9721aa1f128 100644 --- a/src/Parsers/queryToString.cpp +++ b/src/Parsers/queryToString.cpp @@ -3,11 +3,6 @@ namespace DB { - String queryToStringNullable(const ASTPtr & query) - { - return query ? queryToString(query) : ""; - } - String queryToString(const ASTPtr & query) { return queryToString(*query); diff --git a/src/Parsers/queryToString.h b/src/Parsers/queryToString.h index 3acd560b1e2..873de218293 100644 --- a/src/Parsers/queryToString.h +++ b/src/Parsers/queryToString.h @@ -6,5 +6,4 @@ namespace DB { String queryToString(const ASTPtr & query); String queryToString(const IAST & query); - String queryToStringNullable(const ASTPtr & query); } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index f3057a8254f..87f23b0da2a 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -81,7 +81,6 @@ void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const Par auto minmax_column_types = data.getMinMaxColumnsTypes(partition_key); size_t minmax_idx_size = minmax_column_types.size(); - hyperrectangle.clear(); hyperrectangle.reserve(minmax_idx_size); for (size_t i = 0; i < minmax_idx_size; ++i) { @@ -105,39 +104,6 @@ void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const Par initialized = true; } -Block IMergeTreeDataPart::MinMaxIndex::getBlock(const MergeTreeData & data) const -{ - if (!initialized) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to get block from uninitialized MinMax index."); - - Block block; - - const auto metadata_snapshot = data.getInMemoryMetadataPtr(); - const auto & partition_key = metadata_snapshot->getPartitionKey(); - - const auto minmax_column_names = data.getMinMaxColumnsNames(partition_key); - const auto minmax_column_types = data.getMinMaxColumnsTypes(partition_key); - const auto minmax_idx_size = minmax_column_types.size(); - - for (size_t i = 0; i < minmax_idx_size; ++i) - { - const auto & data_type = minmax_column_types[i]; - const auto & column_name = minmax_column_names[i]; - - const auto column = data_type->createColumn(); - - const auto min_val = hyperrectangle.at(i).left; - const auto max_val = hyperrectangle.at(i).right; - - column->insert(min_val); - column->insert(max_val); - - block.insert(ColumnWithTypeAndName(column->getPtr(), data_type, column_name)); - } - - return block; -} - IMergeTreeDataPart::MinMaxIndex::WrittenFiles IMergeTreeDataPart::MinMaxIndex::store( const MergeTreeData & data, IDataPartStorage & part_storage, Checksums & out_checksums) const { @@ -219,7 +185,8 @@ void IMergeTreeDataPart::MinMaxIndex::merge(const MinMaxIndex & other) if (!initialized) { - *this = other; + hyperrectangle = other.hyperrectangle; + initialized = true; } else { diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 29f0f54d419..640a1f1d0a3 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -336,7 +336,6 @@ public: } void load(const MergeTreeData & data, const PartMetadataManagerPtr & manager); - Block getBlock(const MergeTreeData & data) const; using WrittenFiles = std::vector>; diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index e5bcb11091f..d5922ae1bc2 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1,37 +1,36 @@ -#include -#include +#include +#include +#include #include #include #include #include -#include #include -#include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include +#include #include -#include +#include +#include #include -#include #include -#include -#include -#include -#include -#include -#include -#include +#include +#include +#include +#include #include #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include @@ -837,6 +836,21 @@ bool KeyCondition::getConstant(const ASTPtr & expr, Block & block_with_constants return node.tryGetConstant(out_value, out_type); } + +static Field applyFunctionForField( + const FunctionBasePtr & func, + const DataTypePtr & arg_type, + const Field & arg_value) +{ + ColumnsWithTypeAndName columns + { + { arg_type->createColumnConst(1, arg_value), arg_type, "x" }, + }; + + auto col = func->execute(columns, func->getResultType(), 1); + return (*col)[0]; +} + /// The case when arguments may have types different than in the primary key. static std::pair applyFunctionForFieldOfUnknownType( const FunctionBasePtr & func, @@ -876,6 +890,33 @@ static std::pair applyBinaryFunctionForFieldOfUnknownType( return {std::move(result), std::move(return_type)}; } + +static FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & current_type, const FieldRef & field) +{ + /// Fallback for fields without block reference. + if (field.isExplicit()) + return applyFunctionForField(func, current_type, field); + + String result_name = "_" + func->getName() + "_" + toString(field.column_idx); + const auto & columns = field.columns; + size_t result_idx = columns->size(); + + for (size_t i = 0; i < result_idx; ++i) + { + if ((*columns)[i].name == result_name) + result_idx = i; + } + + if (result_idx == columns->size()) + { + ColumnsWithTypeAndName args{(*columns)[field.column_idx]}; + field.columns->emplace_back(ColumnWithTypeAndName {nullptr, func->getResultType(), result_name}); + (*columns)[result_idx].column = func->execute(args, (*columns)[result_idx].type, columns->front().column->size()); + } + + return {field.columns, field.row_idx, result_idx}; +} + /** When table's key has expression with these functions from a column, * and when a column in a query is compared with a constant, such as: * CREATE TABLE (x String) ORDER BY toDate(x) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c3e348a549a..61332a4ff38 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8,6 +8,21 @@ #include #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include #include @@ -28,20 +43,19 @@ #include #include #include +#include +#include #include #include #include #include #include #include -#include -#include #include -#include #include -#include -#include #include +#include +#include #include #include #include @@ -50,41 +64,26 @@ #include #include #include +#include #include #include #include #include #include #include -#include #include #include -#include #include #include #include #include #include -#include #include #include +#include #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include #include @@ -198,50 +197,6 @@ namespace ErrorCodes extern const int LIMIT_EXCEEDED; } -static size_t getPartitionAstFieldsCount(const ASTPartition & partition_ast, ASTPtr partition_value_ast) -{ - if (partition_ast.fields_count.has_value()) - return *partition_ast.fields_count; - - if (partition_value_ast->as()) - return 1; - - const auto * tuple_ast = partition_value_ast->as(); - - if (!tuple_ast) - { - throw Exception( - ErrorCodes::INVALID_PARTITION_VALUE, "Expected literal or tuple for partition key, got {}", partition_value_ast->getID()); - } - - if (tuple_ast->name != "tuple") - { - if (!isFunctionCast(tuple_ast)) - throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name); - - if (tuple_ast->arguments->as()->children.empty()) - throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name); - - auto first_arg = tuple_ast->arguments->as()->children.at(0); - if (const auto * inner_tuple = first_arg->as(); inner_tuple && inner_tuple->name == "tuple") - { - const auto * arguments_ast = tuple_ast->arguments->as(); - return arguments_ast ? arguments_ast->children.size() : 0; - } - else if (const auto * inner_literal_tuple = first_arg->as(); inner_literal_tuple) - { - return inner_literal_tuple->value.getType() == Field::Types::Tuple ? inner_literal_tuple->value.safeGet().size() : 1; - } - - throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name); - } - else - { - const auto * arguments_ast = tuple_ast->arguments->as(); - return arguments_ast ? arguments_ast->children.size() : 0; - } -} - static void checkSuspiciousIndices(const ASTFunction * index_function) { std::unordered_set unique_index_expression_hashes; @@ -4899,7 +4854,7 @@ void MergeTreeData::removePartContributionToColumnAndSecondaryIndexSizes(const D } void MergeTreeData::checkAlterPartitionIsPossible( - const PartitionCommands & commands, const StorageMetadataPtr & /*metadata_snapshot*/, const Settings & settings, ContextPtr) const + const PartitionCommands & commands, const StorageMetadataPtr & /*metadata_snapshot*/, const Settings & settings, ContextPtr local_context) const { for (const auto & command : commands) { @@ -4927,15 +4882,7 @@ void MergeTreeData::checkAlterPartitionIsPossible( throw DB::Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only support DROP/DETACH PARTITION ALL currently"); } else - { - // The below `getPartitionIDFromQuery` call will not work for attach / replace because it assumes the partition expressions - // are the same and deliberately uses this storage. Later on, `MergeTreeData::replaceFrom` is called, and it makes the right - // call to `getPartitionIDFromQuery` using source storage. - // Note: `PartitionCommand::REPLACE_PARTITION` is used both for `REPLACE PARTITION` and `ATTACH PARTITION FROM` queries. - // But not for `ATTACH PARTITION` queries. - if (command.type != PartitionCommand::REPLACE_PARTITION) - getPartitionIDFromQuery(command.partition, getContext()); - } + getPartitionIDFromQuery(command.partition, local_context); } } } @@ -5669,8 +5616,69 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc MergeTreePartInfo::validatePartitionID(partition_ast.id->clone(), format_version); return partition_ast.id->as()->value.safeGet(); } + size_t partition_ast_fields_count = 0; ASTPtr partition_value_ast = partition_ast.value->clone(); - auto partition_ast_fields_count = getPartitionAstFieldsCount(partition_ast, partition_value_ast); + if (!partition_ast.fields_count.has_value()) + { + if (partition_value_ast->as()) + { + partition_ast_fields_count = 1; + } + else if (const auto * tuple_ast = partition_value_ast->as()) + { + if (tuple_ast->name != "tuple") + { + if (isFunctionCast(tuple_ast)) + { + if (tuple_ast->arguments->as()->children.empty()) + { + throw Exception( + ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name); + } + auto first_arg = tuple_ast->arguments->as()->children.at(0); + if (const auto * inner_tuple = first_arg->as(); inner_tuple && inner_tuple->name == "tuple") + { + const auto * arguments_ast = tuple_ast->arguments->as(); + if (arguments_ast) + partition_ast_fields_count = arguments_ast->children.size(); + else + partition_ast_fields_count = 0; + } + else if (const auto * inner_literal_tuple = first_arg->as(); inner_literal_tuple) + { + if (inner_literal_tuple->value.getType() == Field::Types::Tuple) + partition_ast_fields_count = inner_literal_tuple->value.safeGet().size(); + else + partition_ast_fields_count = 1; + } + else + { + throw Exception( + ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name); + } + } + else + throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name); + } + else + { + const auto * arguments_ast = tuple_ast->arguments->as(); + if (arguments_ast) + partition_ast_fields_count = arguments_ast->children.size(); + else + partition_ast_fields_count = 0; + } + } + else + { + throw Exception( + ErrorCodes::INVALID_PARTITION_VALUE, "Expected literal or tuple for partition key, got {}", partition_value_ast->getID()); + } + } + else + { + partition_ast_fields_count = *partition_ast.fields_count; + } if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { @@ -7006,35 +7014,23 @@ MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(IStorage & sour if (my_snapshot->getColumns().getAllPhysical().sizeOfDifference(src_snapshot->getColumns().getAllPhysical())) throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Tables have different structure"); - if (queryToStringNullable(my_snapshot->getSortingKeyAST()) != queryToStringNullable(src_snapshot->getSortingKeyAST())) + auto query_to_string = [] (const ASTPtr & ast) + { + return ast ? queryToString(ast) : ""; + }; + + if (query_to_string(my_snapshot->getSortingKeyAST()) != query_to_string(src_snapshot->getSortingKeyAST())) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different ordering"); + if (query_to_string(my_snapshot->getPartitionKeyAST()) != query_to_string(src_snapshot->getPartitionKeyAST())) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different partition key"); + if (format_version != src_data->format_version) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different format_version"); - if (queryToStringNullable(my_snapshot->getPrimaryKeyAST()) != queryToStringNullable(src_snapshot->getPrimaryKeyAST())) + if (query_to_string(my_snapshot->getPrimaryKeyAST()) != query_to_string(src_snapshot->getPrimaryKeyAST())) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different primary key"); - const auto is_a_subset_of = [](const auto & lhs, const auto & rhs) - { - if (lhs.size() > rhs.size()) - return false; - - const auto rhs_set = NameSet(rhs.begin(), rhs.end()); - for (const auto & lhs_element : lhs) - if (!rhs_set.contains(lhs_element)) - return false; - - return true; - }; - - if (!is_a_subset_of(my_snapshot->getColumnsRequiredForPartitionKey(), src_snapshot->getColumnsRequiredForPartitionKey())) - { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Destination table partition expression columns must be a subset of source table partition expression columns"); - } - const auto check_definitions = [](const auto & my_descriptions, const auto & src_descriptions) { if (my_descriptions.size() != src_descriptions.size()) @@ -7075,56 +7071,128 @@ std::pair MergeTreeData::cloneAn const ReadSettings & read_settings, const WriteSettings & write_settings) { - return MergeTreeDataPartCloner::clone( - this, src_part, metadata_snapshot, dst_part_info, tmp_part_prefix, require_part_metadata, params, read_settings, write_settings); -} + /// Check that the storage policy contains the disk where the src_part is located. + bool does_storage_policy_allow_same_disk = false; + for (const DiskPtr & disk : getStoragePolicy()->getDisks()) + { + if (disk->getName() == src_part->getDataPartStorage().getDiskName()) + { + does_storage_policy_allow_same_disk = true; + break; + } + } + if (!does_storage_policy_allow_same_disk) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Could not clone and load part {} because disk does not belong to storage policy", + quoteString(src_part->getDataPartStorage().getFullPath())); -std::pair MergeTreeData::cloneAndLoadPartOnSameDiskWithDifferentPartitionKey( - const MergeTreeData::DataPartPtr & src_part, - const MergeTreePartition & new_partition, - const String & partition_id, - const IMergeTreeDataPart::MinMaxIndex & min_max_index, - const String & tmp_part_prefix, - const StorageMetadataPtr & my_metadata_snapshot, - const IDataPartStorage::ClonePartParams & clone_params, - ContextPtr local_context, - Int64 min_block, - Int64 max_block -) -{ - MergeTreePartInfo dst_part_info(partition_id, min_block, max_block, src_part->info.level); + String dst_part_name = src_part->getNewName(dst_part_info); + String tmp_dst_part_name = tmp_part_prefix + dst_part_name; + auto temporary_directory_lock = getTemporaryPartDirectoryHolder(tmp_dst_part_name); - return MergeTreeDataPartCloner::cloneWithDistinctPartitionExpression( - this, - src_part, - my_metadata_snapshot, - dst_part_info, - tmp_part_prefix, - local_context->getReadSettings(), - local_context->getWriteSettings(), - new_partition, - min_max_index, - false, - clone_params); -} + /// Why it is needed if we only hardlink files? + auto reservation = src_part->getDataPartStorage().reserve(src_part->getBytesOnDisk()); + auto src_part_storage = src_part->getDataPartStoragePtr(); -std::pair MergeTreeData::createPartitionAndMinMaxIndexFromSourcePart( - const MergeTreeData::DataPartPtr & src_part, - const StorageMetadataPtr & metadata_snapshot, - ContextPtr local_context) -{ - const auto & src_data = src_part->storage; + scope_guard src_flushed_tmp_dir_lock; + MergeTreeData::MutableDataPartPtr src_flushed_tmp_part; - auto metadata_manager = std::make_shared(src_part.get()); - IMergeTreeDataPart::MinMaxIndex min_max_index; + /// If source part is in memory, flush it to disk and clone it already in on-disk format + /// Protect tmp dir from removing by cleanup thread with src_flushed_tmp_dir_lock + /// Construct src_flushed_tmp_part in order to delete part with its directory at destructor + if (auto src_part_in_memory = asInMemoryPart(src_part)) + { + auto flushed_part_path = *src_part_in_memory->getRelativePathForPrefix(tmp_part_prefix); - min_max_index.load(src_data, metadata_manager); + auto tmp_src_part_file_name = fs::path(tmp_dst_part_name).filename(); + src_flushed_tmp_dir_lock = src_part->storage.getTemporaryPartDirectoryHolder(tmp_src_part_file_name); - MergeTreePartition new_partition; + auto flushed_part_storage = src_part_in_memory->flushToDisk(flushed_part_path, metadata_snapshot); - new_partition.create(metadata_snapshot, min_max_index.getBlock(src_data), 0u, local_context); + src_flushed_tmp_part = MergeTreeDataPartBuilder(*this, src_part->name, flushed_part_storage) + .withPartInfo(src_part->info) + .withPartFormatFromDisk() + .build(); - return {new_partition, min_max_index}; + src_flushed_tmp_part->is_temp = true; + src_part_storage = flushed_part_storage; + } + + String with_copy; + if (params.copy_instead_of_hardlink) + with_copy = " (copying data)"; + + auto dst_part_storage = src_part_storage->freeze( + relative_data_path, + tmp_dst_part_name, + read_settings, + write_settings, + /* save_metadata_callback= */ {}, + params); + + if (params.metadata_version_to_write.has_value()) + { + chassert(!params.keep_metadata_version); + auto out_metadata = dst_part_storage->writeFile(IMergeTreeDataPart::METADATA_VERSION_FILE_NAME, 4096, getContext()->getWriteSettings()); + writeText(metadata_snapshot->getMetadataVersion(), *out_metadata); + out_metadata->finalize(); + if (getSettings()->fsync_after_insert) + out_metadata->sync(); + } + + LOG_DEBUG(log, "Clone{} part {} to {}{}", + src_flushed_tmp_part ? " flushed" : "", + src_part_storage->getFullPath(), + std::string(fs::path(dst_part_storage->getFullRootPath()) / tmp_dst_part_name), + with_copy); + + auto dst_data_part = MergeTreeDataPartBuilder(*this, dst_part_name, dst_part_storage) + .withPartFormatFromDisk() + .build(); + + if (!params.copy_instead_of_hardlink && params.hardlinked_files) + { + params.hardlinked_files->source_part_name = src_part->name; + params.hardlinked_files->source_table_shared_id = src_part->storage.getTableSharedID(); + + for (auto it = src_part->getDataPartStorage().iterate(); it->isValid(); it->next()) + { + if (!params.files_to_copy_instead_of_hardlinks.contains(it->name()) + && it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED + && it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME) + { + params.hardlinked_files->hardlinks_from_source_part.insert(it->name()); + } + } + + auto projections = src_part->getProjectionParts(); + for (const auto & [name, projection_part] : projections) + { + const auto & projection_storage = projection_part->getDataPartStorage(); + for (auto it = projection_storage.iterate(); it->isValid(); it->next()) + { + auto file_name_with_projection_prefix = fs::path(projection_storage.getPartDirectory()) / it->name(); + if (!params.files_to_copy_instead_of_hardlinks.contains(file_name_with_projection_prefix) + && it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED + && it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME) + { + params.hardlinked_files->hardlinks_from_source_part.insert(file_name_with_projection_prefix); + } + } + } + } + + /// We should write version metadata on part creation to distinguish it from parts that were created without transaction. + TransactionID tid = params.txn ? params.txn->tid : Tx::PrehistoricTID; + dst_data_part->version.setCreationTID(tid, nullptr); + dst_data_part->storeVersionMetadata(); + + dst_data_part->is_temp = true; + + dst_data_part->loadColumnsChecksumsIndexes(require_part_metadata, true); + dst_data_part->modification_time = dst_part_storage->getLastModified().epochTime(); + return std::make_pair(dst_data_part, std::move(temporary_directory_lock)); } String MergeTreeData::getFullPathOnDisk(const DiskPtr & disk) const diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 9c433e11b84..f0dbaf0e307 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -231,7 +231,6 @@ public: } }; - using DataParts = std::set; using MutableDataParts = std::set; using DataPartsVector = std::vector; @@ -849,23 +848,6 @@ public: const ReadSettings & read_settings, const WriteSettings & write_settings); - std::pair cloneAndLoadPartOnSameDiskWithDifferentPartitionKey( - const MergeTreeData::DataPartPtr & src_part, - const MergeTreePartition & new_partition, - const String & partition_id, - const IMergeTreeDataPart::MinMaxIndex & min_max_index, - const String & tmp_part_prefix, - const StorageMetadataPtr & my_metadata_snapshot, - const IDataPartStorage::ClonePartParams & clone_params, - ContextPtr local_context, - Int64 min_block, - Int64 max_block); - - static std::pair createPartitionAndMinMaxIndexFromSourcePart( - const MergeTreeData::DataPartPtr & src_part, - const StorageMetadataPtr & metadata_snapshot, - ContextPtr local_context); - virtual std::vector getMutationsStatus() const = 0; /// Returns true if table can create new parts with adaptive granularity diff --git a/src/Storages/MergeTree/MergeTreeDataPartCloner.cpp b/src/Storages/MergeTree/MergeTreeDataPartCloner.cpp deleted file mode 100644 index 78cb9aa0624..00000000000 --- a/src/Storages/MergeTree/MergeTreeDataPartCloner.cpp +++ /dev/null @@ -1,320 +0,0 @@ -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ -extern const int BAD_ARGUMENTS; -} - -static Poco::Logger * log = &Poco::Logger::get("MergeTreeDataPartCloner"); - -namespace DistinctPartitionExpression -{ -std::unique_ptr updatePartitionFile( - const MergeTreeData & merge_tree_data, - const MergeTreePartition & partition, - const MergeTreeData::MutableDataPartPtr & dst_part, - IDataPartStorage & storage) -{ - storage.removeFile("partition.dat"); - // Leverage already implemented MergeTreePartition::store to create & store partition.dat. - // Checksum is re-calculated later. - return partition.store(merge_tree_data, storage, dst_part->checksums); -} - -IMergeTreeDataPart::MinMaxIndex::WrittenFiles updateMinMaxFiles( - const MergeTreeData & merge_tree_data, - const MergeTreeData::MutableDataPartPtr & dst_part, - IDataPartStorage & storage, - const StorageMetadataPtr & metadata_snapshot) -{ - for (const auto & column_name : MergeTreeData::getMinMaxColumnsNames(metadata_snapshot->partition_key)) - { - auto file = "minmax_" + escapeForFileName(column_name) + ".idx"; - storage.removeFile(file); - } - - return dst_part->minmax_idx->store(merge_tree_data, storage, dst_part->checksums); -} - -void finalizeNewFiles(const std::vector> & files, bool sync_new_files) -{ - for (const auto & file : files) - { - file->finalize(); - if (sync_new_files) - file->sync(); - } -} - -void updateNewPartFiles( - const MergeTreeData & merge_tree_data, - const MergeTreeData::MutableDataPartPtr & dst_part, - const MergeTreePartition & new_partition, - const IMergeTreeDataPart::MinMaxIndex & new_min_max_index, - const StorageMetadataPtr & src_metadata_snapshot, - bool sync_new_files) -{ - auto & storage = dst_part->getDataPartStorage(); - - *dst_part->minmax_idx = new_min_max_index; - - auto partition_file = updatePartitionFile(merge_tree_data, new_partition, dst_part, storage); - - auto min_max_files = updateMinMaxFiles(merge_tree_data, dst_part, storage, src_metadata_snapshot); - - IMergeTreeDataPart::MinMaxIndex::WrittenFiles written_files; - - if (partition_file) - written_files.emplace_back(std::move(partition_file)); - - written_files.insert(written_files.end(), std::make_move_iterator(min_max_files.begin()), std::make_move_iterator(min_max_files.end())); - - finalizeNewFiles(written_files, sync_new_files); - - // MergeTreeDataPartCloner::finalize_part calls IMergeTreeDataPart::loadColumnsChecksumsIndexes, which will re-create - // the checksum file if it doesn't exist. Relying on that is cumbersome, but this refactoring is simply a code extraction - // with small improvements. It can be further improved in the future. - storage.removeFile("checksums.txt"); -} -} - -namespace -{ -bool doesStoragePolicyAllowSameDisk(MergeTreeData * merge_tree_data, const MergeTreeData::DataPartPtr & src_part) -{ - for (const DiskPtr & disk : merge_tree_data->getStoragePolicy()->getDisks()) - if (disk->getName() == src_part->getDataPartStorage().getDiskName()) - return true; - return false; -} - -DataPartStoragePtr flushPartStorageToDiskIfInMemory( - MergeTreeData * merge_tree_data, - const MergeTreeData::DataPartPtr & src_part, - const StorageMetadataPtr & metadata_snapshot, - const String & tmp_part_prefix, - const String & tmp_dst_part_name, - scope_guard & src_flushed_tmp_dir_lock, - MergeTreeData::MutableDataPartPtr src_flushed_tmp_part) -{ - if (auto src_part_in_memory = asInMemoryPart(src_part)) - { - auto flushed_part_path = src_part_in_memory->getRelativePathForPrefix(tmp_part_prefix); - auto tmp_src_part_file_name = fs::path(tmp_dst_part_name).filename(); - - src_flushed_tmp_dir_lock = src_part->storage.getTemporaryPartDirectoryHolder(tmp_src_part_file_name); - - auto flushed_part_storage = src_part_in_memory->flushToDisk(*flushed_part_path, metadata_snapshot); - - src_flushed_tmp_part = MergeTreeDataPartBuilder(*merge_tree_data, src_part->name, flushed_part_storage) - .withPartInfo(src_part->info) - .withPartFormatFromDisk() - .build(); - - src_flushed_tmp_part->is_temp = true; - - return flushed_part_storage; - } - - return src_part->getDataPartStoragePtr(); -} - -std::shared_ptr hardlinkAllFiles( - MergeTreeData * merge_tree_data, - const DB::ReadSettings & read_settings, - const DB::WriteSettings & write_settings, - const DataPartStoragePtr & storage, - const String & path, - const DB::IDataPartStorage::ClonePartParams & params) -{ - return storage->freeze( - merge_tree_data->getRelativeDataPath(), - path, - read_settings, - write_settings, - /*save_metadata_callback=*/{}, - params); -} - -std::pair cloneSourcePart( - MergeTreeData * merge_tree_data, - const MergeTreeData::DataPartPtr & src_part, - const StorageMetadataPtr & metadata_snapshot, - const MergeTreePartInfo & dst_part_info, - const String & tmp_part_prefix, - const ReadSettings & read_settings, - const WriteSettings & write_settings, - const DB::IDataPartStorage::ClonePartParams & params) -{ - const auto dst_part_name = src_part->getNewName(dst_part_info); - - const auto tmp_dst_part_name = tmp_part_prefix + dst_part_name; - - auto temporary_directory_lock = merge_tree_data->getTemporaryPartDirectoryHolder(tmp_dst_part_name); - - src_part->getDataPartStorage().reserve(src_part->getBytesOnDisk()); - - scope_guard src_flushed_tmp_dir_lock; - MergeTreeData::MutableDataPartPtr src_flushed_tmp_part; - - auto src_part_storage = flushPartStorageToDiskIfInMemory( - merge_tree_data, src_part, metadata_snapshot, tmp_part_prefix, tmp_dst_part_name, src_flushed_tmp_dir_lock, src_flushed_tmp_part); - - auto dst_part_storage = hardlinkAllFiles(merge_tree_data, read_settings, write_settings, src_part_storage, tmp_dst_part_name, params); - - if (params.metadata_version_to_write.has_value()) - { - chassert(!params.keep_metadata_version); - auto out_metadata = dst_part_storage->writeFile( - IMergeTreeDataPart::METADATA_VERSION_FILE_NAME, 4096, merge_tree_data->getContext()->getWriteSettings()); - writeText(metadata_snapshot->getMetadataVersion(), *out_metadata); - out_metadata->finalize(); - if (merge_tree_data->getSettings()->fsync_after_insert) - out_metadata->sync(); - } - - LOG_DEBUG( - log, - "Clone {} part {} to {}{}", - src_flushed_tmp_part ? "flushed" : "", - src_part_storage->getFullPath(), - std::string(fs::path(dst_part_storage->getFullRootPath()) / tmp_dst_part_name), - false); - - - auto part = MergeTreeDataPartBuilder(*merge_tree_data, dst_part_name, dst_part_storage).withPartFormatFromDisk().build(); - - return std::make_pair(part, std::move(temporary_directory_lock)); -} - -void handleHardLinkedParameterFiles(const MergeTreeData::DataPartPtr & src_part, const DB::IDataPartStorage::ClonePartParams & params) -{ - const auto & hardlinked_files = params.hardlinked_files; - - hardlinked_files->source_part_name = src_part->name; - hardlinked_files->source_table_shared_id = src_part->storage.getTableSharedID(); - - for (auto it = src_part->getDataPartStorage().iterate(); it->isValid(); it->next()) - { - if (!params.files_to_copy_instead_of_hardlinks.contains(it->name()) - && it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED - && it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME) - { - hardlinked_files->hardlinks_from_source_part.insert(it->name()); - } - } -} - -void handleProjections(const MergeTreeData::DataPartPtr & src_part, const DB::IDataPartStorage::ClonePartParams & params) -{ - auto projections = src_part->getProjectionParts(); - for (const auto & [name, projection_part] : projections) - { - const auto & projection_storage = projection_part->getDataPartStorage(); - for (auto it = projection_storage.iterate(); it->isValid(); it->next()) - { - auto file_name_with_projection_prefix = fs::path(projection_storage.getPartDirectory()) / it->name(); - if (!params.files_to_copy_instead_of_hardlinks.contains(file_name_with_projection_prefix) - && it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED - && it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME) - { - params.hardlinked_files->hardlinks_from_source_part.insert(file_name_with_projection_prefix); - } - } - } -} - -MergeTreeData::MutableDataPartPtr finalizePart( - const MergeTreeData::MutableDataPartPtr & dst_part, const DB::IDataPartStorage::ClonePartParams & params, bool require_part_metadata) -{ - /// We should write version metadata on part creation to distinguish it from parts that were created without transaction. - TransactionID tid = params.txn ? params.txn->tid : Tx::PrehistoricTID; - dst_part->version.setCreationTID(tid, nullptr); - dst_part->storeVersionMetadata(); - - dst_part->is_temp = true; - - dst_part->loadColumnsChecksumsIndexes(require_part_metadata, true); - - dst_part->modification_time = dst_part->getDataPartStorage().getLastModified().epochTime(); - - return dst_part; -} - -std::pair cloneAndHandleHardlinksAndProjections( - MergeTreeData * merge_tree_data, - const DataPartPtr & src_part, - const StorageMetadataPtr & metadata_snapshot, - const MergeTreePartInfo & dst_part_info, - const String & tmp_part_prefix, - const ReadSettings & read_settings, - const WriteSettings & write_settings, - const IDataPartStorage::ClonePartParams & params) -{ - if (!doesStoragePolicyAllowSameDisk(merge_tree_data, src_part)) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Could not clone and load part {} because disk does not belong to storage policy", - quoteString(src_part->getDataPartStorage().getFullPath())); - - auto [destination_part, temporary_directory_lock] = cloneSourcePart( - merge_tree_data, src_part, metadata_snapshot, dst_part_info, tmp_part_prefix, read_settings, write_settings, params); - - if (!params.copy_instead_of_hardlink && params.hardlinked_files) - { - handleHardLinkedParameterFiles(src_part, params); - handleProjections(src_part, params); - } - - return std::make_pair(destination_part, std::move(temporary_directory_lock)); -} -} - -std::pair MergeTreeDataPartCloner::clone( - MergeTreeData * merge_tree_data, - const DataPartPtr & src_part, - const StorageMetadataPtr & metadata_snapshot, - const MergeTreePartInfo & dst_part_info, - const String & tmp_part_prefix, - bool require_part_metadata, - const IDataPartStorage::ClonePartParams & params, - const ReadSettings & read_settings, - const WriteSettings & write_settings) -{ - auto [destination_part, temporary_directory_lock] = cloneAndHandleHardlinksAndProjections( - merge_tree_data, src_part, metadata_snapshot, dst_part_info, tmp_part_prefix, read_settings, write_settings, params); - - return std::make_pair(finalizePart(destination_part, params, require_part_metadata), std::move(temporary_directory_lock)); -} - -std::pair MergeTreeDataPartCloner::cloneWithDistinctPartitionExpression( - MergeTreeData * merge_tree_data, - const DataPartPtr & src_part, - const StorageMetadataPtr & metadata_snapshot, - const MergeTreePartInfo & dst_part_info, - const String & tmp_part_prefix, - const ReadSettings & read_settings, - const WriteSettings & write_settings, - const MergeTreePartition & new_partition, - const IMergeTreeDataPart::MinMaxIndex & new_min_max_index, - bool sync_new_files, - const IDataPartStorage::ClonePartParams & params) -{ - auto [destination_part, temporary_directory_lock] = cloneAndHandleHardlinksAndProjections( - merge_tree_data, src_part, metadata_snapshot, dst_part_info, tmp_part_prefix, read_settings, write_settings, params); - - DistinctPartitionExpression::updateNewPartFiles( - *merge_tree_data, destination_part, new_partition, new_min_max_index, src_part->storage.getInMemoryMetadataPtr(), sync_new_files); - - return std::make_pair(finalizePart(destination_part, params, false), std::move(temporary_directory_lock)); -} - -} diff --git a/src/Storages/MergeTree/MergeTreeDataPartCloner.h b/src/Storages/MergeTree/MergeTreeDataPartCloner.h deleted file mode 100644 index 53585f20b7f..00000000000 --- a/src/Storages/MergeTree/MergeTreeDataPartCloner.h +++ /dev/null @@ -1,43 +0,0 @@ -#pragma once - -namespace DB -{ - -struct StorageInMemoryMetadata; -using StorageMetadataPtr = std::shared_ptr; -struct MergeTreePartition; -class IMergeTreeDataPart; - -class MergeTreeDataPartCloner -{ -public: - using DataPart = IMergeTreeDataPart; - using MutableDataPartPtr = std::shared_ptr; - using DataPartPtr = std::shared_ptr; - - static std::pair clone( - MergeTreeData * merge_tree_data, - const DataPartPtr & src_part, - const StorageMetadataPtr & metadata_snapshot, - const MergeTreePartInfo & dst_part_info, - const String & tmp_part_prefix, - bool require_part_metadata, - const IDataPartStorage::ClonePartParams & params, - const ReadSettings & read_settings, - const WriteSettings & write_settings); - - static std::pair cloneWithDistinctPartitionExpression( - MergeTreeData * merge_tree_data, - const DataPartPtr & src_part, - const StorageMetadataPtr & metadata_snapshot, - const MergeTreePartInfo & dst_part_info, - const String & tmp_part_prefix, - const ReadSettings & read_settings, - const WriteSettings & write_settings, - const MergeTreePartition & new_partition, - const IMergeTreeDataPart::MinMaxIndex & new_min_max_index, - bool sync_new_files, - const IDataPartStorage::ClonePartParams & params); -}; - -} diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 76ef3be25b3..ddeaf69136a 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -467,45 +467,6 @@ void MergeTreePartition::create(const StorageMetadataPtr & metadata_snapshot, Bl } } -void MergeTreePartition::createAndValidateMinMaxPartitionIds( - const StorageMetadataPtr & metadata_snapshot, Block block_with_min_max_partition_ids, ContextPtr context) -{ - if (!metadata_snapshot->hasPartitionKey()) - return; - - auto partition_key_names_and_types = executePartitionByExpression(metadata_snapshot, block_with_min_max_partition_ids, context); - value.resize(partition_key_names_and_types.size()); - - /// Executing partition_by expression adds new columns to passed block according to partition functions. - /// The block is passed by reference and is used afterwards. `moduloLegacy` needs to be substituted back - /// with just `modulo`, because it was a temporary substitution. - static constexpr std::string_view modulo_legacy_function_name = "moduloLegacy"; - - size_t i = 0; - for (const auto & element : partition_key_names_and_types) - { - auto & partition_column = block_with_min_max_partition_ids.getByName(element.name); - - if (element.name.starts_with(modulo_legacy_function_name)) - partition_column.name.replace(0, modulo_legacy_function_name.size(), "modulo"); - - Field extracted_min_partition_id_field; - Field extracted_max_partition_id_field; - - partition_column.column->get(0, extracted_min_partition_id_field); - partition_column.column->get(1, extracted_max_partition_id_field); - - if (extracted_min_partition_id_field != extracted_max_partition_id_field) - { - throw Exception( - ErrorCodes::INVALID_PARTITION_VALUE, - "Can not create the partition. A partition can not contain values that have different partition ids"); - } - - partition_column.column->get(0u, value[i++]); - } -} - NamesAndTypesList MergeTreePartition::executePartitionByExpression(const StorageMetadataPtr & metadata_snapshot, Block & block, ContextPtr context) { auto adjusted_partition_key = adjustPartitionKey(metadata_snapshot, context); diff --git a/src/Storages/MergeTree/MergeTreePartition.h b/src/Storages/MergeTree/MergeTreePartition.h index fd7ae02cde4..78b141f26ec 100644 --- a/src/Storages/MergeTree/MergeTreePartition.h +++ b/src/Storages/MergeTree/MergeTreePartition.h @@ -1,12 +1,11 @@ #pragma once -#include +#include #include #include #include #include -#include -#include +#include namespace DB { @@ -52,11 +51,6 @@ public: void create(const StorageMetadataPtr & metadata_snapshot, Block block, size_t row, ContextPtr context); - /// Copy of MergeTreePartition::create, but also validates if min max partition keys are equal. If they are different, - /// it means the partition can't be created because the data doesn't belong to the same partition. - void createAndValidateMinMaxPartitionIds( - const StorageMetadataPtr & metadata_snapshot, Block block_with_min_max_partition_ids, ContextPtr context); - static void appendFiles(const MergeTreeData & storage, Strings & files); /// Adjust partition key and execute its expression on block. Return sample block according to used expression. diff --git a/src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.cpp b/src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.cpp deleted file mode 100644 index 21bcdb84a96..00000000000 --- a/src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.cpp +++ /dev/null @@ -1,91 +0,0 @@ -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ -extern const int BAD_ARGUMENTS; -} - -namespace -{ -bool isDestinationPartitionExpressionMonotonicallyIncreasing( - const std::vector & hyperrectangle, const MergeTreeData & destination_storage) -{ - auto destination_table_metadata = destination_storage.getInMemoryMetadataPtr(); - - auto key_description = destination_table_metadata->getPartitionKey(); - auto definition_ast = key_description.definition_ast->clone(); - - auto table_identifier = std::make_shared(destination_storage.getStorageID().getTableName()); - auto table_with_columns - = TableWithColumnNamesAndTypes{DatabaseAndTableWithAlias(table_identifier), destination_table_metadata->getColumns().getOrdinary()}; - - auto expression_list = extractKeyExpressionList(definition_ast); - - MonotonicityCheckVisitor::Data data{{table_with_columns}, destination_storage.getContext(), /*group_by_function_hashes*/ {}}; - - for (auto i = 0u; i < expression_list->children.size(); i++) - { - data.range = hyperrectangle[i]; - - MonotonicityCheckVisitor(data).visit(expression_list->children[i]); - - if (!data.monotonicity.is_monotonic || !data.monotonicity.is_positive) - return false; - } - - return true; -} - -bool isExpressionDirectSubsetOf(const ASTPtr source, const ASTPtr destination) -{ - auto source_expression_list = extractKeyExpressionList(source); - auto destination_expression_list = extractKeyExpressionList(destination); - - std::unordered_set source_columns; - - for (auto i = 0u; i < source_expression_list->children.size(); ++i) - source_columns.insert(source_expression_list->children[i]->getColumnName()); - - for (auto i = 0u; i < destination_expression_list->children.size(); ++i) - if (!source_columns.contains(destination_expression_list->children[i]->getColumnName())) - return false; - - return true; -} -} - -void MergeTreePartitionCompatibilityVerifier::verify( - const MergeTreeData & source_storage, const MergeTreeData & destination_storage, const DataPartsVector & source_parts) -{ - const auto source_metadata = source_storage.getInMemoryMetadataPtr(); - const auto destination_metadata = destination_storage.getInMemoryMetadataPtr(); - - const auto source_partition_key_ast = source_metadata->getPartitionKeyAST(); - const auto destination_partition_key_ast = destination_metadata->getPartitionKeyAST(); - - // If destination partition expression columns are a subset of source partition expression columns, - // there is no need to check for monotonicity. - if (isExpressionDirectSubsetOf(source_partition_key_ast, destination_partition_key_ast)) - return; - - const auto src_global_min_max_indexes = MergeTreePartitionGlobalMinMaxIdxCalculator::calculate(source_parts, destination_storage); - - assert(!src_global_min_max_indexes.hyperrectangle.empty()); - - if (!isDestinationPartitionExpressionMonotonicallyIncreasing(src_global_min_max_indexes.hyperrectangle, destination_storage)) - throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Destination table partition expression is not monotonically increasing"); - - MergeTreePartition().createAndValidateMinMaxPartitionIds( - destination_storage.getInMemoryMetadataPtr(), - src_global_min_max_indexes.getBlock(destination_storage), - destination_storage.getContext()); -} - -} diff --git a/src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.h b/src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.h deleted file mode 100644 index 1682add3ebd..00000000000 --- a/src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.h +++ /dev/null @@ -1,30 +0,0 @@ -#pragma once - -#include -#include - -namespace DB -{ - -/* - * Verifies that source and destination partitions are compatible. - * To be compatible, one of the following criteria must be met: - * 1. Destination partition expression columns are a subset of source partition columns; or - * 2. Destination partition expression is monotonic on the source global min_max idx Range AND the computer partition id for - * the source global min_max idx range is the same. - * - * If not, an exception is thrown. - * */ - -class MergeTreePartitionCompatibilityVerifier -{ -public: - using DataPart = IMergeTreeDataPart; - using DataPartPtr = std::shared_ptr; - using DataPartsVector = std::vector; - - static void - verify(const MergeTreeData & source_storage, const MergeTreeData & destination_storage, const DataPartsVector & source_parts); -}; - -} diff --git a/src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.cpp b/src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.cpp deleted file mode 100644 index 0871efadf0c..00000000000 --- a/src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.cpp +++ /dev/null @@ -1,25 +0,0 @@ -#include - -namespace DB -{ - -IMergeTreeDataPart::MinMaxIndex -MergeTreePartitionGlobalMinMaxIdxCalculator::calculate(const DataPartsVector & parts, const MergeTreeData & storage) -{ - IMergeTreeDataPart::MinMaxIndex global_min_max_indexes; - - for (const auto & part : parts) - { - auto metadata_manager = std::make_shared(part.get()); - - auto local_min_max_index = MergeTreeData::DataPart::MinMaxIndex(); - - local_min_max_index.load(storage, metadata_manager); - - global_min_max_indexes.merge(local_min_max_index); - } - - return global_min_max_indexes; -} - -} diff --git a/src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.h b/src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.h deleted file mode 100644 index 4f271177246..00000000000 --- a/src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.h +++ /dev/null @@ -1,24 +0,0 @@ -#pragma once - -#include - -#include -#include - -namespace DB -{ - -/* - * Calculates global min max indexes for a given set of parts on given storage. - * */ -class MergeTreePartitionGlobalMinMaxIdxCalculator -{ - using DataPart = IMergeTreeDataPart; - using DataPartPtr = std::shared_ptr; - using DataPartsVector = std::vector; - -public: - static IMergeTreeDataPart::MinMaxIndex calculate(const DataPartsVector & parts, const MergeTreeData & storage); -}; - -} diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index fd5354a00a9..4761ccd8b58 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -5,9 +5,9 @@ #include #include +#include #include #include -#include #include "Common/Exception.h" #include #include @@ -20,30 +20,25 @@ #include #include #include -#include #include +#include #include #include #include #include #include -#include #include -#include +#include #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include #include -#include -#include +#include +#include +#include +#include +#include +#include #include #include #include @@ -2044,74 +2039,42 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con ProfileEventsScope profile_events_scope; MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table, source_metadata_snapshot, my_metadata_snapshot); - String partition_id = src_data.getPartitionIDFromQuery(partition, local_context); + String partition_id = getPartitionIDFromQuery(partition, local_context); DataPartsVector src_parts = src_data.getVisibleDataPartsVectorInPartition(local_context, partition_id); - - bool attach_empty_partition = !replace && src_parts.empty(); - if (attach_empty_partition) - return; - MutableDataPartsVector dst_parts; std::vector dst_parts_locks; static const String TMP_PREFIX = "tmp_replace_from_"; - const auto my_partition_expression = my_metadata_snapshot->getPartitionKeyAST(); - const auto src_partition_expression = source_metadata_snapshot->getPartitionKeyAST(); - const auto is_partition_exp_different = queryToStringNullable(my_partition_expression) != queryToStringNullable(src_partition_expression); - - if (is_partition_exp_different && !src_parts.empty()) - MergeTreePartitionCompatibilityVerifier::verify(src_data, /* destination_storage */ *this, src_parts); - - for (DataPartPtr & src_part : src_parts) + for (const DataPartPtr & src_part : src_parts) { if (!canReplacePartition(src_part)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot replace partition '{}' because part '{}' has inconsistent granularity with table", partition_id, src_part->name); - IDataPartStorage::ClonePartParams clone_params{.txn = local_context->getCurrentTransaction()}; /// This will generate unique name in scope of current server process. - auto index = insert_increment.get(); + Int64 temp_index = insert_increment.get(); + MergeTreePartInfo dst_part_info(partition_id, temp_index, temp_index, src_part->info.level); - if (is_partition_exp_different) - { - auto [new_partition, new_min_max_index] = createPartitionAndMinMaxIndexFromSourcePart( - src_part, my_metadata_snapshot, local_context); - - auto [dst_part, part_lock] = cloneAndLoadPartOnSameDiskWithDifferentPartitionKey( - src_part, - new_partition, - new_partition.getID(*this), - new_min_max_index, - TMP_PREFIX, - my_metadata_snapshot, - clone_params, - local_context, - index, - index); - - dst_parts.emplace_back(std::move(dst_part)); - dst_parts_locks.emplace_back(std::move(part_lock)); - } - else - { - MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); - - auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk( - src_part, - TMP_PREFIX, - dst_part_info, - my_metadata_snapshot, - clone_params, - local_context->getReadSettings(), - local_context->getWriteSettings()); - dst_parts.emplace_back(std::move(dst_part)); - dst_parts_locks.emplace_back(std::move(part_lock)); - } + IDataPartStorage::ClonePartParams clone_params{.txn = local_context->getCurrentTransaction()}; + auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk( + src_part, + TMP_PREFIX, + dst_part_info, + my_metadata_snapshot, + clone_params, + local_context->getReadSettings(), + local_context->getWriteSettings()); + dst_parts.emplace_back(std::move(dst_part)); + dst_parts_locks.emplace_back(std::move(part_lock)); } + /// ATTACH empty part set + if (!replace && dst_parts.empty()) + return; + MergeTreePartInfo drop_range; if (replace) { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 512811e39d7..f7e6783dbc2 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -26,21 +26,22 @@ #include +#include #include #include #include #include #include +#include #include #include +#include #include #include #include #include #include -#include #include -#include #include #include #include @@ -52,11 +53,9 @@ #include #include #include -#include #include #include #include -#include #include #include @@ -2714,48 +2713,16 @@ bool StorageReplicatedMergeTree::executeReplaceRange(LogEntry & entry) .copy_instead_of_hardlink = storage_settings_ptr->always_use_copy_instead_of_hardlinks || ((our_zero_copy_enabled || source_zero_copy_enabled) && part_desc->src_table_part->isStoredOnRemoteDiskWithZeroCopySupport()), .metadata_version_to_write = metadata_snapshot->getMetadataVersion() }; - - const auto my_partition_expression = metadata_snapshot->getPartitionKeyAST(); - const auto src_partition_expression = source_table->getInMemoryMetadataPtr()->getPartitionKeyAST(); - - const auto is_partition_exp_different = queryToStringNullable(my_partition_expression) != queryToStringNullable(src_partition_expression); - - if (is_partition_exp_different) - { - auto [new_partition, new_min_max_index] = createPartitionAndMinMaxIndexFromSourcePart( - part_desc->src_table_part, metadata_snapshot, getContext()); - - auto partition_id = new_partition.getID(*this); - - auto [res_part, temporary_part_lock] = cloneAndLoadPartOnSameDiskWithDifferentPartitionKey( - part_desc->src_table_part, - new_partition, - partition_id, - new_min_max_index, - TMP_PREFIX + "clone_", - metadata_snapshot, - clone_params, - getContext(), - part_desc->new_part_info.min_block, - part_desc->new_part_info.max_block); - - part_desc->res_part = std::move(res_part); - part_desc->temporary_part_lock = std::move(temporary_part_lock); - } - else - { - auto [res_part, temporary_part_lock] = cloneAndLoadDataPartOnSameDisk( - part_desc->src_table_part, - TMP_PREFIX + "clone_", - part_desc->new_part_info, - metadata_snapshot, - clone_params, - getContext()->getReadSettings(), - getContext()->getWriteSettings()); - - part_desc->res_part = std::move(res_part); - part_desc->temporary_part_lock = std::move(temporary_part_lock); - } + auto [res_part, temporary_part_lock] = cloneAndLoadDataPartOnSameDisk( + part_desc->src_table_part, + TMP_PREFIX + "clone_", + part_desc->new_part_info, + metadata_snapshot, + clone_params, + getContext()->getReadSettings(), + getContext()->getWriteSettings()); + part_desc->res_part = std::move(res_part); + part_desc->temporary_part_lock = std::move(temporary_part_lock); } else if (!part_desc->replica.empty()) { @@ -7885,22 +7852,11 @@ void StorageReplicatedMergeTree::replacePartitionFrom( ProfileEventsScope profile_events_scope; MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table, source_metadata_snapshot, metadata_snapshot); - String partition_id = src_data.getPartitionIDFromQuery(partition, query_context); + String partition_id = getPartitionIDFromQuery(partition, query_context); /// NOTE: Some covered parts may be missing in src_all_parts if corresponding log entries are not executed yet. DataPartsVector src_all_parts = src_data.getVisibleDataPartsVectorInPartition(query_context, partition_id); - bool attach_empty_partition = !replace && src_all_parts.empty(); - if (attach_empty_partition) - return; - - const auto my_partition_expression = metadata_snapshot->getPartitionKeyAST(); - const auto src_partition_expression = source_metadata_snapshot->getPartitionKeyAST(); - const auto is_partition_exp_different = queryToStringNullable(my_partition_expression) != queryToStringNullable(src_partition_expression); - - if (is_partition_exp_different && !src_all_parts.empty()) - MergeTreePartitionCompatibilityVerifier::verify(src_data, /* destination_storage */ *this, src_all_parts); - LOG_DEBUG(log, "Cloning {} parts", src_all_parts.size()); static const String TMP_PREFIX = "tmp_replace_from_"; @@ -7955,18 +7911,6 @@ void StorageReplicatedMergeTree::replacePartitionFrom( "Cannot replace partition '{}' because part '{}" "' has inconsistent granularity with table", partition_id, src_part->name); - IMergeTreeDataPart::MinMaxIndex min_max_index = *src_part->minmax_idx; - MergeTreePartition merge_tree_partition = src_part->partition; - - if (is_partition_exp_different) - { - auto [new_partition, new_min_max_index] = createPartitionAndMinMaxIndexFromSourcePart(src_part, metadata_snapshot, query_context); - - merge_tree_partition = new_partition; - min_max_index = new_min_max_index; - partition_id = merge_tree_partition.getID(*this); - } - String hash_hex = src_part->checksums.getTotalChecksumHex(); const bool is_duplicated_part = replaced_parts.contains(hash_hex); replaced_parts.insert(hash_hex); @@ -7985,52 +7929,27 @@ void StorageReplicatedMergeTree::replacePartitionFrom( continue; } + UInt64 index = lock->getNumber(); + MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); + bool zero_copy_enabled = storage_settings_ptr->allow_remote_fs_zero_copy_replication || dynamic_cast(source_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; - - UInt64 index = lock->getNumber(); - IDataPartStorage::ClonePartParams clone_params { .copy_instead_of_hardlink = storage_settings_ptr->always_use_copy_instead_of_hardlinks || (zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport()), .metadata_version_to_write = metadata_snapshot->getMetadataVersion() }; - - if (is_partition_exp_different) - { - auto [dst_part, part_lock] = cloneAndLoadPartOnSameDiskWithDifferentPartitionKey( - src_part, - merge_tree_partition, - partition_id, - min_max_index, - TMP_PREFIX, - metadata_snapshot, - clone_params, - query_context, - index, - index); - - dst_parts.emplace_back(dst_part); - dst_parts_locks.emplace_back(std::move(part_lock)); - } - else - { - MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); - - auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk( - src_part, - TMP_PREFIX, - dst_part_info, - metadata_snapshot, - clone_params, - query_context->getReadSettings(), - query_context->getWriteSettings()); - - dst_parts.emplace_back(dst_part); - dst_parts_locks.emplace_back(std::move(part_lock)); - } - + auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk( + src_part, + TMP_PREFIX, + dst_part_info, + metadata_snapshot, + clone_params, + query_context->getReadSettings(), + query_context->getWriteSettings()); src_parts.emplace_back(src_part); + dst_parts.emplace_back(dst_part); + dst_parts_locks.emplace_back(std::move(part_lock)); ephemeral_locks.emplace_back(std::move(*lock)); block_id_paths.emplace_back(block_id_path); part_checksums.emplace_back(hash_hex); diff --git a/tests/integration/test_attach_partition_distinct_expression_replicated/__init__.py b/tests/integration/test_attach_partition_distinct_expression_replicated/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_attach_partition_distinct_expression_replicated/configs/remote_servers.xml b/tests/integration/test_attach_partition_distinct_expression_replicated/configs/remote_servers.xml deleted file mode 100644 index b40730e9f7d..00000000000 --- a/tests/integration/test_attach_partition_distinct_expression_replicated/configs/remote_servers.xml +++ /dev/null @@ -1,17 +0,0 @@ - - - - - true - - replica1 - 9000 - - - replica2 - 9000 - - - - - diff --git a/tests/integration/test_attach_partition_distinct_expression_replicated/test.py b/tests/integration/test_attach_partition_distinct_expression_replicated/test.py deleted file mode 100644 index 1d8ac4e9e37..00000000000 --- a/tests/integration/test_attach_partition_distinct_expression_replicated/test.py +++ /dev/null @@ -1,214 +0,0 @@ -import pytest -from helpers.cluster import ClickHouseCluster -from helpers.test_tools import assert_eq_with_retry - -cluster = ClickHouseCluster(__file__) - -replica1 = cluster.add_instance( - "replica1", with_zookeeper=True, main_configs=["configs/remote_servers.xml"] -) -replica2 = cluster.add_instance( - "replica2", with_zookeeper=True, main_configs=["configs/remote_servers.xml"] -) - - -@pytest.fixture(scope="module") -def start_cluster(): - try: - cluster.start() - yield cluster - except Exception as ex: - print(ex) - finally: - cluster.shutdown() - - -def cleanup(nodes): - for node in nodes: - node.query("DROP TABLE IF EXISTS source SYNC") - node.query("DROP TABLE IF EXISTS destination SYNC") - - -def create_table(node, table_name, replicated): - replica = node.name - engine = ( - f"ReplicatedMergeTree('/clickhouse/tables/1/{table_name}', '{replica}')" - if replicated - else "MergeTree()" - ) - partition_expression = ( - "toYYYYMMDD(timestamp)" if table_name == "source" else "toYYYYMM(timestamp)" - ) - node.query_with_retry( - """ - CREATE TABLE {table_name}(timestamp DateTime) - ENGINE = {engine} - ORDER BY tuple() PARTITION BY {partition_expression} - SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, max_cleanup_delay_period=1; - """.format( - table_name=table_name, - engine=engine, - partition_expression=partition_expression, - ) - ) - - -def test_both_replicated(start_cluster): - for node in [replica1, replica2]: - create_table(node, "source", True) - create_table(node, "destination", True) - - replica1.query("INSERT INTO source VALUES ('2010-03-02 02:01:01')") - replica1.query("SYSTEM SYNC REPLICA source") - replica1.query("SYSTEM SYNC REPLICA destination") - replica1.query( - f"ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source" - ) - - assert_eq_with_retry( - replica1, f"SELECT * FROM destination", "2010-03-02 02:01:01\n" - ) - assert_eq_with_retry( - replica1, - f"SELECT * FROM destination", - replica2.query(f"SELECT * FROM destination"), - ) - - cleanup([replica1, replica2]) - - -def test_only_destination_replicated(start_cluster): - create_table(replica1, "source", False) - create_table(replica1, "destination", True) - create_table(replica2, "destination", True) - - replica1.query("INSERT INTO source VALUES ('2010-03-02 02:01:01')") - replica1.query("SYSTEM SYNC REPLICA destination") - replica1.query( - f"ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source" - ) - - assert_eq_with_retry( - replica1, f"SELECT * FROM destination", "2010-03-02 02:01:01\n" - ) - assert_eq_with_retry( - replica1, - f"SELECT * FROM destination", - replica2.query(f"SELECT * FROM destination"), - ) - - cleanup([replica1, replica2]) - - -def test_both_replicated_partitioned_to_unpartitioned(start_cluster): - def create_tables(nodes): - for node in nodes: - source_engine = ( - f"ReplicatedMergeTree('/clickhouse/tables/1/source', '{node.name}')" - ) - node.query( - """ - CREATE TABLE source(timestamp DateTime) - ENGINE = {engine} - ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp) - SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, max_cleanup_delay_period=1; - """.format( - engine=source_engine, - ) - ) - - destination_engine = f"ReplicatedMergeTree('/clickhouse/tables/1/destination', '{node.name}')" - node.query( - """ - CREATE TABLE destination(timestamp DateTime) - ENGINE = {engine} - ORDER BY tuple() PARTITION BY tuple() - SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, max_cleanup_delay_period=1; - """.format( - engine=destination_engine, - ) - ) - - create_tables([replica1, replica2]) - - replica1.query("INSERT INTO source VALUES ('2010-03-02 02:01:01')") - replica1.query("INSERT INTO source VALUES ('2010-03-03 02:01:01')") - replica1.query("SYSTEM SYNC REPLICA source") - replica1.query("SYSTEM SYNC REPLICA destination") - - replica1.query( - f"ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source" - ) - replica1.query( - f"ALTER TABLE destination ATTACH PARTITION ID '20100303' FROM source" - ) - - assert_eq_with_retry( - replica1, - f"SELECT * FROM destination ORDER BY timestamp", - "2010-03-02 02:01:01\n2010-03-03 02:01:01\n", - ) - assert_eq_with_retry( - replica1, - f"SELECT * FROM destination ORDER BY timestamp", - replica2.query(f"SELECT * FROM destination ORDER BY timestamp"), - ) - - cleanup([replica1, replica2]) - - -def test_both_replicated_different_exp_same_id(start_cluster): - def create_tables(nodes): - for node in nodes: - source_engine = ( - f"ReplicatedMergeTree('/clickhouse/tables/1/source', '{node.name}')" - ) - node.query( - """ - CREATE TABLE source(a UInt16,b UInt16,c UInt16,extra UInt64,Path String,Time DateTime,Value Float64,Timestamp Int64,sign Int8) - ENGINE = {engine} - ORDER BY tuple() PARTITION BY a % 3 - SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, max_cleanup_delay_period=1; - """.format( - engine=source_engine, - ) - ) - - destination_engine = f"ReplicatedMergeTree('/clickhouse/tables/1/destination', '{node.name}')" - node.query( - """ - CREATE TABLE destination(a UInt16,b UInt16,c UInt16,extra UInt64,Path String,Time DateTime,Value Float64,Timestamp Int64,sign Int8) - ENGINE = {engine} - ORDER BY tuple() PARTITION BY a - SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, max_cleanup_delay_period=1; - """.format( - engine=destination_engine, - ) - ) - - create_tables([replica1, replica2]) - - replica1.query( - "INSERT INTO source (a, b, c, extra, sign) VALUES (1, 5, 9, 1000, 1)" - ) - replica1.query( - "INSERT INTO source (a, b, c, extra, sign) VALUES (2, 6, 10, 1000, 1)" - ) - replica1.query("SYSTEM SYNC REPLICA source") - replica1.query("SYSTEM SYNC REPLICA destination") - - replica1.query(f"ALTER TABLE destination ATTACH PARTITION 1 FROM source") - replica1.query(f"ALTER TABLE destination ATTACH PARTITION 2 FROM source") - - assert_eq_with_retry( - replica1, - f"SELECT * FROM destination ORDER BY a", - "1\t5\t9\t1000\t\t1970-01-01 00:00:00\t0\t0\t1\n2\t6\t10\t1000\t\t1970-01-01 00:00:00\t0\t0\t1\n", - ) - assert_eq_with_retry( - replica1, - f"SELECT * FROM destination ORDER BY a", - replica2.query(f"SELECT * FROM destination ORDER BY a"), - ) - - cleanup([replica1, replica2]) diff --git a/tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.reference b/tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.reference deleted file mode 100644 index f1d036b08bf..00000000000 --- a/tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.reference +++ /dev/null @@ -1,467 +0,0 @@ --- { echoOn } --- Should be allowed since destination partition expr is monotonically increasing and compatible -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); -ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -201003 -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION '20100302' FROM source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -201003 --- Should be allowed since destination partition expr is monotonically increasing and compatible. Note that even though --- the destination partition expression is more granular, the data would still fall in the same partition. Thus, it is valid -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); -ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -20100302 -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION '201003' FROM source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -20100302 --- Should be allowed since destination partition expr is monotonically increasing and compatible for those specific values -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY intDiv(A, 6); -CREATE TABLE destination (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY A; -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01', 1), ('2010-03-02 02:01:03', 1); -ALTER TABLE destination ATTACH PARTITION ID '0' FROM source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 1 -2010-03-02 02:01:03 1 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 1 -2010-03-02 02:01:03 1 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -1 -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION 0 FROM source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 1 -2010-03-02 02:01:03 1 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 1 -2010-03-02 02:01:03 1 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -1 --- Should be allowed because dst partition exp is monot inc and data is not split -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY cityHash64(category); -CREATE TABLE destination (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(category); -INSERT INTO TABLE source VALUES ('spaghetti', 'food'), ('mop', 'general'); -INSERT INTO TABLE source VALUES ('rice', 'food'); -ALTER TABLE destination ATTACH PARTITION ID '17908065610379824077' from source; -SELECT * FROM source ORDER BY productName; -mop general -rice food -spaghetti food -SELECT * FROM destination ORDER BY productName; -rice food -spaghetti food -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -59532f3c39a412a413f0f014c7750a9d -59532f3c39a412a413f0f014c7750a9d -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION '17908065610379824077' from source; -SELECT * FROM source ORDER BY productName; -mop general -rice food -spaghetti food -SELECT * FROM destination ORDER BY productName; -rice food -spaghetti food -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -59532f3c39a412a413f0f014c7750a9d -59532f3c39a412a413f0f014c7750a9d --- Should be allowed, extra test case to validate https://github.com/ClickHouse/ClickHouse/pull/39507#issuecomment-1747574133 - -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (timestamp Int64) engine=MergeTree ORDER BY (timestamp) PARTITION BY intDiv(timestamp, 86400000); -CREATE TABLE destination (timestamp Int64) engine=MergeTree ORDER BY (timestamp) PARTITION BY toYear(toDateTime(intDiv(timestamp, 1000))); -INSERT INTO TABLE source VALUES (1267495261123); -ALTER TABLE destination ATTACH PARTITION ID '14670' FROM source; -SELECT * FROM source ORDER BY timestamp; -1267495261123 -SELECT * FROM destination ORDER BY timestamp; -1267495261123 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -2010 -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION '14670' from source; -SELECT * FROM source ORDER BY timestamp; -1267495261123 -SELECT * FROM destination ORDER BY timestamp; -1267495261123 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -2010 --- Should be allowed, extra test case to validate https://github.com/ClickHouse/ClickHouse/pull/39507#issuecomment-1747511726 - -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (timestamp DateTime('UTC'), key Int64, f Float64) engine=MergeTree ORDER BY (key, timestamp) PARTITION BY toYear(timestamp); -CREATE TABLE destination (timestamp DateTime('UTC'), key Int64, f Float64) engine=MergeTree ORDER BY (key, timestamp) PARTITION BY (intDiv(toUInt32(timestamp),86400)); -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01',1,1),('2010-03-02 02:01:01',1,1),('2011-02-02 02:01:03',1,1); -ALTER TABLE destination ATTACH PARTITION ID '2010' FROM source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 1 1 -2010-03-02 02:01:01 1 1 -2011-02-02 02:01:03 1 1 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 1 1 -2010-03-02 02:01:01 1 1 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -14670 -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION '2010' from source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 1 1 -2010-03-02 02:01:01 1 1 -2011-02-02 02:01:03 1 1 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 1 1 -2010-03-02 02:01:01 1 1 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -14670 --- Should be allowed, partitioned table to unpartitioned. Since the destination is unpartitioned, parts would ultimately --- fall into the same partition. --- Destination partition by expression is omitted, which causes StorageMetadata::getPartitionKeyAST() to be nullptr. -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple(); -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); -ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -all -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION '201003' from source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -all --- Same as above, but destination partition by expression is explicitly defined. Test case required to validate that --- partition by tuple() is accepted. -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY tuple(); -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); -ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -all -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION '201003' from source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -all --- Should be allowed because the destination partition expression columns are a subset of the source partition expression columns --- Columns in this case refer to the expression elements, not to the actual table columns -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY (a, b, c); -CREATE TABLE destination (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY (a, b); -INSERT INTO TABLE source VALUES (1, 2, 3), (1, 2, 4); -ALTER TABLE destination ATTACH PARTITION ID '1-2-3' FROM source; -SELECT * FROM source ORDER BY (a, b, c); -1 2 3 -1 2 4 -SELECT * FROM destination ORDER BY (a, b, c); -1 2 3 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -1-2 -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION (1, 2, 3) from source; -SELECT * FROM source ORDER BY (a, b, c); -1 2 3 -1 2 4 -SELECT * FROM destination ORDER BY (a, b, c); -1 2 3 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -1-2 --- Should be allowed because the destination partition expression columns are a subset of the source partition expression columns --- Columns in this case refer to the expression elements, not to the actual table columns -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY (a, b, c); -CREATE TABLE destination (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY a; -INSERT INTO TABLE source VALUES (1, 2, 3), (1, 2, 4); -ALTER TABLE destination ATTACH PARTITION ID '1-2-3' FROM source; -SELECT * FROM source ORDER BY (a, b, c); -1 2 3 -1 2 4 -SELECT * FROM destination ORDER BY (a, b, c); -1 2 3 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -1 -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION (1, 2, 3) from source; -SELECT * FROM source ORDER BY (a, b, c); -1 2 3 -1 2 4 -SELECT * FROM destination ORDER BY (a, b, c); -1 2 3 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -1 --- Should be allowed. Special test case, tricky to explain. First column of source partition expression is --- timestamp, while first column of destination partition expression is `A`. One of the previous implementations --- would not match the columns, which could lead to `timestamp` min max being used to calculate monotonicity of `A`. -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (`timestamp` DateTime, `A` Int64) ENGINE = MergeTree PARTITION BY tuple(toYYYYMM(timestamp), intDiv(A, 6)) ORDER BY timestamp; -CREATE TABLE destination (`timestamp` DateTime, `A` Int64) ENGINE = MergeTree PARTITION BY A ORDER BY timestamp; -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01', 5); -ALTER TABLE destination ATTACH PARTITION ID '201003-0' FROM source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 5 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 5 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -5 -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION (201003, 0) from source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 5 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 5 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -5 --- Should be allowed. Destination partition expression contains multiple expressions, but all of them are monotonically --- increasing in the source partition min max indexes. -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(A, B) ORDER BY tuple(); -CREATE TABLE destination (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(intDiv(A, 2), intDiv(B, 2)) ORDER BY tuple(); -INSERT INTO TABLE source VALUES (6, 12); -ALTER TABLE destination ATTACH PARTITION ID '6-12' FROM source; -SELECT * FROM source ORDER BY A; -6 12 -SELECT * FROM destination ORDER BY A; -6 12 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -3-6 -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION (6, 12) from source; -SELECT * FROM source ORDER BY A; -6 12 -SELECT * FROM destination ORDER BY A; -6 12 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -3-6 --- Should be allowed. The same scenario as above, but partition expressions inverted. -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(intDiv(A, 2), intDiv(B, 2)) ORDER BY tuple(); -CREATE TABLE destination (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(A, B) ORDER BY tuple(); -INSERT INTO TABLE source VALUES (6, 12); -ALTER TABLE destination ATTACH PARTITION ID '3-6' FROM source; -SELECT * FROM source ORDER BY A; -6 12 -SELECT * FROM destination ORDER BY A; -6 12 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -6-12 -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION (3, 6) from source; -SELECT * FROM source ORDER BY A; -6 12 -SELECT * FROM destination ORDER BY A; -6 12 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -6-12 --- Should be allowed, it is a local operation, no different than regular attach. Replicated to replicated. -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE - source(timestamp DateTime) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/source_replicated_to_replicated_distinct_expression', '1') - PARTITION BY toYYYYMMDD(timestamp) - ORDER BY tuple(); -CREATE TABLE - destination(timestamp DateTime) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/destination_replicated_to_replicated_distinct_expression', '1') - PARTITION BY toYYYYMM(timestamp) - ORDER BY tuple(); -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); -ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -201003 -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION '20100302' from source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -201003 --- Should be allowed, it is a local operation, no different than regular attach. Non replicated to replicated -DROP TABLE IF EXISTS source SYNC; -DROP TABLE IF EXISTS destination SYNC; -CREATE TABLE source(timestamp DateTime) ENGINE = MergeTree() PARTITION BY toYYYYMMDD(timestamp) ORDER BY tuple(); -CREATE TABLE - destination(timestamp DateTime) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/destination_non_replicated_to_replicated_distinct_expression', '1') - PARTITION BY toYYYYMM(timestamp) - ORDER BY tuple(); -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); -ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -201003 -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION '20100302' from source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -201003 --- Should not be allowed because data would be split into two different partitions -DROP TABLE IF EXISTS source SYNC; -DROP TABLE IF EXISTS destination SYNC; -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-03 02:01:03'); -ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; -- { serverError 248 } -ALTER TABLE destination ATTACH PARTITION '201003' from source; -- { serverError 248 } --- Should not be allowed because data would be split into two different partitions -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY intDiv(A, 6); -CREATE TABLE destination (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY A; -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01', 1), ('2010-03-02 02:01:03', 2); -ALTER TABLE destination ATTACH PARTITION ID '0' FROM source; -- { serverError 248 } -ALTER TABLE destination ATTACH PARTITION 0 FROM source; -- { serverError 248 } --- Should not be allowed because dst partition exp takes more than two arguments, so it's not considered monotonically inc -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(category); -CREATE TABLE destination (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY substring(category, 1, 2); -INSERT INTO TABLE source VALUES ('spaghetti', 'food'), ('mop', 'general'); -INSERT INTO TABLE source VALUES ('rice', 'food'); -ALTER TABLE destination ATTACH PARTITION ID '4590ba78048910b74a47d5bfb308abed' from source; -- { serverError 36 } -ALTER TABLE destination ATTACH PARTITION 'food' from source; -- { serverError 36 } --- Should not be allowed because dst partition exp depends on a different set of columns -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(category); -CREATE TABLE destination (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(productName); -INSERT INTO TABLE source VALUES ('spaghetti', 'food'), ('mop', 'general'); -INSERT INTO TABLE source VALUES ('rice', 'food'); -ALTER TABLE destination ATTACH PARTITION ID '4590ba78048910b74a47d5bfb308abed' from source; -- { serverError 36 } -ALTER TABLE destination ATTACH PARTITION 'food' from source; -- { serverError 36 } --- Should not be allowed because dst partition exp is not monotonically increasing -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (productName String) engine=MergeTree ORDER BY tuple() PARTITION BY left(productName, 2); -CREATE TABLE destination (productName String) engine=MergeTree ORDER BY tuple() PARTITION BY cityHash64(productName); -INSERT INTO TABLE source VALUES ('bread'), ('mop'); -INSERT INTO TABLE source VALUES ('broccoli'); -ALTER TABLE destination ATTACH PARTITION ID '4589453b7ee96ce9de1265bd57674496' from source; -- { serverError 36 } -ALTER TABLE destination ATTACH PARTITION 'br' from source; -- { serverError 36 } --- Empty/ non-existent partition, same partition expression. Nothing should happen -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -ALTER TABLE destination ATTACH PARTITION ID '1' FROM source; -ALTER TABLE destination ATTACH PARTITION 1 FROM source; -SELECT * FROM destination; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; --- Empty/ non-existent partition, different partition expression. Nothing should happen --- https://github.com/ClickHouse/ClickHouse/pull/39507#discussion_r1399839045 -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -ALTER TABLE destination ATTACH PARTITION ID '1' FROM source; -ALTER TABLE destination ATTACH PARTITION 1 FROM source; -SELECT * FROM destination; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; --- Replace instead of attach. Empty/ non-existent partition, same partition expression. Nothing should happen --- https://github.com/ClickHouse/ClickHouse/pull/39507#discussion_r1399839045 -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -ALTER TABLE destination REPLACE PARTITION '1' FROM source; -SELECT * FROM destination; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; --- Replace instead of attach. Empty/ non-existent partition to non-empty partition, same partition id. --- https://github.com/ClickHouse/ClickHouse/pull/39507#discussion_r1399839045 -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (A Int) engine=MergeTree ORDER BY tuple() PARTITION BY A; -CREATE TABLE destination (A Int) engine=MergeTree ORDER BY tuple() PARTITION BY A; -INSERT INTO TABLE destination VALUES (1); -ALTER TABLE destination REPLACE PARTITION '1' FROM source; -SELECT * FROM destination; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; diff --git a/tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.sql b/tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.sql deleted file mode 100644 index 9547d6ae249..00000000000 --- a/tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.sql +++ /dev/null @@ -1,485 +0,0 @@ --- { echoOn } --- Should be allowed since destination partition expr is monotonically increasing and compatible -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); - -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); - -ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION '20100302' FROM source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Should be allowed since destination partition expr is monotonically increasing and compatible. Note that even though --- the destination partition expression is more granular, the data would still fall in the same partition. Thus, it is valid -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); - -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); - -ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION '201003' FROM source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Should be allowed since destination partition expr is monotonically increasing and compatible for those specific values -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY intDiv(A, 6); - -CREATE TABLE destination (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY A; - -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01', 1), ('2010-03-02 02:01:03', 1); - -ALTER TABLE destination ATTACH PARTITION ID '0' FROM source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION 0 FROM source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Should be allowed because dst partition exp is monot inc and data is not split -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY cityHash64(category); -CREATE TABLE destination (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(category); - -INSERT INTO TABLE source VALUES ('spaghetti', 'food'), ('mop', 'general'); -INSERT INTO TABLE source VALUES ('rice', 'food'); - -ALTER TABLE destination ATTACH PARTITION ID '17908065610379824077' from source; - -SELECT * FROM source ORDER BY productName; -SELECT * FROM destination ORDER BY productName; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION '17908065610379824077' from source; - -SELECT * FROM source ORDER BY productName; -SELECT * FROM destination ORDER BY productName; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Should be allowed, extra test case to validate https://github.com/ClickHouse/ClickHouse/pull/39507#issuecomment-1747574133 - -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (timestamp Int64) engine=MergeTree ORDER BY (timestamp) PARTITION BY intDiv(timestamp, 86400000); -CREATE TABLE destination (timestamp Int64) engine=MergeTree ORDER BY (timestamp) PARTITION BY toYear(toDateTime(intDiv(timestamp, 1000))); - -INSERT INTO TABLE source VALUES (1267495261123); - -ALTER TABLE destination ATTACH PARTITION ID '14670' FROM source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION '14670' from source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Should be allowed, extra test case to validate https://github.com/ClickHouse/ClickHouse/pull/39507#issuecomment-1747511726 - -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (timestamp DateTime('UTC'), key Int64, f Float64) engine=MergeTree ORDER BY (key, timestamp) PARTITION BY toYear(timestamp); -CREATE TABLE destination (timestamp DateTime('UTC'), key Int64, f Float64) engine=MergeTree ORDER BY (key, timestamp) PARTITION BY (intDiv(toUInt32(timestamp),86400)); - -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01',1,1),('2010-03-02 02:01:01',1,1),('2011-02-02 02:01:03',1,1); - -ALTER TABLE destination ATTACH PARTITION ID '2010' FROM source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION '2010' from source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Should be allowed, partitioned table to unpartitioned. Since the destination is unpartitioned, parts would ultimately --- fall into the same partition. --- Destination partition by expression is omitted, which causes StorageMetadata::getPartitionKeyAST() to be nullptr. -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple(); - -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); - -ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION '201003' from source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Same as above, but destination partition by expression is explicitly defined. Test case required to validate that --- partition by tuple() is accepted. -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY tuple(); - -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); - -ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION '201003' from source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Should be allowed because the destination partition expression columns are a subset of the source partition expression columns --- Columns in this case refer to the expression elements, not to the actual table columns -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY (a, b, c); -CREATE TABLE destination (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY (a, b); - -INSERT INTO TABLE source VALUES (1, 2, 3), (1, 2, 4); - -ALTER TABLE destination ATTACH PARTITION ID '1-2-3' FROM source; - -SELECT * FROM source ORDER BY (a, b, c); -SELECT * FROM destination ORDER BY (a, b, c); -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION (1, 2, 3) from source; - -SELECT * FROM source ORDER BY (a, b, c); -SELECT * FROM destination ORDER BY (a, b, c); -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Should be allowed because the destination partition expression columns are a subset of the source partition expression columns --- Columns in this case refer to the expression elements, not to the actual table columns -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY (a, b, c); -CREATE TABLE destination (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY a; - -INSERT INTO TABLE source VALUES (1, 2, 3), (1, 2, 4); - -ALTER TABLE destination ATTACH PARTITION ID '1-2-3' FROM source; - -SELECT * FROM source ORDER BY (a, b, c); -SELECT * FROM destination ORDER BY (a, b, c); -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION (1, 2, 3) from source; - -SELECT * FROM source ORDER BY (a, b, c); -SELECT * FROM destination ORDER BY (a, b, c); -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Should be allowed. Special test case, tricky to explain. First column of source partition expression is --- timestamp, while first column of destination partition expression is `A`. One of the previous implementations --- would not match the columns, which could lead to `timestamp` min max being used to calculate monotonicity of `A`. -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (`timestamp` DateTime, `A` Int64) ENGINE = MergeTree PARTITION BY tuple(toYYYYMM(timestamp), intDiv(A, 6)) ORDER BY timestamp; -CREATE TABLE destination (`timestamp` DateTime, `A` Int64) ENGINE = MergeTree PARTITION BY A ORDER BY timestamp; - -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01', 5); - -ALTER TABLE destination ATTACH PARTITION ID '201003-0' FROM source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION (201003, 0) from source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Should be allowed. Destination partition expression contains multiple expressions, but all of them are monotonically --- increasing in the source partition min max indexes. -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(A, B) ORDER BY tuple(); -CREATE TABLE destination (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(intDiv(A, 2), intDiv(B, 2)) ORDER BY tuple(); - -INSERT INTO TABLE source VALUES (6, 12); - -ALTER TABLE destination ATTACH PARTITION ID '6-12' FROM source; - -SELECT * FROM source ORDER BY A; -SELECT * FROM destination ORDER BY A; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION (6, 12) from source; - -SELECT * FROM source ORDER BY A; -SELECT * FROM destination ORDER BY A; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Should be allowed. The same scenario as above, but partition expressions inverted. -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(intDiv(A, 2), intDiv(B, 2)) ORDER BY tuple(); -CREATE TABLE destination (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(A, B) ORDER BY tuple(); - -INSERT INTO TABLE source VALUES (6, 12); - -ALTER TABLE destination ATTACH PARTITION ID '3-6' FROM source; - -SELECT * FROM source ORDER BY A; -SELECT * FROM destination ORDER BY A; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION (3, 6) from source; - -SELECT * FROM source ORDER BY A; -SELECT * FROM destination ORDER BY A; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Should be allowed, it is a local operation, no different than regular attach. Replicated to replicated. -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE - source(timestamp DateTime) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/source_replicated_to_replicated_distinct_expression', '1') - PARTITION BY toYYYYMMDD(timestamp) - ORDER BY tuple(); - -CREATE TABLE - destination(timestamp DateTime) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/destination_replicated_to_replicated_distinct_expression', '1') - PARTITION BY toYYYYMM(timestamp) - ORDER BY tuple(); - -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); - -ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION '20100302' from source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Should be allowed, it is a local operation, no different than regular attach. Non replicated to replicated -DROP TABLE IF EXISTS source SYNC; -DROP TABLE IF EXISTS destination SYNC; -CREATE TABLE source(timestamp DateTime) ENGINE = MergeTree() PARTITION BY toYYYYMMDD(timestamp) ORDER BY tuple(); - -CREATE TABLE - destination(timestamp DateTime) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/destination_non_replicated_to_replicated_distinct_expression', '1') - PARTITION BY toYYYYMM(timestamp) - ORDER BY tuple(); - -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); - -ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION '20100302' from source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Should not be allowed because data would be split into two different partitions -DROP TABLE IF EXISTS source SYNC; -DROP TABLE IF EXISTS destination SYNC; - -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); - -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-03 02:01:03'); - -ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; -- { serverError 248 } -ALTER TABLE destination ATTACH PARTITION '201003' from source; -- { serverError 248 } - --- Should not be allowed because data would be split into two different partitions -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY intDiv(A, 6); - -CREATE TABLE destination (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY A; - -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01', 1), ('2010-03-02 02:01:03', 2); - -ALTER TABLE destination ATTACH PARTITION ID '0' FROM source; -- { serverError 248 } -ALTER TABLE destination ATTACH PARTITION 0 FROM source; -- { serverError 248 } - --- Should not be allowed because dst partition exp takes more than two arguments, so it's not considered monotonically inc -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(category); -CREATE TABLE destination (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY substring(category, 1, 2); - -INSERT INTO TABLE source VALUES ('spaghetti', 'food'), ('mop', 'general'); -INSERT INTO TABLE source VALUES ('rice', 'food'); - -ALTER TABLE destination ATTACH PARTITION ID '4590ba78048910b74a47d5bfb308abed' from source; -- { serverError 36 } -ALTER TABLE destination ATTACH PARTITION 'food' from source; -- { serverError 36 } - --- Should not be allowed because dst partition exp depends on a different set of columns -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(category); -CREATE TABLE destination (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(productName); - -INSERT INTO TABLE source VALUES ('spaghetti', 'food'), ('mop', 'general'); -INSERT INTO TABLE source VALUES ('rice', 'food'); - -ALTER TABLE destination ATTACH PARTITION ID '4590ba78048910b74a47d5bfb308abed' from source; -- { serverError 36 } -ALTER TABLE destination ATTACH PARTITION 'food' from source; -- { serverError 36 } - --- Should not be allowed because dst partition exp is not monotonically increasing -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (productName String) engine=MergeTree ORDER BY tuple() PARTITION BY left(productName, 2); -CREATE TABLE destination (productName String) engine=MergeTree ORDER BY tuple() PARTITION BY cityHash64(productName); - -INSERT INTO TABLE source VALUES ('bread'), ('mop'); -INSERT INTO TABLE source VALUES ('broccoli'); - -ALTER TABLE destination ATTACH PARTITION ID '4589453b7ee96ce9de1265bd57674496' from source; -- { serverError 36 } -ALTER TABLE destination ATTACH PARTITION 'br' from source; -- { serverError 36 } - --- Empty/ non-existent partition, same partition expression. Nothing should happen -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); - -ALTER TABLE destination ATTACH PARTITION ID '1' FROM source; -ALTER TABLE destination ATTACH PARTITION 1 FROM source; - -SELECT * FROM destination; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Empty/ non-existent partition, different partition expression. Nothing should happen --- https://github.com/ClickHouse/ClickHouse/pull/39507#discussion_r1399839045 -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); - -ALTER TABLE destination ATTACH PARTITION ID '1' FROM source; -ALTER TABLE destination ATTACH PARTITION 1 FROM source; - -SELECT * FROM destination; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Replace instead of attach. Empty/ non-existent partition, same partition expression. Nothing should happen --- https://github.com/ClickHouse/ClickHouse/pull/39507#discussion_r1399839045 -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); - -ALTER TABLE destination REPLACE PARTITION '1' FROM source; - -SELECT * FROM destination; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Replace instead of attach. Empty/ non-existent partition to non-empty partition, same partition id. --- https://github.com/ClickHouse/ClickHouse/pull/39507#discussion_r1399839045 -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (A Int) engine=MergeTree ORDER BY tuple() PARTITION BY A; -CREATE TABLE destination (A Int) engine=MergeTree ORDER BY tuple() PARTITION BY A; - -INSERT INTO TABLE destination VALUES (1); - -ALTER TABLE destination REPLACE PARTITION '1' FROM source; - -SELECT * FROM destination; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; From 5d78cf1a381e78354df7ac11eff7791cf1ca9a93 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Tue, 23 Jan 2024 10:31:32 -0500 Subject: [PATCH 153/264] ipv6 bloom filter indexes incompatibility --- docs/changelogs/v23.9.1.1854-stable.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/changelogs/v23.9.1.1854-stable.md b/docs/changelogs/v23.9.1.1854-stable.md index 655dd54d81b..aa27cd34478 100644 --- a/docs/changelogs/v23.9.1.1854-stable.md +++ b/docs/changelogs/v23.9.1.1854-stable.md @@ -11,6 +11,7 @@ sidebar_label: 2023 * Remove the `status_info` configuration option and dictionaries status from the default Prometheus handler. [#54090](https://github.com/ClickHouse/ClickHouse/pull/54090) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * The experimental parts metadata cache is removed from the codebase. [#54215](https://github.com/ClickHouse/ClickHouse/pull/54215) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Disable setting `input_format_json_try_infer_numbers_from_strings` by default, so we don't try to infer numbers from strings in JSON formats by default to avoid possible parsing errors when sample data contains strings that looks like a number. [#55099](https://github.com/ClickHouse/ClickHouse/pull/55099) ([Kruglov Pavel](https://github.com/Avogar)). +* IPv6 bloom filter indexes created prior to March 2023 are not compatible with current version and have to be rebuilt. [#54200](https://github.com/ClickHouse/ClickHouse/pull/54200) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). #### New Feature * Added new type of authentication based on SSH keys. It works only for Native TCP protocol. [#41109](https://github.com/ClickHouse/ClickHouse/pull/41109) ([George Gamezardashvili](https://github.com/InfJoker)). From d64b491b45d0fcf171bdef9a8d6620b12f11a2d2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 23 Jan 2024 17:00:21 +0000 Subject: [PATCH 154/264] Fixing build --- src/Storages/StorageMergeTree.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 417e709b3f9..fbdde15c2af 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -29,6 +29,8 @@ #include #include #include +#include +#include #include #include #include From 94a79c081da87ffedc31841cc5b45083693f4323 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Tue, 23 Jan 2024 09:21:17 -0800 Subject: [PATCH 155/264] Fix aggregation issue in mixed cluster of x86_64 and aarch64 --- src/Common/HashTable/StringHashTable.h | 22 ++++++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/src/Common/HashTable/StringHashTable.h b/src/Common/HashTable/StringHashTable.h index bfa369ef8e4..f3ba54e7d6e 100644 --- a/src/Common/HashTable/StringHashTable.h +++ b/src/Common/HashTable/StringHashTable.h @@ -71,6 +71,28 @@ struct StringHashTableHash res = _mm_crc32_u64(res, key.c); return res; } +#elif defined(__aarch64__) && defined(__ARM_FEATURE_CRC32) + size_t ALWAYS_INLINE operator()(StringKey8 key) const + { + size_t res = -1ULL; + res = __crc32cd(static_cast(res), key); + return res; + } + size_t ALWAYS_INLINE operator()(StringKey16 key) const + { + size_t res = -1ULL; + res = __crc32cd(static_cast(res), key.items[0]); + res = __crc32cd(static_cast(res), key.items[1]); + return res; + } + size_t ALWAYS_INLINE operator()(StringKey24 key) const + { + size_t res = -1ULL; + res = __crc32cd(static_cast(res), key.a); + res = __crc32cd(static_cast(res), key.b); + res = __crc32cd(static_cast(res), key.c); + return res; + } #elif defined(__s390x__) && __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ size_t ALWAYS_INLINE operator()(StringKey8 key) const { From be3b5dc45fdfd76585cb752b8a34fae9e34d7451 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 23 Jan 2024 17:45:41 +0000 Subject: [PATCH 156/264] Allow to ignore schema evolution in Iceberg table engine under a setting --- docs/en/operations/settings/settings.md | 4 ++ src/Core/Settings.h | 3 +- src/Core/SettingsChangesHistory.h | 3 +- .../DataLakes/Iceberg/IcebergMetadata.cpp | 57 +++++++++++++++---- .../integration/test_storage_iceberg/test.py | 7 +++ 5 files changed, 60 insertions(+), 14 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index f085fe1abcd..6444b76ba0e 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5197,3 +5197,7 @@ The value 0 means that you can delete all tables without any restrictions. :::note This query setting overwrites its server setting equivalent, see [max_table_size_to_drop](/docs/en/operations/server-configuration-parameters/settings.md/#max-table-size-to-drop) ::: + +## iceberg_engine_ignore_schema_evolution {#iceberg_engine_ignore_schema_evolution} + +Allow to ignore schema evolution in Iceberg table engine and read all data using latest schema saved on storage creation. \ No newline at end of file diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 305d6466658..77a8c0ed766 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -856,7 +856,8 @@ class IColumn; M(UInt64, cache_warmer_threads, 4, "Only available in ClickHouse Cloud", 0) \ M(Int64, ignore_cold_parts_seconds, 0, "Only available in ClickHouse Cloud", 0) \ M(Int64, prefer_warmed_unmerged_parts_seconds, 0, "Only available in ClickHouse Cloud", 0) \ - M(Bool, enable_order_by_all, true, "Enable sorting expression ORDER BY ALL.", 0)\ + M(Bool, enable_order_by_all, true, "Enable sorting expression ORDER BY ALL.", 0) \ + M(Bool, iceberg_engine_ignore_schema_evolution, false, "Ignore schema evolution in Iceberg table engine and read all data using latest schema saved on table creation. Note that it can lead to incorrect result", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS, move obsolete settings to OBSOLETE_SETTINGS and obsolete format settings to OBSOLETE_FORMAT_SETTINGS. diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index ef37aa5fb47..859ba99b5f7 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -98,7 +98,8 @@ static std::map sett {"max_rows_in_set_to_optimize_join", 100000, 0, "Disable join optimization as it prevents from read in order optimization"}, {"output_format_pretty_color", true, "auto", "Setting is changed to allow also for auto value, disabling ANSI escapes if output is not a tty"}, {"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"}, - {"max_estimated_execution_time", 0, 0, "Separate max_execution_time and max_estimated_execution_time"}}}, + {"max_estimated_execution_time", 0, 0, "Separate max_execution_time and max_estimated_execution_time"}, + {"iceberg_engine_ignore_schema_evolution", false, false, "Allow to ignore schema evolution in Iceberg table engine"}}}, {"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}, {"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, diff --git a/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp b/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp index b4ac00507a3..e0c7e26a2e1 100644 --- a/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp +++ b/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp @@ -240,7 +240,7 @@ DataTypePtr getFieldType(const Poco::JSON::Object::Ptr & field, const String & t } -std::pair parseTableSchema(const Poco::JSON::Object::Ptr & metadata_object, int format_version) +std::pair parseTableSchema(const Poco::JSON::Object::Ptr & metadata_object, int format_version, bool ignore_schema_evolution) { Poco::JSON::Object::Ptr schema; Int32 current_schema_id; @@ -253,13 +253,39 @@ std::pair parseTableSchema(const Poco::JSON::Object::P { current_schema_id = metadata_object->getValue("current-schema-id"); auto schemas = metadata_object->get("schemas").extract(); - if (schemas->size() != 1) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot read Iceberg table: the table schema has been changed at least 1 time, reading tables with evolved schema is not supported"); + if (schemas->size() == 0) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse Iceberg table schema: schemas field is empty"); - /// Now we sure that there is only one schema. - schema = schemas->getObject(0); - if (schema->getValue("schema-id") != current_schema_id) - throw Exception(ErrorCodes::BAD_ARGUMENTS, R"(Field "schema-id" of the schema doesn't match "current-schema-id" in metadata)"); + if (ignore_schema_evolution) + { + /// If we ignore schema evolution, we will just use latest schema for all data files. + /// Find schema with 'schema-id' equal to 'current_schema_id'. + for (uint32_t i = 0; i != schemas->size(); ++i) + { + auto current_schema = schemas->getObject(i); + if (current_schema->getValue("schema-id") == current_schema_id) + { + schema = current_schema; + break; + } + } + + if (!schema) + throw Exception(ErrorCodes::BAD_ARGUMENTS, R"(There is no schema with "schema-id" that matches "current-schema-id" in metadata)"); + } + else + { + if (schemas->size() != 1) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Cannot read Iceberg table: the table schema has been changed at least 1 time, reading tables with evolved schema is " + "supported. If you want to ignore schema evolution and read all files using latest schema saved on table creation, enable setting " + "iceberg_engine_ignore_schema_evolution (Note: enabling this setting can lead to incorrect result)"); + + /// Now we sure that there is only one schema. + schema = schemas->getObject(0); + if (schema->getValue("schema-id") != current_schema_id) + throw Exception(ErrorCodes::BAD_ARGUMENTS, R"(Field "schema-id" of the schema doesn't match "current-schema-id" in metadata)"); + } } else { @@ -267,8 +293,11 @@ std::pair parseTableSchema(const Poco::JSON::Object::P current_schema_id = schema->getValue("schema-id"); /// Field "schemas" is optional for version 1, but after version 2 was introduced, /// in most cases this field is added for new tables in version 1 as well. - if (metadata_object->has("schemas") && metadata_object->get("schemas").extract()->size() > 1) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot read Iceberg table: the table schema has been changed at least 1 time, reading tables with evolved schema is not supported"); + if (!ignore_schema_evolution && metadata_object->has("schemas") && metadata_object->get("schemas").extract()->size() > 1) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Cannot read Iceberg table: the table schema has been changed at least 1 time, reading tables with evolved schema is not " + "supported. If you want to ignore schema evolution and read all files using latest schema saved on table creation, enable setting " + "iceberg_engine_ignore_schema_evolution (Note: enabling this setting can lead to incorrect result)"); } NamesAndTypesList names_and_types; @@ -356,7 +385,7 @@ std::unique_ptr parseIcebergMetadata(const StorageS3::Configura Poco::JSON::Object::Ptr object = json.extract(); auto format_version = object->getValue("format-version"); - auto [schema, schema_id] = parseTableSchema(object, format_version); + auto [schema, schema_id] = parseTableSchema(object, format_version, context_->getSettingsRef().iceberg_engine_ignore_schema_evolution); auto current_snapshot_id = object->getValue("current-snapshot-id"); auto snapshots = object->get("snapshots").extract(); @@ -453,8 +482,12 @@ Strings IcebergMetadata::getDataFiles() Poco::JSON::Parser parser; Poco::Dynamic::Var json = parser.parse(schema_json_string); Poco::JSON::Object::Ptr schema_object = json.extract(); - if (schema_object->getValue("schema-id") != current_schema_id) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot read Iceberg table: the table schema has been changed at least 1 time, reading tables with evolved schema is not supported"); + if (!getContext()->getSettingsRef().iceberg_engine_ignore_schema_evolution && schema_object->getValue("schema-id") != current_schema_id) + throw Exception( + ErrorCodes::UNSUPPORTED_METHOD, + "Cannot read Iceberg table: the table schema has been changed at least 1 time, reading tables with evolved schema is not " + "supported. If you want to ignore schema evolution and read all files using latest schema saved on table creation, enable setting " + "iceberg_engine_ignore_schema_evolution (Note: enabling this setting can lead to incorrect result)"); avro::NodePtr root_node = manifest_file_reader->dataSchema().root(); size_t leaves_num = root_node->leaves(); diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index 9a75dc50d61..30962dc619c 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -399,6 +399,8 @@ def test_evolved_schema(started_cluster, format_version): assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 + expected_data = instance.query(f"SELECT * FROM {TABLE_NAME} order by a, b") + spark.sql(f"ALTER TABLE {TABLE_NAME} ADD COLUMNS (x bigint)") files = upload_directory( minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" @@ -407,6 +409,11 @@ def test_evolved_schema(started_cluster, format_version): error = instance.query_and_get_error(f"SELECT * FROM {TABLE_NAME}") assert "UNSUPPORTED_METHOD" in error + data = instance.query( + f"SELECT * FROM {TABLE_NAME} SETTINGS iceberg_engine_ignore_schema_evolution=1" + ) + assert data == expected_data + def test_row_based_deletes(started_cluster): instance = started_cluster.instances["node1"] From 1fd4b26c3c3429528e8b21479654216bee98a75d Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 23 Jan 2024 17:48:50 +0000 Subject: [PATCH 157/264] Better docs --- docs/en/operations/settings/settings.md | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 6444b76ba0e..75d05d55366 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5200,4 +5200,10 @@ This query setting overwrites its server setting equivalent, see [max_table_size ## iceberg_engine_ignore_schema_evolution {#iceberg_engine_ignore_schema_evolution} -Allow to ignore schema evolution in Iceberg table engine and read all data using latest schema saved on storage creation. \ No newline at end of file +Allow to ignore schema evolution in Iceberg table engine and read all data using schema specified by the user on table creation or latest schema parsed from metadata on table creation. + +:::note +Enabling this setting can lead to incorrect result as in case of evolved schema all data files will be read using the same schema. +::: + +Default value: 'false'. \ No newline at end of file From 8728ab768fbf85f946e9765cafab07d9a11c3c08 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 23 Jan 2024 19:01:57 +0000 Subject: [PATCH 158/264] Fix not started database shutdown deadlock --- programs/server/Server.cpp | 1 - src/Databases/DatabaseAtomic.cpp | 32 +++++++++++--- src/Databases/DatabaseAtomic.h | 5 ++- src/Databases/DatabaseOnDisk.cpp | 12 +++--- src/Databases/DatabaseOrdinary.cpp | 33 ++++++++++++-- src/Databases/DatabaseOrdinary.h | 5 ++- src/Databases/DatabaseReplicated.cpp | 43 +++++++++++++------ src/Databases/DatabaseReplicated.h | 5 ++- src/Databases/IDatabase.h | 7 ++- .../MySQL/DatabaseMaterializedMySQL.cpp | 25 +++++++++-- .../MySQL/DatabaseMaterializedMySQL.h | 5 ++- .../DatabaseMaterializedPostgreSQL.cpp | 25 +++++++++-- .../DatabaseMaterializedPostgreSQL.h | 5 ++- 13 files changed, 152 insertions(+), 51 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 96f3119e073..4295a989248 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1748,7 +1748,6 @@ try LOG_INFO(log, "Stopping AsyncLoader."); // Waits for all currently running jobs to finish and do not run any other pending jobs. - // Pending jobs will be canceled and destructed later by `load_metadata_tasks` dtor. global_context->getAsyncLoader().stop(); ); diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 8a5ba5f033f..94ae00be634 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -9,6 +9,7 @@ #include #include #include +#include "Databases/DatabaseOrdinary.h" #include #include #include @@ -76,7 +77,7 @@ String DatabaseAtomic::getTableDataPath(const ASTCreateQuery & query) const void DatabaseAtomic::drop(ContextPtr) { - waitDatabaseStarted(false); + waitDatabaseStarted(); assert(TSA_SUPPRESS_WARNING_FOR_READ(tables).empty()); try { @@ -115,7 +116,7 @@ StoragePtr DatabaseAtomic::detachTable(ContextPtr /* context */, const String & void DatabaseAtomic::dropTable(ContextPtr local_context, const String & table_name, bool sync) { - waitDatabaseStarted(false); + waitDatabaseStarted(); auto table = tryGetTable(table_name, local_context); /// Remove the inner table (if any) to avoid deadlock /// (due to attempt to execute DROP from the worker thread) @@ -179,7 +180,7 @@ void DatabaseAtomic::renameTable(ContextPtr local_context, const String & table_ if (exchange && !supportsAtomicRename()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "RENAME EXCHANGE is not supported"); - waitDatabaseStarted(false); + waitDatabaseStarted(); auto & other_db = dynamic_cast(to_database); bool inside_database = this == &other_db; @@ -468,13 +469,30 @@ LoadTaskPtr DatabaseAtomic::startupDatabaseAsync(AsyncLoader & async_loader, Loa for (const auto & table : table_names) tryCreateSymlink(table.first, table.second, true); }); + std::scoped_lock lock(mutex); return startup_atomic_database_task = makeLoadTask(async_loader, {job}); } -void DatabaseAtomic::waitDatabaseStarted(bool no_throw) const +void DatabaseAtomic::waitDatabaseStarted() const { - if (startup_atomic_database_task) - waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), startup_atomic_database_task, no_throw); + LoadTaskPtr task; + { + std::scoped_lock lock(mutex); + task = startup_atomic_database_task; + } + if (task) + waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), task, false); +} + +void DatabaseAtomic::stopLoading() +{ + LoadTaskPtr stop_atomic_database; + { + std::scoped_lock lock(mutex); + stop_atomic_database.swap(startup_atomic_database_task); + } + stop_atomic_database.reset(); + DatabaseOrdinary::stopLoading(); } void DatabaseAtomic::tryCreateSymlink(const String & table_name, const String & actual_data_path, bool if_data_path_exist) @@ -544,7 +562,7 @@ void DatabaseAtomic::renameDatabase(ContextPtr query_context, const String & new { /// CREATE, ATTACH, DROP, DETACH and RENAME DATABASE must hold DDLGuard - waitDatabaseStarted(false); + waitDatabaseStarted(); bool check_ref_deps = query_context->getSettingsRef().check_referential_table_dependencies; bool check_loading_deps = !check_ref_deps && query_context->getSettingsRef().check_table_dependencies; diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index 83cb51be1ff..404478f7cd1 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -51,7 +51,8 @@ public: void beforeLoadingMetadata(ContextMutablePtr context, LoadingStrictnessLevel mode) override; LoadTaskPtr startupDatabaseAsync(AsyncLoader & async_loader, LoadJobSet startup_after, LoadingStrictnessLevel mode) override; - void waitDatabaseStarted(bool no_throw) const override; + void waitDatabaseStarted() const override; + void stopLoading() override; /// Atomic database cannot be detached if there is detached table which still in use void assertCanBeDetached(bool cleanup) override; @@ -87,7 +88,7 @@ protected: String path_to_metadata_symlink; const UUID db_uuid; - LoadTaskPtr startup_atomic_database_task; + LoadTaskPtr startup_atomic_database_task TSA_GUARDED_BY(mutex); }; } diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 12b0dc07799..91ee717ef34 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -165,7 +165,7 @@ DatabaseOnDisk::DatabaseOnDisk( void DatabaseOnDisk::shutdown() { - waitDatabaseStarted(/* no_throw = */ true); + stopLoading(); DatabaseWithOwnTablesBase::shutdown(); } @@ -196,7 +196,7 @@ void DatabaseOnDisk::createTable( throw Exception( ErrorCodes::TABLE_ALREADY_EXISTS, "Table {}.{} already exists", backQuote(getDatabaseName()), backQuote(table_name)); - waitDatabaseStarted(false); + waitDatabaseStarted(); String table_metadata_path = getObjectMetadataPath(table_name); @@ -287,7 +287,7 @@ void DatabaseOnDisk::commitCreateTable(const ASTCreateQuery & query, const Stora void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const String & table_name) { - waitDatabaseStarted(false); + waitDatabaseStarted(); auto table = detachTable(query_context, table_name); @@ -305,7 +305,7 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const Stri void DatabaseOnDisk::dropTable(ContextPtr local_context, const String & table_name, bool /*sync*/) { - waitDatabaseStarted(false); + waitDatabaseStarted(); String table_metadata_path = getObjectMetadataPath(table_name); String table_metadata_path_drop = table_metadata_path + drop_suffix; @@ -391,7 +391,7 @@ void DatabaseOnDisk::renameTable( throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Moving tables between databases of different engines is not supported"); } - waitDatabaseStarted(false); + waitDatabaseStarted(); auto table_data_relative_path = getTableDataPath(table_name); TableExclusiveLockHolder table_lock; @@ -534,7 +534,7 @@ ASTPtr DatabaseOnDisk::getCreateDatabaseQuery() const void DatabaseOnDisk::drop(ContextPtr local_context) { - waitDatabaseStarted(false); + waitDatabaseStarted(); assert(TSA_SUPPRESS_WARNING_FOR_READ(tables).empty()); if (local_context->getSettingsRef().force_remove_data_recursively_on_drop) diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index ba1b2cdacad..644bed23350 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -238,6 +238,7 @@ LoadTaskPtr DatabaseOrdinary::startupDatabaseAsync( // 1) startup should be done after tables loading // 2) load or startup errors for tables should not lead to not starting up the whole database }); + std::scoped_lock lock(mutex); return startup_database_task = makeLoadTask(async_loader, {job}); } @@ -255,11 +256,35 @@ void DatabaseOrdinary::waitTableStarted(const String & name) const waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), task); } -void DatabaseOrdinary::waitDatabaseStarted(bool no_throw) const +void DatabaseOrdinary::waitDatabaseStarted() const { /// Prioritize load and startup of all tables and database itself and wait for them synchronously - if (startup_database_task) - waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), startup_database_task, no_throw); + LoadTaskPtr task; + { + std::scoped_lock lock(mutex); + task = startup_database_task; + } + if (task) + waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), task); +} + +void DatabaseOrdinary::stopLoading() +{ + std::unordered_map stop_load_table; + std::unordered_map stop_startup_table; + LoadTaskPtr stop_startup_database; + { + std::scoped_lock lock(mutex); + stop_load_table.swap(load_table); + stop_startup_table.swap(startup_table); + stop_startup_database.swap(startup_database_task); + } + + // Cancel pending tasks and wait for currently running tasks + // Note that order must be backward of how it was created to make sure no dependent task is run after waiting for current task + stop_startup_database.reset(); + stop_startup_table.clear(); + stop_load_table.clear(); } DatabaseTablesIteratorPtr DatabaseOrdinary::getTablesIterator(ContextPtr local_context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name) const @@ -272,7 +297,7 @@ DatabaseTablesIteratorPtr DatabaseOrdinary::getTablesIterator(ContextPtr local_c void DatabaseOrdinary::alterTable(ContextPtr local_context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) { - waitDatabaseStarted(false); + waitDatabaseStarted(); String table_name = table_id.table_name; diff --git a/src/Databases/DatabaseOrdinary.h b/src/Databases/DatabaseOrdinary.h index d1d7dfd83fa..bd169155fa6 100644 --- a/src/Databases/DatabaseOrdinary.h +++ b/src/Databases/DatabaseOrdinary.h @@ -51,7 +51,8 @@ public: void waitTableStarted(const String & name) const override; - void waitDatabaseStarted(bool no_throw) const override; + void waitDatabaseStarted() const override; + void stopLoading() override; LoadTaskPtr startupDatabaseAsync(AsyncLoader & async_loader, LoadJobSet startup_after, LoadingStrictnessLevel mode) override; @@ -76,7 +77,7 @@ protected: std::unordered_map load_table TSA_GUARDED_BY(mutex); std::unordered_map startup_table TSA_GUARDED_BY(mutex); - LoadTaskPtr startup_database_task; + LoadTaskPtr startup_database_task TSA_GUARDED_BY(mutex); std::atomic total_tables_to_startup{0}; std::atomic tables_started{0}; AtomicStopwatch startup_watch; diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index d484b223706..9f75bcb5529 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -576,13 +576,30 @@ LoadTaskPtr DatabaseReplicated::startupDatabaseAsync(AsyncLoader & async_loader, ddl_worker->startup(); ddl_worker_initialized = true; }); + std::scoped_lock lock(mutex); return startup_replicated_database_task = makeLoadTask(async_loader, {job}); } -void DatabaseReplicated::waitDatabaseStarted(bool no_throw) const +void DatabaseReplicated::waitDatabaseStarted() const { - if (startup_replicated_database_task) - waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), startup_replicated_database_task, no_throw); + LoadTaskPtr task; + { + std::scoped_lock lock(mutex); + task = startup_replicated_database_task; + } + if (task) + waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), task); +} + +void DatabaseReplicated::stopLoading() +{ + LoadTaskPtr stop_startup_replicated_database; + { + std::scoped_lock lock(mutex); + stop_startup_replicated_database.swap(startup_replicated_database_task); + } + stop_startup_replicated_database.reset(); + DatabaseAtomic::stopLoading(); } bool DatabaseReplicated::checkDigestValid(const ContextPtr & local_context, bool debug_check /* = true */) const @@ -743,7 +760,7 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, ContextPtr query_ BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, ContextPtr query_context, QueryFlags flags) { - waitDatabaseStarted(false); + waitDatabaseStarted(); if (query_context->getCurrentTransaction() && query_context->getSettingsRef().throw_on_unsupported_query_inside_transaction) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Distributed DDL queries inside transactions are not supported"); @@ -807,7 +824,7 @@ static UUID getTableUUIDIfReplicated(const String & metadata, ContextPtr context void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeeper, UInt32 our_log_ptr, UInt32 & max_log_ptr) { - waitDatabaseStarted(false); + waitDatabaseStarted(); is_recovering = true; SCOPE_EXIT({ is_recovering = false; }); @@ -1258,7 +1275,7 @@ void DatabaseReplicated::drop(ContextPtr context_) return; } - waitDatabaseStarted(false); + waitDatabaseStarted(); auto current_zookeeper = getZooKeeper(); current_zookeeper->set(replica_path, DROPPED_MARK, -1); @@ -1277,7 +1294,7 @@ void DatabaseReplicated::drop(ContextPtr context_) void DatabaseReplicated::stopReplication() { - waitDatabaseStarted(/* no_throw = */ true); + stopLoading(); if (ddl_worker) ddl_worker->shutdown(); } @@ -1293,7 +1310,7 @@ void DatabaseReplicated::shutdown() void DatabaseReplicated::dropTable(ContextPtr local_context, const String & table_name, bool sync) { - waitDatabaseStarted(false); + waitDatabaseStarted(); auto txn = local_context->getZooKeeperMetadataTransaction(); assert(!ddl_worker || !ddl_worker->isCurrentlyActive() || txn || startsWith(table_name, ".inner_id.")); @@ -1337,7 +1354,7 @@ void DatabaseReplicated::renameTable(ContextPtr local_context, const String & ta if (exchange && !to_database.isTableExist(to_table_name, local_context)) throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} does not exist", to_table_name); - waitDatabaseStarted(false); + waitDatabaseStarted(); String statement = readMetadataFile(table_name); String statement_to; @@ -1439,7 +1456,7 @@ bool DatabaseReplicated::canExecuteReplicatedMetadataAlter() const void DatabaseReplicated::detachTablePermanently(ContextPtr local_context, const String & table_name) { - waitDatabaseStarted(false); + waitDatabaseStarted(); auto txn = local_context->getZooKeeperMetadataTransaction(); assert(!ddl_worker->isCurrentlyActive() || txn); @@ -1464,7 +1481,7 @@ void DatabaseReplicated::detachTablePermanently(ContextPtr local_context, const void DatabaseReplicated::removeDetachedPermanentlyFlag(ContextPtr local_context, const String & table_name, const String & table_metadata_path, bool attach) { - waitDatabaseStarted(false); + waitDatabaseStarted(); auto txn = local_context->getZooKeeperMetadataTransaction(); assert(!ddl_worker->isCurrentlyActive() || txn); @@ -1502,7 +1519,7 @@ String DatabaseReplicated::readMetadataFile(const String & table_name) const std::vector> DatabaseReplicated::getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr &) const { - waitDatabaseStarted(false); + waitDatabaseStarted(); /// Here we read metadata from ZooKeeper. We could do that by simple call of DatabaseAtomic::getTablesForBackup() however /// reading from ZooKeeper is better because thus we won't be dependent on how fast the replication queue of this database is. @@ -1545,7 +1562,7 @@ void DatabaseReplicated::createTableRestoredFromBackup( std::shared_ptr restore_coordination, UInt64 timeout_ms) { - waitDatabaseStarted(false); + waitDatabaseStarted(); /// Because of the replication multiple nodes can try to restore the same tables again and failed with "Table already exists" /// because of some table could be restored already on other node and then replicated to this node. diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 8a3999e70e9..55bcf963d37 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -126,7 +126,8 @@ private: UInt64 getMetadataHash(const String & table_name) const; bool checkDigestValid(const ContextPtr & local_context, bool debug_check = true) const TSA_REQUIRES(metadata_mutex); - void waitDatabaseStarted(bool no_throw) const override; + void waitDatabaseStarted() const override; + void stopLoading() override; String zookeeper_path; String shard_name; @@ -155,7 +156,7 @@ private: mutable ClusterPtr cluster; - LoadTaskPtr startup_replicated_database_task; + LoadTaskPtr startup_replicated_database_task TSA_GUARDED_BY(mutex); }; } diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 15e453371b7..2f448cd7036 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -219,8 +219,11 @@ public: virtual void waitTableStarted(const String & /*name*/) const {} /// Waits for the database to be started up, i.e. task returned by `startupDatabaseAsync()` is done - /// NOTE: `no_throw` wait should be used during shutdown to (1) prevent race with startup and (2) avoid exceptions if startup failed - virtual void waitDatabaseStarted(bool /*no_throw*/) const {} + virtual void waitDatabaseStarted() const {} + + /// Cancels all load and startup tasks and waits for currently running tasks to finish. + /// Should be used during shutdown to (1) prevent race with startup, (2) stop any not yet started task and (3) avoid exceptions if startup failed + virtual void stopLoading() {} /// Check the existence of the table in memory (attached). virtual bool isTableExist(const String & name, ContextPtr context) const = 0; diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp index 99dd337189c..0f0d73ae16f 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp @@ -90,13 +90,30 @@ LoadTaskPtr DatabaseMaterializedMySQL::startupDatabaseAsync(AsyncLoader & async_ materialize_thread.startSynchronization(); started_up = true; }); + std::scoped_lock lock(mutex); return startup_mysql_database_task = makeLoadTask(async_loader, {job}); } -void DatabaseMaterializedMySQL::waitDatabaseStarted(bool no_throw) const +void DatabaseMaterializedMySQL::waitDatabaseStarted() const { - if (startup_mysql_database_task) - waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), startup_mysql_database_task, no_throw); + LoadTaskPtr task; + { + std::scoped_lock lock(mutex); + task = startup_mysql_database_task; + } + if (task) + waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), task); +} + +void DatabaseMaterializedMySQL::stopLoading() +{ + LoadTaskPtr stop_startup_mysql_database; + { + std::scoped_lock lock(mutex); + stop_startup_mysql_database.swap(startup_mysql_database_task); + } + stop_startup_mysql_database.reset(); + DatabaseAtomic::stopLoading(); } void DatabaseMaterializedMySQL::createTable(ContextPtr context_, const String & name, const StoragePtr & table, const ASTPtr & query) @@ -184,7 +201,7 @@ void DatabaseMaterializedMySQL::checkIsInternalQuery(ContextPtr context_, const void DatabaseMaterializedMySQL::stopReplication() { - waitDatabaseStarted(/* no_throw = */ true); + stopLoading(); materialize_thread.stopSynchronization(); started_up = false; } diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.h b/src/Databases/MySQL/DatabaseMaterializedMySQL.h index 4d7871d49d6..d2976adcadb 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.h +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.h @@ -48,13 +48,14 @@ protected: std::atomic_bool started_up{false}; - LoadTaskPtr startup_mysql_database_task; + LoadTaskPtr startup_mysql_database_task TSA_GUARDED_BY(mutex); public: String getEngineName() const override { return "MaterializedMySQL"; } LoadTaskPtr startupDatabaseAsync(AsyncLoader & async_loader, LoadJobSet startup_after, LoadingStrictnessLevel mode) override; - void waitDatabaseStarted(bool no_throw) const override; + void waitDatabaseStarted() const override; + void stopLoading() override; void createTable(ContextPtr context_, const String & name, const StoragePtr & table, const ASTPtr & query) override; diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index 37db70eb9b7..b44bc136b1f 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -152,13 +152,30 @@ LoadTaskPtr DatabaseMaterializedPostgreSQL::startupDatabaseAsync(AsyncLoader & a { startup_task->activateAndSchedule(); }); + std::scoped_lock lock(mutex); return startup_postgresql_database_task = makeLoadTask(async_loader, {job}); } -void DatabaseMaterializedPostgreSQL::waitDatabaseStarted(bool no_throw) const +void DatabaseMaterializedPostgreSQL::waitDatabaseStarted() const { - if (startup_postgresql_database_task) - waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), startup_postgresql_database_task, no_throw); + LoadTaskPtr task; + { + std::scoped_lock lock(mutex); + task = startup_postgresql_database_task; + } + if (task) + waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), task); +} + +void DatabaseMaterializedPostgreSQL::stopLoading() +{ + LoadTaskPtr stop_startup_postgresql_database; + { + std::scoped_lock lock(mutex); + stop_startup_postgresql_database.swap(startup_postgresql_database_task); + } + stop_startup_postgresql_database.reset(); + DatabaseAtomic::stopLoading(); } void DatabaseMaterializedPostgreSQL::applySettingsChanges(const SettingsChanges & settings_changes, ContextPtr query_context) @@ -438,7 +455,7 @@ void DatabaseMaterializedPostgreSQL::shutdown() void DatabaseMaterializedPostgreSQL::stopReplication() { - waitDatabaseStarted(/* no_throw = */ true); + stopLoading(); std::lock_guard lock(handler_mutex); if (replication_handler) diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h index 8feb5a014e1..dfa53fa61d7 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h @@ -41,7 +41,8 @@ public: String getMetadataPath() const override { return metadata_path; } LoadTaskPtr startupDatabaseAsync(AsyncLoader & async_loader, LoadJobSet startup_after, LoadingStrictnessLevel mode) override; - void waitDatabaseStarted(bool no_throw) const override; + void waitDatabaseStarted() const override; + void stopLoading() override; DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name) const override; @@ -94,7 +95,7 @@ private: BackgroundSchedulePool::TaskHolder startup_task; bool shutdown_called = false; - LoadTaskPtr startup_postgresql_database_task; + LoadTaskPtr startup_postgresql_database_task TSA_GUARDED_BY(mutex); }; } From c36090ec99b2d33869591f8a21bc29b49b5239d8 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 23 Jan 2024 19:14:49 +0000 Subject: [PATCH 159/264] cleanup --- src/Databases/DatabaseAtomic.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 94ae00be634..0bf7c8af4b4 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -9,7 +9,6 @@ #include #include #include -#include "Databases/DatabaseOrdinary.h" #include #include #include From c8adf35298499838e5a4b1004879b7e5c205f176 Mon Sep 17 00:00:00 2001 From: Nikolay Edigaryev Date: Wed, 24 Jan 2024 02:34:17 +0400 Subject: [PATCH 160/264] =?UTF-8?q?DisksApp.cpp:=20fix=20typo=20(specifige?= =?UTF-8?q?d=20=E2=86=92=20specified)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- programs/disks/DisksApp.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/disks/DisksApp.cpp b/programs/disks/DisksApp.cpp index ded324fd0da..4b3b83238a0 100644 --- a/programs/disks/DisksApp.cpp +++ b/programs/disks/DisksApp.cpp @@ -160,7 +160,7 @@ int DisksApp::main(const std::vector & /*args*/) } else { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "No config-file specifiged"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "No config-file specified"); } if (config().has("save-logs")) From 95829e074eb208ab84e6065c5907d042990ef532 Mon Sep 17 00:00:00 2001 From: skyoct Date: Wed, 24 Jan 2024 03:49:21 +0000 Subject: [PATCH 161/264] Opt code --- src/IO/ReadWriteBufferFromHTTP.cpp | 12 +++--------- 1 file changed, 3 insertions(+), 9 deletions(-) diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index ea18b369c52..fccbf13a519 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -213,10 +213,7 @@ void ReadWriteBufferFromHTTPBase::getHeadResponse(Poco::Net } catch (const Poco::Exception & e) { - if (i == settings.http_max_tries - 1 || !isRetriableError(response.getStatus())) - throw; - - if (e.code() == ErrorCodes::TOO_MANY_REDIRECTS) + if (i == settings.http_max_tries - 1 || e.code() == ErrorCodes::TOO_MANY_REDIRECTS || !isRetriableError(response.getStatus())) throw; LOG_ERROR(log, "Failed to make HTTP_HEAD request to {}. Error: {}", uri.toString(), e.displayText()); @@ -544,11 +541,8 @@ bool ReadWriteBufferFromHTTPBase::nextImpl() } catch (const Poco::Exception & e) { - /// Too many open files - non-retryable. - if (e.code() == POCO_EMFILE) - throw; - - if (e.code() == ErrorCodes::TOO_MANY_REDIRECTS) + /// Too many open files or redirects - non-retryable. + if (e.code() == POCO_EMFILE || e.code() == ErrorCodes::TOO_MANY_REDIRECTS) throw; /** Retry request unconditionally if nothing has been read yet. From e112288c0b47d793d5c1a732655c4ae240b526ca Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 24 Jan 2024 09:27:27 +0000 Subject: [PATCH 162/264] Capability check for s3_plain --- .../ObjectStorages/ObjectStorageFactory.cpp | 50 ++++++++++++------- 1 file changed, 31 insertions(+), 19 deletions(-) diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index 54f8fb31acc..08ab7f210d1 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -80,9 +80,10 @@ ObjectStoragePtr ObjectStorageFactory::create( } #if USE_AWS_S3 -static S3::URI getS3URI(const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - const ContextPtr & context) +namespace +{ + +S3::URI getS3URI(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const ContextPtr & context) { String endpoint = context->getMacros()->expand(config.getString(config_prefix + ".endpoint")); S3::URI uri(endpoint); @@ -94,6 +95,23 @@ static S3::URI getS3URI(const Poco::Util::AbstractConfiguration & config, return uri; } +void checkS3Capabilities( + S3ObjectStorage & storage, const S3Capabilities s3_capabilities, const String & name, const String & key_with_trailing_slash) +{ + /// If `support_batch_delete` is turned on (default), check and possibly switch it off. + if (s3_capabilities.support_batch_delete && !checkBatchRemove(storage, key_with_trailing_slash)) + { + LOG_WARNING( + &Poco::Logger::get("S3ObjectStorage"), + "Storage for disk {} does not support batch delete operations, " + "so `s3_capabilities.support_batch_delete` was automatically turned off during the access check. " + "To remove this message set `s3_capabilities.support_batch_delete` for the disk to `false`.", + name); + storage.setCapabilitiesSupportBatchDelete(false); + } +} +} + void registerS3ObjectStorage(ObjectStorageFactory & factory) { static constexpr auto disk_type = "s3"; @@ -116,20 +134,8 @@ void registerS3ObjectStorage(ObjectStorageFactory & factory) /// NOTE: should we still perform this check for clickhouse-disks? if (!skip_access_check) - { - /// If `support_batch_delete` is turned on (default), check and possibly switch it off. - if (s3_capabilities.support_batch_delete && !checkBatchRemove(*object_storage, uri.key)) - { - LOG_WARNING( - &Poco::Logger::get("S3ObjectStorage"), - "Storage for disk {} does not support batch delete operations, " - "so `s3_capabilities.support_batch_delete` was automatically turned off during the access check. " - "To remove this message set `s3_capabilities.support_batch_delete` for the disk to `false`.", - name - ); - object_storage->setCapabilitiesSupportBatchDelete(false); - } - } + checkS3Capabilities(*object_storage, s3_capabilities, name, uri.key); + return object_storage; }); } @@ -143,7 +149,7 @@ void registerS3PlainObjectStorage(ObjectStorageFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const ContextPtr & context, - bool /* skip_access_check */) -> ObjectStoragePtr + bool skip_access_check) -> ObjectStoragePtr { /// send_metadata changes the filenames (includes revision), while /// s3_plain do not care about this, and expect that the file name @@ -159,8 +165,14 @@ void registerS3PlainObjectStorage(ObjectStorageFactory & factory) auto client = getClient(config, config_prefix, context, *settings); auto key_generator = getKeyGenerator(disk_type, uri, config, config_prefix); - return std::make_shared( + auto object_storage = std::make_shared( std::move(client), std::move(settings), uri, s3_capabilities, key_generator, name); + + /// NOTE: should we still perform this check for clickhouse-disks? + if (!skip_access_check) + checkS3Capabilities(*object_storage, s3_capabilities, name, uri.key); + + return object_storage; }); } #endif From 75e27444645c4d3652d7eb5cf2091e3a4966e839 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 24 Jan 2024 09:45:17 +0000 Subject: [PATCH 163/264] Remove timeout --- .../02015_async_inserts_stress_long.sh | 30 +++++++++++-------- 1 file changed, 18 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/02015_async_inserts_stress_long.sh b/tests/queries/0_stateless/02015_async_inserts_stress_long.sh index 437df01d445..2f7e15f201a 100755 --- a/tests/queries/0_stateless/02015_async_inserts_stress_long.sh +++ b/tests/queries/0_stateless/02015_async_inserts_stress_long.sh @@ -11,7 +11,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function insert1() { url="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=0" - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do ${CLICKHOUSE_CURL} -sS "$url" -d 'INSERT INTO async_inserts FORMAT CSV 1,"a" 2,"b" @@ -22,7 +23,8 @@ function insert1() function insert2() { url="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=0" - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do ${CLICKHOUSE_CURL} -sS "$url" -d 'INSERT INTO async_inserts FORMAT JSONEachRow {"id": 5, "s": "e"} {"id": 6, "s": "f"}' done } @@ -30,28 +32,32 @@ function insert2() function insert3() { url="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=0" - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do ${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO FUNCTION remote('127.0.0.1', $CLICKHOUSE_DATABASE, async_inserts) VALUES (7, 'g') (8, 'h')" done } function select1() { - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do ${CLICKHOUSE_CLIENT} -q "SELECT * FROM async_inserts FORMAT Null" done } function select2() { - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do ${CLICKHOUSE_CLIENT} -q "SELECT * FROM system.asynchronous_inserts FORMAT Null" done } function truncate1() { - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do sleep 0.1 ${CLICKHOUSE_CLIENT} -q "TRUNCATE TABLE async_inserts" done @@ -70,14 +76,14 @@ export -f select2 export -f truncate1 for _ in {1..5}; do - timeout $TIMEOUT bash -c insert1 & - timeout $TIMEOUT bash -c insert2 & - timeout $TIMEOUT bash -c insert3 & + insert1 $TIMEOUT & + insert2 $TIMEOUT & + insert3 $TIMEOUT & done -timeout $TIMEOUT bash -c select1 & -timeout $TIMEOUT bash -c select2 & -timeout $TIMEOUT bash -c truncate1 & +select1 $TIMEOUT & +select2 $TIMEOUT & +truncate1 $TIMEOUT & wait echo "OK" From bcfe69195410ea428d47db9fdc390d3a018a4657 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 24 Jan 2024 09:54:18 +0000 Subject: [PATCH 164/264] Use different names for `nothing` function with different result types --- .../AggregateFunctionNothing.cpp | 26 ++++-- .../AggregateFunctionNothing.h | 42 +++++++-- .../Combinators/AggregateFunctionNull.cpp | 4 +- src/Analyzer/FunctionNode.cpp | 12 --- src/Analyzer/Utils.cpp | 2 +- ..._analyzer_aggregate_nothing_type.reference | 84 ++++++++++++++++++ .../02922_analyzer_aggregate_nothing_type.sql | 86 ++++++++++++++++++- 7 files changed, 224 insertions(+), 32 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionNothing.cpp b/src/AggregateFunctions/AggregateFunctionNothing.cpp index ebeffffc71e..abdcf3dd1da 100644 --- a/src/AggregateFunctions/AggregateFunctionNothing.cpp +++ b/src/AggregateFunctions/AggregateFunctionNothing.cpp @@ -10,12 +10,28 @@ struct Settings; void registerAggregateFunctionNothing(AggregateFunctionFactory & factory) { - factory.registerFunction("nothing", [](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) - { - assertNoParameters(name, parameters); + factory.registerFunction(NameAggregateFunctionNothing::name, + [](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) + { + assertNoParameters(name, parameters); + return std::make_shared(argument_types, parameters); + }); - auto result_type = argument_types.empty() ? std::make_shared(std::make_shared()) : argument_types.front(); - return std::make_shared(argument_types, parameters, result_type); + factory.registerFunction(NameAggregateFunctionNothingNull::name, + [](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) + { + assertNoParameters(name, parameters); + return std::make_shared(argument_types, parameters); + }); + + + factory.registerFunction(NameAggregateFunctionNothingUInt64::name, { + [](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) + { + assertNoParameters(name, parameters); + return std::make_shared(argument_types, parameters); + }, + AggregateFunctionProperties{ .returns_default_when_only_null = true } }); } diff --git a/src/AggregateFunctions/AggregateFunctionNothing.h b/src/AggregateFunctions/AggregateFunctionNothing.h index 8c1b95c26b0..46366ded043 100644 --- a/src/AggregateFunctions/AggregateFunctionNothing.h +++ b/src/AggregateFunctions/AggregateFunctionNothing.h @@ -6,7 +6,8 @@ #include #include #include -#include "DataTypes/IDataType.h" +#include +#include namespace DB @@ -18,20 +19,42 @@ namespace ErrorCodes extern const int INCORRECT_DATA; } +/// Returns the same type as the first argument +struct NameAggregateFunctionNothing { static constexpr auto name = "nothing"; }; +/// Always returns Nullable(Nothing) +struct NameAggregateFunctionNothingNull { static constexpr auto name = "nothingNull"; }; +/// Always returns UInt64 +struct NameAggregateFunctionNothingUInt64 { static constexpr auto name = "nothingUInt64"; }; + +template class AggregateFunctionNothingImpl; + +using AggregateFunctionNothing = AggregateFunctionNothingImpl; +using AggregateFunctionNothingNull = AggregateFunctionNothingImpl; +using AggregateFunctionNothingUInt64 = AggregateFunctionNothingImpl; + /** Aggregate function that takes arbitrary number of arbitrary arguments and does nothing. */ -class AggregateFunctionNothing final : public IAggregateFunctionHelper +template +class AggregateFunctionNothingImpl final : public IAggregateFunctionHelper> { -public: - AggregateFunctionNothing(const DataTypes & arguments, const Array & params, const DataTypePtr & result_type_) - : IAggregateFunctionHelper(arguments, params, result_type_) {} - - String getName() const override + static DataTypePtr getReturnType(const DataTypes & arguments [[maybe_unused]]) { - return "nothing"; + if constexpr (std::is_same_v) + return std::make_shared(); + else if constexpr (std::is_same_v) + return std::make_shared(std::make_shared()); + return arguments.empty() ? std::make_shared(std::make_shared()) : arguments.front(); } +public: + AggregateFunctionNothingImpl(const DataTypes & arguments, const Array & params) + : IAggregateFunctionHelper>(arguments, params, getReturnType(arguments)) + { + } + + String getName() const override { return Name::name; } + bool allocatesMemoryInArena() const override { return false; } void create(AggregateDataPtr __restrict) const override @@ -75,7 +98,8 @@ public: [[maybe_unused]] char symbol; readChar(symbol, buf); if (symbol != '\0') - throw Exception(ErrorCodes::INCORRECT_DATA, "Incorrect state of aggregate function 'nothing', it should contain exactly one zero byte, while it is {}.", static_cast(symbol)); + throw Exception(ErrorCodes::INCORRECT_DATA, "Incorrect state of aggregate function '{}', it should contain exactly one zero byte, while it is {}", + getName(), static_cast(symbol)); } void insertResultInto(AggregateDataPtr __restrict, IColumn & to, Arena *) const override diff --git a/src/AggregateFunctions/Combinators/AggregateFunctionNull.cpp b/src/AggregateFunctions/Combinators/AggregateFunctionNull.cpp index 7a1f18a8904..7ffacd3af16 100644 --- a/src/AggregateFunctions/Combinators/AggregateFunctionNull.cpp +++ b/src/AggregateFunctions/Combinators/AggregateFunctionNull.cpp @@ -111,9 +111,9 @@ public: * To address this, we handle `nothing` in a special way in `FunctionNode::toASTImpl`. */ if (properties.returns_default_when_only_null) - return std::make_shared(arguments, params, std::make_shared()); + return std::make_shared(arguments, params); else - return std::make_shared(arguments, params, std::make_shared(std::make_shared())); + return std::make_shared(arguments, params); } assert(nested_function); diff --git a/src/Analyzer/FunctionNode.cpp b/src/Analyzer/FunctionNode.cpp index f3bf41cb1e9..e902ac2274e 100644 --- a/src/Analyzer/FunctionNode.cpp +++ b/src/Analyzer/FunctionNode.cpp @@ -210,18 +210,6 @@ ASTPtr FunctionNode::toASTImpl(const ConvertToASTOptions & options) const function_ast->name = function_name; function_ast->nulls_action = nulls_action; - if (function_name == "nothing") - { - /** Inside AggregateFunctionCombinatorNull we may replace functions with `NULL` in arguments with `nothing`. - * Result type of `nothing` depends on `returns_default_when_only_null` property of nested function. - * If we convert `nothing` to AST, we will lose this information, so we use original function name instead. - */ - const auto & original_ast = getOriginalAST(); - const auto & original_function_ast = original_ast ? original_ast->as() : nullptr; - if (original_function_ast) - function_ast->name = original_function_ast->name; - } - if (isWindowFunction()) { function_ast->is_window_function = true; diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index 53fcf534f64..af20ff4854d 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -626,7 +626,7 @@ void rerunFunctionResolve(FunctionNode * function_node, ContextPtr context) } else if (function_node->isAggregateFunction()) { - if (name == "nothing") + if (name == "nothing" || name == "nothingUInt64" || name == "nothingNull") return; function_node->resolveAsAggregateFunction(resolveAggregateFunction(function_node)); } diff --git a/tests/queries/0_stateless/02922_analyzer_aggregate_nothing_type.reference b/tests/queries/0_stateless/02922_analyzer_aggregate_nothing_type.reference index f9c7b26d245..feebf7dbf49 100644 --- a/tests/queries/0_stateless/02922_analyzer_aggregate_nothing_type.reference +++ b/tests/queries/0_stateless/02922_analyzer_aggregate_nothing_type.reference @@ -1,13 +1,97 @@ +\N +\N +[nan,nan] +\N [nan,nan] 0 \N 0 0 0 +\N +\N + +\N +[nan,nan] +[nan,nan] + +[nan,nan] +-- notinhgs: +\N Nullable(Nothing) +0 UInt8 +\N Nullable(Nothing) +0 UInt64 +0 UInt64 +0 UInt64 +\N Nullable(Nothing) +\N Nullable(Nothing) +\N Nullable(Nothing) +-- quantile: +\N +\N +\N +\N +1 +1 +1 +-- quantiles: +[nan,nan] +[nan,nan] +[nan,nan] +[nan,nan] +[1,1] +[1,1] +[1,1] +-- nothing: +\N +\N +\N +\N +0 +0 +0 +-- nothing(UInt64): +0 +0 +0 +0 +0 +0 +0 +-- nothing(Nullable(Nothing)): +\N +\N +\N +\N +\N +\N +\N +-- sum: +\N +\N +\N +\N +6 +6 +6 +-- count: +0 +0 +0 +0 +6 +6 +6 0 0 0 \N +0 \N +0 \N + 0 \N 0 0 +\N \N [nan,nan] + +\N \N [nan,nan] diff --git a/tests/queries/0_stateless/02922_analyzer_aggregate_nothing_type.sql b/tests/queries/0_stateless/02922_analyzer_aggregate_nothing_type.sql index a064c091df0..5b9343c6e13 100644 --- a/tests/queries/0_stateless/02922_analyzer_aggregate_nothing_type.sql +++ b/tests/queries/0_stateless/02922_analyzer_aggregate_nothing_type.sql @@ -1,7 +1,87 @@ -#!/usr/bin/env -S ${HOME}/clickhouse-client --progress --queries-file +select sum(NULL); +select quantile(0.5)(NULL); +select quantiles(0.1, 0.2)(NULL :: Nullable(UInt32)); +select quantile(0.5)(NULL), quantiles(0.1, 0.2)(NULL :: Nullable(UInt32)), count(NULL), sum(NULL); SELECT count(NULL) FROM remote('127.0.0.{1,2}', numbers(3)) GROUP BY number % 2 WITH TOTALS; +SELECT quantile(0.5)(NULL) FROM remote('127.0.0.{1,2}', numbers(3)) GROUP BY number % 2 WITH TOTALS; +SELECT quantiles(0.1, 0.2)(NULL :: Nullable(UInt32)) FROM remote('127.0.0.{1,2}', numbers(3)) GROUP BY number % 2 WITH TOTALS; + +SELECT '-- notinhgs:'; +SELECT nothing() as n, toTypeName(n); +SELECT nothing(1) as n, toTypeName(n); +SELECT nothing(NULL) as n, toTypeName(n); +SELECT nothingUInt64() as n, toTypeName(n); +SELECT nothingUInt64(1) as n, toTypeName(n); +SELECT nothingUInt64(NULL) as n, toTypeName(n); +SELECT nothingNull() as n, toTypeName(n); +SELECT nothingNull(1) as n, toTypeName(n); +SELECT nothingNull(NULL) as n, toTypeName(n); + +SELECT '-- quantile:'; +SELECT quantileArray(0.5)([NULL, NULL]) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT quantileArrayIf(0.5)([NULL], 1) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT quantileArrayIf(0.5)([NULL], 0) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT quantileIfArray(0.5)([NULL, NULL], [1, 0]) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT quantileIfArray(0.5)([1, NULL], [1, 0]) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT quantileIfArrayIf(0.5)([1, NULL], [1, 0], 1) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT quantileIfArrayArray(0.5)([[1, NULL]], [[1, 0]]) AS x FROM remote('127.0.0.{1,2}', numbers(3)); + +SELECT '-- quantiles:'; +select quantilesArray(0.5, 0.9)([NULL :: Nullable(UInt64), NULL]) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT quantilesArrayIf(0.5, 0.9)([NULL :: Nullable(UInt64)], 1) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT quantilesArrayIf(0.5, 0.9)([NULL :: Nullable(UInt64)], 0) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT quantilesIfArray(0.5, 0.9)([NULL :: Nullable(UInt64), NULL], [1, 0]) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT quantilesIfArray(0.5, 0.9)([1, NULL], [1, 0]) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT quantilesIfArrayIf(0.5, 0.9)([1, NULL], [1, 0], 1) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT quantilesIfArrayArray(0.5, 0.9)([[1, NULL]], [[1, 0]]) AS x FROM remote('127.0.0.{1,2}', numbers(3)); + +SELECT '-- nothing:'; +SELECT nothingArray([NULL, NULL]) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT nothingArrayIf([NULL], 1) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT nothingArrayIf([NULL], 0) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT nothingIfArray([NULL, NULL], [1, 0]) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT nothingIfArray([1, NULL], [1, 0]) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT nothingIfArrayIf([1, NULL], [1, 0], 1) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT nothingIfArrayArray([[1, NULL]], [[1, 0]]) AS x FROM remote('127.0.0.{1,2}', numbers(3)); + +SELECT '-- nothing(UInt64):'; +SELECT nothingUInt64Array([NULL, NULL]) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT nothingUInt64ArrayIf([NULL], 1) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT nothingUInt64ArrayIf([NULL], 0) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT nothingUInt64IfArray([NULL, NULL], [1, 0]) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT nothingUInt64IfArray([1, NULL], [1, 0]) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT nothingUInt64IfArrayIf([1, NULL], [1, 0], 1) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT nothingUInt64IfArrayArray([[1, NULL]], [[1, 0]]) AS x FROM remote('127.0.0.{1,2}', numbers(3)); + +SELECT '-- nothing(Nullable(Nothing)):'; +SELECT nothingNullArray([NULL, NULL]) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT nothingNullArrayIf([NULL], 1) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT nothingNullArrayIf([NULL], 0) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT nothingNullIfArray([NULL, NULL], [1, 0]) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT nothingNullIfArray([1, NULL], [1, 0]) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT nothingNullIfArrayIf([1, NULL], [1, 0], 1) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT nothingNullIfArrayArray([[1, NULL]], [[1, 0]]) AS x FROM remote('127.0.0.{1,2}', numbers(3)); + +SELECT '-- sum:'; +SELECT sumArray([NULL, NULL]) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT sumArrayIf([NULL], 1) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT sumArrayIf([NULL], 0) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT sumIfArray([NULL, NULL], [1, 0]) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT sumIfArray([1, NULL], [1, 0]) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT sumIfArrayIf([1, NULL], [1, 0], 1) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT sumIfArrayArray([[1, NULL]], [[1, 0]]) AS x FROM remote('127.0.0.{1,2}', numbers(3)); + +SELECT '-- count:'; +SELECT countArray([NULL, NULL]) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT countArrayIf([NULL], 1) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT countArrayIf([NULL], 0) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT countIfArray([NULL, NULL], [1, 0]) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT countIfArray([1, NULL], [1, 0]) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT countIfArrayIf([1, NULL], [1, 0], 1) AS x FROM remote('127.0.0.{1,2}', numbers(3)); +SELECT countIfArrayArray([[1, NULL]], [[1, 0]]) AS x FROM remote('127.0.0.{1,2}', numbers(3)); + DROP TABLE IF EXISTS t1; CREATE TABLE t1 (`n` UInt64) ENGINE = MergeTree ORDER BY tuple(); @@ -18,7 +98,7 @@ SET SELECT count(NULL) FROM t1 WITH TOTALS; SELECT count(NULL as a), a FROM t1 WITH TOTALS; --- result differs in old and new analyzer: --- SELECT count(NULL as a), sum(a) FROM t1 WITH TOTALS; +SELECT count(NULL as a), sum(a) FROM t1 WITH TOTALS; SELECT uniq(NULL) FROM t1 WITH TOTALS; +SELECT quantile(0.5)(NULL), quantile(0.9)(NULL), quantiles(0.1, 0.2)(NULL :: Nullable(UInt32)) FROM t1 WITH TOTALS; From 62a38478887625a78e478b080116b23cb7b0a6d4 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 24 Jan 2024 11:15:52 +0100 Subject: [PATCH 165/264] Update src/Disks/ObjectStorages/ObjectStorageFactory.cpp --- src/Disks/ObjectStorages/ObjectStorageFactory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index 08ab7f210d1..ec6f7081c85 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -165,7 +165,7 @@ void registerS3PlainObjectStorage(ObjectStorageFactory & factory) auto client = getClient(config, config_prefix, context, *settings); auto key_generator = getKeyGenerator(disk_type, uri, config, config_prefix); - auto object_storage = std::make_shared( + auto object_storage = std::make_shared( std::move(client), std::move(settings), uri, s3_capabilities, key_generator, name); /// NOTE: should we still perform this check for clickhouse-disks? From 4b40b14bba4c4d4472be47235a51538587a85242 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 24 Jan 2024 11:23:13 +0100 Subject: [PATCH 166/264] Update test.py --- tests/integration/test_storage_iceberg/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index c4b48ce884c..fc2193854f6 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -564,8 +564,8 @@ def test_restart_broken(started_cluster): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session minio_client = started_cluster.minio_client - bucket = "broken" - TABLE_NAME = "test_restart_broken" + bucket = "broken2" + TABLE_NAME = "test_restart_broken_table_function" if not minio_client.bucket_exists(bucket): minio_client.make_bucket(bucket) From 718064bb43e98da65889d8381b81e2c64df07329 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 24 Jan 2024 12:19:10 +0100 Subject: [PATCH 167/264] Minor change to restart CI --- src/IO/ReadWriteBufferFromHTTP.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index fccbf13a519..723a12d0e5b 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -4,8 +4,8 @@ namespace ProfileEvents { -extern const Event ReadBufferSeekCancelConnection; -extern const Event ReadWriteBufferFromHTTPPreservedSessions; + extern const Event ReadBufferSeekCancelConnection; + extern const Event ReadWriteBufferFromHTTPPreservedSessions; } namespace DB From 0d7710ee4bd65657813c1a90148e957c58cfcff5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 24 Jan 2024 12:39:01 +0100 Subject: [PATCH 168/264] Fix build --- src/Interpreters/Cache/FileSegment.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index cce088081e7..f3791bf2c23 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -339,8 +339,6 @@ void FileSegment::write(const char * from, size_t size, size_t offset) if (!size) throw Exception(ErrorCodes::LOGICAL_ERROR, "Writing zero size is not allowed"); - const auto file_segment_path = getPath(); - { auto lock = lockFileSegment(); assertIsDownloaderUnlocked("write", lock); From 0badcffdc159f06ed90f2574b296603c6b139d06 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 24 Jan 2024 12:01:32 +0000 Subject: [PATCH 169/264] test: table metadata load failure should shutdowns the server properly --- tests/integration/helpers/cluster.py | 29 +++++++++++- .../test_replicated_database/test.py | 44 +++++++++++++++++++ 2 files changed, 72 insertions(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index c0b145b047a..66b46f1ab2f 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -3776,7 +3776,7 @@ class ClickHouseInstance: except Exception as e: logging.warning(f"Stop ClickHouse raised an error {e}") - def start_clickhouse(self, start_wait_sec=60, retry_start=True): + def start_clickhouse(self, start_wait_sec=60, retry_start=True, expected_to_fail=False): if not self.stay_alive: raise Exception( "ClickHouse can be started again only with stay_alive=True instance" @@ -3794,10 +3794,15 @@ class ClickHouseInstance: ["bash", "-c", "{} --daemon".format(self.clickhouse_start_command)], user=str(os.getuid()), ) + if expected_to_fail: + self.wait_start_failed(start_wait_sec + start_time - time.time()) + return time.sleep(1) continue else: logging.debug("Clickhouse process running.") + if expected_to_fail: + raise Exception("ClickHouse was expected not to be running.") try: self.wait_start(start_wait_sec + start_time - time.time()) return @@ -3849,6 +3854,28 @@ class ClickHouseInstance: if last_err is not None: raise last_err + def wait_start_failed(self, start_wait_sec): + start_time = time.time() + while time.time() <= start_time + start_wait_sec: + pid = self.get_process_pid("clickhouse") + if pid is None: + return + time.sleep(1) + logging.error( + f"No time left to shutdown. Process is still running. Will dump threads." + ) + ps_clickhouse = self.exec_in_container( + ["bash", "-c", "ps -C clickhouse"], nothrow=True, user="root" + ) + logging.info(f"PS RESULT:\n{ps_clickhouse}") + pid = self.get_process_pid("clickhouse") + if pid is not None: + self.exec_in_container( + ["bash", "-c", f"gdb -batch -ex 'thread apply all bt full' -p {pid}"], + user="root", + ) + raise Exception("ClickHouse server is still running, but was expected to shutdown. Check logs.") + def restart_clickhouse(self, stop_start_wait_sec=60, kill=False): self.stop_clickhouse(stop_start_wait_sec, kill) self.start_clickhouse(stop_start_wait_sec) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 9846e88a3c0..b47f86a843d 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -1398,3 +1398,47 @@ def test_modify_comment(started_cluster): main_node.query("DROP DATABASE modify_comment_db SYNC") dummy_node.query("DROP DATABASE modify_comment_db SYNC") + + +def test_table_metadata_corruption(started_cluster): + main_node.query("DROP DATABASE IF EXISTS table_metadata_corruption") + dummy_node.query("DROP DATABASE IF EXISTS table_metadata_corruption") + + main_node.query( + "CREATE DATABASE table_metadata_corruption ENGINE = Replicated('/clickhouse/databases/table_metadata_corruption', 'shard1', 'replica1');" + ) + dummy_node.query( + "CREATE DATABASE table_metadata_corruption ENGINE = Replicated('/clickhouse/databases/table_metadata_corruption', 'shard1', 'replica2');" + ) + + create_some_tables("table_metadata_corruption") + + main_node.query("SYSTEM SYNC DATABASE REPLICA table_metadata_corruption") + dummy_node.query("SYSTEM SYNC DATABASE REPLICA table_metadata_corruption") + + # Server should handle this by throwing an exception during table loading, which should lead to server shutdown + corrupt = "sed --follow-symlinks -i 's/ReplicatedMergeTree/CorruptedMergeTree/' /var/lib/clickhouse/metadata/table_metadata_corruption/rmt1.sql" + + print(f"Corrupting metadata using `{corrupt}`") + dummy_node.stop_clickhouse(kill=True) + dummy_node.exec_in_container(["bash", "-c", corrupt]) + + query = ( + "SELECT name, uuid, create_table_query FROM system.tables WHERE database='table_metadata_corruption' AND name NOT LIKE '.inner_id.%' " + "ORDER BY name SETTINGS show_table_uuid_in_table_create_query_if_not_nil=1" + ) + expected = main_node.query(query) + + # We expect clickhouse server to shutdown without LOGICAL_ERRORs or deadlocks + dummy_node.start_clickhouse(expected_to_fail=True) + assert not dummy_node.contains_in_log("LOGICAL_ERROR") + + fix_corrupt = "sed --follow-symlinks -i 's/CorruptedMergeTree/ReplicatedMergeTree/' /var/lib/clickhouse/metadata/table_metadata_corruption/rmt1.sql" + print(f"Fix corrupted metadata using `{fix_corrupt}`") + dummy_node.exec_in_container(["bash", "-c", fix_corrupt]) + + dummy_node.start_clickhouse() + assert_eq_with_retry(dummy_node, query, expected) + + main_node.query("DROP DATABASE IF EXISTS table_metadata_corruption") + dummy_node.query("DROP DATABASE IF EXISTS table_metadata_corruption") From 41e6dd7a9d91eef031a9c375278183a35acc927e Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 24 Jan 2024 12:15:19 +0000 Subject: [PATCH 170/264] Automatic style fix --- tests/integration/helpers/cluster.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 66b46f1ab2f..1d96563251b 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -3776,7 +3776,9 @@ class ClickHouseInstance: except Exception as e: logging.warning(f"Stop ClickHouse raised an error {e}") - def start_clickhouse(self, start_wait_sec=60, retry_start=True, expected_to_fail=False): + def start_clickhouse( + self, start_wait_sec=60, retry_start=True, expected_to_fail=False + ): if not self.stay_alive: raise Exception( "ClickHouse can be started again only with stay_alive=True instance" @@ -3874,7 +3876,9 @@ class ClickHouseInstance: ["bash", "-c", f"gdb -batch -ex 'thread apply all bt full' -p {pid}"], user="root", ) - raise Exception("ClickHouse server is still running, but was expected to shutdown. Check logs.") + raise Exception( + "ClickHouse server is still running, but was expected to shutdown. Check logs." + ) def restart_clickhouse(self, stop_start_wait_sec=60, kill=False): self.stop_clickhouse(stop_start_wait_sec, kill) From 698aef50c2c074a816e8fed668fcaf61c9393d57 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 24 Jan 2024 12:43:01 +0000 Subject: [PATCH 171/264] Fix: LIMIT BY in distriubted query --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- .../02971_limit_by_distributed.reference | 16 ++++++++++++ .../02971_limit_by_distributed.sql | 25 +++++++++++++++++++ 3 files changed, 42 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02971_limit_by_distributed.reference create mode 100644 tests/queries/0_stateless/02971_limit_by_distributed.sql diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index c0e9aeaae1d..4e4e25617e4 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1572,7 +1572,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

Date: Wed, 24 Jan 2024 14:26:50 +0100 Subject: [PATCH 172/264] Fix url encoding issue --- tests/ci/pr_info.py | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index d1867ebf786..d1be459666f 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -3,15 +3,20 @@ import json import logging import os from typing import Dict, List, Set, Union +from urllib.parse import quote +# isort: off +# for some reason this line moves to the end from unidiff import PatchSet # type: ignore +# isort: on + from build_download_helper import get_gh_api from env_helper import ( - GITHUB_REPOSITORY, - GITHUB_SERVER_URL, - GITHUB_RUN_URL, GITHUB_EVENT_PATH, + GITHUB_REPOSITORY, + GITHUB_RUN_URL, + GITHUB_SERVER_URL, ) FORCE_TESTS_LABEL = "force tests" @@ -295,9 +300,10 @@ class PRInfo: @staticmethod def compare_url(first: str, second: str) -> str: + """the first and second are URL encoded to not fail on '#' and other symbols""" return ( "https://api.github.com/repos/" - f"{GITHUB_REPOSITORY}/compare/{first}...{second}" + f"{GITHUB_REPOSITORY}/compare/{quote(first)}...{quote(second)}" ) def fetch_changed_files(self): From 383ae86ebb0da8a962521f55b08186331eb0f676 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 24 Jan 2024 15:06:16 +0000 Subject: [PATCH 173/264] Upgrade simdjson to v3.6.3 See https://github.com/ClickHouse/ClickHouse/issues/38621#issuecomment-1908219067 --- contrib/simdjson | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/simdjson b/contrib/simdjson index 1075e8609c4..6060be2fdf6 160000 --- a/contrib/simdjson +++ b/contrib/simdjson @@ -1 +1 @@ -Subproject commit 1075e8609c4afa253162d441437af929c29e31bb +Subproject commit 6060be2fdf62edf4a8f51a8b0883d57d09397b30 From 7826706ff923f2ef85d2f6fd02c539a29e7b3421 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 24 Jan 2024 16:09:26 +0100 Subject: [PATCH 174/264] Small fix --- src/Analyzer/Utils.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index da6ab844dd3..1315af85d4c 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -360,8 +360,7 @@ QueryTreeNodes extractAllTableReferences(const QueryTreeNodePtr & tree) } case QueryTreeNodeType::TABLE_FUNCTION: { - for (const auto & argument_node : node_to_process->as()->getArgumentsNode()->getChildren()) - nodes_to_process.push_back(argument_node); + // Arguments of table function can't contain TableNodes. break; } case QueryTreeNodeType::ARRAY_JOIN: From a4186dc4d3e8b54736942dede7e096baee900c97 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 24 Jan 2024 16:34:29 +0100 Subject: [PATCH 175/264] Decrease log level for one log message --- src/Interpreters/Cache/Metadata.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index aafa94f04be..953413a8ef2 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -924,7 +924,7 @@ KeyMetadata::iterator LockedKey::removeFileSegmentImpl( { auto file_segment = it->second->file_segment; - LOG_DEBUG( + LOG_TEST( key_metadata->logger(), "Remove from cache. Key: {}, offset: {}, size: {}", getKey(), file_segment->offset(), file_segment->reserved_size); From 50a0494df78af4cb3b12600d8eefe959a4c4104f Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 24 Jan 2024 16:53:10 +0100 Subject: [PATCH 176/264] Update test.py --- tests/integration/test_storage_delta/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index 413245fdaa3..25f0b58e0f5 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -463,8 +463,8 @@ def test_restart_broken_table_function(started_cluster): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session minio_client = started_cluster.minio_client - bucket = "broken" - TABLE_NAME = "test_restart_broken" + bucket = "broken2" + TABLE_NAME = "test_restart_broken_table_function" if not minio_client.bucket_exists(bucket): minio_client.make_bucket(bucket) From 235980d52ef6c20797e834e60223ae64a8069bfe Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 24 Jan 2024 17:28:01 +0100 Subject: [PATCH 177/264] Update FileCache.cpp --- src/Interpreters/Cache/FileCache.cpp | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 6d6822b7ad5..51f725e1699 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -881,18 +881,12 @@ void FileCache::iterate(IterateFunc && func, const UserID & user_id) void FileCache::removeKey(const Key & key, const UserID & user_id) { assertInitialized(); -#ifdef ABORT_ON_LOGICAL_ERROR - assertCacheCorrectness(); -#endif metadata.removeKey(key, /* if_exists */false, /* if_releasable */true, user_id); } void FileCache::removeKeyIfExists(const Key & key, const UserID & user_id) { assertInitialized(); -#ifdef ABORT_ON_LOGICAL_ERROR - assertCacheCorrectness(); -#endif metadata.removeKey(key, /* if_exists */true, /* if_releasable */true, user_id); } From dedc889e49a4922eec99caab3087d7f4b824fd53 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 24 Jan 2024 15:13:06 +0100 Subject: [PATCH 178/264] Fix broken cache for non-existing temp_path --- tests/ci/cache_utils.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/cache_utils.py b/tests/ci/cache_utils.py index ccb2049d1e6..062207fadd1 100644 --- a/tests/ci/cache_utils.py +++ b/tests/ci/cache_utils.py @@ -116,6 +116,7 @@ class Cache: self.s3_helper = s3_helper def _download(self, url: str, ignore_error: bool = False) -> None: + self.temp_path.mkdir(parents=True, exist_ok=True) compressed_cache = self.temp_path / self.archive_name try: if url.startswith("file://"): From 9038f0710a3ad541c5baf477dc8b188a6dea0907 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 24 Jan 2024 18:26:11 +0100 Subject: [PATCH 179/264] Minor --- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 1 + src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 1 + src/Disks/ObjectStorages/Cached/CachedObjectStorage.h | 4 ++-- src/Interpreters/Cache/FileCache.cpp | 2 +- src/Interpreters/Context.cpp | 1 + src/Interpreters/InterpreterSystemQuery.cpp | 9 +++++---- src/Interpreters/TemporaryDataOnDisk.cpp | 1 + src/Interpreters/TemporaryDataOnDisk.h | 3 ++- 8 files changed, 14 insertions(+), 8 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index ac87305270f..2b169bd9c7d 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include #include diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 3bc86f86250..63bacaa99e9 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -4,6 +4,7 @@ #include #include +#include #include #include #include diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index 7dcede2c0bc..028b0f7c510 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include namespace Poco @@ -121,7 +121,7 @@ public: static bool canUseReadThroughCache(const ReadSettings & settings); private: - FileCache::Key getCacheKey(const std::string & path) const; + FileCacheKey getCacheKey(const std::string & path) const; ReadSettings patchSettings(const ReadSettings & read_settings) const override; diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 51f725e1699..9c29a9c4a47 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -972,7 +972,7 @@ void FileCache::loadMetadataImpl() std::mutex set_exception_mutex; std::atomic stop_loading = false; - LOG_INFO(log, "Loading filesystem cache with {} threads", load_metadata_threads); + LOG_INFO(log, "Loading filesystem cache with {} threads from {}", load_metadata_threads, metadata.getBaseDirectory()); for (size_t i = 0; i < load_metadata_threads; ++i) { diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 38a033bee1d..9778a02f1fa 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -43,6 +43,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 1411e7e017b..8a242cee213 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -379,27 +379,28 @@ BlockIO InterpreterSystemQuery::execute() case Type::DROP_FILESYSTEM_CACHE: { getContext()->checkAccess(AccessType::SYSTEM_DROP_FILESYSTEM_CACHE); + const auto user_id = FileCache::getCommonUser().user_id; if (query.filesystem_cache_name.empty()) { auto caches = FileCacheFactory::instance().getAll(); for (const auto & [_, cache_data] : caches) - cache_data->cache->removeAllReleasable(FileCache::getCommonUser().user_id); + cache_data->cache->removeAllReleasable(user_id); } else { auto cache = FileCacheFactory::instance().getByName(query.filesystem_cache_name)->cache; if (query.key_to_drop.empty()) { - cache->removeAllReleasable(FileCache::getCommonUser().user_id); + cache->removeAllReleasable(user_id); } else { auto key = FileCacheKey::fromKeyString(query.key_to_drop); if (query.offset_to_drop.has_value()) - cache->removeFileSegment(key, query.offset_to_drop.value(), FileCache::getCommonUser().user_id); + cache->removeFileSegment(key, query.offset_to_drop.value(), user_id); else - cache->removeKey(key, FileCache::getCommonUser().user_id); + cache->removeKey(key, user_id); } } break; diff --git a/src/Interpreters/TemporaryDataOnDisk.cpp b/src/Interpreters/TemporaryDataOnDisk.cpp index 7bf5e86eb41..3ad72af95f9 100644 --- a/src/Interpreters/TemporaryDataOnDisk.cpp +++ b/src/Interpreters/TemporaryDataOnDisk.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Interpreters/TemporaryDataOnDisk.h b/src/Interpreters/TemporaryDataOnDisk.h index 0c35cd63d13..e57d9130369 100644 --- a/src/Interpreters/TemporaryDataOnDisk.h +++ b/src/Interpreters/TemporaryDataOnDisk.h @@ -7,7 +7,6 @@ #include #include #include -#include namespace CurrentMetrics @@ -27,6 +26,8 @@ using TemporaryDataOnDiskPtr = std::unique_ptr; class TemporaryFileStream; using TemporaryFileStreamPtr = std::unique_ptr; +class FileCache; + /* * Used to account amount of temporary data written to disk. * If limit is set, throws exception if limit is exceeded. From 4deb1eab19c582ce4f3158578084c90f7e36641c Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 24 Jan 2024 18:17:19 +0000 Subject: [PATCH 180/264] init --- src/Functions/FunctionsConversion.h | 15 ++++++++++----- .../02972_to_string_nullable_timezone.reference | 1 + .../02972_to_string_nullable_timezone.sql | 2 ++ 3 files changed, 13 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/02972_to_string_nullable_timezone.reference create mode 100644 tests/queries/0_stateless/02972_to_string_nullable_timezone.sql diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index e04d547e579..cd13a6f5cd0 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -1076,6 +1076,7 @@ struct ConvertImpl) { auto datetime_arg = arguments[0]; + auto timezone_arg = arguments[1]; const DateLUTImpl * time_zone = nullptr; const ColumnConst * time_zone_column = nullptr; @@ -1088,13 +1089,14 @@ struct ConvertImplconvertToFullColumnIfConst(); + timezone_arg.column = timezone_arg.column->convertToFullColumnIfConst(); if constexpr (std::is_same_v || std::is_same_v) time_zone = &DateLUT::instance(); /// For argument of Date or DateTime type, second argument with time zone could be specified. if constexpr (std::is_same_v || std::is_same_v) { - if ((time_zone_column = checkAndGetColumnConst(arguments[1].column.get()))) + if ((time_zone_column = checkAndGetColumnConst(timezone_arg.column.get()))) { auto non_null_args = createBlockWithNestedColumns(arguments); time_zone = &extractTimeZoneFromFunctionArguments(non_null_args, 1, 0); @@ -1130,14 +1132,17 @@ struct ConvertImpl 1) + null_map = copyNullMap(timezone_arg.column); + if (null_map) { for (size_t i = 0; i < size; ++i) { if (!time_zone_column && arguments.size() > 1) { - if (!arguments[1].column.get()->getDataAt(i).toString().empty()) - time_zone = &DateLUT::instance(arguments[1].column.get()->getDataAt(i).toString()); + if (!timezone_arg.column.get()->getDataAt(i).toString().empty()) + time_zone = &DateLUT::instance(timezone_arg.column.get()->getDataAt(i).toString()); else throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Provided time zone must be non-empty"); } @@ -1153,8 +1158,8 @@ struct ConvertImpl 1) { - if (!arguments[1].column.get()->getDataAt(i).toString().empty()) - time_zone = &DateLUT::instance(arguments[1].column.get()->getDataAt(i).toString()); + if (!timezone_arg.column.get()->getDataAt(i).toString().empty()) + time_zone = &DateLUT::instance(timezone_arg.column.get()->getDataAt(i).toString()); else throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Provided time zone must be non-empty"); } diff --git a/tests/queries/0_stateless/02972_to_string_nullable_timezone.reference b/tests/queries/0_stateless/02972_to_string_nullable_timezone.reference new file mode 100644 index 00000000000..c3514956520 --- /dev/null +++ b/tests/queries/0_stateless/02972_to_string_nullable_timezone.reference @@ -0,0 +1 @@ +2022-01-01 11:13:14 diff --git a/tests/queries/0_stateless/02972_to_string_nullable_timezone.sql b/tests/queries/0_stateless/02972_to_string_nullable_timezone.sql new file mode 100644 index 00000000000..25b787f892a --- /dev/null +++ b/tests/queries/0_stateless/02972_to_string_nullable_timezone.sql @@ -0,0 +1,2 @@ +SET session_timezone='Europe/Amsterdam'; +SELECT toString(toDateTime('2022-01-01 12:13:14'), CAST('UTC', 'Nullable(String)')); \ No newline at end of file From 3d03c16fc81d17f9eccd3dba0d209c2b3d30be26 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 24 Jan 2024 21:02:22 +0100 Subject: [PATCH 181/264] Fix another place with special symbols in the URL --- tests/ci/workflow_approve_rerun_lambda/app.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py index e511d773577..5e68f2d4b53 100644 --- a/tests/ci/workflow_approve_rerun_lambda/app.py +++ b/tests/ci/workflow_approve_rerun_lambda/app.py @@ -1,12 +1,12 @@ #!/usr/bin/env python3 -from collections import namedtuple import fnmatch import json import time +from collections import namedtuple +from urllib.parse import quote import requests # type: ignore - from lambda_shared.pr import TRUSTED_CONTRIBUTORS from lambda_shared.token import get_cached_access_token @@ -129,7 +129,7 @@ def _exec_post_with_retry(url, token, data=None): def _get_pull_requests_from(repo_url, owner, branch, token): - url = f"{repo_url}/pulls?head={owner}:{branch}" + url = f"{repo_url}/pulls?head={quote(owner)}:{quote(branch)}" return _exec_get_with_retry(url, token) From 2a508bdd3463aee5c3e2580f1faeeb2ba7042bb6 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 24 Jan 2024 22:42:20 +0000 Subject: [PATCH 182/264] enhance tests --- .../0_stateless/02972_to_string_nullable_timezone.reference | 1 + .../queries/0_stateless/02972_to_string_nullable_timezone.sql | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02972_to_string_nullable_timezone.reference b/tests/queries/0_stateless/02972_to_string_nullable_timezone.reference index c3514956520..d1e3a643a5f 100644 --- a/tests/queries/0_stateless/02972_to_string_nullable_timezone.reference +++ b/tests/queries/0_stateless/02972_to_string_nullable_timezone.reference @@ -1 +1,2 @@ 2022-01-01 11:13:14 +2022-01-01 11:13:14 diff --git a/tests/queries/0_stateless/02972_to_string_nullable_timezone.sql b/tests/queries/0_stateless/02972_to_string_nullable_timezone.sql index 25b787f892a..d4bdb835336 100644 --- a/tests/queries/0_stateless/02972_to_string_nullable_timezone.sql +++ b/tests/queries/0_stateless/02972_to_string_nullable_timezone.sql @@ -1,2 +1,3 @@ SET session_timezone='Europe/Amsterdam'; -SELECT toString(toDateTime('2022-01-01 12:13:14'), CAST('UTC', 'Nullable(String)')); \ No newline at end of file +SELECT toString(toDateTime('2022-01-01 12:13:14'), CAST('UTC', 'Nullable(String)')); +SELECT toString(toDateTime('2022-01-01 12:13:14'), materialize(CAST('UTC', 'Nullable(String)'))); From 42029f42e7bb84f8e6a273764d3d58f381b0b68e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 15 Jan 2024 18:54:32 +0300 Subject: [PATCH 183/264] Table CREATE DROP Poco::Logger memory leak fix --- src/Common/Exception.cpp | 10 ++++++++++ src/Common/Exception.h | 3 +++ src/Common/LoggerPtr.cpp | 15 +++++++++++++++ src/Common/LoggerPtr.h | 14 ++++++++++++++ src/Common/logger_useful.h | 14 ++++++++------ src/Processors/QueryPlan/ReadFromMergeTree.cpp | 6 +++--- src/Processors/QueryPlan/ReadFromMergeTree.h | 4 ++-- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 8 ++++---- src/Storages/MergeTree/MergeTreeData.cpp | 10 +++++----- src/Storages/MergeTree/MergeTreeData.h | 3 ++- .../MergeTree/MergeTreeDataMergerMutator.cpp | 2 +- .../MergeTree/MergeTreeDataMergerMutator.h | 2 +- .../MergeTree/MergeTreeDataSelectExecutor.h | 2 +- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataWriter.h | 7 ++++--- src/Storages/StorageMergeTree.cpp | 6 +++--- src/Storages/StorageReplicatedMergeTree.cpp | 16 ++++++++-------- 17 files changed, 85 insertions(+), 39 deletions(-) create mode 100644 src/Common/LoggerPtr.cpp create mode 100644 src/Common/LoggerPtr.h diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index e5e8cf2c818..fcf44d96b43 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -251,6 +251,11 @@ void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_ tryLogCurrentExceptionImpl(logger, start_of_message); } +void tryLogCurrentException(const LoggerPtr & logger, const std::string & start_of_message) +{ + tryLogCurrentException(logger.get(), start_of_message); +} + static void getNoSpaceLeftInfoMessage(std::filesystem::path path, String & msg) { path = std::filesystem::absolute(path); @@ -523,6 +528,11 @@ void tryLogException(std::exception_ptr e, Poco::Logger * logger, const std::str } } +void tryLogException(std::exception_ptr e, const LoggerPtr & logger, const std::string & start_of_message) +{ + tryLogException(e, logger.get(), start_of_message); +} + std::string getExceptionMessage(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace) { return getExceptionMessageAndPattern(e, with_stacktrace, check_embedded_stacktrace).text; diff --git a/src/Common/Exception.h b/src/Common/Exception.h index 6f30fde3876..77ba2c6db5b 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -10,6 +10,7 @@ #include #include #include +#include #include #include @@ -242,6 +243,7 @@ using Exceptions = std::vector; */ void tryLogCurrentException(const char * log_name, const std::string & start_of_message = ""); void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_message = ""); +void tryLogCurrentException(const LoggerPtr & logger, const std::string & start_of_message = ""); /** Prints current exception in canonical format. @@ -287,6 +289,7 @@ struct ExecutionStatus void tryLogException(std::exception_ptr e, const char * log_name, const std::string & start_of_message = ""); void tryLogException(std::exception_ptr e, Poco::Logger * logger, const std::string & start_of_message = ""); +void tryLogException(std::exception_ptr e, const LoggerPtr & logger, const std::string & start_of_message = ""); std::string getExceptionMessage(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace = false); PreformattedMessage getExceptionMessageAndPattern(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace = false); diff --git a/src/Common/LoggerPtr.cpp b/src/Common/LoggerPtr.cpp new file mode 100644 index 00000000000..33caaf81a81 --- /dev/null +++ b/src/Common/LoggerPtr.cpp @@ -0,0 +1,15 @@ +#include + +struct LoggerDeleter +{ + void operator()(const Poco::Logger * logger) + { + Poco::Logger::destroy(logger->name()); + } +}; + +LoggerPtr getLogger(const std::string & name) +{ + Poco::Logger * logger_raw_ptr = &Poco::Logger::get(name); + return std::shared_ptr(logger_raw_ptr, LoggerDeleter()); +} diff --git a/src/Common/LoggerPtr.h b/src/Common/LoggerPtr.h new file mode 100644 index 00000000000..cd0529ea258 --- /dev/null +++ b/src/Common/LoggerPtr.h @@ -0,0 +1,14 @@ +#pragma once + +#include + +namespace Poco +{ + +class Logger; + +} + +using LoggerPtr = std::shared_ptr; + +LoggerPtr getLogger(const std::string & name); diff --git a/src/Common/logger_useful.h b/src/Common/logger_useful.h index d9fe5ac9190..fda374befc9 100644 --- a/src/Common/logger_useful.h +++ b/src/Common/logger_useful.h @@ -8,6 +8,7 @@ #include #include #include +#include namespace Poco { class Logger; } @@ -19,11 +20,12 @@ using LogSeriesLimiterPtr = std::shared_ptr; namespace { - [[maybe_unused]] const ::Poco::Logger * getLogger(const ::Poco::Logger * logger) { return logger; } - [[maybe_unused]] const ::Poco::Logger * getLogger(const std::atomic<::Poco::Logger *> & logger) { return logger.load(); } - [[maybe_unused]] std::unique_ptr getLogger(std::unique_ptr && logger) { return logger; } - [[maybe_unused]] std::unique_ptr getLogger(std::unique_ptr && logger) { return logger; } - [[maybe_unused]] LogSeriesLimiterPtr getLogger(LogSeriesLimiterPtr & logger) { return logger; } + [[maybe_unused]] const ::Poco::Logger * getLoggerHelper(const LoggerPtr & logger) { return logger.get(); } + [[maybe_unused]] const ::Poco::Logger * getLoggerHelper(const ::Poco::Logger * logger) { return logger; } + [[maybe_unused]] const ::Poco::Logger * getLoggerHelper(const std::atomic<::Poco::Logger *> & logger) { return logger.load(); } + [[maybe_unused]] std::unique_ptr getLoggerHelper(std::unique_ptr && logger) { return logger; } + [[maybe_unused]] std::unique_ptr getLoggerHelper(std::unique_ptr && logger) { return logger; } + [[maybe_unused]] LogSeriesLimiterPtr getLoggerHelper(LogSeriesLimiterPtr & logger) { return logger; } } #define LOG_IMPL_FIRST_ARG(X, ...) X @@ -62,7 +64,7 @@ namespace #define LOG_IMPL(logger, priority, PRIORITY, ...) do \ { \ - auto _logger = ::getLogger(logger); \ + auto _logger = ::getLoggerHelper(logger); \ const bool _is_clients_log = (DB::CurrentThread::getGroup() != nullptr) && \ (DB::CurrentThread::get().getClientLogsLevel() >= (priority)); \ if (!_is_clients_log && !_logger->is((PRIORITY))) \ diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index d02e387afc3..62d4538cb84 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -247,7 +247,7 @@ ReadFromMergeTree::ReadFromMergeTree( size_t num_streams_, bool sample_factor_column_queried_, std::shared_ptr max_block_numbers_to_read_, - Poco::Logger * log_, + LoggerPtr log_, AnalysisResultPtr analyzed_result_ptr_, bool enable_parallel_reading) : SourceStepWithFilter(DataStream{.header = MergeTreeSelectProcessor::transformHeader( @@ -274,7 +274,7 @@ ReadFromMergeTree::ReadFromMergeTree( , requested_num_streams(num_streams_) , sample_factor_column_queried(sample_factor_column_queried_) , max_block_numbers_to_read(std::move(max_block_numbers_to_read_)) - , log(log_) + , log(std::move(log_)) , analyzed_result_ptr(analyzed_result_ptr_) , is_parallel_reading_from_replicas(enable_parallel_reading) { @@ -1281,7 +1281,7 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( data, real_column_names, sample_factor_column_queried, - log, + log.get(), indexes); } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index aed2a270ca1..3f2ac7ea931 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -120,7 +120,7 @@ public: size_t num_streams_, bool sample_factor_column_queried_, std::shared_ptr max_block_numbers_to_read_, - Poco::Logger * log_, + LoggerPtr log_, AnalysisResultPtr analyzed_result_ptr_, bool enable_parallel_reading); @@ -259,7 +259,7 @@ private: /// Pre-computed value, needed to trigger sets creating for PK mutable std::optional indexes; - Poco::Logger * log; + LoggerPtr log; UInt64 selected_parts = 0; UInt64 selected_rows = 0; UInt64 selected_marks = 0; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 87f23b0da2a..0c1bcff2c50 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1663,7 +1663,7 @@ try metadata_manager->deleteAll(true); metadata_manager->assertAllDeleted(true); - getDataPartStorage().rename(to.parent_path(), to.filename(), storage.log, remove_new_dir_if_exists, fsync_dir); + getDataPartStorage().rename(to.parent_path(), to.filename(), storage.log.get(), remove_new_dir_if_exists, fsync_dir); metadata_manager->updateAll(true); auto new_projection_root_path = to.string(); @@ -1758,7 +1758,7 @@ void IMergeTreeDataPart::remove() } bool is_temporary_part = is_temp || state == MergeTreeDataPartState::Temporary; - getDataPartStorage().remove(std::move(can_remove_callback), checksums, projection_checksums, is_temporary_part, storage.log); + getDataPartStorage().remove(std::move(can_remove_callback), checksums, projection_checksums, is_temporary_part, storage.log.get()); } std::optional IMergeTreeDataPart::getRelativePathForPrefix(const String & prefix, bool detached, bool broken) const @@ -1775,7 +1775,7 @@ std::optional IMergeTreeDataPart::getRelativePathForPrefix(const String if (detached && parent_part) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot detach projection"); - return getDataPartStorage().getRelativePathForPrefix(storage.log, prefix, detached, broken); + return getDataPartStorage().getRelativePathForPrefix(storage.log.get(), prefix, detached, broken); } std::optional IMergeTreeDataPart::getRelativePathForDetachedPart(const String & prefix, bool broken) const @@ -1841,7 +1841,7 @@ MutableDataPartStoragePtr IMergeTreeDataPart::makeCloneOnDisk( throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not clone data part {} to empty directory.", name); String path_to_clone = fs::path(storage.relative_data_path) / directory_name / ""; - return getDataPartStorage().clonePart(path_to_clone, getDataPartStorage().getPartDirectory(), disk, read_settings, write_settings, storage.log, cancellation_hook); + return getDataPartStorage().clonePart(path_to_clone, getDataPartStorage().getPartDirectory(), disk, read_settings, write_settings, storage.log.get(), cancellation_hook); } UInt64 IMergeTreeDataPart::getIndexSizeFromFile() const diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 61332a4ff38..ef18a1d46a8 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -355,7 +355,7 @@ MergeTreeData::MergeTreeData( , require_part_metadata(require_part_metadata_) , broken_part_callback(broken_part_callback_) , log_name(std::make_shared(table_id_.getNameForLogs())) - , log(&Poco::Logger::get(*log_name)) + , log(getLogger(*log_name)) , storage_settings(std::move(storage_settings_)) , pinned_part_uuids(std::make_shared()) , data_parts_by_info(data_parts_indexes.get()) @@ -1222,7 +1222,7 @@ MergeTreeData::PartLoadingTree::build(PartLoadingInfos nodes) } static std::optional calculatePartSizeSafe( - const MergeTreeData::DataPartPtr & part, Poco::Logger * log) + const MergeTreeData::DataPartPtr & part, const LoggerPtr & log) { try { @@ -2114,7 +2114,7 @@ size_t MergeTreeData::clearOldTemporaryDirectories(const String & root_path, siz { /// Actually we don't rely on temporary_directories_lifetime when removing old temporaries directories, /// it's just an extra level of protection just in case we have a bug. - LOG_INFO(LogFrequencyLimiter(log, 10), "{} is in use (by merge/mutation/INSERT) (consider increasing temporary_directories_lifetime setting)", full_path); + LOG_INFO(LogFrequencyLimiter(log.get(), 10), "{} is in use (by merge/mutation/INSERT) (consider increasing temporary_directories_lifetime setting)", full_path); continue; } else if (!disk->exists(it->path())) @@ -2735,7 +2735,7 @@ void MergeTreeData::renameInMemory(const StorageID & new_table_id) { IStorage::renameInMemory(new_table_id); std::atomic_store(&log_name, std::make_shared(new_table_id.getNameForLogs())); - log = &Poco::Logger::get(*log_name); + log = getLogger(*log_name); } void MergeTreeData::dropAllData() @@ -8181,7 +8181,7 @@ ReservationPtr MergeTreeData::balancedReservation( } // Record submerging big parts in the tagger to clean them up. - tagger_ptr->emplace(*this, part_name, std::move(covered_parts), log); + tagger_ptr->emplace(*this, part_name, std::move(covered_parts), log.get()); } } } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index f0dbaf0e307..7ac46e12e1a 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -1117,7 +1118,7 @@ protected: /// log_name will change during table RENAME. Use atomic_shared_ptr to allow concurrent RW. /// NOTE clang-14 doesn't have atomic_shared_ptr yet. Use std::atomic* operations for now. std::shared_ptr log_name; - std::atomic log; + LoggerPtr log; /// Storage settings. /// Use get and set to receive readonly versions. diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 8c03aef6f99..58fddde7b54 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -66,7 +66,7 @@ static const double DISK_USAGE_COEFFICIENT_TO_SELECT = 2; static const double DISK_USAGE_COEFFICIENT_TO_RESERVE = 1.1; MergeTreeDataMergerMutator::MergeTreeDataMergerMutator(MergeTreeData & data_) - : data(data_), log(&Poco::Logger::get(data.getLogName() + " (MergerMutator)")) + : data(data_), log(getLogger(data.getLogName() + " (MergerMutator)")) { } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 6eab0ee0c37..f3a3f51b6c3 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -213,7 +213,7 @@ public : private: MergeTreeData & data; - Poco::Logger * log; + LoggerPtr log; /// When the last time you wrote to the log that the disk space was running out (not to write about this too often). time_t disk_space_warning_time = 0; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index ba1f20054f0..93eac427f77 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -75,7 +75,7 @@ public: private: const MergeTreeData & data; - Poco::Logger * log; + LoggerPtr log; /// Get the approximate value (bottom estimate - only by full marks) of the number of rows falling under the index. static size_t getApproximateTotalRowsToRead( diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 3c0b2d2b42e..25785b8aea0 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -591,7 +591,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( auto projection_block = projection.calculate(block, context); if (projection_block.rows()) { - auto proj_temp_part = writeProjectionPart(data, log, projection_block, projection, new_data_part.get()); + auto proj_temp_part = writeProjectionPart(data, log.get(), projection_block, projection, new_data_part.get()); new_data_part->addProjectionPart(projection.name, std::move(proj_temp_part.part)); for (auto & stream : proj_temp_part.streams) temp_part.streams.emplace_back(std::move(stream)); diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.h b/src/Storages/MergeTree/MergeTreeDataWriter.h index 2fb6b1f22d4..aaa0f71eccf 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.h +++ b/src/Storages/MergeTree/MergeTreeDataWriter.h @@ -45,8 +45,9 @@ class MergeTreeDataWriter public: explicit MergeTreeDataWriter(MergeTreeData & data_) : data(data_) - , log(&Poco::Logger::get(data.getLogName() + " (Writer)")) - {} + , log(getLogger(data.getLogName() + " (Writer)")) + { + } /** Split the block to blocks, each of them must be written as separate part. * (split rows by partition) @@ -131,7 +132,7 @@ private: const ProjectionDescription & projection); MergeTreeData & data; - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index fbdde15c2af..ab2f7ea2989 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -683,7 +683,7 @@ std::optional StorageMergeTree::getIncompleteMutationsS const auto & mutation_entry = current_mutation_it->second; - auto txn = tryGetTransactionForMutation(mutation_entry, log); + auto txn = tryGetTransactionForMutation(mutation_entry, log.get()); /// There's no way a transaction may finish before a mutation that was started by the transaction. /// But sometimes we need to check status of an unrelated mutation, in this case we don't care about transactions. assert(txn || mutation_entry.tid.isPrehistoric() || from_another_mutation); @@ -829,7 +829,7 @@ CancellationCode StorageMergeTree::killMutation(const String & mutation_id) if (!to_kill) return CancellationCode::NotFound; - if (auto txn = tryGetTransactionForMutation(*to_kill, log)) + if (auto txn = tryGetTransactionForMutation(*to_kill, log.get())) { LOG_TRACE(log, "Cancelling transaction {} which had started mutation {}", to_kill->tid, mutation_id); TransactionLog::instance().rollbackTransaction(txn); @@ -1222,7 +1222,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate( if (!part->version.isVisible(first_mutation_tid.start_csn, first_mutation_tid)) continue; - txn = tryGetTransactionForMutation(mutations_begin_it->second, log); + txn = tryGetTransactionForMutation(mutations_begin_it->second, log.get()); if (!txn) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find transaction {} that has started mutation {} " "that is going to be applied to part {}", diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 715cbab9eea..1e86f3f70f5 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -320,7 +320,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( attach, [this] (const std::string & name) { enqueuePartForCheck(name); }) , zookeeper_name(zkutil::extractZooKeeperName(zookeeper_path_)) - , zookeeper_path(zkutil::extractZooKeeperPath(zookeeper_path_, /* check_starts_with_slash */ !attach, log)) + , zookeeper_path(zkutil::extractZooKeeperPath(zookeeper_path_, /* check_starts_with_slash */ !attach, log.get())) , replica_name(replica_name_) , replica_path(fs::path(zookeeper_path) / "replicas" / replica_name_) , reader(*this) @@ -812,7 +812,7 @@ bool StorageReplicatedMergeTree::createTableIfNotExists(const StorageMetadataPtr else { auto metadata_drop_lock = zkutil::EphemeralNodeHolder::existing(drop_lock_path, *zookeeper); - if (!removeTableNodesFromZooKeeper(zookeeper, zookeeper_path, metadata_drop_lock, log)) + if (!removeTableNodesFromZooKeeper(zookeeper, zookeeper_path, metadata_drop_lock, log.get())) { /// Someone is recursively removing table right now, we cannot create new table until old one is removed continue; @@ -1128,7 +1128,7 @@ void StorageReplicatedMergeTree::drop() if (lost_part_count > 0) LOG_INFO(log, "Dropping table with non-zero lost_part_count equal to {}", lost_part_count); } - dropReplica(zookeeper, zookeeper_path, replica_name, log, getSettings(), &has_metadata_in_zookeeper); + dropReplica(zookeeper, zookeeper_path, replica_name, log.get(), getSettings(), &has_metadata_in_zookeeper); } } @@ -4181,7 +4181,7 @@ void StorageReplicatedMergeTree::startBeingLeader() return; } - zkutil::checkNoOldLeaders(log, *zookeeper, fs::path(zookeeper_path) / "leader_election"); + zkutil::checkNoOldLeaders(log.get(), *zookeeper, fs::path(zookeeper_path) / "leader_election"); LOG_INFO(log, "Became leader"); is_leader = true; @@ -4275,7 +4275,7 @@ void StorageReplicatedMergeTree::waitForUniquePartsToBeFetchedByOtherReplicas(St auto zookeeper = getZooKeeperIfTableShutDown(); - auto unique_parts_set = findReplicaUniqueParts(replica_name, zookeeper_path, format_version, zookeeper, log); + auto unique_parts_set = findReplicaUniqueParts(replica_name, zookeeper_path, format_version, zookeeper, log.get()); if (unique_parts_set.empty()) { LOG_INFO(log, "Will not wait for unique parts to be fetched because we don't have any unique parts"); @@ -9348,7 +9348,7 @@ StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & part, co return unlockSharedDataByID( part.getUniqueId(), shared_id, part.info, replica_name, - part.getDataPartStorage().getDiskType(), zookeeper, *getSettings(), log, zookeeper_path, format_version); + part.getDataPartStorage().getDiskType(), zookeeper, *getSettings(), log.get(), zookeeper_path, format_version); } namespace @@ -10301,7 +10301,7 @@ void StorageReplicatedMergeTree::backupData( bool exists = false; Strings mutation_ids; { - ZooKeeperRetriesControl retries_ctl("getMutations", log, zookeeper_retries_info, nullptr); + ZooKeeperRetriesControl retries_ctl("getMutations", log.get(), zookeeper_retries_info, nullptr); retries_ctl.retryLoop([&]() { if (!zookeeper || zookeeper->expired()) @@ -10320,7 +10320,7 @@ void StorageReplicatedMergeTree::backupData( bool mutation_id_exists = false; String mutation; - ZooKeeperRetriesControl retries_ctl("getMutation", log, zookeeper_retries_info, nullptr); + ZooKeeperRetriesControl retries_ctl("getMutation", log.get(), zookeeper_retries_info, nullptr); retries_ctl.retryLoop([&]() { if (!zookeeper || zookeeper->expired()) From bd8bbdaeb31dfc388a13ffea25b401ad899676ea Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 23 Jan 2024 20:00:11 +0300 Subject: [PATCH 184/264] Poco update RefCountedObject with std::atomic --- .../include/Poco/RefCountedObject.h | 28 +++++++++++-------- 1 file changed, 16 insertions(+), 12 deletions(-) diff --git a/base/poco/Foundation/include/Poco/RefCountedObject.h b/base/poco/Foundation/include/Poco/RefCountedObject.h index 4ad32e30cad..db966089e00 100644 --- a/base/poco/Foundation/include/Poco/RefCountedObject.h +++ b/base/poco/Foundation/include/Poco/RefCountedObject.h @@ -38,15 +38,15 @@ public: /// Creates the RefCountedObject. /// The initial reference count is one. - void duplicate() const; - /// Increments the object's reference count. + size_t duplicate() const; + /// Increments the object's reference count, returns reference count before call. - void release() const throw(); + size_t release() const throw(); /// Decrements the object's reference count /// and deletes the object if the count - /// reaches zero. + /// reaches zero, returns reference count before call. - int referenceCount() const; + size_t referenceCount() const; /// Returns the reference count. protected: @@ -57,36 +57,40 @@ private: RefCountedObject(const RefCountedObject &); RefCountedObject & operator=(const RefCountedObject &); - mutable AtomicCounter _counter; + mutable std::atomic _counter; }; // // inlines // -inline int RefCountedObject::referenceCount() const +inline size_t RefCountedObject::referenceCount() const { - return _counter.value(); + return _counter.load(std::memory_order_acquire); } -inline void RefCountedObject::duplicate() const +inline size_t RefCountedObject::duplicate() const { - ++_counter; + return _counter.fetch_add(1, std::memory_order_acq_rel); } -inline void RefCountedObject::release() const throw() +inline size_t RefCountedObject::release() const throw() { + size_t reference_count_before = _counter.fetch_sub(1, std::memory_order_acq_rel); + try { - if (--_counter == 0) + if (reference_count_before == 1) delete this; } catch (...) { poco_unexpected(); } + + return reference_count_before; } From 6f698b40ae58aef2fc5e474ac081758a3078619c Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 23 Jan 2024 20:03:51 +0300 Subject: [PATCH 185/264] Poco update Logger to allow create shared LoggerPtr --- base/poco/Foundation/include/Poco/Logger.h | 15 ++- base/poco/Foundation/src/Logger.cpp | 111 +++++++++++++++++---- 2 files changed, 105 insertions(+), 21 deletions(-) diff --git a/base/poco/Foundation/include/Poco/Logger.h b/base/poco/Foundation/include/Poco/Logger.h index ffe3766dfec..79d71c506d5 100644 --- a/base/poco/Foundation/include/Poco/Logger.h +++ b/base/poco/Foundation/include/Poco/Logger.h @@ -33,7 +33,8 @@ namespace Poco class Exception; - +class Logger; +using LoggerPtr = std::shared_ptr; class Foundation_API Logger : public Channel /// Logger is a special Channel that acts as the main @@ -870,6 +871,11 @@ public: /// If the Logger does not yet exist, it is created, based /// on its parent logger. + static LoggerPtr getShared(const std::string & name); + /// Returns a shared pointer to the Logger with the given name. + /// If the Logger does not yet exist, it is created, based + /// on its parent logger. + static Logger & unsafeGet(const std::string & name); /// Returns a reference to the Logger with the given name. /// If the Logger does not yet exist, it is created, based @@ -885,6 +891,11 @@ public: /// given name. The Logger's Channel and log level as set as /// specified. + static LoggerPtr createShared(const std::string & name, Channel * pChannel, int level = Message::PRIO_INFORMATION); + /// Creates and returns a shared pointer to a Logger with the + /// given name. The Logger's Channel and log level as set as + /// specified. + static Logger & root(); /// Returns a reference to the root logger, which is the ultimate /// ancestor of all Loggers. @@ -938,6 +949,7 @@ protected: void log(const std::string & text, Message::Priority prio, const char * file, int line); static std::string format(const std::string & fmt, int argc, std::string argv[]); + static Logger & unsafeCreate(const std::string & name, Channel * pChannel, int level = Message::PRIO_INFORMATION); static Logger & parent(const std::string & name); static void add(Logger * pLogger); static Logger * find(const std::string & name); @@ -952,7 +964,6 @@ private: std::atomic_int _level; static LoggerMap * _pLoggerMap; - static Mutex _mapMtx; }; diff --git a/base/poco/Foundation/src/Logger.cpp b/base/poco/Foundation/src/Logger.cpp index 3d5de585b4f..7b562217268 100644 --- a/base/poco/Foundation/src/Logger.cpp +++ b/base/poco/Foundation/src/Logger.cpp @@ -20,12 +20,43 @@ #include "Poco/NumberParser.h" #include "Poco/String.h" +#include + +namespace +{ + +std::atomic is_logging_infrastructure_destroyed{false}; + +class LoggerMutex +{ +public: + void lock() + { + mutex.lock(); + } + + void unlock() + { + mutex.unlock(); + } + + ~LoggerMutex() + { + is_logging_infrastructure_destroyed.store(true, std::memory_order_release); + } + +private: + std::mutex mutex; +}; + +LoggerMutex logger_mutex; + +} namespace Poco { Logger::LoggerMap* Logger::_pLoggerMap = 0; -Mutex Logger::_mapMtx; const std::string Logger::ROOT; @@ -73,7 +104,7 @@ void Logger::setProperty(const std::string& name, const std::string& value) setChannel(LoggingRegistry::defaultRegistry().channelForName(value)); else if (name == "level") setLevel(value); - else + else Channel::setProperty(name, value); } @@ -112,14 +143,14 @@ void Logger::dump(const std::string& msg, const void* buffer, std::size_t length void Logger::setLevel(const std::string& name, int level) { - Mutex::ScopedLock lock(_mapMtx); + std::lock_guard lock(logger_mutex); if (_pLoggerMap) { std::string::size_type len = name.length(); for (LoggerMap::iterator it = _pLoggerMap->begin(); it != _pLoggerMap->end(); ++it) { - if (len == 0 || + if (len == 0 || (it->first.compare(0, len, name) == 0 && (it->first.length() == len || it->first[len] == '.'))) { it->second->setLevel(level); @@ -131,7 +162,7 @@ void Logger::setLevel(const std::string& name, int level) void Logger::setChannel(const std::string& name, Channel* pChannel) { - Mutex::ScopedLock lock(_mapMtx); + std::lock_guard lock(logger_mutex); if (_pLoggerMap) { @@ -150,7 +181,7 @@ void Logger::setChannel(const std::string& name, Channel* pChannel) void Logger::setProperty(const std::string& loggerName, const std::string& propertyName, const std::string& value) { - Mutex::ScopedLock lock(_mapMtx); + std::lock_guard lock(logger_mutex); if (_pLoggerMap) { @@ -280,13 +311,44 @@ void Logger::formatDump(std::string& message, const void* buffer, std::size_t le } +namespace +{ + +struct LoggerDeleter +{ + void operator()(Poco::Logger * logger) + { + if (is_logging_infrastructure_destroyed.load(std::memory_order_acquire)) + { + logger->release(); + return; + } + + Logger::destroy(logger->name()); + } +}; + +inline LoggerPtr makeLoggerPtr(Logger & logger) +{ + logger.duplicate(); + return std::shared_ptr(&logger, LoggerDeleter()); +} + +} + Logger& Logger::get(const std::string& name) { - Mutex::ScopedLock lock(_mapMtx); + std::lock_guard lock(logger_mutex); return unsafeGet(name); } +LoggerPtr Logger::getShared(const std::string & name) +{ + std::lock_guard lock(logger_mutex); + + return makeLoggerPtr(unsafeGet(name)); +} Logger& Logger::unsafeGet(const std::string& name) { @@ -310,18 +372,21 @@ Logger& Logger::unsafeGet(const std::string& name) Logger& Logger::create(const std::string& name, Channel* pChannel, int level) { - Mutex::ScopedLock lock(_mapMtx); + std::lock_guard lock(logger_mutex); - if (find(name)) throw ExistsException(); - Logger* pLogger = new Logger(name, pChannel, level); - add(pLogger); - return *pLogger; + return unsafeCreate(name, pChannel, level); } +LoggerPtr Logger::createShared(const std::string & name, Channel * pChannel, int level) +{ + std::lock_guard lock(logger_mutex); + + return makeLoggerPtr(unsafeCreate(name, pChannel, level)); +} Logger& Logger::root() { - Mutex::ScopedLock lock(_mapMtx); + std::lock_guard lock(logger_mutex); return unsafeGet(ROOT); } @@ -329,7 +394,7 @@ Logger& Logger::root() Logger* Logger::has(const std::string& name) { - Mutex::ScopedLock lock(_mapMtx); + std::lock_guard lock(logger_mutex); return find(name); } @@ -337,7 +402,7 @@ Logger* Logger::has(const std::string& name) void Logger::shutdown() { - Mutex::ScopedLock lock(_mapMtx); + std::lock_guard lock(logger_mutex); if (_pLoggerMap) { @@ -365,15 +430,15 @@ Logger* Logger::find(const std::string& name) void Logger::destroy(const std::string& name) { - Mutex::ScopedLock lock(_mapMtx); + std::lock_guard lock(logger_mutex); if (_pLoggerMap) { LoggerMap::iterator it = _pLoggerMap->find(name); if (it != _pLoggerMap->end()) { - it->second->release(); - _pLoggerMap->erase(it); + if (it->second->release() == 1) + _pLoggerMap->erase(it); } } } @@ -381,7 +446,7 @@ void Logger::destroy(const std::string& name) void Logger::names(std::vector& names) { - Mutex::ScopedLock lock(_mapMtx); + std::lock_guard lock(logger_mutex); names.clear(); if (_pLoggerMap) @@ -393,6 +458,14 @@ void Logger::names(std::vector& names) } } +Logger& Logger::unsafeCreate(const std::string & name, Channel * pChannel, int level) +{ + if (find(name)) throw ExistsException(); + Logger* pLogger = new Logger(name, pChannel, level); + add(pLogger); + + return *pLogger; +} Logger& Logger::parent(const std::string& name) { From 2a327107b637ef4db2bd403e44ab953ec02595f9 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 23 Jan 2024 20:04:50 +0300 Subject: [PATCH 186/264] Updated implementation --- programs/copier/ClusterCopier.h | 4 +- programs/copier/ZooKeeperStaff.h | 2 +- programs/keeper-client/KeeperClient.cpp | 2 +- programs/keeper-converter/KeeperConverter.cpp | 2 +- programs/keeper/Keeper.cpp | 2 +- .../CatBoostLibraryHandlerFactory.cpp | 2 +- .../CatBoostLibraryHandlerFactory.h | 2 +- .../ExternalDictionaryLibraryAPI.cpp | 34 ++++++------- ...xternalDictionaryLibraryHandlerFactory.cpp | 2 +- .../LibraryBridgeHandlerFactory.cpp | 2 +- .../LibraryBridgeHandlerFactory.h | 2 +- .../library-bridge/LibraryBridgeHandlers.cpp | 10 ++-- .../library-bridge/LibraryBridgeHandlers.h | 8 +-- programs/local/LocalServer.cpp | 4 +- programs/odbc-bridge/ColumnInfoHandler.h | 4 +- programs/odbc-bridge/IdentifierQuoteHandler.h | 4 +- programs/odbc-bridge/MainHandler.h | 4 +- programs/odbc-bridge/ODBCBlockInputStream.cpp | 2 +- programs/odbc-bridge/ODBCBlockInputStream.h | 2 +- .../odbc-bridge/ODBCBlockOutputStream.cpp | 2 +- programs/odbc-bridge/ODBCBlockOutputStream.h | 2 +- programs/odbc-bridge/ODBCHandlerFactory.cpp | 2 +- programs/odbc-bridge/ODBCHandlerFactory.h | 2 +- .../odbc-bridge/ODBCPooledConnectionFactory.h | 2 +- programs/odbc-bridge/SchemaAllowedHandler.h | 4 +- programs/odbc-bridge/getIdentifierQuote.cpp | 2 +- programs/server/Server.cpp | 8 +-- src/Access/AccessControl.cpp | 2 +- src/Access/AccessRights.cpp | 4 +- src/Access/Common/AllowedClientHosts.cpp | 4 +- src/Access/ContextAccess.cpp | 2 +- src/Access/ContextAccess.h | 3 +- src/Access/DiskAccessStorage.cpp | 6 +-- src/Access/ExternalAuthenticators.cpp | 2 +- src/Access/ExternalAuthenticators.h | 2 +- src/Access/GSSAcceptor.cpp | 4 +- src/Access/GSSAcceptor.h | 3 +- src/Access/IAccessStorage.cpp | 14 +++--- src/Access/IAccessStorage.h | 9 ++-- src/Access/KerberosInit.cpp | 2 +- src/Access/LDAPClient.cpp | 2 +- src/Access/RowPolicyCache.cpp | 2 +- src/Access/SettingsAuthResponseParser.cpp | 2 +- src/Backups/BackupCoordinationLocal.cpp | 2 +- src/Backups/BackupCoordinationLocal.h | 2 +- src/Backups/BackupCoordinationRemote.cpp | 2 +- src/Backups/BackupCoordinationRemote.h | 2 +- src/Backups/BackupCoordinationStageSync.cpp | 2 +- src/Backups/BackupCoordinationStageSync.h | 4 +- src/Backups/BackupEntriesCollector.cpp | 2 +- src/Backups/BackupEntriesCollector.h | 2 +- src/Backups/BackupFileInfo.cpp | 6 +-- src/Backups/BackupFileInfo.h | 5 +- src/Backups/BackupIO_Default.cpp | 4 +- src/Backups/BackupIO_Default.h | 8 +-- src/Backups/BackupIO_Disk.cpp | 4 +- src/Backups/BackupIO_File.cpp | 4 +- src/Backups/BackupIO_S3.cpp | 4 +- src/Backups/BackupImpl.cpp | 4 +- src/Backups/BackupImpl.h | 2 +- src/Backups/BackupsWorker.cpp | 2 +- src/Backups/BackupsWorker.h | 2 +- src/Backups/RestoreCoordinationLocal.cpp | 2 +- src/Backups/RestoreCoordinationLocal.h | 2 +- src/Backups/RestoreCoordinationRemote.cpp | 2 +- src/Backups/RestoreCoordinationRemote.h | 2 +- src/Backups/RestorerFromBackup.cpp | 2 +- src/Backups/RestorerFromBackup.h | 2 +- src/Backups/WithRetries.cpp | 2 +- src/Backups/WithRetries.h | 4 +- src/BridgeHelper/IBridgeHelper.h | 2 +- src/BridgeHelper/LibraryBridgeHelper.cpp | 2 +- src/BridgeHelper/LibraryBridgeHelper.h | 4 +- src/BridgeHelper/XDBCBridgeHelper.h | 6 +-- src/Client/Connection.h | 11 ++-- src/Client/ConnectionEstablisher.cpp | 4 +- src/Client/ConnectionEstablisher.h | 6 +-- src/Client/ConnectionPool.h | 2 +- src/Client/ConnectionPoolWithFailover.cpp | 2 +- src/Client/HedgedConnectionsFactory.cpp | 2 +- src/Client/HedgedConnectionsFactory.h | 2 +- src/Common/Allocator.cpp | 2 +- src/Common/AsyncLoader.cpp | 4 +- src/Common/AsyncLoader.h | 5 +- src/Common/AsynchronousMetrics.cpp | 6 +-- src/Common/AsynchronousMetrics.h | 2 +- src/Common/Config/ConfigProcessor.cpp | 12 ++--- src/Common/Config/ConfigProcessor.h | 6 +-- src/Common/Config/ConfigReloader.h | 2 +- src/Common/DNSResolver.cpp | 4 +- src/Common/DNSResolver.h | 2 +- .../EnvironmentProxyConfigurationResolver.cpp | 2 +- src/Common/ErrorHandlers.h | 2 +- src/Common/Exception.cpp | 14 ++---- src/Common/Exception.h | 10 ++-- src/Common/FileChecker.cpp | 2 +- src/Common/FileChecker.h | 3 +- src/Common/FrequencyHolder.cpp | 6 +-- src/Common/Jemalloc.cpp | 10 ++-- src/Common/LRUResourceCache.h | 8 +-- src/Common/Logger.cpp | 27 ++++++++++ src/Common/Logger.h | 50 +++++++++++++++++++ src/Common/LoggerPtr.cpp | 15 ------ src/Common/LoggerPtr.h | 14 ------ src/Common/LoggingFormatStringHelpers.cpp | 4 +- src/Common/LoggingFormatStringHelpers.h | 18 +++---- src/Common/Macros.cpp | 4 ++ src/Common/Macros.h | 1 + src/Common/MemoryTracker.cpp | 8 +-- .../NamedCollections/NamedCollectionUtils.cpp | 6 +-- src/Common/NetlinkMetricsProvider.cpp | 2 +- src/Common/OptimizedRegularExpression.cpp | 2 +- src/Common/PipeFDs.cpp | 2 +- src/Common/PoolBase.h | 4 +- src/Common/PoolWithFailoverBase.h | 4 +- .../ProxyConfigurationResolverProvider.cpp | 4 +- src/Common/ProxyListConfigurationResolver.cpp | 2 +- src/Common/QueryProfiler.cpp | 4 +- src/Common/QueryProfiler.h | 6 ++- .../RemoteProxyConfigurationResolver.cpp | 2 +- src/Common/SensitiveDataMasker.cpp | 2 +- src/Common/ShellCommand.cpp | 4 +- src/Common/ShellCommand.h | 2 +- src/Common/StatusFile.cpp | 8 +-- src/Common/SystemLogBase.cpp | 2 +- src/Common/SystemLogBase.h | 2 +- src/Common/TLDListsHolder.cpp | 2 +- src/Common/ThreadProfileEvents.cpp | 18 +++---- src/Common/ThreadStatus.cpp | 2 +- src/Common/ThreadStatus.h | 2 +- src/Common/ZooKeeper/ZooKeeper.cpp | 6 +-- src/Common/ZooKeeper/ZooKeeper.h | 8 +-- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 4 +- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 2 +- src/Common/ZooKeeper/ZooKeeperImpl.h | 2 +- src/Common/ZooKeeper/ZooKeeperLock.cpp | 2 +- src/Common/ZooKeeper/ZooKeeperLock.h | 2 +- .../ZooKeeper/ZooKeeperWithFaultInjection.cpp | 2 +- .../ZooKeeper/ZooKeeperWithFaultInjection.h | 6 +-- src/Common/logger_useful.h | 3 +- src/Common/makeSocketAddress.cpp | 5 ++ src/Common/makeSocketAddress.h | 4 ++ src/Common/mysqlxx/Query.cpp | 2 +- src/Common/mysqlxx/mysqlxx/Pool.h | 2 +- src/Common/parseRemoteDescription.cpp | 2 +- src/Common/tests/gtest_log.cpp | 8 +-- src/Common/tests/gtest_poolbase.cpp | 2 +- .../CompressionCodecDeflateQpl.cpp | 4 +- src/Compression/CompressionCodecDeflateQpl.h | 2 +- src/Compression/CompressionCodecEncrypted.cpp | 2 +- src/Compression/CompressionCodecZSTDQAT.cpp | 4 +- src/Coordination/Changelog.cpp | 8 +-- src/Coordination/Changelog.h | 4 +- src/Coordination/FourLetterCommand.cpp | 2 +- src/Coordination/KeeperContext.cpp | 6 +-- src/Coordination/KeeperDispatcher.cpp | 2 +- src/Coordination/KeeperDispatcher.h | 2 +- src/Coordination/KeeperFeatureFlags.cpp | 2 +- src/Coordination/KeeperFeatureFlags.h | 2 +- src/Coordination/KeeperLogStore.cpp | 2 +- src/Coordination/KeeperLogStore.h | 2 +- src/Coordination/KeeperServer.cpp | 4 +- src/Coordination/KeeperServer.h | 2 +- src/Coordination/KeeperSnapshotManager.cpp | 8 +-- src/Coordination/KeeperSnapshotManager.h | 2 +- src/Coordination/KeeperSnapshotManagerS3.cpp | 2 +- src/Coordination/KeeperSnapshotManagerS3.h | 2 +- src/Coordination/KeeperStateMachine.cpp | 6 +-- src/Coordination/KeeperStateMachine.h | 2 +- src/Coordination/KeeperStateManager.cpp | 6 +-- src/Coordination/KeeperStateManager.h | 2 +- src/Coordination/KeeperStorage.cpp | 6 +-- src/Coordination/LoggerWrapper.h | 4 +- src/Coordination/ZooKeeperDataReader.cpp | 12 ++--- src/Coordination/ZooKeeperDataReader.h | 8 +-- src/Coordination/tests/gtest_coordination.cpp | 6 +-- src/Core/BackgroundSchedulePool.cpp | 10 ++-- src/Core/BaseSettings.cpp | 3 +- src/Core/MySQL/Authentication.cpp | 2 +- src/Core/MySQL/Authentication.h | 4 +- src/Core/PostgreSQL/Connection.cpp | 4 +- src/Core/PostgreSQL/Connection.h | 3 +- src/Core/PostgreSQL/PoolWithFailover.cpp | 6 +-- src/Core/PostgreSQL/PoolWithFailover.h | 2 +- src/Core/PostgreSQLProtocol.h | 2 +- src/Core/ServerUUID.h | 6 +-- src/Core/SettingsQuirks.cpp | 4 +- src/Core/SettingsQuirks.h | 7 +-- src/Core/SortDescription.cpp | 5 +- src/Daemon/BaseDaemon.cpp | 4 +- src/Daemon/SentryWriter.cpp | 4 +- src/Databases/DatabaseDictionary.cpp | 2 +- src/Databases/DatabaseDictionary.h | 2 +- src/Databases/DatabaseFilesystem.cpp | 2 +- src/Databases/DatabaseFilesystem.h | 2 +- src/Databases/DatabaseHDFS.cpp | 2 +- src/Databases/DatabaseHDFS.h | 2 +- src/Databases/DatabaseOnDisk.cpp | 2 +- src/Databases/DatabaseOnDisk.h | 2 +- src/Databases/DatabaseS3.cpp | 2 +- src/Databases/DatabaseS3.h | 2 +- src/Databases/DatabasesCommon.cpp | 2 +- src/Databases/DatabasesCommon.h | 2 +- src/Databases/DatabasesOverlay.cpp | 2 +- src/Databases/DatabasesOverlay.h | 2 +- .../MySQL/MaterializedMySQLSyncThread.cpp | 8 +-- .../MySQL/MaterializedMySQLSyncThread.h | 2 +- src/Databases/MySQL/MySQLBinlogClient.cpp | 2 +- src/Databases/MySQL/MySQLBinlogClient.h | 2 +- .../MySQL/MySQLBinlogEventsDispatcher.cpp | 6 +-- .../MySQL/MySQLBinlogEventsDispatcher.h | 2 +- .../PostgreSQL/DatabasePostgreSQL.cpp | 4 +- src/Databases/PostgreSQL/DatabasePostgreSQL.h | 2 +- src/Databases/SQLite/DatabaseSQLite.cpp | 2 +- src/Databases/SQLite/DatabaseSQLite.h | 2 +- src/Databases/SQLite/SQLiteUtils.cpp | 4 +- src/Databases/TablesDependencyGraph.cpp | 4 +- src/Databases/TablesDependencyGraph.h | 4 +- src/Databases/TablesLoader.cpp | 2 +- src/Databases/TablesLoader.h | 2 +- src/Dictionaries/CacheDictionary.cpp | 2 +- src/Dictionaries/CacheDictionary.h | 2 +- .../CassandraDictionarySource.cpp | 2 +- src/Dictionaries/CassandraDictionarySource.h | 2 +- src/Dictionaries/CassandraHelpers.cpp | 2 +- src/Dictionaries/ClickHouseDictionarySource.h | 2 +- src/Dictionaries/DictionaryFactory.cpp | 2 +- src/Dictionaries/DictionarySourceFactory.cpp | 2 +- src/Dictionaries/DictionarySourceFactory.h | 2 +- src/Dictionaries/DirectDictionary.cpp | 4 +- .../Embedded/RegionsHierarchies.cpp | 2 +- .../Embedded/RegionsHierarchy.cpp | 2 +- src/Dictionaries/Embedded/RegionsNames.cpp | 2 +- .../ExecutableDictionarySource.cpp | 4 +- src/Dictionaries/ExecutableDictionarySource.h | 2 +- .../ExecutablePoolDictionarySource.cpp | 4 +- .../ExecutablePoolDictionarySource.h | 2 +- src/Dictionaries/FileDictionarySource.cpp | 2 +- src/Dictionaries/HTTPDictionarySource.cpp | 4 +- src/Dictionaries/HTTPDictionarySource.h | 2 +- src/Dictionaries/HashedArrayDictionary.cpp | 4 +- src/Dictionaries/HashedArrayDictionary.h | 2 +- src/Dictionaries/HashedDictionary.h | 4 +- src/Dictionaries/IPAddressDictionary.cpp | 2 +- src/Dictionaries/IPAddressDictionary.h | 2 +- src/Dictionaries/LibraryDictionarySource.cpp | 4 +- src/Dictionaries/LibraryDictionarySource.h | 2 +- src/Dictionaries/MySQLDictionarySource.cpp | 4 +- src/Dictionaries/MySQLDictionarySource.h | 2 +- src/Dictionaries/NullDictionarySource.cpp | 2 +- src/Dictionaries/PolygonDictionaryUtils.cpp | 2 +- src/Dictionaries/PolygonDictionaryUtils.h | 2 +- .../PostgreSQLDictionarySource.cpp | 4 +- src/Dictionaries/PostgreSQLDictionarySource.h | 2 +- src/Dictionaries/RegExpTreeDictionary.cpp | 4 +- src/Dictionaries/RegExpTreeDictionary.h | 2 +- src/Dictionaries/XDBCDictionarySource.cpp | 4 +- src/Dictionaries/XDBCDictionarySource.h | 2 +- .../YAMLRegExpTreeDictionarySource.cpp | 2 +- .../YAMLRegExpTreeDictionarySource.h | 2 +- src/Dictionaries/registerHashedDictionary.cpp | 2 +- src/Disks/DiskLocal.cpp | 8 +-- src/Disks/DiskLocal.h | 2 +- src/Disks/DiskLocalCheckThread.cpp | 2 +- src/Disks/DiskLocalCheckThread.h | 2 +- src/Disks/DiskSelector.cpp | 2 +- src/Disks/IDisk.cpp | 4 +- .../IO/AsynchronousBoundedReadBuffer.cpp | 2 +- src/Disks/IO/AsynchronousBoundedReadBuffer.h | 2 +- .../IO/CachedOnDiskReadBufferFromFile.cpp | 4 +- src/Disks/IO/CachedOnDiskReadBufferFromFile.h | 2 +- .../IO/CachedOnDiskWriteBufferFromFile.cpp | 4 +- .../IO/CachedOnDiskWriteBufferFromFile.h | 4 +- src/Disks/IO/IOUringReader.cpp | 2 +- src/Disks/IO/IOUringReader.h | 2 +- src/Disks/IO/ReadBufferFromAzureBlobStorage.h | 2 +- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 2 +- src/Disks/IO/ReadBufferFromRemoteFSGather.h | 2 +- src/Disks/IO/ReadBufferFromWebServer.cpp | 2 +- src/Disks/IO/ReadBufferFromWebServer.h | 2 +- .../IO/WriteBufferFromAzureBlobStorage.cpp | 2 +- .../IO/WriteBufferFromAzureBlobStorage.h | 2 +- .../AzureBlobStorage/AzureObjectStorage.cpp | 2 +- .../AzureBlobStorage/AzureObjectStorage.h | 2 +- .../Cached/CachedObjectStorage.cpp | 2 +- .../Cached/CachedObjectStorage.h | 2 +- .../Cached/registerDiskCache.cpp | 2 +- .../ObjectStorages/DiskObjectStorage.cpp | 2 +- src/Disks/ObjectStorages/DiskObjectStorage.h | 2 +- .../DiskObjectStorageMetadata.cpp | 4 +- .../DiskObjectStorageTransaction.cpp | 12 ++--- .../Local/LocalObjectStorage.cpp | 2 +- .../ObjectStorages/Local/LocalObjectStorage.h | 2 +- .../ObjectStorages/S3/S3ObjectStorage.cpp | 2 +- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 4 +- .../ObjectStorages/Web/WebObjectStorage.cpp | 4 +- .../ObjectStorages/Web/WebObjectStorage.h | 2 +- src/Disks/StoragePolicy.cpp | 6 +-- src/Disks/StoragePolicy.h | 2 +- src/Disks/TemporaryFileOnDisk.cpp | 2 +- src/Disks/VolumeJBOD.cpp | 2 +- src/Disks/getOrCreateDiskFromAST.cpp | 2 +- src/Formats/FormatFactory.cpp | 4 +- src/Formats/ProtobufSerializer.cpp | 4 +- ...alUserDefinedExecutableFunctionsLoader.cpp | 2 +- .../UserDefinedSQLObjectsDiskStorage.cpp | 2 +- .../UserDefinedSQLObjectsDiskStorage.h | 2 +- .../UserDefinedSQLObjectsZooKeeperStorage.cpp | 2 +- .../UserDefinedSQLObjectsZooKeeperStorage.h | 2 +- src/Functions/logTrace.cpp | 2 +- src/IO/Archives/ZipArchiveWriter.cpp | 2 +- src/IO/HTTPCommon.cpp | 6 +-- src/IO/ParallelReadBuffer.cpp | 2 +- src/IO/ReadBufferFromS3.h | 2 +- src/IO/ReadWriteBufferFromHTTP.cpp | 2 +- src/IO/ReadWriteBufferFromHTTP.h | 2 +- src/IO/S3/AWSLogger.cpp | 2 +- src/IO/S3/AWSLogger.h | 5 +- src/IO/S3/Client.cpp | 6 +-- src/IO/S3/Client.h | 2 +- src/IO/S3/Credentials.cpp | 16 +++--- src/IO/S3/Credentials.h | 10 ++-- src/IO/S3/PocoHTTPClient.cpp | 2 +- src/IO/S3/copyS3File.cpp | 8 +-- src/IO/WriteBuffer.cpp | 2 +- src/IO/WriteBufferFromEncryptedFile.h | 2 +- src/IO/WriteBufferFromHTTP.cpp | 2 +- src/IO/WriteBufferFromS3.h | 2 +- src/Interpreters/Aggregator.cpp | 6 +-- src/Interpreters/Aggregator.h | 2 +- src/Interpreters/AsynchronousInsertQueue.cpp | 4 +- src/Interpreters/AsynchronousInsertQueue.h | 4 +- src/Interpreters/Cache/FileCache.cpp | 2 +- src/Interpreters/Cache/FileCache.h | 2 +- src/Interpreters/Cache/FileSegment.cpp | 4 +- src/Interpreters/Cache/FileSegment.h | 2 +- src/Interpreters/Cache/LRUFileCachePriority.h | 2 +- src/Interpreters/Cache/Metadata.cpp | 4 +- src/Interpreters/Cache/Metadata.h | 4 +- src/Interpreters/Cache/QueryCache.h | 4 +- .../Cache/SLRUFileCachePriority.h | 2 +- .../Cache/WriteBufferToFileSegment.cpp | 2 +- src/Interpreters/ClusterDiscovery.cpp | 6 +-- src/Interpreters/ClusterDiscovery.h | 2 +- .../ClusterProxy/SelectStreamFactory.cpp | 4 +- .../ClusterProxy/executeQuery.cpp | 12 ++--- src/Interpreters/ClusterProxy/executeQuery.h | 4 +- src/Interpreters/Context.cpp | 6 +-- src/Interpreters/CrossToInnerJoinVisitor.cpp | 2 +- src/Interpreters/DDLTask.cpp | 4 +- src/Interpreters/DDLTask.h | 4 +- src/Interpreters/DDLWorker.cpp | 2 +- src/Interpreters/DDLWorker.h | 2 +- src/Interpreters/DNSCacheUpdater.cpp | 4 +- src/Interpreters/DatabaseCatalog.cpp | 2 +- src/Interpreters/DatabaseCatalog.h | 2 +- src/Interpreters/DirectJoin.cpp | 2 +- src/Interpreters/DirectJoin.h | 2 +- src/Interpreters/EmbeddedDictionaries.cpp | 2 +- src/Interpreters/EmbeddedDictionaries.h | 2 +- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- src/Interpreters/ExpressionJIT.cpp | 5 +- .../ExternalDictionariesLoader.cpp | 2 +- src/Interpreters/ExternalLoader.cpp | 10 ++-- src/Interpreters/ExternalLoader.h | 5 +- src/Interpreters/FullSortingMergeJoin.h | 2 +- src/Interpreters/GraceHashJoin.cpp | 8 +-- src/Interpreters/GraceHashJoin.h | 2 +- src/Interpreters/HashJoin.cpp | 2 +- src/Interpreters/HashJoin.h | 2 +- src/Interpreters/InternalTextLogsQueue.cpp | 2 +- src/Interpreters/InterpreterCheckQuery.cpp | 10 ++-- src/Interpreters/InterpreterCheckQuery.h | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 4 +- .../InterpreterKillQueryQuery.cpp | 4 +- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Interpreters/InterpreterSelectQuery.h | 2 +- src/Interpreters/InterpreterSystemQuery.cpp | 4 +- src/Interpreters/InterpreterSystemQuery.h | 4 +- src/Interpreters/InterserverCredentials.cpp | 2 +- src/Interpreters/JoinedTables.cpp | 4 +- src/Interpreters/MergeJoin.cpp | 2 +- src/Interpreters/MergeJoin.h | 2 +- src/Interpreters/MutationsInterpreter.cpp | 2 +- src/Interpreters/PartLog.cpp | 2 +- src/Interpreters/PasteJoin.h | 2 +- src/Interpreters/ProcessList.cpp | 6 +-- src/Interpreters/Session.cpp | 4 +- src/Interpreters/Session.h | 2 +- src/Interpreters/Set.h | 4 +- src/Interpreters/SystemLog.cpp | 8 +-- src/Interpreters/SystemLog.h | 2 +- src/Interpreters/TableJoin.cpp | 4 +- src/Interpreters/TemporaryDataOnDisk.cpp | 8 +-- src/Interpreters/TraceCollector.cpp | 2 +- src/Interpreters/TransactionLog.cpp | 4 +- src/Interpreters/TransactionLog.h | 2 +- .../TransactionVersionMetadata.cpp | 2 +- src/Interpreters/TransactionVersionMetadata.h | 2 +- src/Interpreters/TransactionsInfoLog.cpp | 2 +- src/Interpreters/TransactionsInfoLog.h | 2 +- src/Interpreters/TreeRewriter.cpp | 4 +- src/Interpreters/executeDDLQueryOnCluster.cpp | 10 ++-- src/Interpreters/executeQuery.cpp | 20 ++++---- src/Interpreters/loadMetadata.cpp | 10 ++-- .../removeOnClusterClauseIfNeeded.cpp | 2 +- src/Parsers/DumpASTNode.h | 4 +- src/Planner/Planner.cpp | 10 ++-- src/Planner/PlannerJoinTree.cpp | 10 ++-- src/Planner/PlannerJoins.cpp | 8 +-- src/Processors/Executors/PipelineExecutor.h | 2 +- src/Processors/Formats/IRowInputFormat.cpp | 2 +- src/Processors/Formats/ISchemaReader.cpp | 2 +- .../Formats/Impl/AvroRowInputFormat.cpp | 4 +- .../Formats/Impl/DWARFBlockInputFormat.cpp | 8 +-- .../Impl/ParallelFormattingOutputFormat.h | 2 +- .../Formats/Impl/ParallelParsingInputFormat.h | 2 +- .../Formats/Impl/ValuesBlockInputFormat.cpp | 2 +- .../Algorithms/CollapsingSortedAlgorithm.cpp | 2 +- .../Algorithms/CollapsingSortedAlgorithm.h | 4 +- src/Processors/Merges/Algorithms/RowRef.h | 4 +- .../Merges/CollapsingSortedTransform.h | 2 +- .../Merges/MergingSortedTransform.cpp | 2 +- .../CreateSetAndFilterOnTheFlyStep.h | 2 +- .../Optimizations/filterPushDown.cpp | 2 +- .../Optimizations/optimizePrewhere.cpp | 2 +- .../optimizeUseAggregateProjection.cpp | 38 +++++++------- .../Optimizations/removeRedundantDistinct.cpp | 4 +- src/Processors/QueryPlan/QueryPlanVisitor.h | 2 +- .../QueryPlan/ReadFromMergeTree.cpp | 6 +-- src/Processors/QueryPlan/ReadFromMergeTree.h | 4 +- src/Processors/QueryPlan/ReadFromRemote.cpp | 8 +-- src/Processors/QueryPlan/ReadFromRemote.h | 8 +-- src/Processors/Sources/MySQLSource.cpp | 4 +- src/Processors/Sources/MySQLSource.h | 2 +- src/Processors/Sources/ShellCommandSource.cpp | 4 +- .../Transforms/AggregatingInOrderTransform.h | 2 +- .../Transforms/AggregatingTransform.h | 2 +- .../Transforms/ColumnGathererTransform.cpp | 2 +- .../Transforms/ColumnGathererTransform.h | 2 +- .../CreateSetAndFilterOnTheFlyTransform.h | 4 +- .../Transforms/CreatingSetsTransform.h | 2 +- .../Transforms/FillingTransform.cpp | 2 +- .../Transforms/JoiningTransform.cpp | 4 +- .../Transforms/MergeJoinTransform.cpp | 4 +- .../Transforms/MergeJoinTransform.h | 4 +- .../Transforms/MergeSortingTransform.cpp | 4 +- .../Transforms/MergeSortingTransform.h | 2 +- .../Transforms/MergingAggregatedTransform.h | 2 +- .../Transforms/PasteJoinTransform.cpp | 4 +- .../Transforms/PasteJoinTransform.h | 4 +- .../Transforms/TTLCalcTransform.cpp | 2 +- src/Processors/Transforms/TTLCalcTransform.h | 2 +- src/Processors/Transforms/TTLTransform.cpp | 2 +- src/Processors/Transforms/TTLTransform.h | 2 +- .../Transforms/buildPushingToViewsChain.cpp | 10 ++-- src/QueryPipeline/RemoteQueryExecutor.h | 4 +- src/Server/CertificateReloader.cpp | 2 +- src/Server/CertificateReloader.h | 5 +- src/Server/GRPCServer.cpp | 10 ++-- src/Server/GRPCServer.h | 3 +- src/Server/HTTP/HTTPServerRequest.cpp | 2 +- src/Server/HTTPHandler.cpp | 4 +- src/Server/HTTPHandler.h | 2 +- src/Server/HTTPRequestHandlerFactoryMain.cpp | 2 +- src/Server/HTTPRequestHandlerFactoryMain.h | 2 +- src/Server/InterserverIOHTTPHandler.h | 4 +- src/Server/KeeperReadinessHandler.cpp | 2 +- src/Server/KeeperTCPHandler.cpp | 2 +- src/Server/KeeperTCPHandler.h | 2 +- src/Server/KeeperTCPHandlerFactory.h | 4 +- src/Server/MySQLHandler.cpp | 2 +- src/Server/MySQLHandler.h | 2 +- src/Server/MySQLHandlerFactory.cpp | 2 +- src/Server/MySQLHandlerFactory.h | 2 +- src/Server/PostgreSQLHandler.h | 2 +- src/Server/PostgreSQLHandlerFactory.cpp | 2 +- src/Server/PostgreSQLHandlerFactory.h | 2 +- src/Server/ProxyV1Handler.h | 4 +- src/Server/ProxyV1HandlerFactory.h | 4 +- src/Server/ReplicasStatusHandler.cpp | 2 +- src/Server/TCPHandler.cpp | 4 +- src/Server/TCPHandler.h | 2 +- src/Server/TCPHandlerFactory.h | 4 +- src/Server/TCPProtocolStackFactory.h | 4 +- src/Server/TLSHandlerFactory.h | 4 +- src/Storages/Cache/ExternalDataSourceCache.h | 2 +- src/Storages/Cache/RemoteCacheController.cpp | 2 +- src/Storages/Cache/RemoteCacheController.h | 2 +- .../DataLakes/DeltaLakeMetadataParser.cpp | 2 +- src/Storages/DataLakes/HudiMetadataParser.cpp | 2 +- src/Storages/DataLakes/IStorageDataLake.h | 6 +-- .../DataLakes/Iceberg/IcebergMetadata.cpp | 4 +- .../DataLakes/Iceberg/IcebergMetadata.h | 2 +- src/Storages/DataLakes/S3MetadataReader.cpp | 2 +- .../DistributedAsyncInsertDirectoryQueue.cpp | 4 +- .../DistributedAsyncInsertDirectoryQueue.h | 2 +- .../DistributedAsyncInsertHeader.cpp | 2 +- .../DistributedAsyncInsertHeader.h | 2 +- .../DistributedAsyncInsertHelpers.cpp | 2 +- .../DistributedAsyncInsertHelpers.h | 8 ++- .../DistributedAsyncInsertSource.cpp | 4 +- src/Storages/Distributed/DistributedSink.cpp | 6 +-- src/Storages/Distributed/DistributedSink.h | 2 +- src/Storages/FileLog/FileLogConsumer.cpp | 2 +- src/Storages/FileLog/FileLogConsumer.h | 2 +- .../FileLog/FileLogDirectoryWatcher.cpp | 2 +- .../FileLog/FileLogDirectoryWatcher.h | 2 +- src/Storages/FileLog/StorageFileLog.cpp | 2 +- src/Storages/FileLog/StorageFileLog.h | 2 +- src/Storages/Freeze.cpp | 2 +- src/Storages/Freeze.h | 2 +- .../HDFS/AsynchronousReadBufferFromHDFS.cpp | 2 +- .../HDFS/AsynchronousReadBufferFromHDFS.h | 2 +- src/Storages/HDFS/HDFSCommon.cpp | 10 ++-- src/Storages/HDFS/StorageHDFS.h | 2 +- src/Storages/HDFS/StorageHDFSCluster.cpp | 2 +- src/Storages/Hive/HiveCommon.cpp | 2 +- src/Storages/Hive/HiveCommon.h | 4 +- src/Storages/Hive/StorageHive.cpp | 6 +-- src/Storages/Hive/StorageHive.h | 2 +- src/Storages/IMessageProducer.cpp | 2 +- src/Storages/IMessageProducer.h | 6 +-- src/Storages/IStorageCluster.cpp | 6 +-- src/Storages/IStorageCluster.h | 4 +- src/Storages/Kafka/KafkaConsumer.cpp | 2 +- src/Storages/Kafka/KafkaConsumer.h | 4 +- src/Storages/Kafka/KafkaProducer.cpp | 2 +- src/Storages/Kafka/KafkaSource.cpp | 2 +- src/Storages/Kafka/KafkaSource.h | 4 +- src/Storages/Kafka/StorageKafka.cpp | 2 +- src/Storages/Kafka/StorageKafka.h | 2 +- src/Storages/LiveView/StorageLiveView.cpp | 2 +- src/Storages/LiveView/StorageLiveView.h | 2 +- src/Storages/MaterializedView/RefreshTask.cpp | 2 +- src/Storages/MaterializedView/RefreshTask.h | 2 +- src/Storages/MergeTree/AsyncBlockIDsCache.cpp | 2 +- src/Storages/MergeTree/AsyncBlockIDsCache.h | 2 +- .../MergeTree/DataPartStorageOnDiskBase.cpp | 10 ++-- .../MergeTree/DataPartStorageOnDiskBase.h | 10 ++-- src/Storages/MergeTree/DataPartsExchange.cpp | 4 +- src/Storages/MergeTree/DataPartsExchange.h | 4 +- .../MergeTree/EphemeralLockInZooKeeper.cpp | 10 ++-- src/Storages/MergeTree/IDataPartStorage.h | 8 +-- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 8 +-- src/Storages/MergeTree/InsertBlockInfo.cpp | 2 +- src/Storages/MergeTree/InsertBlockInfo.h | 6 +-- src/Storages/MergeTree/LeaderElection.h | 2 +- .../MergeTree/MergeFromLogEntryTask.cpp | 2 +- src/Storages/MergeTree/MergeTask.h | 4 +- .../MergeTree/MergeTreeBackgroundExecutor.cpp | 2 +- .../MergeTree/MergeTreeBackgroundExecutor.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 4 +- src/Storages/MergeTree/MergeTreeData.h | 6 +-- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 18 +++---- .../MergeTree/MergeTreeDataSelectExecutor.h | 16 +++--- .../MergeTree/MergeTreeDataWriter.cpp | 12 ++--- src/Storages/MergeTree/MergeTreeDataWriter.h | 6 +-- src/Storages/MergeTree/MergeTreePartsMover.h | 4 +- .../MergeTree/MergeTreePrefetchedReadPool.cpp | 2 +- .../MergeTree/MergeTreePrefetchedReadPool.h | 2 +- src/Storages/MergeTree/MergeTreeRangeReader.h | 6 +-- src/Storages/MergeTree/MergeTreeReadPool.h | 2 +- .../MergeTreeReadPoolParallelReplicas.h | 2 +- .../MergeTree/MergeTreeSelectProcessor.h | 2 +- .../MergeTree/MergeTreeSequentialSource.cpp | 8 +-- .../MergeTree/MergeTreeSequentialSource.h | 2 +- src/Storages/MergeTree/MergeTreeSettings.cpp | 2 +- .../MergeTree/MergeTreeWhereOptimizer.cpp | 2 +- .../MergeTree/MergeTreeWhereOptimizer.h | 4 +- .../MergeTree/MergeTreeWriteAheadLog.cpp | 2 +- .../MergeTree/MergeTreeWriteAheadLog.h | 2 +- .../MergeTree/MergedBlockOutputStream.cpp | 2 +- .../MergeTree/MutateFromLogEntryTask.h | 2 +- .../MergeTree/MutatePlainMergeTreeTask.cpp | 2 +- src/Storages/MergeTree/MutateTask.cpp | 8 +-- .../ParallelReplicasReadingCoordinator.cpp | 4 +- .../PartMovesBetweenShardsOrchestrator.cpp | 2 +- .../PartMovesBetweenShardsOrchestrator.h | 2 +- src/Storages/MergeTree/PartitionPruner.cpp | 2 +- .../MergeTree/ReplicatedMergeMutateTaskBase.h | 4 +- .../ReplicatedMergeTreeAttachThread.cpp | 2 +- .../ReplicatedMergeTreeAttachThread.h | 2 +- .../ReplicatedMergeTreeCleanupThread.cpp | 2 +- .../ReplicatedMergeTreeCleanupThread.h | 2 +- .../ReplicatedMergeTreePartCheckThread.cpp | 2 +- .../ReplicatedMergeTreePartCheckThread.h | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 4 +- .../MergeTree/ReplicatedMergeTreeQueue.h | 4 +- .../ReplicatedMergeTreeRestartingThread.cpp | 2 +- .../ReplicatedMergeTreeRestartingThread.h | 2 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 6 +-- .../MergeTree/ReplicatedMergeTreeSink.h | 2 +- src/Storages/MergeTree/ZooKeeperRetries.h | 6 +-- src/Storages/MergeTree/checkDataPart.cpp | 2 +- src/Storages/MessageQueueSink.cpp | 2 +- src/Storages/NATS/NATSConnection.cpp | 6 +-- src/Storages/NATS/NATSConnection.h | 4 +- src/Storages/NATS/NATSConsumer.cpp | 2 +- src/Storages/NATS/NATSConsumer.h | 4 +- src/Storages/NATS/NATSHandler.cpp | 2 +- src/Storages/NATS/NATSHandler.h | 6 +-- src/Storages/NATS/NATSProducer.cpp | 2 +- src/Storages/NATS/NATSProducer.h | 2 +- src/Storages/NATS/StorageNATS.cpp | 2 +- src/Storages/NATS/StorageNATS.h | 2 +- .../MaterializedPostgreSQLConsumer.cpp | 4 +- .../MaterializedPostgreSQLConsumer.h | 4 +- .../PostgreSQLReplicationHandler.cpp | 2 +- .../PostgreSQL/PostgreSQLReplicationHandler.h | 2 +- .../StorageMaterializedPostgreSQL.cpp | 8 +-- .../StorageMaterializedPostgreSQL.h | 2 +- src/Storages/RabbitMQ/RabbitMQConnection.cpp | 2 +- src/Storages/RabbitMQ/RabbitMQConnection.h | 4 +- src/Storages/RabbitMQ/RabbitMQConsumer.cpp | 2 +- src/Storages/RabbitMQ/RabbitMQConsumer.h | 4 +- src/Storages/RabbitMQ/RabbitMQHandler.cpp | 2 +- src/Storages/RabbitMQ/RabbitMQHandler.h | 4 +- src/Storages/RabbitMQ/RabbitMQProducer.cpp | 2 +- src/Storages/RabbitMQ/RabbitMQProducer.h | 2 +- src/Storages/RabbitMQ/RabbitMQSource.cpp | 2 +- src/Storages/RabbitMQ/RabbitMQSource.h | 2 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 2 +- src/Storages/RabbitMQ/StorageRabbitMQ.h | 2 +- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 4 +- src/Storages/S3Queue/S3QueueFilesMetadata.h | 4 +- src/Storages/S3Queue/S3QueueSource.cpp | 4 +- src/Storages/S3Queue/S3QueueSource.h | 4 +- src/Storages/S3Queue/StorageS3Queue.cpp | 6 +-- src/Storages/S3Queue/StorageS3Queue.h | 2 +- src/Storages/StorageAzureBlob.h | 2 +- src/Storages/StorageAzureBlobCluster.cpp | 2 +- src/Storages/StorageBuffer.cpp | 4 +- src/Storages/StorageBuffer.h | 2 +- src/Storages/StorageDistributed.cpp | 2 +- src/Storages/StorageDistributed.h | 2 +- src/Storages/StorageExecutable.cpp | 2 +- src/Storages/StorageExecutable.h | 2 +- src/Storages/StorageFile.cpp | 2 +- src/Storages/StorageFile.h | 2 +- src/Storages/StorageFileCluster.cpp | 2 +- src/Storages/StorageJoin.cpp | 2 +- src/Storages/StorageKeeperMap.cpp | 6 +-- src/Storages/StorageKeeperMap.h | 2 +- src/Storages/StorageMaterializedView.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 8 +-- src/Storages/StorageMySQL.cpp | 2 +- src/Storages/StorageMySQL.h | 2 +- src/Storages/StoragePostgreSQL.cpp | 2 +- src/Storages/StoragePostgreSQL.h | 2 +- src/Storages/StorageRedis.cpp | 2 +- src/Storages/StorageRedis.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 32 ++++++------ src/Storages/StorageReplicatedMergeTree.h | 10 ++-- src/Storages/StorageS3.cpp | 4 +- src/Storages/StorageS3.h | 2 +- src/Storages/StorageS3Cluster.cpp | 2 +- src/Storages/StorageSQLite.cpp | 2 +- src/Storages/StorageSQLite.h | 2 +- src/Storages/StorageSet.cpp | 4 +- src/Storages/StorageStripeLog.cpp | 2 +- src/Storages/StorageStripeLog.h | 2 +- src/Storages/StorageURL.cpp | 2 +- src/Storages/StorageURLCluster.cpp | 2 +- src/Storages/StorageXDBC.cpp | 2 +- src/Storages/StorageXDBC.h | 2 +- .../System/StorageSystemDatabases.cpp | 2 +- src/Storages/System/StorageSystemJemalloc.cpp | 2 +- src/Storages/System/StorageSystemReplicas.cpp | 4 +- .../System/StorageSystemStackTrace.cpp | 12 ++--- src/Storages/System/StorageSystemStackTrace.h | 2 +- src/Storages/UVLoop.h | 2 +- src/Storages/WindowView/StorageWindowView.cpp | 2 +- src/Storages/WindowView/StorageWindowView.h | 2 +- src/TableFunctions/Hive/TableFunctionHive.cpp | 2 +- 675 files changed, 1298 insertions(+), 1239 deletions(-) create mode 100644 src/Common/Logger.cpp create mode 100644 src/Common/Logger.h delete mode 100644 src/Common/LoggerPtr.cpp delete mode 100644 src/Common/LoggerPtr.h diff --git a/programs/copier/ClusterCopier.h b/programs/copier/ClusterCopier.h index 063b13e9078..01f8b30f546 100644 --- a/programs/copier/ClusterCopier.h +++ b/programs/copier/ClusterCopier.h @@ -20,7 +20,7 @@ public: const String & host_id_, const String & proxy_database_name_, ContextMutablePtr context_, - Poco::Logger * log_) + LoggerRawPtr log_) : WithMutableContext(context_), task_zookeeper_path(task_path_), host_id(host_id_), @@ -230,7 +230,7 @@ private: bool experimental_use_sample_offset{false}; - Poco::Logger * log; + LoggerRawPtr log; UInt64 max_table_tries = 3; UInt64 max_shard_partition_tries = 3; diff --git a/programs/copier/ZooKeeperStaff.h b/programs/copier/ZooKeeperStaff.h index 36dcfa50842..bbdec230d2d 100644 --- a/programs/copier/ZooKeeperStaff.h +++ b/programs/copier/ZooKeeperStaff.h @@ -177,7 +177,7 @@ public: auto watch_callback = [my_stale = stale] (const Coordination::WatchResponse & rsp) { - auto logger = &Poco::Logger::get("ClusterCopier"); + auto logger = getLogger("ClusterCopier"); if (rsp.error == Coordination::Error::ZOK) { switch (rsp.type) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 7ed4499efbd..fa66a69687c 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -375,7 +375,7 @@ int KeeperClient::main(const std::vector & /* args */) if (!config().has("host") && !config().has("port") && !keys.empty()) { - LOG_INFO(&Poco::Logger::get("KeeperClient"), "Found keeper node in the config.xml, will use it for connection"); + LOG_INFO(getLogger("KeeperClient"), "Found keeper node in the config.xml, will use it for connection"); for (const auto & key : keys) { diff --git a/programs/keeper-converter/KeeperConverter.cpp b/programs/keeper-converter/KeeperConverter.cpp index 20448aafa2f..2b2759412ab 100644 --- a/programs/keeper-converter/KeeperConverter.cpp +++ b/programs/keeper-converter/KeeperConverter.cpp @@ -28,7 +28,7 @@ int mainEntryClickHouseKeeperConverter(int argc, char ** argv) po::store(po::command_line_parser(argc, argv).options(desc).run(), options); Poco::AutoPtr console_channel(new Poco::ConsoleChannel); - Poco::Logger * logger = &Poco::Logger::get("KeeperConverter"); + LoggerPtr logger = getLogger("KeeperConverter"); logger->setChannel(console_channel); if (options.count("help")) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 109884ec899..c751702dc6f 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -624,7 +624,7 @@ catch (...) void Keeper::logRevision() const { - LOG_INFO(&Poco::Logger::get("Application"), + LOG_INFO(getLogger("Application"), "Starting ClickHouse Keeper {} (revision: {}, git hash: {}, build id: {}), PID {}", VERSION_STRING, ClickHouseRevision::getVersionRevision(), diff --git a/programs/library-bridge/CatBoostLibraryHandlerFactory.cpp b/programs/library-bridge/CatBoostLibraryHandlerFactory.cpp index 6ee078f6c5c..7ce896636e7 100644 --- a/programs/library-bridge/CatBoostLibraryHandlerFactory.cpp +++ b/programs/library-bridge/CatBoostLibraryHandlerFactory.cpp @@ -13,7 +13,7 @@ CatBoostLibraryHandlerFactory & CatBoostLibraryHandlerFactory::instance() } CatBoostLibraryHandlerFactory::CatBoostLibraryHandlerFactory() - : log(&Poco::Logger::get("CatBoostLibraryHandlerFactory")) + : log(getLogger("CatBoostLibraryHandlerFactory")) { } diff --git a/programs/library-bridge/CatBoostLibraryHandlerFactory.h b/programs/library-bridge/CatBoostLibraryHandlerFactory.h index 6ba3fe84ec9..e29834cbe79 100644 --- a/programs/library-bridge/CatBoostLibraryHandlerFactory.h +++ b/programs/library-bridge/CatBoostLibraryHandlerFactory.h @@ -31,7 +31,7 @@ private: /// map: model path --> catboost library handler std::unordered_map library_handlers TSA_GUARDED_BY(mutex); std::mutex mutex; - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/programs/library-bridge/ExternalDictionaryLibraryAPI.cpp b/programs/library-bridge/ExternalDictionaryLibraryAPI.cpp index 70cd6fca375..4fa5c991f0f 100644 --- a/programs/library-bridge/ExternalDictionaryLibraryAPI.cpp +++ b/programs/library-bridge/ExternalDictionaryLibraryAPI.cpp @@ -9,40 +9,40 @@ const char DICT_LOGGER_NAME[] = "LibraryDictionarySourceExternal"; void ExternalDictionaryLibraryAPI::log(LogLevel level, CString msg) { - auto & logger = Poco::Logger::get(DICT_LOGGER_NAME); + auto logger = getLogger(DICT_LOGGER_NAME); switch (level) { case LogLevel::TRACE: - if (logger.trace()) - logger.trace(msg); + if (logger->trace()) + logger->trace(msg); break; case LogLevel::DEBUG: - if (logger.debug()) - logger.debug(msg); + if (logger->debug()) + logger->debug(msg); break; case LogLevel::INFORMATION: - if (logger.information()) - logger.information(msg); + if (logger->information()) + logger->information(msg); break; case LogLevel::NOTICE: - if (logger.notice()) - logger.notice(msg); + if (logger->notice()) + logger->notice(msg); break; case LogLevel::WARNING: - if (logger.warning()) - logger.warning(msg); + if (logger->warning()) + logger->warning(msg); break; case LogLevel::ERROR: - if (logger.error()) - logger.error(msg); + if (logger->error()) + logger->error(msg); break; case LogLevel::CRITICAL: - if (logger.critical()) - logger.critical(msg); + if (logger->critical()) + logger->critical(msg); break; case LogLevel::FATAL: - if (logger.fatal()) - logger.fatal(msg); + if (logger->fatal()) + logger->fatal(msg); break; } } diff --git a/programs/library-bridge/ExternalDictionaryLibraryHandlerFactory.cpp b/programs/library-bridge/ExternalDictionaryLibraryHandlerFactory.cpp index 6acd9af20ed..1b2b57beeb1 100644 --- a/programs/library-bridge/ExternalDictionaryLibraryHandlerFactory.cpp +++ b/programs/library-bridge/ExternalDictionaryLibraryHandlerFactory.cpp @@ -26,7 +26,7 @@ void ExternalDictionaryLibraryHandlerFactory::create( if (library_handlers.contains(dictionary_id)) { - LOG_WARNING(&Poco::Logger::get("ExternalDictionaryLibraryHandlerFactory"), "Library handler with dictionary id {} already exists", dictionary_id); + LOG_WARNING(getLogger("ExternalDictionaryLibraryHandlerFactory"), "Library handler with dictionary id {} already exists", dictionary_id); return; } diff --git a/programs/library-bridge/LibraryBridgeHandlerFactory.cpp b/programs/library-bridge/LibraryBridgeHandlerFactory.cpp index 4af1f8355e8..e5ab22f2d40 100644 --- a/programs/library-bridge/LibraryBridgeHandlerFactory.cpp +++ b/programs/library-bridge/LibraryBridgeHandlerFactory.cpp @@ -12,7 +12,7 @@ LibraryBridgeHandlerFactory::LibraryBridgeHandlerFactory( size_t keep_alive_timeout_, ContextPtr context_) : WithContext(context_) - , log(&Poco::Logger::get(name_)) + , log(getLogger(name_)) , name(name_) , keep_alive_timeout(keep_alive_timeout_) { diff --git a/programs/library-bridge/LibraryBridgeHandlerFactory.h b/programs/library-bridge/LibraryBridgeHandlerFactory.h index 7565052c4cb..5b0f088bc29 100644 --- a/programs/library-bridge/LibraryBridgeHandlerFactory.h +++ b/programs/library-bridge/LibraryBridgeHandlerFactory.h @@ -19,7 +19,7 @@ public: std::unique_ptr createRequestHandler(const HTTPServerRequest & request) override; private: - Poco::Logger * log; + LoggerPtr log; const std::string name; const size_t keep_alive_timeout; }; diff --git a/programs/library-bridge/LibraryBridgeHandlers.cpp b/programs/library-bridge/LibraryBridgeHandlers.cpp index b0b465460e0..ab146f458df 100644 --- a/programs/library-bridge/LibraryBridgeHandlers.cpp +++ b/programs/library-bridge/LibraryBridgeHandlers.cpp @@ -47,7 +47,7 @@ namespace if (!response.sent()) *response.send() << message << '\n'; - LOG_WARNING(&Poco::Logger::get("LibraryBridge"), fmt::runtime(message)); + LOG_WARNING(getLogger("LibraryBridge"), fmt::runtime(message)); } std::shared_ptr parseColumns(String && column_string) @@ -92,7 +92,7 @@ static void writeData(Block data, OutputFormatPtr format) ExternalDictionaryLibraryBridgeRequestHandler::ExternalDictionaryLibraryBridgeRequestHandler(size_t keep_alive_timeout_, ContextPtr context_) : WithContext(context_) , keep_alive_timeout(keep_alive_timeout_) - , log(&Poco::Logger::get("ExternalDictionaryLibraryBridgeRequestHandler")) + , log(getLogger("ExternalDictionaryLibraryBridgeRequestHandler")) { } @@ -380,7 +380,7 @@ void ExternalDictionaryLibraryBridgeRequestHandler::handleRequest(HTTPServerRequ ExternalDictionaryLibraryBridgeExistsHandler::ExternalDictionaryLibraryBridgeExistsHandler(size_t keep_alive_timeout_, ContextPtr context_) : WithContext(context_) , keep_alive_timeout(keep_alive_timeout_) - , log(&Poco::Logger::get("ExternalDictionaryLibraryBridgeExistsHandler")) + , log(getLogger("ExternalDictionaryLibraryBridgeExistsHandler")) { } @@ -419,7 +419,7 @@ CatBoostLibraryBridgeRequestHandler::CatBoostLibraryBridgeRequestHandler( size_t keep_alive_timeout_, ContextPtr context_) : WithContext(context_) , keep_alive_timeout(keep_alive_timeout_) - , log(&Poco::Logger::get("CatBoostLibraryBridgeRequestHandler")) + , log(getLogger("CatBoostLibraryBridgeRequestHandler")) { } @@ -623,7 +623,7 @@ void CatBoostLibraryBridgeRequestHandler::handleRequest(HTTPServerRequest & requ CatBoostLibraryBridgeExistsHandler::CatBoostLibraryBridgeExistsHandler(size_t keep_alive_timeout_, ContextPtr context_) : WithContext(context_) , keep_alive_timeout(keep_alive_timeout_) - , log(&Poco::Logger::get("CatBoostLibraryBridgeExistsHandler")) + , log(getLogger("CatBoostLibraryBridgeExistsHandler")) { } diff --git a/programs/library-bridge/LibraryBridgeHandlers.h b/programs/library-bridge/LibraryBridgeHandlers.h index 4f08d7a6084..1db71eb24cb 100644 --- a/programs/library-bridge/LibraryBridgeHandlers.h +++ b/programs/library-bridge/LibraryBridgeHandlers.h @@ -26,7 +26,7 @@ private: static constexpr inline auto FORMAT = "RowBinary"; const size_t keep_alive_timeout; - Poco::Logger * log; + LoggerPtr log; }; @@ -40,7 +40,7 @@ public: private: const size_t keep_alive_timeout; - Poco::Logger * log; + LoggerPtr log; }; @@ -69,7 +69,7 @@ public: private: const size_t keep_alive_timeout; - Poco::Logger * log; + LoggerPtr log; }; @@ -83,7 +83,7 @@ public: private: const size_t keep_alive_timeout; - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 4e0b9eeb731..443d4a52fa3 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -221,7 +221,7 @@ void LocalServer::tryInitPath() { // The path is not provided explicitly - use a unique path in the system temporary directory // (or in the current dir if temporary don't exist) - Poco::Logger * log = &logger(); + LoggerRawPtr log = &logger(); std::filesystem::path parent_folder; std::filesystem::path default_path; @@ -631,7 +631,7 @@ void LocalServer::processConfig() tryInitPath(); - Poco::Logger * log = &logger(); + LoggerRawPtr log = &logger(); /// Maybe useless if (config().has("macros")) diff --git a/programs/odbc-bridge/ColumnInfoHandler.h b/programs/odbc-bridge/ColumnInfoHandler.h index e3087701182..ca7044fdf32 100644 --- a/programs/odbc-bridge/ColumnInfoHandler.h +++ b/programs/odbc-bridge/ColumnInfoHandler.h @@ -18,7 +18,7 @@ class ODBCColumnsInfoHandler : public HTTPRequestHandler, WithContext public: ODBCColumnsInfoHandler(size_t keep_alive_timeout_, ContextPtr context_) : WithContext(context_) - , log(&Poco::Logger::get("ODBCColumnsInfoHandler")) + , log(getLogger("ODBCColumnsInfoHandler")) , keep_alive_timeout(keep_alive_timeout_) { } @@ -26,7 +26,7 @@ public: void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; private: - Poco::Logger * log; + LoggerPtr log; size_t keep_alive_timeout; }; diff --git a/programs/odbc-bridge/IdentifierQuoteHandler.h b/programs/odbc-bridge/IdentifierQuoteHandler.h index ff5c02ca07b..7b78c5b4b93 100644 --- a/programs/odbc-bridge/IdentifierQuoteHandler.h +++ b/programs/odbc-bridge/IdentifierQuoteHandler.h @@ -16,7 +16,7 @@ class IdentifierQuoteHandler : public HTTPRequestHandler, WithContext public: IdentifierQuoteHandler(size_t keep_alive_timeout_, ContextPtr context_) : WithContext(context_) - , log(&Poco::Logger::get("IdentifierQuoteHandler")) + , log(getLogger("IdentifierQuoteHandler")) , keep_alive_timeout(keep_alive_timeout_) { } @@ -24,7 +24,7 @@ public: void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; private: - Poco::Logger * log; + LoggerPtr log; size_t keep_alive_timeout; }; diff --git a/programs/odbc-bridge/MainHandler.h b/programs/odbc-bridge/MainHandler.h index 7977245ff82..ed0c6b2e28c 100644 --- a/programs/odbc-bridge/MainHandler.h +++ b/programs/odbc-bridge/MainHandler.h @@ -24,7 +24,7 @@ public: ContextPtr context_, const String & mode_) : WithContext(context_) - , log(&Poco::Logger::get("ODBCHandler")) + , log(getLogger("ODBCHandler")) , keep_alive_timeout(keep_alive_timeout_) , mode(mode_) { @@ -33,7 +33,7 @@ public: void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; private: - Poco::Logger * log; + LoggerPtr log; size_t keep_alive_timeout; String mode; diff --git a/programs/odbc-bridge/ODBCBlockInputStream.cpp b/programs/odbc-bridge/ODBCBlockInputStream.cpp index 3aa3d9a652b..c46144c3dc8 100644 --- a/programs/odbc-bridge/ODBCBlockInputStream.cpp +++ b/programs/odbc-bridge/ODBCBlockInputStream.cpp @@ -23,7 +23,7 @@ namespace ErrorCodes ODBCSource::ODBCSource( nanodbc::ConnectionHolderPtr connection_holder, const std::string & query_str, const Block & sample_block, const UInt64 max_block_size_) : ISource(sample_block) - , log(&Poco::Logger::get("ODBCSource")) + , log(getLogger("ODBCSource")) , max_block_size{max_block_size_} , query(query_str) { diff --git a/programs/odbc-bridge/ODBCBlockInputStream.h b/programs/odbc-bridge/ODBCBlockInputStream.h index 79d5816ad01..dedd98f930f 100644 --- a/programs/odbc-bridge/ODBCBlockInputStream.h +++ b/programs/odbc-bridge/ODBCBlockInputStream.h @@ -30,7 +30,7 @@ private: column.insertFrom(sample_column, 0); } - Poco::Logger * log; + LoggerPtr log; const UInt64 max_block_size; ExternalResultDescription description; diff --git a/programs/odbc-bridge/ODBCBlockOutputStream.cpp b/programs/odbc-bridge/ODBCBlockOutputStream.cpp index eb5901ad3e1..87c09d1e757 100644 --- a/programs/odbc-bridge/ODBCBlockOutputStream.cpp +++ b/programs/odbc-bridge/ODBCBlockOutputStream.cpp @@ -19,7 +19,7 @@ ODBCSink::ODBCSink( ContextPtr local_context_, IdentifierQuotingStyle quoting_) : ISink(sample_block_) - , log(&Poco::Logger::get("ODBCSink")) + , log(getLogger("ODBCSink")) , connection_holder(std::move(connection_holder_)) , db_name(remote_database_name_) , table_name(remote_table_name_) diff --git a/programs/odbc-bridge/ODBCBlockOutputStream.h b/programs/odbc-bridge/ODBCBlockOutputStream.h index f5e7b4e3a2d..06edce92e1a 100644 --- a/programs/odbc-bridge/ODBCBlockOutputStream.h +++ b/programs/odbc-bridge/ODBCBlockOutputStream.h @@ -30,7 +30,7 @@ protected: void consume(Chunk chunk) override; private: - Poco::Logger * log; + LoggerPtr log; nanodbc::ConnectionHolderPtr connection_holder; std::string db_name; diff --git a/programs/odbc-bridge/ODBCHandlerFactory.cpp b/programs/odbc-bridge/ODBCHandlerFactory.cpp index dd21358df8c..eebb0c24c7a 100644 --- a/programs/odbc-bridge/ODBCHandlerFactory.cpp +++ b/programs/odbc-bridge/ODBCHandlerFactory.cpp @@ -11,7 +11,7 @@ namespace DB ODBCBridgeHandlerFactory::ODBCBridgeHandlerFactory(const std::string & name_, size_t keep_alive_timeout_, ContextPtr context_) : WithContext(context_) - , log(&Poco::Logger::get(name_)) + , log(getLogger(name_)) , name(name_) , keep_alive_timeout(keep_alive_timeout_) { diff --git a/programs/odbc-bridge/ODBCHandlerFactory.h b/programs/odbc-bridge/ODBCHandlerFactory.h index 3e3da7c9f24..4aaf1b55453 100644 --- a/programs/odbc-bridge/ODBCHandlerFactory.h +++ b/programs/odbc-bridge/ODBCHandlerFactory.h @@ -22,7 +22,7 @@ public: std::unique_ptr createRequestHandler(const HTTPServerRequest & request) override; private: - Poco::Logger * log; + LoggerPtr log; std::string name; size_t keep_alive_timeout; }; diff --git a/programs/odbc-bridge/ODBCPooledConnectionFactory.h b/programs/odbc-bridge/ODBCPooledConnectionFactory.h index a10055c6659..b70e45f2b9d 100644 --- a/programs/odbc-bridge/ODBCPooledConnectionFactory.h +++ b/programs/odbc-bridge/ODBCPooledConnectionFactory.h @@ -97,7 +97,7 @@ T execute(nanodbc::ConnectionHolderPtr connection_holder, std::function & servers, std::vector & servers_to_start_before_tables) { - Poco::Logger * log = &logger(); + LoggerRawPtr log = &logger(); const auto listen_hosts = getListenHosts(config); const auto interserver_listen_hosts = getInterserverListenHosts(config); diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 5de405d9f8f..71ad219110d 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -47,7 +47,7 @@ namespace const Poco::Util::AbstractConfiguration & config, const std::string & config_path, const std::string & users_config_path, - Poco::Logger * log) + LoggerPtr log) { if (config.getBool("skip_check_for_incorrect_settings", false)) return; diff --git a/src/Access/AccessRights.cpp b/src/Access/AccessRights.cpp index 520a11bd696..72cbeca4f11 100644 --- a/src/Access/AccessRights.cpp +++ b/src/Access/AccessRights.cpp @@ -443,7 +443,7 @@ public: optimizeTree(); } - void logTree(Poco::Logger * log, const String & title) const + void logTree(LoggerPtr log, const String & title) const { LOG_TRACE(log, "Tree({}): level={}, name={}, flags={}, min_flags={}, max_flags={}, num_children={}", title, level, node_name ? *node_name : "NULL", flags.toString(), @@ -1158,7 +1158,7 @@ AccessRights AccessRights::getFullAccess() void AccessRights::logTree() const { - auto * log = &Poco::Logger::get("AccessRights"); + auto log = getLogger("AccessRights"); if (root) { root->logTree(log, ""); diff --git a/src/Access/Common/AllowedClientHosts.cpp b/src/Access/Common/AllowedClientHosts.cpp index 801ccd3748b..c677465a7a1 100644 --- a/src/Access/Common/AllowedClientHosts.cpp +++ b/src/Access/Common/AllowedClientHosts.cpp @@ -514,7 +514,7 @@ bool AllowedClientHosts::contains(const IPAddress & client_address) const throw; /// Try to ignore DNS errors: if host cannot be resolved, skip it and try next. LOG_WARNING( - &Poco::Logger::get("AddressPatterns"), + getLogger("AddressPatterns"), "Failed to check if the allowed client hosts contain address {}. {}, code = {}", client_address.toString(), e.displayText(), e.code()); return false; @@ -556,7 +556,7 @@ bool AllowedClientHosts::contains(const IPAddress & client_address) const throw; /// Try to ignore DNS errors: if host cannot be resolved, skip it and try next. LOG_WARNING( - &Poco::Logger::get("AddressPatterns"), + getLogger("AddressPatterns"), "Failed to check if the allowed client hosts contain address {}. {}, code = {}", client_address.toString(), e.displayText(), e.code()); return false; diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 567b131c00e..0943e797e3f 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -298,7 +298,7 @@ void ContextAccess::setUser(const UserPtr & user_) const } user_name = user->getName(); - trace_log = &Poco::Logger::get("ContextAccess (" + user_name + ")"); + trace_log = getLogger("ContextAccess (" + user_name + ")"); std::vector current_roles, current_roles_with_admin_option; if (params.use_default_roles) diff --git a/src/Access/ContextAccess.h b/src/Access/ContextAccess.h index caf903b85bd..237c423d261 100644 --- a/src/Access/ContextAccess.h +++ b/src/Access/ContextAccess.h @@ -185,9 +185,10 @@ private: mutable std::atomic initialized = false; // can be removed after Bug 5504 is resolved mutable std::atomic user_was_dropped = false; - mutable std::atomic trace_log = nullptr; mutable std::mutex mutex; + /// TODO: Fix race + mutable LoggerPtr trace_log; mutable UserPtr user TSA_GUARDED_BY(mutex); mutable String user_name TSA_GUARDED_BY(mutex); mutable scope_guard subscription_for_user_change TSA_GUARDED_BY(mutex); diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp index 190c7567b85..3c20ef3d102 100644 --- a/src/Access/DiskAccessStorage.cpp +++ b/src/Access/DiskAccessStorage.cpp @@ -47,7 +47,7 @@ namespace } - AccessEntityPtr tryReadEntityFile(const String & file_path, Poco::Logger & log) + AccessEntityPtr tryReadEntityFile(const String & file_path, LoggerPtr log) { try { @@ -55,7 +55,7 @@ namespace } catch (...) { - tryLogCurrentException(&log); + tryLogCurrentException(log); return nullptr; } } @@ -378,7 +378,7 @@ void DiskAccessStorage::reloadAllAndRebuildLists() continue; const auto access_entity_file_path = getEntityFilePath(directory_path, id); - auto entity = tryReadEntityFile(access_entity_file_path, *getLogger()); + auto entity = tryReadEntityFile(access_entity_file_path, getLogger()); if (!entity) continue; diff --git a/src/Access/ExternalAuthenticators.cpp b/src/Access/ExternalAuthenticators.cpp index 351bcb95c73..77812ac5eb5 100644 --- a/src/Access/ExternalAuthenticators.cpp +++ b/src/Access/ExternalAuthenticators.cpp @@ -279,7 +279,7 @@ void ExternalAuthenticators::reset() resetImpl(); } -void ExternalAuthenticators::setConfiguration(const Poco::Util::AbstractConfiguration & config, Poco::Logger * log) +void ExternalAuthenticators::setConfiguration(const Poco::Util::AbstractConfiguration & config, LoggerPtr log) { std::lock_guard lock(mutex); resetImpl(); diff --git a/src/Access/ExternalAuthenticators.h b/src/Access/ExternalAuthenticators.h index 46c51f0d2f3..3a710e6df26 100644 --- a/src/Access/ExternalAuthenticators.h +++ b/src/Access/ExternalAuthenticators.h @@ -36,7 +36,7 @@ class ExternalAuthenticators { public: void reset(); - void setConfiguration(const Poco::Util::AbstractConfiguration & config, Poco::Logger * log); + void setConfiguration(const Poco::Util::AbstractConfiguration & config, LoggerPtr log); // The name and readiness of the credentials must be verified before calling these. bool checkLDAPCredentials(const String & server, const BasicCredentials & credentials, diff --git a/src/Access/GSSAcceptor.cpp b/src/Access/GSSAcceptor.cpp index 02946f0d74d..cfa1af6a200 100644 --- a/src/Access/GSSAcceptor.cpp +++ b/src/Access/GSSAcceptor.cpp @@ -328,7 +328,7 @@ void GSSAcceptorContext::initHandles() } } -String GSSAcceptorContext::processToken(const String & input_token, Poco::Logger * log) +String GSSAcceptorContext::processToken(const String & input_token, LoggerPtr log) { std::lock_guard lock(gss_global_mutex); @@ -455,7 +455,7 @@ void GSSAcceptorContext::initHandles() { } -String GSSAcceptorContext::processToken(const String &, Poco::Logger *) +String GSSAcceptorContext::processToken(const String &, LoggerPtr) { throw Exception(ErrorCodes::FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME, "ClickHouse was built without GSS-API/Kerberos support"); } diff --git a/src/Access/GSSAcceptor.h b/src/Access/GSSAcceptor.h index ba448ae474e..8d490fb47ae 100644 --- a/src/Access/GSSAcceptor.h +++ b/src/Access/GSSAcceptor.h @@ -3,6 +3,7 @@ #include "config.h" #include +#include #include #include @@ -42,7 +43,7 @@ public: const String & getRealm() const; bool isFailed() const; - MAYBE_NORETURN String processToken(const String & input_token, Poco::Logger * log); + MAYBE_NORETURN String processToken(const String & input_token, LoggerPtr log); private: void reset(); diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index 222f38b41b6..fbe9e231002 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -615,7 +616,7 @@ UUID IAccessStorage::generateRandomID() } -void IAccessStorage::clearConflictsInEntitiesList(std::vector> & entities, const Poco::Logger * log_) +void IAccessStorage::clearConflictsInEntitiesList(std::vector> & entities, const LoggerPtr log_) { std::unordered_map positions_by_id; std::unordered_map positions_by_type_and_name[static_cast(AccessEntityType::MAX)]; @@ -671,12 +672,13 @@ void IAccessStorage::clearConflictsInEntitiesList(std::vector #include #include +#include #include #include @@ -225,9 +226,9 @@ protected: SettingsChanges & settings) const; virtual bool isAddressAllowed(const User & user, const Poco::Net::IPAddress & address) const; static UUID generateRandomID(); - Poco::Logger * getLogger() const; + LoggerPtr getLogger() const; static String formatEntityTypeWithName(AccessEntityType type, const String & name) { return AccessEntityTypeInfo::get(type).formatEntityNameWithType(name); } - static void clearConflictsInEntitiesList(std::vector> & entities, const Poco::Logger * log_); + static void clearConflictsInEntitiesList(std::vector> & entities, const LoggerPtr log_); [[noreturn]] void throwNotFound(const UUID & id) const; [[noreturn]] void throwNotFound(AccessEntityType type, const String & name) const; [[noreturn]] static void throwBadCast(const UUID & id, AccessEntityType type, const String & name, AccessEntityType required_type); @@ -246,7 +247,9 @@ protected: private: const String storage_name; - mutable std::atomic log = nullptr; + + mutable OnceFlag log_initialized; + mutable LoggerPtr log = nullptr; }; diff --git a/src/Access/KerberosInit.cpp b/src/Access/KerberosInit.cpp index 772938ad9b2..3cda1c8e13c 100644 --- a/src/Access/KerberosInit.cpp +++ b/src/Access/KerberosInit.cpp @@ -63,7 +63,7 @@ String KerberosInit::fmtError(krb5_error_code code) const void KerberosInit::init(const String & keytab_file, const String & principal, const String & cache_name) { - auto * log = &Poco::Logger::get("KerberosInit"); + auto log = getLogger("KerberosInit"); LOG_TRACE(log,"Trying to authenticate with Kerberos v5"); krb5_error_code ret; diff --git a/src/Access/LDAPClient.cpp b/src/Access/LDAPClient.cpp index 7926298061d..3a0b82b9a76 100644 --- a/src/Access/LDAPClient.cpp +++ b/src/Access/LDAPClient.cpp @@ -532,7 +532,7 @@ LDAPClient::SearchResults LDAPClient::search(const SearchParams & search_params) for (size_t i = 0; referrals[i]; ++i) { - LOG_WARNING(&Poco::Logger::get("LDAPClient"), "Received reference during LDAP search but not following it: {}", referrals[i]); + LOG_WARNING(getLogger("LDAPClient"), "Received reference during LDAP search but not following it: {}", referrals[i]); } } diff --git a/src/Access/RowPolicyCache.cpp b/src/Access/RowPolicyCache.cpp index bb9da674477..13140099a63 100644 --- a/src/Access/RowPolicyCache.cpp +++ b/src/Access/RowPolicyCache.cpp @@ -91,7 +91,7 @@ void RowPolicyCache::PolicyInfo::setPolicy(const RowPolicyPtr & policy_) catch (...) { tryLogCurrentException( - &Poco::Logger::get("RowPolicy"), + getLogger("RowPolicy"), String("Could not parse the condition ") + toString(filter_type) + " of row policy " + backQuote(policy->getName())); } diff --git a/src/Access/SettingsAuthResponseParser.cpp b/src/Access/SettingsAuthResponseParser.cpp index 62d15f1dcfc..a90ae61f93a 100644 --- a/src/Access/SettingsAuthResponseParser.cpp +++ b/src/Access/SettingsAuthResponseParser.cpp @@ -37,7 +37,7 @@ SettingsAuthResponseParser::parse(const Poco::Net::HTTPResponse & response, std: } catch (...) { - LOG_INFO(&Poco::Logger::get("HTTPAuthentication"), "Failed to parse settings from authentication response. Skip it."); + LOG_INFO(getLogger("HTTPAuthentication"), "Failed to parse settings from authentication response. Skip it."); } return result; } diff --git a/src/Backups/BackupCoordinationLocal.cpp b/src/Backups/BackupCoordinationLocal.cpp index fb91bae2303..9964de2ad6e 100644 --- a/src/Backups/BackupCoordinationLocal.cpp +++ b/src/Backups/BackupCoordinationLocal.cpp @@ -9,7 +9,7 @@ namespace DB { BackupCoordinationLocal::BackupCoordinationLocal(bool plain_backup_) - : log(&Poco::Logger::get("BackupCoordinationLocal")), file_infos(plain_backup_) + : log(getLogger("BackupCoordinationLocal")), file_infos(plain_backup_) { } diff --git a/src/Backups/BackupCoordinationLocal.h b/src/Backups/BackupCoordinationLocal.h index f73cbbe29a8..7719ffd3e52 100644 --- a/src/Backups/BackupCoordinationLocal.h +++ b/src/Backups/BackupCoordinationLocal.h @@ -57,7 +57,7 @@ public: bool hasConcurrentBackups(const std::atomic & num_active_backups) const override; private: - Poco::Logger * const log; + LoggerPtr const log; BackupCoordinationReplicatedTables TSA_GUARDED_BY(replicated_tables_mutex) replicated_tables; BackupCoordinationReplicatedAccess TSA_GUARDED_BY(replicated_access_mutex) replicated_access; diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index 4662f436aba..9c509858b2a 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -173,7 +173,7 @@ BackupCoordinationRemote::BackupCoordinationRemote( , current_host_index(findCurrentHostIndex(all_hosts, current_host)) , plain_backup(plain_backup_) , is_internal(is_internal_) - , log(&Poco::Logger::get("BackupCoordinationRemote")) + , log(getLogger("BackupCoordinationRemote")) , with_retries( log, get_zookeeper_, diff --git a/src/Backups/BackupCoordinationRemote.h b/src/Backups/BackupCoordinationRemote.h index 81980ee5637..ce891699bd2 100644 --- a/src/Backups/BackupCoordinationRemote.h +++ b/src/Backups/BackupCoordinationRemote.h @@ -102,7 +102,7 @@ private: const size_t current_host_index; const bool plain_backup; const bool is_internal; - Poco::Logger * const log; + LoggerPtr const log; /// The order of these two fields matters, because stage_sync holds a reference to with_retries object mutable WithRetries with_retries; diff --git a/src/Backups/BackupCoordinationStageSync.cpp b/src/Backups/BackupCoordinationStageSync.cpp index 2eba3440be9..17ef163ce35 100644 --- a/src/Backups/BackupCoordinationStageSync.cpp +++ b/src/Backups/BackupCoordinationStageSync.cpp @@ -24,7 +24,7 @@ namespace ErrorCodes BackupCoordinationStageSync::BackupCoordinationStageSync( const String & root_zookeeper_path_, WithRetries & with_retries_, - Poco::Logger * log_) + LoggerPtr log_) : zookeeper_path(root_zookeeper_path_ + "/stage") , with_retries(with_retries_) , log(log_) diff --git a/src/Backups/BackupCoordinationStageSync.h b/src/Backups/BackupCoordinationStageSync.h index e34fbcc099b..a06c5c61041 100644 --- a/src/Backups/BackupCoordinationStageSync.h +++ b/src/Backups/BackupCoordinationStageSync.h @@ -12,7 +12,7 @@ public: BackupCoordinationStageSync( const String & root_zookeeper_path_, WithRetries & with_retries_, - Poco::Logger * log_); + LoggerPtr log_); /// Sets the stage of the current host and signal other hosts if there were other hosts waiting for that. void set(const String & current_host, const String & new_stage, const String & message, const bool & all_hosts = false); @@ -36,7 +36,7 @@ private: String zookeeper_path; /// A reference to the field of parent object - BackupCoordinationRemote or RestoreCoordinationRemote WithRetries & with_retries; - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/src/Backups/BackupEntriesCollector.cpp b/src/Backups/BackupEntriesCollector.cpp index bf9cf50a67a..5c0b80aa894 100644 --- a/src/Backups/BackupEntriesCollector.cpp +++ b/src/Backups/BackupEntriesCollector.cpp @@ -97,7 +97,7 @@ BackupEntriesCollector::BackupEntriesCollector( , max_sleep_before_next_attempt_to_collect_metadata( context->getConfigRef().getUInt64("backups.max_sleep_before_next_attempt_to_collect_metadata", 5000)) , compare_collected_metadata(context->getConfigRef().getBool("backups.compare_collected_metadata", true)) - , log(&Poco::Logger::get("BackupEntriesCollector")) + , log(getLogger("BackupEntriesCollector")) , global_zookeeper_retries_info( context->getSettingsRef().backup_restore_keeper_max_retries, context->getSettingsRef().backup_restore_keeper_retry_initial_backoff_ms, diff --git a/src/Backups/BackupEntriesCollector.h b/src/Backups/BackupEntriesCollector.h index fcbc5e5985f..bad67e494c4 100644 --- a/src/Backups/BackupEntriesCollector.h +++ b/src/Backups/BackupEntriesCollector.h @@ -129,7 +129,7 @@ private: /// Whether we should collect the metadata after a successful attempt one more time and check that nothing has changed. const bool compare_collected_metadata; - Poco::Logger * log; + LoggerPtr log; /// Unfortunately we can use ZooKeeper for collecting information for backup /// and we need to retry... ZooKeeperRetriesInfo global_zookeeper_retries_info; diff --git a/src/Backups/BackupFileInfo.cpp b/src/Backups/BackupFileInfo.cpp index 2a1642f3b84..f14b955149e 100644 --- a/src/Backups/BackupFileInfo.cpp +++ b/src/Backups/BackupFileInfo.cpp @@ -102,7 +102,7 @@ BackupFileInfo buildFileInfoForBackupEntry( const BackupEntryPtr & backup_entry, const BackupPtr & base_backup, const ReadSettings & read_settings, - Poco::Logger * log) + LoggerPtr log) { auto adjusted_path = removeLeadingSlash(file_name); @@ -129,7 +129,7 @@ BackupFileInfo buildFileInfoForBackupEntry( } if (!log) - log = &Poco::Logger::get("FileInfoFromBackupEntry"); + log = getLogger("FileInfoFromBackupEntry"); std::optional base_backup_file_info = getInfoAboutFileFromBaseBackupIfExists(base_backup, adjusted_path); @@ -216,7 +216,7 @@ BackupFileInfos buildFileInfosForBackupEntries(const BackupEntries & backup_entr std::exception_ptr exception; auto thread_group = CurrentThread::getGroup(); - Poco::Logger * log = &Poco::Logger::get("FileInfosFromBackupEntries"); + LoggerPtr log = getLogger("FileInfosFromBackupEntries"); for (size_t i = 0; i != backup_entries.size(); ++i) { diff --git a/src/Backups/BackupFileInfo.h b/src/Backups/BackupFileInfo.h index 15c38595002..a1405a9cafb 100644 --- a/src/Backups/BackupFileInfo.h +++ b/src/Backups/BackupFileInfo.h @@ -2,8 +2,7 @@ #include #include - -namespace Poco { class Logger; } +#include namespace DB { @@ -77,7 +76,7 @@ struct BackupFileInfo using BackupFileInfos = std::vector; /// Builds a BackupFileInfo for a specified backup entry. -BackupFileInfo buildFileInfoForBackupEntry(const String & file_name, const BackupEntryPtr & backup_entry, const BackupPtr & base_backup, const ReadSettings & read_settings, Poco::Logger * log); +BackupFileInfo buildFileInfoForBackupEntry(const String & file_name, const BackupEntryPtr & backup_entry, const BackupPtr & base_backup, const ReadSettings & read_settings, LoggerPtr log); /// Builds a vector of BackupFileInfos for specified backup entries. BackupFileInfos buildFileInfosForBackupEntries(const BackupEntries & backup_entries, const BackupPtr & base_backup, const ReadSettings & read_settings, ThreadPool & thread_pool, QueryStatusPtr process_list_element); diff --git a/src/Backups/BackupIO_Default.cpp b/src/Backups/BackupIO_Default.cpp index 5ac522695ce..77fd4532c8f 100644 --- a/src/Backups/BackupIO_Default.cpp +++ b/src/Backups/BackupIO_Default.cpp @@ -10,7 +10,7 @@ namespace DB { -BackupReaderDefault::BackupReaderDefault(const ReadSettings & read_settings_, const WriteSettings & write_settings_, Poco::Logger * log_) +BackupReaderDefault::BackupReaderDefault(const ReadSettings & read_settings_, const WriteSettings & write_settings_, LoggerPtr log_) : log(log_) , read_settings(read_settings_) , write_settings(write_settings_) @@ -36,7 +36,7 @@ void BackupReaderDefault::copyFileToDisk(const String & path_in_backup, size_t f write_buffer->finalize(); } -BackupWriterDefault::BackupWriterDefault(const ReadSettings & read_settings_, const WriteSettings & write_settings_, Poco::Logger * log_) +BackupWriterDefault::BackupWriterDefault(const ReadSettings & read_settings_, const WriteSettings & write_settings_, LoggerPtr log_) : log(log_) , read_settings(read_settings_) , write_settings(write_settings_) diff --git a/src/Backups/BackupIO_Default.h b/src/Backups/BackupIO_Default.h index b4888fecd2f..639293f22d9 100644 --- a/src/Backups/BackupIO_Default.h +++ b/src/Backups/BackupIO_Default.h @@ -18,7 +18,7 @@ enum class WriteMode; class BackupReaderDefault : public IBackupReader { public: - BackupReaderDefault(const ReadSettings & read_settings_, const WriteSettings & write_settings_, Poco::Logger * log_); + BackupReaderDefault(const ReadSettings & read_settings_, const WriteSettings & write_settings_, LoggerPtr log_); ~BackupReaderDefault() override = default; /// The function copyFileToDisk() can be much faster than reading the file with readFile() and then writing it to some disk. @@ -33,7 +33,7 @@ public: size_t getWriteBufferSize() const override { return write_buffer_size; } protected: - Poco::Logger * const log; + LoggerPtr const log; const ReadSettings read_settings; /// The write settings are used to write to the source disk in copyFileToDisk(). @@ -45,7 +45,7 @@ protected: class BackupWriterDefault : public IBackupWriter { public: - BackupWriterDefault(const ReadSettings & read_settings_, const WriteSettings & write_settings_, Poco::Logger * log_); + BackupWriterDefault(const ReadSettings & read_settings_, const WriteSettings & write_settings_, LoggerPtr log_); ~BackupWriterDefault() override = default; bool fileContentsEqual(const String & file_name, const String & expected_file_contents) override; @@ -60,7 +60,7 @@ protected: /// Here readFile() is used only to implement fileContentsEqual(). virtual std::unique_ptr readFile(const String & file_name, size_t expected_file_size) = 0; - Poco::Logger * const log; + LoggerPtr const log; /// The read settings are used to read from the source disk in copyFileFromDisk(). const ReadSettings read_settings; diff --git a/src/Backups/BackupIO_Disk.cpp b/src/Backups/BackupIO_Disk.cpp index 91e8b97bc20..27b594f6bb8 100644 --- a/src/Backups/BackupIO_Disk.cpp +++ b/src/Backups/BackupIO_Disk.cpp @@ -9,7 +9,7 @@ namespace DB { BackupReaderDisk::BackupReaderDisk(const DiskPtr & disk_, const String & root_path_, const ReadSettings & read_settings_, const WriteSettings & write_settings_) - : BackupReaderDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupReaderDisk")) + : BackupReaderDefault(read_settings_, write_settings_, getLogger("BackupReaderDisk")) , disk(disk_) , root_path(root_path_) , data_source_description(disk->getDataSourceDescription()) @@ -57,7 +57,7 @@ void BackupReaderDisk::copyFileToDisk(const String & path_in_backup, size_t file BackupWriterDisk::BackupWriterDisk(const DiskPtr & disk_, const String & root_path_, const ReadSettings & read_settings_, const WriteSettings & write_settings_) - : BackupWriterDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupWriterDisk")) + : BackupWriterDefault(read_settings_, write_settings_, getLogger("BackupWriterDisk")) , disk(disk_) , root_path(root_path_) , data_source_description(disk->getDataSourceDescription()) diff --git a/src/Backups/BackupIO_File.cpp b/src/Backups/BackupIO_File.cpp index 5384637a969..35544a526f1 100644 --- a/src/Backups/BackupIO_File.cpp +++ b/src/Backups/BackupIO_File.cpp @@ -17,7 +17,7 @@ namespace ErrorCodes } BackupReaderFile::BackupReaderFile(const String & root_path_, const ReadSettings & read_settings_, const WriteSettings & write_settings_) - : BackupReaderDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupReaderFile")) + : BackupReaderDefault(read_settings_, write_settings_, getLogger("BackupReaderFile")) , root_path(root_path_) , data_source_description(DiskLocal::getLocalDataSourceDescription(root_path)) { @@ -75,7 +75,7 @@ void BackupReaderFile::copyFileToDisk(const String & path_in_backup, size_t file BackupWriterFile::BackupWriterFile(const String & root_path_, const ReadSettings & read_settings_, const WriteSettings & write_settings_) - : BackupWriterDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupWriterFile")) + : BackupWriterDefault(read_settings_, write_settings_, getLogger("BackupWriterFile")) , root_path(root_path_) , data_source_description(DiskLocal::getLocalDataSourceDescription(root_path)) { diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 381c58dd045..fa4c1af3698 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -124,7 +124,7 @@ BackupReaderS3::BackupReaderS3( const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_) - : BackupReaderDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupReaderS3")) + : BackupReaderDefault(read_settings_, write_settings_, getLogger("BackupReaderS3")) , s3_uri(s3_uri_) , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::S3, MetadataStorageType::None, s3_uri.endpoint, false, false} , s3_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString())) @@ -214,7 +214,7 @@ BackupWriterS3::BackupWriterS3( const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_) - : BackupWriterDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupWriterS3")) + : BackupWriterDefault(read_settings_, write_settings_, getLogger("BackupWriterS3")) , s3_uri(s3_uri_) , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::S3, MetadataStorageType::None, s3_uri.endpoint, false, false} , s3_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString())) diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index 9ac68bc2437..0961c867cab 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -105,7 +105,7 @@ BackupImpl::BackupImpl( , version(INITIAL_BACKUP_VERSION) , base_backup_info(base_backup_info_) , use_same_s3_credentials_for_base_backup(use_same_s3_credentials_for_base_backup_) - , log(&Poco::Logger::get("BackupImpl")) + , log(getLogger("BackupImpl")) { open(); } @@ -136,7 +136,7 @@ BackupImpl::BackupImpl( , base_backup_info(base_backup_info_) , deduplicate_files(deduplicate_files_) , use_same_s3_credentials_for_base_backup(use_same_s3_credentials_for_base_backup_) - , log(&Poco::Logger::get("BackupImpl")) + , log(getLogger("BackupImpl")) { open(); } diff --git a/src/Backups/BackupImpl.h b/src/Backups/BackupImpl.h index b369fe00171..e9803b46bb4 100644 --- a/src/Backups/BackupImpl.h +++ b/src/Backups/BackupImpl.h @@ -153,7 +153,7 @@ private: bool writing_finalized = false; bool deduplicate_files = true; bool use_same_s3_credentials_for_base_backup = false; - const Poco::Logger * log; + const LoggerPtr log; }; } diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 9c1b6d8af97..c19be22c749 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -380,7 +380,7 @@ BackupsWorker::BackupsWorker(ContextMutablePtr global_context, size_t num_backup , allow_concurrent_backups(allow_concurrent_backups_) , allow_concurrent_restores(allow_concurrent_restores_) , test_inject_sleep(test_inject_sleep_) - , log(&Poco::Logger::get("BackupsWorker")) + , log(getLogger("BackupsWorker")) , backup_log(global_context->getBackupLog()) , process_list(global_context->getProcessList()) { diff --git a/src/Backups/BackupsWorker.h b/src/Backups/BackupsWorker.h index 7a514e7032b..73c8bf19473 100644 --- a/src/Backups/BackupsWorker.h +++ b/src/Backups/BackupsWorker.h @@ -127,7 +127,7 @@ private: const bool allow_concurrent_restores; const bool test_inject_sleep; - Poco::Logger * log; + LoggerPtr log; struct ExtendedOperationInfo { diff --git a/src/Backups/RestoreCoordinationLocal.cpp b/src/Backups/RestoreCoordinationLocal.cpp index cf606151b53..4a91b8d95a5 100644 --- a/src/Backups/RestoreCoordinationLocal.cpp +++ b/src/Backups/RestoreCoordinationLocal.cpp @@ -6,7 +6,7 @@ namespace DB { -RestoreCoordinationLocal::RestoreCoordinationLocal() : log(&Poco::Logger::get("RestoreCoordinationLocal")) +RestoreCoordinationLocal::RestoreCoordinationLocal() : log(getLogger("RestoreCoordinationLocal")) { } diff --git a/src/Backups/RestoreCoordinationLocal.h b/src/Backups/RestoreCoordinationLocal.h index 7f6ffe1eeec..5e51b719d63 100644 --- a/src/Backups/RestoreCoordinationLocal.h +++ b/src/Backups/RestoreCoordinationLocal.h @@ -51,7 +51,7 @@ public: bool hasConcurrentRestores(const std::atomic & num_active_restores) const override; private: - Poco::Logger * const log; + LoggerPtr const log; std::set> acquired_tables_in_replicated_databases; std::unordered_set acquired_data_in_replicated_tables; diff --git a/src/Backups/RestoreCoordinationRemote.cpp b/src/Backups/RestoreCoordinationRemote.cpp index 0d2b3832bad..84106737fc9 100644 --- a/src/Backups/RestoreCoordinationRemote.cpp +++ b/src/Backups/RestoreCoordinationRemote.cpp @@ -32,7 +32,7 @@ RestoreCoordinationRemote::RestoreCoordinationRemote( , current_host(current_host_) , current_host_index(BackupCoordinationRemote::findCurrentHostIndex(all_hosts, current_host)) , is_internal(is_internal_) - , log(&Poco::Logger::get("RestoreCoordinationRemote")) + , log(getLogger("RestoreCoordinationRemote")) , with_retries( log, get_zookeeper_, diff --git a/src/Backups/RestoreCoordinationRemote.h b/src/Backups/RestoreCoordinationRemote.h index f7e678645df..9c299865cfa 100644 --- a/src/Backups/RestoreCoordinationRemote.h +++ b/src/Backups/RestoreCoordinationRemote.h @@ -73,7 +73,7 @@ private: const String current_host; const size_t current_host_index; const bool is_internal; - Poco::Logger * const log; + LoggerPtr const log; mutable WithRetries with_retries; std::optional stage_sync; diff --git a/src/Backups/RestorerFromBackup.cpp b/src/Backups/RestorerFromBackup.cpp index f218410e599..6f18c070cd7 100644 --- a/src/Backups/RestorerFromBackup.cpp +++ b/src/Backups/RestorerFromBackup.cpp @@ -89,7 +89,7 @@ RestorerFromBackup::RestorerFromBackup( , process_list_element(context->getProcessListElement()) , on_cluster_first_sync_timeout(context->getConfigRef().getUInt64("backups.on_cluster_first_sync_timeout", 180000)) , create_table_timeout(context->getConfigRef().getUInt64("backups.create_table_timeout", 300000)) - , log(&Poco::Logger::get("RestorerFromBackup")) + , log(getLogger("RestorerFromBackup")) , tables_dependencies("RestorerFromBackup") { } diff --git a/src/Backups/RestorerFromBackup.h b/src/Backups/RestorerFromBackup.h index fad79a3a2e6..5e4ee0c3832 100644 --- a/src/Backups/RestorerFromBackup.h +++ b/src/Backups/RestorerFromBackup.h @@ -79,7 +79,7 @@ private: QueryStatusPtr process_list_element; std::chrono::milliseconds on_cluster_first_sync_timeout; std::chrono::milliseconds create_table_timeout; - Poco::Logger * log; + LoggerPtr log; Strings all_hosts; DDLRenamingMap renaming_map; diff --git a/src/Backups/WithRetries.cpp b/src/Backups/WithRetries.cpp index db36bc92d92..66851fa42ce 100644 --- a/src/Backups/WithRetries.cpp +++ b/src/Backups/WithRetries.cpp @@ -21,7 +21,7 @@ WithRetries::KeeperSettings WithRetries::KeeperSettings::fromContext(ContextPtr } WithRetries::WithRetries( - Poco::Logger * log_, zkutil::GetZooKeeper get_zookeeper_, const KeeperSettings & settings_, QueryStatusPtr process_list_element_, RenewerCallback callback_) + LoggerPtr log_, zkutil::GetZooKeeper get_zookeeper_, const KeeperSettings & settings_, QueryStatusPtr process_list_element_, RenewerCallback callback_) : log(log_) , get_zookeeper(get_zookeeper_) , settings(settings_) diff --git a/src/Backups/WithRetries.h b/src/Backups/WithRetries.h index edfccc658d9..3a6e28996b9 100644 --- a/src/Backups/WithRetries.h +++ b/src/Backups/WithRetries.h @@ -52,7 +52,7 @@ public: }; RetriesControlHolder createRetriesControlHolder(const String & name); - WithRetries(Poco::Logger * log, zkutil::GetZooKeeper get_zookeeper_, const KeeperSettings & settings, QueryStatusPtr process_list_element_, RenewerCallback callback); + WithRetries(LoggerPtr log, zkutil::GetZooKeeper get_zookeeper_, const KeeperSettings & settings, QueryStatusPtr process_list_element_, RenewerCallback callback); /// Used to re-establish new connection inside a retry loop. void renewZooKeeper(FaultyKeeper my_faulty_zookeeper) const; @@ -62,7 +62,7 @@ private: /// This will provide a special wrapper which is useful for testing FaultyKeeper getFaultyZooKeeper() const; - Poco::Logger * log; + LoggerPtr log; zkutil::GetZooKeeper get_zookeeper; KeeperSettings settings; QueryStatusPtr process_list_element; diff --git a/src/BridgeHelper/IBridgeHelper.h b/src/BridgeHelper/IBridgeHelper.h index 272d97c8a78..6812bd04a03 100644 --- a/src/BridgeHelper/IBridgeHelper.h +++ b/src/BridgeHelper/IBridgeHelper.h @@ -51,7 +51,7 @@ protected: virtual const Poco::Util::AbstractConfiguration & getConfig() const = 0; - virtual Poco::Logger * getLog() const = 0; + virtual LoggerPtr getLog() const = 0; virtual Poco::Timespan getHTTPTimeout() const = 0; diff --git a/src/BridgeHelper/LibraryBridgeHelper.cpp b/src/BridgeHelper/LibraryBridgeHelper.cpp index e83707595b9..84bfe096e79 100644 --- a/src/BridgeHelper/LibraryBridgeHelper.cpp +++ b/src/BridgeHelper/LibraryBridgeHelper.cpp @@ -8,7 +8,7 @@ namespace DB LibraryBridgeHelper::LibraryBridgeHelper(ContextPtr context_) : IBridgeHelper(context_) , config(context_->getConfigRef()) - , log(&Poco::Logger::get("LibraryBridgeHelper")) + , log(getLogger("LibraryBridgeHelper")) , http_timeout(context_->getGlobalContext()->getSettingsRef().http_receive_timeout.value) , bridge_host(config.getString("library_bridge.host", DEFAULT_HOST)) , bridge_port(config.getUInt("library_bridge.port", DEFAULT_PORT)) diff --git a/src/BridgeHelper/LibraryBridgeHelper.h b/src/BridgeHelper/LibraryBridgeHelper.h index 1723d1f8fb4..8940f9d1c9e 100644 --- a/src/BridgeHelper/LibraryBridgeHelper.h +++ b/src/BridgeHelper/LibraryBridgeHelper.h @@ -31,7 +31,7 @@ protected: const Poco::Util::AbstractConfiguration & getConfig() const override { return config; } - Poco::Logger * getLog() const override { return log; } + LoggerPtr getLog() const override { return log; } Poco::Timespan getHTTPTimeout() const override { return http_timeout; } @@ -40,7 +40,7 @@ protected: static constexpr inline size_t DEFAULT_PORT = 9012; const Poco::Util::AbstractConfiguration & config; - Poco::Logger * log; + LoggerPtr log; const Poco::Timespan http_timeout; std::string bridge_host; size_t bridge_port; diff --git a/src/BridgeHelper/XDBCBridgeHelper.h b/src/BridgeHelper/XDBCBridgeHelper.h index 060de74b5b1..d208b8ddab0 100644 --- a/src/BridgeHelper/XDBCBridgeHelper.h +++ b/src/BridgeHelper/XDBCBridgeHelper.h @@ -65,7 +65,7 @@ public: const std::string & connection_string_, bool use_connection_pooling_) : IXDBCBridgeHelper(context_->getGlobalContext()) - , log(&Poco::Logger::get(BridgeHelperMixin::getName() + "BridgeHelper")) + , log(getLogger(BridgeHelperMixin::getName() + "BridgeHelper")) , connection_string(connection_string_) , use_connection_pooling(use_connection_pooling_) , http_timeout(http_timeout_) @@ -123,7 +123,7 @@ protected: const Poco::Util::AbstractConfiguration & getConfig() const override { return config; } - Poco::Logger * getLog() const override { return log; } + LoggerPtr getLog() const override { return log; } bool startBridgeManually() const override { return BridgeHelperMixin::startBridgeManually(); } @@ -146,7 +146,7 @@ protected: private: using Configuration = Poco::Util::AbstractConfiguration; - Poco::Logger * log; + LoggerPtr log; std::string connection_string; bool use_connection_pooling; Poco::Timespan http_timeout; diff --git a/src/Client/Connection.h b/src/Client/Connection.h index 2f209fc92d0..e93a7539d15 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -4,6 +4,7 @@ #include #include +#include #include #include @@ -244,16 +245,18 @@ private: { } - Poco::Logger * get() + LoggerPtr get() { - if (!log) - log = &Poco::Logger::get("Connection (" + parent.getDescription() + ")"); + callOnce(log_initialized, [&] { + log = getLogger("Connection (" + parent.getDescription() + ")"); + }); return log; } private: - std::atomic log; + OnceFlag log_initialized; + LoggerPtr log; Connection & parent; }; diff --git a/src/Client/ConnectionEstablisher.cpp b/src/Client/ConnectionEstablisher.cpp index e5b1347add5..a9009e5bb25 100644 --- a/src/Client/ConnectionEstablisher.cpp +++ b/src/Client/ConnectionEstablisher.cpp @@ -25,7 +25,7 @@ ConnectionEstablisher::ConnectionEstablisher( IConnectionPool * pool_, const ConnectionTimeouts * timeouts_, const Settings & settings_, - Poco::Logger * log_, + LoggerPtr log_, const QualifiedTableName * table_to_check_) : pool(pool_), timeouts(timeouts_), settings(settings_), log(log_), table_to_check(table_to_check_), is_finished(false) { @@ -114,7 +114,7 @@ ConnectionEstablisherAsync::ConnectionEstablisherAsync( IConnectionPool * pool_, const ConnectionTimeouts * timeouts_, const Settings & settings_, - Poco::Logger * log_, + LoggerPtr log_, const QualifiedTableName * table_to_check_) : AsyncTaskExecutor(std::make_unique(*this)), connection_establisher(pool_, timeouts_, settings_, log_, table_to_check_) { diff --git a/src/Client/ConnectionEstablisher.h b/src/Client/ConnectionEstablisher.h index 880e44c8a1a..7ea65708b1d 100644 --- a/src/Client/ConnectionEstablisher.h +++ b/src/Client/ConnectionEstablisher.h @@ -23,7 +23,7 @@ public: ConnectionEstablisher(IConnectionPool * pool_, const ConnectionTimeouts * timeouts_, const Settings & settings_, - Poco::Logger * log, + LoggerPtr log, const QualifiedTableName * table_to_check = nullptr); /// Establish connection and save it in result, write possible exception message in fail_message. @@ -38,7 +38,7 @@ private: IConnectionPool * pool; const ConnectionTimeouts * timeouts; const Settings & settings; - Poco::Logger * log; + LoggerPtr log; const QualifiedTableName * table_to_check; bool is_finished; @@ -61,7 +61,7 @@ public: ConnectionEstablisherAsync(IConnectionPool * pool_, const ConnectionTimeouts * timeouts_, const Settings & settings_, - Poco::Logger * log_, + LoggerPtr log_, const QualifiedTableName * table_to_check_ = nullptr); /// Get file descriptor that can be added in epoll and be polled, diff --git a/src/Client/ConnectionPool.h b/src/Client/ConnectionPool.h index d663c052abc..1886a0431a5 100644 --- a/src/Client/ConnectionPool.h +++ b/src/Client/ConnectionPool.h @@ -64,7 +64,7 @@ public: Protocol::Secure secure_, Priority priority_ = Priority{1}) : Base(max_connections_, - &Poco::Logger::get("ConnectionPool (" + host_ + ":" + toString(port_) + ")")), + getLogger("ConnectionPool (" + host_ + ":" + toString(port_) + ")")), host(host_), port(port_), default_database(default_database_), diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index 43166659b18..c05fbb317c8 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -29,7 +29,7 @@ ConnectionPoolWithFailover::ConnectionPoolWithFailover( LoadBalancing load_balancing, time_t decrease_error_period_, size_t max_error_cap_) - : Base(std::move(nested_pools_), decrease_error_period_, max_error_cap_, &Poco::Logger::get("ConnectionPoolWithFailover")) + : Base(std::move(nested_pools_), decrease_error_period_, max_error_cap_, getLogger("ConnectionPoolWithFailover")) , get_priority_load_balancing(load_balancing) { const std::string & local_hostname = getFQDNOrHostName(); diff --git a/src/Client/HedgedConnectionsFactory.cpp b/src/Client/HedgedConnectionsFactory.cpp index f7b5ceedc96..01f9a32ce75 100644 --- a/src/Client/HedgedConnectionsFactory.cpp +++ b/src/Client/HedgedConnectionsFactory.cpp @@ -34,7 +34,7 @@ HedgedConnectionsFactory::HedgedConnectionsFactory( : pool(pool_) , timeouts(timeouts_) , table_to_check(table_to_check_) - , log(&Poco::Logger::get("HedgedConnectionsFactory")) + , log(getLogger("HedgedConnectionsFactory")) , max_tries(max_tries_) , fallback_to_stale_replicas(fallback_to_stale_replicas_) , max_parallel_replicas(max_parallel_replicas_) diff --git a/src/Client/HedgedConnectionsFactory.h b/src/Client/HedgedConnectionsFactory.h index f187e9b2abb..ce7b553acdd 100644 --- a/src/Client/HedgedConnectionsFactory.h +++ b/src/Client/HedgedConnectionsFactory.h @@ -133,7 +133,7 @@ private: std::shared_ptr table_to_check; int last_used_index = -1; Epoll epoll; - Poco::Logger * log; + LoggerPtr log; std::string fail_messages; /// The maximum number of attempts to connect to replicas. diff --git a/src/Common/Allocator.cpp b/src/Common/Allocator.cpp index 24a96c1c85b..e80c125c2a0 100644 --- a/src/Common/Allocator.cpp +++ b/src/Common/Allocator.cpp @@ -50,7 +50,7 @@ void prefaultPages([[maybe_unused]] void * buf_, [[maybe_unused]] size_t len_) auto [buf, len] = adjustToPageSize(buf_, len_, page_size); if (::madvise(buf, len, MADV_POPULATE_WRITE) < 0) LOG_TRACE( - LogFrequencyLimiter(&Poco::Logger::get("Allocator"), 1), + LogFrequencyLimiter(getLogger("Allocator"), 1), "Attempt to populate pages failed: {} (EINVAL is expected for kernels < 5.14)", errnoToString(errno)); #endif diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 3b7eac3e0d4..e9de95363bc 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -34,7 +34,7 @@ namespace ErrorCodes static constexpr size_t PRINT_MESSAGE_EACH_N_OBJECTS = 256; static constexpr size_t PRINT_MESSAGE_EACH_N_SECONDS = 5; -void logAboutProgress(Poco::Logger * log, size_t processed, size_t total, AtomicStopwatch & watch) +void logAboutProgress(LoggerPtr log, size_t processed, size_t total, AtomicStopwatch & watch) { if (total && (processed % PRINT_MESSAGE_EACH_N_OBJECTS == 0 || watch.compareAndRestart(PRINT_MESSAGE_EACH_N_SECONDS))) { @@ -205,7 +205,7 @@ void LoadTask::detach() AsyncLoader::AsyncLoader(std::vector pool_initializers, bool log_failures_, bool log_progress_) : log_failures(log_failures_) , log_progress(log_progress_) - , log(&Poco::Logger::get("AsyncLoader")) + , log(getLogger("AsyncLoader")) { pools.reserve(pool_initializers.size()); for (auto && init : pool_initializers) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index b02bc2ac06a..c2a9c901f1c 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -15,6 +15,7 @@ #include #include #include +#include namespace Poco { class Logger; } @@ -40,7 +41,7 @@ using LoadTaskPtr = std::shared_ptr; using LoadTaskPtrs = std::vector; class AsyncLoader; -void logAboutProgress(Poco::Logger * log, size_t processed, size_t total, AtomicStopwatch & watch); +void logAboutProgress(LoggerPtr log, size_t processed, size_t total, AtomicStopwatch & watch); // Execution status of a load job. enum class LoadStatus @@ -419,7 +420,7 @@ private: // Logging const bool log_failures; // Worker should log all exceptions caught from job functions. const bool log_progress; // Periodically log total progress - Poco::Logger * log; + LoggerPtr log; mutable std::mutex mutex; // Guards all the fields below. bool is_running = true; diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index ec94d17f590..b24d9bcc301 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -58,7 +58,7 @@ AsynchronousMetrics::AsynchronousMetrics( int update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_) : update_period(update_period_seconds) - , log(&Poco::Logger::get("AsynchronousMetrics")) + , log(getLogger("AsynchronousMetrics")) , protocol_server_metrics_func(protocol_server_metrics_func_) { #if defined(OS_LINUX) @@ -125,7 +125,7 @@ void AsynchronousMetrics::openSensors() TSA_REQUIRES(data_mutex) catch (const ErrnoException & e) { LOG_WARNING( - &Poco::Logger::get("AsynchronousMetrics"), + getLogger("AsynchronousMetrics"), "Thermal monitor '{}' exists but could not be read: {}.", thermal_device_index, errnoToString(e.getErrno())); @@ -254,7 +254,7 @@ void AsynchronousMetrics::openSensorsChips() TSA_REQUIRES(data_mutex) catch (const ErrnoException & e) { LOG_WARNING( - &Poco::Logger::get("AsynchronousMetrics"), + getLogger("AsynchronousMetrics"), "Hardware monitor '{}', sensor '{}' exists but could not be read: {}.", hwmon_name, sensor_index, diff --git a/src/Common/AsynchronousMetrics.h b/src/Common/AsynchronousMetrics.h index b9a5862dbff..305e8136b8a 100644 --- a/src/Common/AsynchronousMetrics.h +++ b/src/Common/AsynchronousMetrics.h @@ -82,7 +82,7 @@ public: protected: const Duration update_period; - Poco::Logger * log; + LoggerPtr log; private: virtual void updateImpl(TimePoint update_time, TimePoint current_time, bool force_update, bool first_run, AsynchronousMetricValues & new_values) = 0; virtual void logImpl(AsynchronousMetricValues &) {} diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 92e66fee489..502ce9f0376 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -77,23 +77,17 @@ ConfigProcessor::ConfigProcessor( , name_pool(new Poco::XML::NamePool(65521)) , dom_parser(name_pool) { - if (log_to_console && !Poco::Logger::has("ConfigProcessor")) + if (log_to_console && !hasLogger("ConfigProcessor")) { channel_ptr = new Poco::ConsoleChannel; - log = &Poco::Logger::create("ConfigProcessor", channel_ptr.get(), Poco::Message::PRIO_TRACE); + log = getLogger("ConfigProcessor", channel_ptr.get(), Poco::Message::PRIO_TRACE); } else { - log = &Poco::Logger::get("ConfigProcessor"); + log = getLogger("ConfigProcessor"); } } -ConfigProcessor::~ConfigProcessor() -{ - if (channel_ptr) /// This means we have created a new console logger in the constructor. - Poco::Logger::destroy("ConfigProcessor"); -} - static std::unordered_map embedded_configs; void ConfigProcessor::registerEmbeddedConfig(std::string name, std::string_view content) diff --git a/src/Common/Config/ConfigProcessor.h b/src/Common/Config/ConfigProcessor.h index 98592d8846e..5712c36d737 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -7,6 +7,8 @@ #include #include +#include + #include #include #include @@ -44,8 +46,6 @@ public: bool log_to_console = false, const Substitutions & substitutions = Substitutions()); - ~ConfigProcessor(); - /// Perform config includes and substitutions and return the resulting XML-document. /// /// Suppose path is "/path/file.xml" @@ -125,7 +125,7 @@ private: bool throw_on_bad_incl; - Poco::Logger * log; + LoggerPtr log; Poco::AutoPtr channel_ptr; Substitutions substitutions; diff --git a/src/Common/Config/ConfigReloader.h b/src/Common/Config/ConfigReloader.h index 2529c7a5236..13a797bad08 100644 --- a/src/Common/Config/ConfigReloader.h +++ b/src/Common/Config/ConfigReloader.h @@ -69,7 +69,7 @@ private: static constexpr auto reload_interval = std::chrono::seconds(2); - Poco::Logger * log = &Poco::Logger::get("ConfigReloader"); + LoggerPtr log = getLogger("ConfigReloader"); std::string config_path; std::vector extra_paths; diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index 9cb352da0ba..fcbbaf6b0be 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -104,7 +104,7 @@ DNSResolver::IPAddresses hostByName(const std::string & host) } catch (const Poco::Net::DNSException & e) { - LOG_WARNING(&Poco::Logger::get("DNSResolver"), "Cannot resolve host ({}), error {}: {}.", host, e.code(), e.name()); + LOG_WARNING(getLogger("DNSResolver"), "Cannot resolve host ({}), error {}: {}.", host, e.code(), e.name()); addresses.clear(); } @@ -201,7 +201,7 @@ struct DNSResolver::Impl }; -DNSResolver::DNSResolver() : impl(std::make_unique()), log(&Poco::Logger::get("DNSResolver")) {} +DNSResolver::DNSResolver() : impl(std::make_unique()), log(getLogger("DNSResolver")) {} Poco::Net::IPAddress DNSResolver::resolveHost(const std::string & host) { diff --git a/src/Common/DNSResolver.h b/src/Common/DNSResolver.h index 1017607a5bd..965688f84f2 100644 --- a/src/Common/DNSResolver.h +++ b/src/Common/DNSResolver.h @@ -73,7 +73,7 @@ private: struct Impl; std::unique_ptr impl; - Poco::Logger * log; + LoggerPtr log; /// Updates cached value and returns true it has been changed. bool updateHost(const String & host); diff --git a/src/Common/EnvironmentProxyConfigurationResolver.cpp b/src/Common/EnvironmentProxyConfigurationResolver.cpp index 58507904ce9..f2c60afa1a8 100644 --- a/src/Common/EnvironmentProxyConfigurationResolver.cpp +++ b/src/Common/EnvironmentProxyConfigurationResolver.cpp @@ -50,7 +50,7 @@ ProxyConfiguration EnvironmentProxyConfigurationResolver::resolve() auto scheme = uri.getScheme(); auto port = uri.getPort(); - LOG_TRACE(&Poco::Logger::get("EnvironmentProxyConfigurationResolver"), "Use proxy from environment: {}://{}:{}", scheme, host, port); + LOG_TRACE(getLogger("EnvironmentProxyConfigurationResolver"), "Use proxy from environment: {}://{}:{}", scheme, host, port); return ProxyConfiguration { host, diff --git a/src/Common/ErrorHandlers.h b/src/Common/ErrorHandlers.h index 301377bff83..a4a7c4683aa 100644 --- a/src/Common/ErrorHandlers.h +++ b/src/Common/ErrorHandlers.h @@ -27,7 +27,7 @@ public: void exception() override { logException(); } private: - Poco::Logger * log = &Poco::Logger::get("ServerErrorHandler"); + LoggerPtr log = getLogger("ServerErrorHandler"); void logException() { diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index fcf44d96b43..ff83f6ba807 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -235,8 +235,9 @@ void tryLogCurrentException(const char * log_name, const std::string & start_of_ /// MemoryTracker until the exception will be logged. LockMemoryExceptionInThread lock_memory_tracker(VariableContext::Global); - /// Poco::Logger::get can allocate memory too - tryLogCurrentExceptionImpl(&Poco::Logger::get(log_name), start_of_message); + /// getLogger can allocate memory too + auto logger = getLogger(log_name); + tryLogCurrentExceptionImpl(logger.get(), start_of_message); } void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_message) @@ -251,7 +252,7 @@ void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_ tryLogCurrentExceptionImpl(logger, start_of_message); } -void tryLogCurrentException(const LoggerPtr & logger, const std::string & start_of_message) +void tryLogCurrentException(LoggerPtr logger, const std::string & start_of_message) { tryLogCurrentException(logger.get(), start_of_message); } @@ -516,7 +517,7 @@ void tryLogException(std::exception_ptr e, const char * log_name, const std::str } } -void tryLogException(std::exception_ptr e, Poco::Logger * logger, const std::string & start_of_message) +void tryLogException(std::exception_ptr e, LoggerPtr logger, const std::string & start_of_message) { try { @@ -528,11 +529,6 @@ void tryLogException(std::exception_ptr e, Poco::Logger * logger, const std::str } } -void tryLogException(std::exception_ptr e, const LoggerPtr & logger, const std::string & start_of_message) -{ - tryLogException(e, logger.get(), start_of_message); -} - std::string getExceptionMessage(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace) { return getExceptionMessageAndPattern(e, with_stacktrace, check_embedded_stacktrace).text; diff --git a/src/Common/Exception.h b/src/Common/Exception.h index 77ba2c6db5b..8afed6034cf 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -10,7 +10,7 @@ #include #include #include -#include +#include #include #include @@ -241,9 +241,10 @@ using Exceptions = std::vector; /** Try to write an exception to the log (and forget about it). * Can be used in destructors in the catch-all block. */ +/// TODO: Logger leak constexpr overload void tryLogCurrentException(const char * log_name, const std::string & start_of_message = ""); void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_message = ""); -void tryLogCurrentException(const LoggerPtr & logger, const std::string & start_of_message = ""); +void tryLogCurrentException(LoggerPtr logger, const std::string & start_of_message = ""); /** Prints current exception in canonical format. @@ -286,10 +287,9 @@ struct ExecutionStatus bool tryDeserializeText(const std::string & data); }; - +/// TODO: Logger leak constexpr overload void tryLogException(std::exception_ptr e, const char * log_name, const std::string & start_of_message = ""); -void tryLogException(std::exception_ptr e, Poco::Logger * logger, const std::string & start_of_message = ""); -void tryLogException(std::exception_ptr e, const LoggerPtr & logger, const std::string & start_of_message = ""); +void tryLogException(std::exception_ptr e, LoggerPtr logger, const std::string & start_of_message = ""); std::string getExceptionMessage(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace = false); PreformattedMessage getExceptionMessageAndPattern(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace = false); diff --git a/src/Common/FileChecker.cpp b/src/Common/FileChecker.cpp index 049dee459a7..098ea4b1ac4 100644 --- a/src/Common/FileChecker.cpp +++ b/src/Common/FileChecker.cpp @@ -29,7 +29,7 @@ FileChecker::FileChecker(const String & file_info_path_) : FileChecker(nullptr, FileChecker::FileChecker(DiskPtr disk_, const String & file_info_path_) : disk(std::move(disk_)) - , log(&Poco::Logger::get("FileChecker")) + , log(getLogger("FileChecker")) { setPath(file_info_path_); try diff --git a/src/Common/FileChecker.h b/src/Common/FileChecker.h index b0dd0814edd..c7ba1b91228 100644 --- a/src/Common/FileChecker.h +++ b/src/Common/FileChecker.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -84,7 +85,7 @@ private: size_t getRealFileSize(const String & path_) const; const DiskPtr disk; - const Poco::Logger * log; + const LoggerPtr log; String files_info_path; std::map map; diff --git a/src/Common/FrequencyHolder.cpp b/src/Common/FrequencyHolder.cpp index 7dc1f622aeb..a2548686586 100644 --- a/src/Common/FrequencyHolder.cpp +++ b/src/Common/FrequencyHolder.cpp @@ -34,7 +34,7 @@ FrequencyHolder::FrequencyHolder() void FrequencyHolder::loadEncodingsFrequency() { - Poco::Logger * log = &Poco::Logger::get("EncodingsFrequency"); + LoggerPtr log = getLogger("EncodingsFrequency"); LOG_TRACE(log, "Loading embedded charset frequencies"); @@ -92,7 +92,7 @@ void FrequencyHolder::loadEncodingsFrequency() void FrequencyHolder::loadEmotionalDict() { - Poco::Logger * log = &Poco::Logger::get("EmotionalDict"); + LoggerPtr log = getLogger("EmotionalDict"); LOG_TRACE(log, "Loading embedded emotional dictionary"); std::string_view resource(reinterpret_cast(gresource_tonality_ru_zstData), gresource_tonality_ru_zstSize); @@ -130,7 +130,7 @@ void FrequencyHolder::loadEmotionalDict() void FrequencyHolder::loadProgrammingFrequency() { - Poco::Logger * log = &Poco::Logger::get("ProgrammingFrequency"); + LoggerPtr log = getLogger("ProgrammingFrequency"); LOG_TRACE(log, "Loading embedded programming languages frequencies loading"); diff --git a/src/Common/Jemalloc.cpp b/src/Common/Jemalloc.cpp index a8b9d70e731..3eb8691a1e1 100644 --- a/src/Common/Jemalloc.cpp +++ b/src/Common/Jemalloc.cpp @@ -25,7 +25,7 @@ namespace ErrorCodes void purgeJemallocArenas() { - LOG_TRACE(&Poco::Logger::get("SystemJemalloc"), "Purging unused memory"); + LOG_TRACE(getLogger("SystemJemalloc"), "Purging unused memory"); Stopwatch watch; mallctl("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge", nullptr, nullptr, nullptr, 0); ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurge); @@ -53,12 +53,12 @@ void setJemallocProfileActive(bool value) mallctl("prof.active", &active, &active_size, nullptr, 0); if (active == value) { - LOG_TRACE(&Poco::Logger::get("SystemJemalloc"), "Profiling is already {}", active ? "enabled" : "disabled"); + LOG_TRACE(getLogger("SystemJemalloc"), "Profiling is already {}", active ? "enabled" : "disabled"); return; } mallctl("prof.active", nullptr, nullptr, &value, sizeof(bool)); - LOG_TRACE(&Poco::Logger::get("SystemJemalloc"), "Profiling is {}", value ? "enabled" : "disabled"); + LOG_TRACE(getLogger("SystemJemalloc"), "Profiling is {}", value ? "enabled" : "disabled"); } std::string flushJemallocProfile(const std::string & file_prefix) @@ -69,7 +69,7 @@ std::string flushJemallocProfile(const std::string & file_prefix) int n = mallctl("opt.prof_prefix", &prefix_buffer, &prefix_size, nullptr, 0); if (!n && std::string_view(prefix_buffer) != "jeprof") { - LOG_TRACE(&Poco::Logger::get("SystemJemalloc"), "Flushing memory profile with prefix {}", prefix_buffer); + LOG_TRACE(getLogger("SystemJemalloc"), "Flushing memory profile with prefix {}", prefix_buffer); mallctl("prof.dump", nullptr, nullptr, nullptr, 0); return prefix_buffer; } @@ -78,7 +78,7 @@ std::string flushJemallocProfile(const std::string & file_prefix) std::string profile_dump_path = fmt::format("{}.{}.{}.heap", file_prefix, getpid(), profile_counter.fetch_add(1)); const auto * profile_dump_path_str = profile_dump_path.c_str(); - LOG_TRACE(&Poco::Logger::get("SystemJemalloc"), "Flushing memory profile to {}", profile_dump_path_str); + LOG_TRACE(getLogger("SystemJemalloc"), "Flushing memory profile to {}", profile_dump_path_str); mallctl("prof.dump", nullptr, nullptr, &profile_dump_path_str, sizeof(profile_dump_path_str)); return profile_dump_path; } diff --git a/src/Common/LRUResourceCache.h b/src/Common/LRUResourceCache.h index 1fe3075a2a3..4ccaa272346 100644 --- a/src/Common/LRUResourceCache.h +++ b/src/Common/LRUResourceCache.h @@ -235,7 +235,7 @@ private: else { // should not reach here - LOG_ERROR(&Poco::Logger::get("LRUResourceCache"), "element is in invalid status."); + LOG_ERROR(getLogger("LRUResourceCache"), "element is in invalid status."); abort(); } } @@ -306,7 +306,7 @@ private: auto it = cells.find(key); if (it == cells.end() || it->second.reference_count == 0) { - LOG_ERROR(&Poco::Logger::get("LRUResourceCache"), "try to release an invalid element"); + LOG_ERROR(getLogger("LRUResourceCache"), "try to release an invalid element"); abort(); } @@ -359,7 +359,7 @@ private: auto cell_it = cells.find(key); if (cell_it == cells.end()) { - LOG_ERROR(&Poco::Logger::get("LRUResourceCache"), "LRUResourceCache became inconsistent. There must be a bug in it."); + LOG_ERROR(getLogger("LRUResourceCache"), "LRUResourceCache became inconsistent. There must be a bug in it."); abort(); } @@ -379,7 +379,7 @@ private: if (loss_weight > current_weight + weight) { - LOG_ERROR(&Poco::Logger::get("LRUResourceCache"), "LRUResourceCache became inconsistent. There must be a bug in it."); + LOG_ERROR(getLogger("LRUResourceCache"), "LRUResourceCache became inconsistent. There must be a bug in it."); abort(); } diff --git a/src/Common/Logger.cpp b/src/Common/Logger.cpp new file mode 100644 index 00000000000..fe1bb68df5c --- /dev/null +++ b/src/Common/Logger.cpp @@ -0,0 +1,27 @@ +#include +#include + +LoggerPtr getLogger(const std::string & name) +{ + return Poco::Logger::getShared(name); +} + +LoggerPtr getLogger(const std::string & name, Poco::Channel * channel, Poco::Message::Priority level) +{ + return Poco::Logger::createShared(name, channel, level); +} + +LoggerRawPtr getRawLogger(const std::string & name) +{ + return &Poco::Logger::get(name); +} + +LoggerRawPtr createRawLogger(const std::string & name, Poco::Channel * channel, Poco::Message::Priority level) +{ + return &Poco::Logger::create(name, channel, level); +} + +bool hasLogger(const std::string & name) +{ + return Poco::Logger::has(name); +} diff --git a/src/Common/Logger.h b/src/Common/Logger.h new file mode 100644 index 00000000000..b8137c13e86 --- /dev/null +++ b/src/Common/Logger.h @@ -0,0 +1,50 @@ +#pragma once + +#include + +#include +#include +#include + +using LoggerPtr = Poco::LoggerPtr; + +using LoggerRawPtr = Poco::Logger *; + +/** RAII wrappers around Poco/Logger.h. + * + * You should use this functions in case Logger instance lifetime needs to be properly + * managed, because otherwise it will leak memory. + * + * For example when Logger is created when table is created and Logger contains table name. + * Then it must be destroyed when underlying table is destroyed. + */ + +/** Get Logger with specified name. If the Logger does not exists, it is created. + * Logger is destroyed, when last shared ptr that refers to Logger with specified name is destroyed. + */ +LoggerPtr getLogger(const std::string & name); + +/** Create Logger with specified name, channel and logging level. + * If Logger already exists, throws exception. + * Logger is destroyed, when last shared ptr that refers to Logger with specified name is destroyed. + */ +LoggerPtr getLogger(const std::string & name, Poco::Channel * channel, Poco::Message::Priority level = Poco::Message::PRIO_INFORMATION); + +/** Create raw Poco::Logger that will not be destroyed before program termination. + * This can be used in cases when specific Logger instance can be singletone. + * + * For example you need to pass Logger into low-level libraries as raw pointer, and using + * RAII wrapper is inconvenient. + * + * Generally you should always use getLogger functions. + */ + +LoggerRawPtr getRawLogger(const std::string & name); + +LoggerRawPtr createRawLogger(const std::string & name, Poco::Channel * channel, Poco::Message::Priority level = Poco::Message::PRIO_INFORMATION); + + +/** Returns true, if currently Logger with specified name is created. + * Otherwise, returns false. + */ +bool hasLogger(const std::string & name); diff --git a/src/Common/LoggerPtr.cpp b/src/Common/LoggerPtr.cpp deleted file mode 100644 index 33caaf81a81..00000000000 --- a/src/Common/LoggerPtr.cpp +++ /dev/null @@ -1,15 +0,0 @@ -#include - -struct LoggerDeleter -{ - void operator()(const Poco::Logger * logger) - { - Poco::Logger::destroy(logger->name()); - } -}; - -LoggerPtr getLogger(const std::string & name) -{ - Poco::Logger * logger_raw_ptr = &Poco::Logger::get(name); - return std::shared_ptr(logger_raw_ptr, LoggerDeleter()); -} diff --git a/src/Common/LoggerPtr.h b/src/Common/LoggerPtr.h deleted file mode 100644 index cd0529ea258..00000000000 --- a/src/Common/LoggerPtr.h +++ /dev/null @@ -1,14 +0,0 @@ -#pragma once - -#include - -namespace Poco -{ - -class Logger; - -} - -using LoggerPtr = std::shared_ptr; - -LoggerPtr getLogger(const std::string & name); diff --git a/src/Common/LoggingFormatStringHelpers.cpp b/src/Common/LoggingFormatStringHelpers.cpp index 074c8dd2803..b3353a59010 100644 --- a/src/Common/LoggingFormatStringHelpers.cpp +++ b/src/Common/LoggingFormatStringHelpers.cpp @@ -80,8 +80,8 @@ void LogFrequencyLimiterIml::cleanup(time_t too_old_threshold_s) std::mutex LogSeriesLimiter::mutex; time_t LogSeriesLimiter::last_cleanup = 0; -LogSeriesLimiter::LogSeriesLimiter(Poco::Logger * logger_, size_t allowed_count_, time_t interval_s_) - : logger(logger_) +LogSeriesLimiter::LogSeriesLimiter(LoggerPtr logger_, size_t allowed_count_, time_t interval_s_) + : logger(std::move(logger_)) { if (allowed_count_ == 0) { diff --git a/src/Common/LoggingFormatStringHelpers.h b/src/Common/LoggingFormatStringHelpers.h index ef7ec0c6144..b0f0a5cd716 100644 --- a/src/Common/LoggingFormatStringHelpers.h +++ b/src/Common/LoggingFormatStringHelpers.h @@ -8,7 +8,7 @@ #include #include #include - +#include struct PreformattedMessage; consteval void formatStringCheckArgsNumImpl(std::string_view str, size_t nargs); @@ -203,10 +203,10 @@ class LogFrequencyLimiterIml static time_t last_cleanup; static std::mutex mutex; - Poco::Logger * logger; + LoggerPtr logger; time_t min_interval_s; public: - LogFrequencyLimiterIml(Poco::Logger * logger_, time_t min_interval_s_) : logger(logger_), min_interval_s(min_interval_s_) {} + LogFrequencyLimiterIml(LoggerPtr logger_, time_t min_interval_s_) : logger(std::move(logger_)), min_interval_s(min_interval_s_) {} LogFrequencyLimiterIml & operator -> () { return *this; } bool is(Poco::Message::Priority priority) { return logger->is(priority); } @@ -218,7 +218,7 @@ public: /// Clears messages that were logged last time more than too_old_threshold_s seconds ago static void cleanup(time_t too_old_threshold_s = 600); - Poco::Logger * getLogger() { return logger; } + LoggerPtr getLogger() { return logger; } }; /// This wrapper helps to avoid too noisy log messages from similar objects. @@ -240,11 +240,11 @@ class LogSeriesLimiter return records; } - Poco::Logger * logger = nullptr; + LoggerPtr logger = nullptr; bool accepted = false; String debug_message; public: - LogSeriesLimiter(Poco::Logger * logger_, size_t allowed_count_, time_t interval_s_); + LogSeriesLimiter(LoggerPtr logger_, size_t allowed_count_, time_t interval_s_); LogSeriesLimiter & operator -> () { return *this; } bool is(Poco::Message::Priority priority) { return logger->is(priority); } @@ -253,18 +253,18 @@ public: void log(Poco::Message & message); - Poco::Logger * getLogger() { return logger; } + LoggerPtr getLogger() { return logger; } }; /// This wrapper is useful to save formatted message into a String before sending it to a logger class LogToStrImpl { String & out_str; - Poco::Logger * logger; + LoggerPtr logger; std::unique_ptr maybe_nested; bool propagate_to_actual_log = true; public: - LogToStrImpl(String & out_str_, Poco::Logger * logger_) : out_str(out_str_), logger(logger_) {} + LogToStrImpl(String & out_str_, LoggerPtr logger_) : out_str(out_str_), logger(std::move(logger_)) {} LogToStrImpl(String & out_str_, std::unique_ptr && maybe_nested_) : out_str(out_str_), logger(maybe_nested_->getLogger()), maybe_nested(std::move(maybe_nested_)) {} LogToStrImpl & operator -> () { return *this; } diff --git a/src/Common/Macros.cpp b/src/Common/Macros.cpp index 0035e7abfe8..9e0977d9bcc 100644 --- a/src/Common/Macros.cpp +++ b/src/Common/Macros.cpp @@ -38,6 +38,10 @@ Macros::Macros(const Poco::Util::AbstractConfiguration & config, const String & } } +Macros::Macros(const Poco::Util::AbstractConfiguration & config, const String & root_key, LoggerPtr log) + : Macros(config, root_key, log.get()) +{} + Macros::Macros(std::map map) { macros = std::move(map); diff --git a/src/Common/Macros.h b/src/Common/Macros.h index 9fe5717effc..8b9eded7dcb 100644 --- a/src/Common/Macros.h +++ b/src/Common/Macros.h @@ -26,6 +26,7 @@ class Macros { public: Macros() = default; + Macros(const Poco::Util::AbstractConfiguration & config, const String & key, LoggerPtr log = nullptr); Macros(const Poco::Util::AbstractConfiguration & config, const String & key, Poco::Logger * log = nullptr); explicit Macros(std::map map); diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 5672bb0ae10..28cfa98666a 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -155,14 +155,14 @@ void MemoryTracker::logPeakMemoryUsage() auto peak_bytes = peak.load(std::memory_order::relaxed); if (peak_bytes < 128 * 1024) return; - LOG_DEBUG(&Poco::Logger::get("MemoryTracker"), + LOG_DEBUG(getLogger("MemoryTracker"), "Peak memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(peak_bytes)); } void MemoryTracker::logMemoryUsage(Int64 current) const { const auto * description = description_ptr.load(std::memory_order_relaxed); - LOG_DEBUG(&Poco::Logger::get("MemoryTracker"), + LOG_DEBUG(getLogger("MemoryTracker"), "Current memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(current)); } @@ -170,7 +170,7 @@ void MemoryTracker::injectFault() const { if (!memoryTrackerCanThrow(level, true)) { - LOG_WARNING(&Poco::Logger::get("MemoryTracker"), + LOG_WARNING(getLogger("MemoryTracker"), "Cannot inject fault at specific point. Uncaught exceptions: {}, stack trace:\n{}", std::uncaught_exceptions(), StackTrace().toString()); return; @@ -201,7 +201,7 @@ void MemoryTracker::debugLogBigAllocationWithoutCheck(Int64 size [[maybe_unused] return; MemoryTrackerBlockerInThread blocker(VariableContext::Global); - LOG_TEST(&Poco::Logger::get("MemoryTracker"), "Too big allocation ({} bytes) without checking memory limits, " + LOG_TEST(getLogger("MemoryTracker"), "Too big allocation ({} bytes) without checking memory limits, " "it may lead to OOM. Stack trace: {}", size, StackTrace().toString()); #else return; /// Avoid trash logging in release builds diff --git a/src/Common/NamedCollections/NamedCollectionUtils.cpp b/src/Common/NamedCollections/NamedCollectionUtils.cpp index d62d54b9f37..fe0f42467c7 100644 --- a/src/Common/NamedCollections/NamedCollectionUtils.cpp +++ b/src/Common/NamedCollections/NamedCollectionUtils.cpp @@ -135,7 +135,7 @@ public: else { LOG_WARNING( - &Poco::Logger::get("NamedCollectionsLoadFromSQL"), + getLogger("NamedCollectionsLoadFromSQL"), "Unexpected file {} in named collections directory", current_path.filename().string()); } @@ -345,7 +345,7 @@ void loadFromConfigUnlocked(const Poco::Util::AbstractConfiguration & config, st { auto named_collections = LoadFromConfig(config).getAll(); LOG_TRACE( - &Poco::Logger::get("NamedCollectionsUtils"), + getLogger("NamedCollectionsUtils"), "Loaded {} collections from config", named_collections.size()); NamedCollectionFactory::instance().add(std::move(named_collections)); @@ -372,7 +372,7 @@ void loadFromSQLUnlocked(ContextPtr context, std::unique_lock &) { auto named_collections = LoadFromSQL(context).getAll(); LOG_TRACE( - &Poco::Logger::get("NamedCollectionsUtils"), + getLogger("NamedCollectionsUtils"), "Loaded {} collections from SQL", named_collections.size()); NamedCollectionFactory::instance().add(std::move(named_collections)); diff --git a/src/Common/NetlinkMetricsProvider.cpp b/src/Common/NetlinkMetricsProvider.cpp index 23173f31689..6969b5b7542 100644 --- a/src/Common/NetlinkMetricsProvider.cpp +++ b/src/Common/NetlinkMetricsProvider.cpp @@ -216,7 +216,7 @@ bool checkPermissionsImpl() { /// This error happens all the time when running inside Docker - consider it ok, /// don't create noise with this error. - LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), getCurrentExceptionMessageAndPattern(/* with_stacktrace */ false)); + LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), getCurrentExceptionMessageAndPattern(/* with_stacktrace */ false)); } else { diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index 8a580469278..b6852964efe 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -463,7 +463,7 @@ catch (...) is_trivial = false; required_substring_is_prefix = false; alternatives.clear(); - LOG_ERROR(&Poco::Logger::get("OptimizeRegularExpression"), "Analyze RegularExpression failed, got error: {}", DB::getCurrentExceptionMessage(false)); + LOG_ERROR(getLogger("OptimizeRegularExpression"), "Analyze RegularExpression failed, got error: {}", DB::getCurrentExceptionMessage(false)); } OptimizedRegularExpression::OptimizedRegularExpression(const std::string & regexp_, int options) diff --git a/src/Common/PipeFDs.cpp b/src/Common/PipeFDs.cpp index f2a913467a9..ceadbb2f983 100644 --- a/src/Common/PipeFDs.cpp +++ b/src/Common/PipeFDs.cpp @@ -97,7 +97,7 @@ void LazyPipeFDs::setNonBlockingReadWrite() void LazyPipeFDs::tryIncreaseSize(int desired_size) { #if defined(OS_LINUX) - Poco::Logger * log = &Poco::Logger::get("Pipe"); + LoggerPtr log = getLogger("Pipe"); /** Increase pipe size to avoid slowdown during fine-grained trace collection. */ diff --git a/src/Common/PoolBase.h b/src/Common/PoolBase.h index 5575b56f299..ef35002c45a 100644 --- a/src/Common/PoolBase.h +++ b/src/Common/PoolBase.h @@ -223,9 +223,9 @@ private: std::condition_variable available; protected: - Poco::Logger * log; + LoggerPtr log; - PoolBase(unsigned max_items_, Poco::Logger * log_, BehaviourOnLimit behaviour_on_limit_ = BehaviourOnLimit::Wait) + PoolBase(unsigned max_items_, LoggerPtr log_, BehaviourOnLimit behaviour_on_limit_ = BehaviourOnLimit::Wait) : max_items(max_items_), behaviour_on_limit(behaviour_on_limit_), log(log_) { items.reserve(max_items); diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index f960d551996..6da4445950c 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -58,7 +58,7 @@ public: NestedPools nested_pools_, time_t decrease_error_period_, size_t max_error_cap_, - Poco::Logger * log_) + LoggerPtr log_) : nested_pools(std::move(nested_pools_)) , decrease_error_period(decrease_error_period_) , max_error_cap(max_error_cap_) @@ -159,7 +159,7 @@ protected: /// The time when error counts were last decreased. time_t last_error_decrease_time = 0; - Poco::Logger * log; + LoggerPtr log; }; diff --git a/src/Common/ProxyConfigurationResolverProvider.cpp b/src/Common/ProxyConfigurationResolverProvider.cpp index 96ebb934643..d15b4d98615 100644 --- a/src/Common/ProxyConfigurationResolverProvider.cpp +++ b/src/Common/ProxyConfigurationResolverProvider.cpp @@ -36,7 +36,7 @@ namespace auto proxy_port = configuration.getUInt(resolver_prefix + ".proxy_port"); auto cache_ttl = configuration.getUInt(resolver_prefix + ".proxy_cache_time", 10); - LOG_DEBUG(&Poco::Logger::get("ProxyConfigurationResolverProvider"), "Configured remote proxy resolver: {}, Scheme: {}, Port: {}", + LOG_DEBUG(getLogger("ProxyConfigurationResolverProvider"), "Configured remote proxy resolver: {}, Scheme: {}, Port: {}", endpoint.toString(), proxy_scheme, proxy_port); auto server_configuration = RemoteProxyConfigurationResolver::RemoteServerConfiguration { @@ -71,7 +71,7 @@ namespace uris.push_back(proxy_uri); - LOG_DEBUG(&Poco::Logger::get("ProxyConfigurationResolverProvider"), "Configured proxy: {}", proxy_uri.toString()); + LOG_DEBUG(getLogger("ProxyConfigurationResolverProvider"), "Configured proxy: {}", proxy_uri.toString()); } } diff --git a/src/Common/ProxyListConfigurationResolver.cpp b/src/Common/ProxyListConfigurationResolver.cpp index 68d676643d6..01a6f52185f 100644 --- a/src/Common/ProxyListConfigurationResolver.cpp +++ b/src/Common/ProxyListConfigurationResolver.cpp @@ -26,7 +26,7 @@ ProxyConfiguration ProxyListConfigurationResolver::resolve() auto & proxy = proxies[index]; - LOG_DEBUG(&Poco::Logger::get("ProxyListConfigurationResolver"), "Use proxy: {}", proxies[index].toString()); + LOG_DEBUG(getLogger("ProxyListConfigurationResolver"), "Use proxy: {}", proxies[index].toString()); return ProxyConfiguration { proxy.getHost(), diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index 16c8d4e223f..34ffbf6c498 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -105,7 +105,7 @@ namespace ErrorCodes #ifndef __APPLE__ Timer::Timer() - : log(&Poco::Logger::get("Timer")) + : log(getLogger("Timer")) {} void Timer::createIfNecessary(UInt64 thread_id, int clock_type, int pause_signal) @@ -211,7 +211,7 @@ void Timer::cleanup() template QueryProfilerBase::QueryProfilerBase(UInt64 thread_id, int clock_type, UInt32 period, int pause_signal_) - : log(&Poco::Logger::get("QueryProfiler")) + : log(getLogger("QueryProfiler")) , pause_signal(pause_signal_) { #if defined(SANITIZER) diff --git a/src/Common/QueryProfiler.h b/src/Common/QueryProfiler.h index 87432a4b699..254b11137cc 100644 --- a/src/Common/QueryProfiler.h +++ b/src/Common/QueryProfiler.h @@ -7,6 +7,8 @@ #include "config.h" +#include + namespace Poco { @@ -43,7 +45,7 @@ public: void cleanup(); private: - Poco::Logger * log; + LoggerPtr log; std::optional timer_id; }; #endif @@ -58,7 +60,7 @@ public: private: void cleanup(); - Poco::Logger * log; + LoggerPtr log; #ifndef __APPLE__ inline static thread_local Timer timer = Timer(); diff --git a/src/Common/RemoteProxyConfigurationResolver.cpp b/src/Common/RemoteProxyConfigurationResolver.cpp index 7342933beff..117c8a34dbb 100644 --- a/src/Common/RemoteProxyConfigurationResolver.cpp +++ b/src/Common/RemoteProxyConfigurationResolver.cpp @@ -27,7 +27,7 @@ RemoteProxyConfigurationResolver::RemoteProxyConfigurationResolver( ProxyConfiguration RemoteProxyConfigurationResolver::resolve() { - auto * logger = &Poco::Logger::get("RemoteProxyConfigurationResolver"); + auto logger = getLogger("RemoteProxyConfigurationResolver"); auto & [endpoint, proxy_protocol, proxy_port, cache_ttl_] = remote_server_configuration; diff --git a/src/Common/SensitiveDataMasker.cpp b/src/Common/SensitiveDataMasker.cpp index 33770c3e78a..70346919f65 100644 --- a/src/Common/SensitiveDataMasker.cpp +++ b/src/Common/SensitiveDataMasker.cpp @@ -112,7 +112,7 @@ SensitiveDataMasker::SensitiveDataMasker(const Poco::Util::AbstractConfiguration { Poco::Util::AbstractConfiguration::Keys keys; config.keys(config_prefix, keys); - Poco::Logger * logger = &Poco::Logger::get("SensitiveDataMaskerConfigRead"); + LoggerPtr logger = getLogger("SensitiveDataMaskerConfigRead"); std::set used_names; diff --git a/src/Common/ShellCommand.cpp b/src/Common/ShellCommand.cpp index f4efc9e3526..98a21b43d76 100644 --- a/src/Common/ShellCommand.cpp +++ b/src/Common/ShellCommand.cpp @@ -54,9 +54,9 @@ ShellCommand::ShellCommand(pid_t pid_, int & in_fd_, int & out_fd_, int & err_fd { } -Poco::Logger * ShellCommand::getLogger() +LoggerPtr ShellCommand::getLogger() { - return &Poco::Logger::get("ShellCommand"); + return ::getLogger("ShellCommand"); } ShellCommand::~ShellCommand() diff --git a/src/Common/ShellCommand.h b/src/Common/ShellCommand.h index da65d2ae494..5ebc1daefa1 100644 --- a/src/Common/ShellCommand.h +++ b/src/Common/ShellCommand.h @@ -97,7 +97,7 @@ private: bool tryWaitProcessWithTimeout(size_t timeout_in_seconds); - static Poco::Logger * getLogger(); + static LoggerPtr getLogger(); /// Print command name and the list of arguments to log. NOTE: No escaping of arguments is performed. static void logCommand(const char * filename, char * const argv[]); diff --git a/src/Common/StatusFile.cpp b/src/Common/StatusFile.cpp index 0a9aa2f2739..56eb1d4d0cb 100644 --- a/src/Common/StatusFile.cpp +++ b/src/Common/StatusFile.cpp @@ -56,9 +56,9 @@ StatusFile::StatusFile(std::string path_, FillFunction fill_) } if (!contents.empty()) - LOG_INFO(&Poco::Logger::get("StatusFile"), "Status file {} already exists - unclean restart. Contents:\n{}", path, contents); + LOG_INFO(getLogger("StatusFile"), "Status file {} already exists - unclean restart. Contents:\n{}", path, contents); else - LOG_INFO(&Poco::Logger::get("StatusFile"), "Status file {} already exists and is empty - probably unclean hardware restart.", path); + LOG_INFO(getLogger("StatusFile"), "Status file {} already exists and is empty - probably unclean hardware restart.", path); } fd = ::open(path.c_str(), O_WRONLY | O_CREAT | O_CLOEXEC, 0666); @@ -99,10 +99,10 @@ StatusFile::StatusFile(std::string path_, FillFunction fill_) StatusFile::~StatusFile() { if (0 != close(fd)) - LOG_ERROR(&Poco::Logger::get("StatusFile"), "Cannot close file {}, {}", path, errnoToString()); + LOG_ERROR(getLogger("StatusFile"), "Cannot close file {}, {}", path, errnoToString()); if (0 != unlink(path.c_str())) - LOG_ERROR(&Poco::Logger::get("StatusFile"), "Cannot unlink file {}, {}", path, errnoToString()); + LOG_ERROR(getLogger("StatusFile"), "Cannot unlink file {}, {}", path, errnoToString()); } } diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index d82b582fee6..4dee6d905d9 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -39,7 +39,7 @@ ISystemLog::~ISystemLog() = default; template SystemLogQueue::SystemLogQueue(const SystemLogQueueSettings & settings_) - : log(&Poco::Logger::get("SystemLogQueue (" + settings_.database + "." +settings_.table + ")")) + : log(getLogger("SystemLogQueue (" + settings_.database + "." +settings_.table + ")")) , settings(settings_) { diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 1ced313b36a..a734c70f285 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -121,7 +121,7 @@ private: /// Data shared between callers of add()/flush()/shutdown(), and the saving thread std::mutex mutex; - Poco::Logger * log; + LoggerPtr log; // Queue is bounded. But its size is quite large to not block in all normal cases. std::vector queue; diff --git a/src/Common/TLDListsHolder.cpp b/src/Common/TLDListsHolder.cpp index 623b88f83a5..c3991b86983 100644 --- a/src/Common/TLDListsHolder.cpp +++ b/src/Common/TLDListsHolder.cpp @@ -55,7 +55,7 @@ void TLDListsHolder::parseConfig(const std::string & top_level_domains_path, con Poco::Util::AbstractConfiguration::Keys config_keys; config.keys("top_level_domains_lists", config_keys); - Poco::Logger * log = &Poco::Logger::get("TLDListsHolder"); + LoggerPtr log = getLogger("TLDListsHolder"); for (const auto & key : config_keys) { diff --git a/src/Common/ThreadProfileEvents.cpp b/src/Common/ThreadProfileEvents.cpp index 256f53df011..990151d73ff 100644 --- a/src/Common/ThreadProfileEvents.cpp +++ b/src/Common/ThreadProfileEvents.cpp @@ -300,7 +300,7 @@ static void enablePerfEvent(int event_fd) { if (ioctl(event_fd, PERF_EVENT_IOC_ENABLE, 0)) { - LOG_WARNING(&Poco::Logger::get("PerfEvents"), + LOG_WARNING(getLogger("PerfEvents"), "Can't enable perf event with file descriptor {}: '{}' ({})", event_fd, errnoToString(), errno); } @@ -310,7 +310,7 @@ static void disablePerfEvent(int event_fd) { if (ioctl(event_fd, PERF_EVENT_IOC_DISABLE, 0)) { - LOG_WARNING(&Poco::Logger::get("PerfEvents"), + LOG_WARNING(getLogger("PerfEvents"), "Can't disable perf event with file descriptor {}: '{}' ({})", event_fd, errnoToString(), errno); } @@ -320,7 +320,7 @@ static void releasePerfEvent(int event_fd) { if (close(event_fd)) { - LOG_WARNING(&Poco::Logger::get("PerfEvents"), + LOG_WARNING(getLogger("PerfEvents"), "Can't close perf event file descriptor {}: {} ({})", event_fd, errnoToString(), errno); } @@ -333,12 +333,12 @@ static bool validatePerfEventDescriptor(int & fd) if (errno == EBADF) { - LOG_WARNING(&Poco::Logger::get("PerfEvents"), + LOG_WARNING(getLogger("PerfEvents"), "Event descriptor {} was closed from the outside; reopening", fd); } else { - LOG_WARNING(&Poco::Logger::get("PerfEvents"), + LOG_WARNING(getLogger("PerfEvents"), "Error while checking availability of event descriptor {}: {} ({})", fd, errnoToString(), errno); @@ -416,7 +416,7 @@ bool PerfEventsCounters::processThreadLocalChanges(const std::string & needed_ev bool has_cap_sys_admin = hasLinuxCapability(CAP_SYS_ADMIN); if (perf_event_paranoid >= 3 && !has_cap_sys_admin) { - LOG_WARNING(&Poco::Logger::get("PerfEvents"), + LOG_WARNING(getLogger("PerfEvents"), "Not enough permissions to record perf events: " "perf_event_paranoid = {} and CAP_SYS_ADMIN = 0", perf_event_paranoid); @@ -444,7 +444,7 @@ bool PerfEventsCounters::processThreadLocalChanges(const std::string & needed_ev // ENOENT means that the event is not supported. Don't log it, because // this is called for each thread and would be too verbose. Log other // error codes because they might signify an error. - LOG_WARNING(&Poco::Logger::get("PerfEvents"), + LOG_WARNING(getLogger("PerfEvents"), "Failed to open perf event {} (event_type={}, event_config={}): " "'{}' ({})", event_info.settings_name, event_info.event_type, event_info.event_config, errnoToString(), errno); @@ -484,7 +484,7 @@ std::vector PerfEventsCounters::eventIndicesFromString(const std::string } else { - LOG_ERROR(&Poco::Logger::get("PerfEvents"), + LOG_ERROR(getLogger("PerfEvents"), "Unknown perf event name '{}' specified in settings", event_name); } } @@ -531,7 +531,7 @@ void PerfEventsCounters::finalizeProfileEvents(ProfileEvents::Counters & profile if (bytes_read != bytes_to_read) { - LOG_WARNING(&Poco::Logger::get("PerfEvents"), + LOG_WARNING(getLogger("PerfEvents"), "Can't read event value from file descriptor {}: '{}' ({})", fd, errnoToString(), errno); current_values[i] = {}; diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index c99823b2dfa..05524a5d6b9 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -76,7 +76,7 @@ ThreadStatus::ThreadStatus(bool check_current_thread_on_destruction_) last_rusage = std::make_unique(); memory_tracker.setDescription("(for thread)"); - log = &Poco::Logger::get("ThreadStatus"); + log = getLogger("ThreadStatus"); current_thread = this; diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index b8bdebf10ed..f7534c35a98 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -236,7 +236,7 @@ private: using Deleter = std::function; Deleter deleter; - Poco::Logger * log = nullptr; + LoggerPtr log = nullptr; bool check_current_thread_on_destruction; diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 70b8df5cd2c..8a8465de491 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -60,7 +60,7 @@ void ZooKeeper::init(ZooKeeperArgs args_) { args = std::move(args_); - log = &Poco::Logger::get("ZooKeeper"); + log = getLogger("ZooKeeper"); if (args.implementation == "zookeeper") { @@ -1455,7 +1455,7 @@ Coordination::RequestPtr makeExistsRequest(const std::string & path) return request; } -std::string normalizeZooKeeperPath(std::string zookeeper_path, bool check_starts_with_slash, Poco::Logger * log) +std::string normalizeZooKeeperPath(std::string zookeeper_path, bool check_starts_with_slash, LoggerPtr log) { if (!zookeeper_path.empty() && zookeeper_path.back() == '/') zookeeper_path.resize(zookeeper_path.size() - 1); @@ -1491,7 +1491,7 @@ String extractZooKeeperName(const String & path) return default_zookeeper_name; } -String extractZooKeeperPath(const String & path, bool check_starts_with_slash, Poco::Logger * log) +String extractZooKeeperPath(const String & path, bool check_starts_with_slash, LoggerPtr log) { if (path.empty()) throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "ZooKeeper path should not be empty"); diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 1f29af0797b..811546fb4b9 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -650,7 +650,7 @@ private: ZooKeeperArgs args; - Poco::Logger * log = nullptr; + LoggerPtr log = nullptr; std::shared_ptr zk_log; AtomicStopwatch session_uptime; @@ -729,7 +729,7 @@ public: else { ProfileEvents::increment(ProfileEvents::CannotRemoveEphemeralNode); - LOG_DEBUG(&Poco::Logger::get("EphemeralNodeHolder"), "Cannot remove {} since session has been expired", path); + LOG_DEBUG(getLogger("EphemeralNodeHolder"), "Cannot remove {} since session has been expired", path); } } catch (...) @@ -749,11 +749,11 @@ private: using EphemeralNodeHolderPtr = EphemeralNodeHolder::Ptr; -String normalizeZooKeeperPath(std::string zookeeper_path, bool check_starts_with_slash, Poco::Logger * log = nullptr); +String normalizeZooKeeperPath(std::string zookeeper_path, bool check_starts_with_slash, LoggerPtr log = nullptr); String extractZooKeeperName(const String & path); -String extractZooKeeperPath(const String & path, bool check_starts_with_slash, Poco::Logger * log = nullptr); +String extractZooKeeperPath(const String & path, bool check_starts_with_slash, LoggerPtr log = nullptr); String getSequentialNodeName(const String & prefix, UInt64 number); diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index 592d142e925..660ae59e81e 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -929,7 +929,7 @@ ZooKeeperRequest::~ZooKeeperRequest() constexpr UInt64 max_request_time_ns = 1000000000ULL; /// 1 sec if (max_request_time_ns < elapsed_ns) { - LOG_TEST(&Poco::Logger::get(__PRETTY_FUNCTION__), "Processing of request xid={} took {} ms", xid, elapsed_ns / 1000000UL); + LOG_TEST(getLogger(__PRETTY_FUNCTION__), "Processing of request xid={} took {} ms", xid, elapsed_ns / 1000000UL); } } @@ -950,7 +950,7 @@ ZooKeeperResponse::~ZooKeeperResponse() constexpr UInt64 max_request_time_ns = 1000000000ULL; /// 1 sec if (max_request_time_ns < elapsed_ns) { - LOG_TEST(&Poco::Logger::get(__PRETTY_FUNCTION__), "Processing of response xid={} took {} ms", xid, elapsed_ns / 1000000UL); + LOG_TEST(getLogger(__PRETTY_FUNCTION__), "Processing of response xid={} took {} ms", xid, elapsed_ns / 1000000UL); } } diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index d732b900d37..1fbadbd7616 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -342,7 +342,7 @@ ZooKeeper::ZooKeeper( std::shared_ptr zk_log_) : args(args_) { - log = &Poco::Logger::get("ZooKeeperClient"); + log = getLogger("ZooKeeperClient"); std::atomic_store(&zk_log, std::move(zk_log_)); if (!args.chroot.empty()) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 13e1dc9e3cd..b63f67bf7a6 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -308,7 +308,7 @@ private: ThreadReference send_thread; ThreadReference receive_thread; - Poco::Logger * log; + LoggerPtr log; void connect( const Nodes & node, diff --git a/src/Common/ZooKeeper/ZooKeeperLock.cpp b/src/Common/ZooKeeper/ZooKeeperLock.cpp index 6ee1c380efb..b90bcfd2b55 100644 --- a/src/Common/ZooKeeper/ZooKeeperLock.cpp +++ b/src/Common/ZooKeeper/ZooKeeperLock.cpp @@ -26,7 +26,7 @@ ZooKeeperLock::ZooKeeperLock( : zookeeper(zookeeper_) , lock_path(fs::path(lock_prefix_) / lock_name_) , lock_message(lock_message_) - , log(&Poco::Logger::get("zkutil::Lock")) + , log(getLogger("zkutil::Lock")) { zookeeper->createIfNotExists(lock_prefix_, ""); } diff --git a/src/Common/ZooKeeper/ZooKeeperLock.h b/src/Common/ZooKeeper/ZooKeeperLock.h index 146527c6c94..6271afaf6e2 100644 --- a/src/Common/ZooKeeper/ZooKeeperLock.h +++ b/src/Common/ZooKeeper/ZooKeeperLock.h @@ -46,7 +46,7 @@ private: std::string lock_path; std::string lock_message; - Poco::Logger * log; + LoggerPtr log; bool locked = false; }; diff --git a/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.cpp b/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.cpp index 72923ca0487..694004dee23 100644 --- a/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.cpp +++ b/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.cpp @@ -9,7 +9,7 @@ ZooKeeperWithFaultInjection::ZooKeeperWithFaultInjection( double fault_injection_probability, UInt64 fault_injection_seed, std::string name_, - Poco::Logger * logger_) + LoggerPtr logger_) : keeper(keeper_) , fault_policy(std::make_unique(fault_injection_probability, fault_injection_seed)) , name(std::move(name_)) diff --git a/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h b/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h index 57e1f0f3b87..2ee456a23b9 100644 --- a/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h +++ b/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h @@ -62,7 +62,7 @@ class ZooKeeperWithFaultInjection std::unique_ptr fault_policy; std::string name; - Poco::Logger * logger = nullptr; + LoggerPtr logger = nullptr; const UInt64 seed = 0; std::vector session_ephemeral_nodes; @@ -87,7 +87,7 @@ public: double fault_injection_probability, UInt64 fault_injection_seed, std::string name_, - Poco::Logger * logger_); + LoggerPtr logger_); explicit ZooKeeperWithFaultInjection(zkutil::ZooKeeper::Ptr const & keeper_) : keeper(keeper_) { } static ZooKeeperWithFaultInjection::Ptr createInstance( @@ -95,7 +95,7 @@ public: UInt64 fault_injection_seed, zkutil::ZooKeeper::Ptr const & zookeeper, std::string name, - Poco::Logger * logger) + LoggerPtr logger) { /// validate all parameters here, constructor just accept everything if (fault_injection_probability < 0.0) diff --git a/src/Common/logger_useful.h b/src/Common/logger_useful.h index fda374befc9..3899d060b7c 100644 --- a/src/Common/logger_useful.h +++ b/src/Common/logger_useful.h @@ -8,7 +8,7 @@ #include #include #include -#include +#include namespace Poco { class Logger; } @@ -22,7 +22,6 @@ namespace { [[maybe_unused]] const ::Poco::Logger * getLoggerHelper(const LoggerPtr & logger) { return logger.get(); } [[maybe_unused]] const ::Poco::Logger * getLoggerHelper(const ::Poco::Logger * logger) { return logger; } - [[maybe_unused]] const ::Poco::Logger * getLoggerHelper(const std::atomic<::Poco::Logger *> & logger) { return logger.load(); } [[maybe_unused]] std::unique_ptr getLoggerHelper(std::unique_ptr && logger) { return logger; } [[maybe_unused]] std::unique_ptr getLoggerHelper(std::unique_ptr && logger) { return logger; } [[maybe_unused]] LogSeriesLimiterPtr getLoggerHelper(LogSeriesLimiterPtr & logger) { return logger; } diff --git a/src/Common/makeSocketAddress.cpp b/src/Common/makeSocketAddress.cpp index b5df6a4ef03..ba5bb53cd20 100644 --- a/src/Common/makeSocketAddress.cpp +++ b/src/Common/makeSocketAddress.cpp @@ -33,4 +33,9 @@ Poco::Net::SocketAddress makeSocketAddress(const std::string & host, uint16_t po return socket_address; } +Poco::Net::SocketAddress makeSocketAddress(const std::string & host, uint16_t port, LoggerPtr log) +{ + return makeSocketAddress(host, port, log.get()); +} + } diff --git a/src/Common/makeSocketAddress.h b/src/Common/makeSocketAddress.h index 9c7d10a0471..439a4ef1e9b 100644 --- a/src/Common/makeSocketAddress.h +++ b/src/Common/makeSocketAddress.h @@ -1,5 +1,7 @@ #pragma once + #include +#include namespace Poco { class Logger; } @@ -8,4 +10,6 @@ namespace DB Poco::Net::SocketAddress makeSocketAddress(const std::string & host, uint16_t port, Poco::Logger * log); +Poco::Net::SocketAddress makeSocketAddress(const std::string & host, uint16_t port, LoggerPtr log); + } diff --git a/src/Common/mysqlxx/Query.cpp b/src/Common/mysqlxx/Query.cpp index e30ed2b75c8..babfc8c7c41 100644 --- a/src/Common/mysqlxx/Query.cpp +++ b/src/Common/mysqlxx/Query.cpp @@ -52,7 +52,7 @@ void Query::executeImpl() { MYSQL* mysql_driver = conn->getDriver(); - LOG_TRACE(&Poco::Logger::get("mysqlxx::Query"), "Running MySQL query using connection {}", mysql_thread_id(mysql_driver)); + LOG_TRACE(getLogger("mysqlxx::Query"), "Running MySQL query using connection {}", mysql_thread_id(mysql_driver)); if (mysql_real_query(mysql_driver, query.data(), query.size())) { const auto err_no = mysql_errno(mysql_driver); diff --git a/src/Common/mysqlxx/mysqlxx/Pool.h b/src/Common/mysqlxx/mysqlxx/Pool.h index bb4d0cefbdc..c85295c4dd0 100644 --- a/src/Common/mysqlxx/mysqlxx/Pool.h +++ b/src/Common/mysqlxx/mysqlxx/Pool.h @@ -202,7 +202,7 @@ public: void removeConnection(Connection * connection); protected: - Poco::Logger * log = &Poco::Logger::get("mysqlxx::Pool"); + LoggerPtr log = getLogger("mysqlxx::Pool"); /// Number of MySQL connections which are created at launch. unsigned default_connections; diff --git a/src/Common/parseRemoteDescription.cpp b/src/Common/parseRemoteDescription.cpp index 7b2045b9de1..df3820b11f9 100644 --- a/src/Common/parseRemoteDescription.cpp +++ b/src/Common/parseRemoteDescription.cpp @@ -179,7 +179,7 @@ std::vector> parseRemoteDescriptionForExternalDataba size_t colon = address.find(':'); if (colon == String::npos) { - LOG_WARNING(&Poco::Logger::get("ParseRemoteDescription"), "Port is not found for host: {}. Using default port {}", address, default_port); + LOG_WARNING(getLogger("ParseRemoteDescription"), "Port is not found for host: {}. Using default port {}", address, default_port); result.emplace_back(std::make_pair(address, default_port)); } else diff --git a/src/Common/tests/gtest_log.cpp b/src/Common/tests/gtest_log.cpp index e755c22ba75..10e85430321 100644 --- a/src/Common/tests/gtest_log.cpp +++ b/src/Common/tests/gtest_log.cpp @@ -15,7 +15,7 @@ TEST(Logger, Log) { Poco::Logger::root().setLevel("none"); Poco::Logger::root().setChannel(Poco::AutoPtr(new Poco::NullChannel())); - Poco::Logger * log = &Poco::Logger::get("Log"); + LoggerPtr log = getLogger("Log"); /// This test checks that we don't pass this string to fmtlib, because it is the only argument. EXPECT_NO_THROW(LOG_INFO(log, fmt::runtime("Hello {} World"))); @@ -27,7 +27,7 @@ TEST(Logger, TestLog) std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM auto my_channel = Poco::AutoPtr(new Poco::StreamChannel(oss)); - auto * log = &Poco::Logger::create("TestLogger", my_channel.get()); + auto log = getLogger("TestLogger", my_channel.get()); log->setLevel("test"); LOG_TEST(log, "Hello World"); @@ -40,7 +40,7 @@ TEST(Logger, TestLog) { std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM auto my_channel = Poco::AutoPtr(new Poco::StreamChannel(oss)); - auto * log = &Poco::Logger::create(std::string{level} + "_Logger", my_channel.get()); + auto log = getLogger(std::string{level} + "_Logger", my_channel.get()); log->setLevel(level); LOG_TEST(log, "Hello World"); @@ -84,7 +84,7 @@ TEST(Logger, SideEffects) { std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM auto my_channel = Poco::AutoPtr(new Poco::StreamChannel(oss)); - auto * log = &Poco::Logger::create("Logger", my_channel.get()); + auto log = getLogger("Logger", my_channel.get()); log->setLevel("trace"); /// Ensure that parameters are evaluated only once diff --git a/src/Common/tests/gtest_poolbase.cpp b/src/Common/tests/gtest_poolbase.cpp index 20c3281c964..879b1b16620 100644 --- a/src/Common/tests/gtest_poolbase.cpp +++ b/src/Common/tests/gtest_poolbase.cpp @@ -18,7 +18,7 @@ public: using Ptr = PoolBase::Ptr; int last_destroy_value = 0; - MyPoolBase() : PoolBase(100, &Poco::Logger::get("MyPoolBase")) { } + MyPoolBase() : PoolBase(100, getLogger("MyPoolBase")) { } protected: ObjectPtr allocObject() override { return std::make_shared(); } diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index ee0356adde5..631a12cc252 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -33,7 +33,7 @@ DeflateQplJobHWPool::DeflateQplJobHWPool() : max_hw_jobs(0) , random_engine(randomSeed()) { - Poco::Logger * log = &Poco::Logger::get("DeflateQplJobHWPool"); + LoggerPtr log = getLogger("DeflateQplJobHWPool"); const char * qpl_version = qpl_get_library_version(); // loop all configured workqueue size to get maximum job number. @@ -141,7 +141,7 @@ void DeflateQplJobHWPool::unLockJob(UInt32 index) } HardwareCodecDeflateQpl::HardwareCodecDeflateQpl(SoftwareCodecDeflateQpl & sw_codec_) - : log(&Poco::Logger::get("HardwareCodecDeflateQpl")) + : log(getLogger("HardwareCodecDeflateQpl")) , sw_codec(sw_codec_) { } diff --git a/src/Compression/CompressionCodecDeflateQpl.h b/src/Compression/CompressionCodecDeflateQpl.h index 3d9a9b13921..c5978335fe8 100644 --- a/src/Compression/CompressionCodecDeflateQpl.h +++ b/src/Compression/CompressionCodecDeflateQpl.h @@ -88,7 +88,7 @@ private: /// For each submission, push job ID && job object into this map; /// For flush, pop out job ID && job object from this map. Use job ID to release job lock and use job object to check job status till complete. std::map decomp_async_job_map; - Poco::Logger * log; + LoggerPtr log; /// Provides a fallback in case of errors. SoftwareCodecDeflateQpl & sw_codec; }; diff --git a/src/Compression/CompressionCodecEncrypted.cpp b/src/Compression/CompressionCodecEncrypted.cpp index 8d945417fc1..3b7f4824069 100644 --- a/src/Compression/CompressionCodecEncrypted.cpp +++ b/src/Compression/CompressionCodecEncrypted.cpp @@ -694,7 +694,7 @@ bool CompressionCodecEncrypted::Configuration::tryLoad(const Poco::Util::Abstrac /// if encryption is disabled, print warning about this. void CompressionCodecEncrypted::Configuration::load(const Poco::Util::AbstractConfiguration & config [[maybe_unused]], const String & config_prefix [[maybe_unused]]) { - LOG_WARNING(&Poco::Logger::get("CompressionCodecEncrypted"), "Server was built without SSL support. Encryption is disabled."); + LOG_WARNING(getLogger("CompressionCodecEncrypted"), "Server was built without SSL support. Encryption is disabled."); } } diff --git a/src/Compression/CompressionCodecZSTDQAT.cpp b/src/Compression/CompressionCodecZSTDQAT.cpp index 4828a71a515..5a4ef70a30a 100644 --- a/src/Compression/CompressionCodecZSTDQAT.cpp +++ b/src/Compression/CompressionCodecZSTDQAT.cpp @@ -34,7 +34,7 @@ protected: private: const int level; - Poco::Logger * log; + LoggerPtr log; static std::atomic qat_state; /// Global initialization status of QAT device, we fall back back to software compression if uninitialized }; @@ -103,7 +103,7 @@ void registerCodecZSTDQAT(CompressionCodecFactory & factory) CompressionCodecZSTDQAT::CompressionCodecZSTDQAT(int level_) : CompressionCodecZSTD(level_) , level(level_) - , log(&Poco::Logger::get("CompressionCodecZSTDQAT")) + , log(getLogger("CompressionCodecZSTDQAT")) { setCodecDescription("ZSTD_QAT", {std::make_shared(static_cast(level))}); } diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 7f1135eec94..5a58932606e 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -116,7 +116,7 @@ public: : existing_changelogs(existing_changelogs_) , log_file_settings(log_file_settings_) , keeper_context(std::move(keeper_context_)) - , log(&Poco::Logger::get("Changelog")) + , log(getLogger("Changelog")) { } @@ -454,7 +454,7 @@ private: KeeperContextPtr keeper_context; - Poco::Logger * const log; + LoggerPtr const log; }; struct ChangelogReadResult @@ -493,7 +493,7 @@ public: } /// start_log_index -- all entries with index < start_log_index will be skipped, but accounted into total_entries_read_from_log - ChangelogReadResult readChangelog(IndexToLogEntry & logs, uint64_t start_log_index, Poco::Logger * log) + ChangelogReadResult readChangelog(IndexToLogEntry & logs, uint64_t start_log_index, LoggerPtr log) { ChangelogReadResult result{}; result.compressed_log = compression_method != CompressionMethod::None; @@ -592,7 +592,7 @@ private: }; Changelog::Changelog( - Poco::Logger * log_, LogFileSettings log_file_settings, FlushSettings flush_settings_, KeeperContextPtr keeper_context_) + LoggerPtr log_, LogFileSettings log_file_settings, FlushSettings flush_settings_, KeeperContextPtr keeper_context_) : changelogs_detached_dir("detached") , rotate_interval(log_file_settings.rotate_interval) , compress_logs(log_file_settings.compress_logs) diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index 20f850e3f62..612c68ab733 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -94,7 +94,7 @@ class Changelog { public: Changelog( - Poco::Logger * log_, + LoggerPtr log_, LogFileSettings log_file_settings, FlushSettings flush_settings, KeeperContextPtr keeper_context_); @@ -185,7 +185,7 @@ private: const String changelogs_detached_dir; const uint64_t rotate_interval; const bool compress_logs; - Poco::Logger * log; + LoggerPtr log; std::mutex writer_mutex; /// Current writer for changelog file diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index af2e4ec5a34..4862acd448f 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -234,7 +234,7 @@ void FourLetterCommandFactory::initializeAllowList(KeeperDispatcher & keeper_dis } else { - auto * log = &Poco::Logger::get("FourLetterCommandFactory"); + auto log = getLogger("FourLetterCommandFactory"); LOG_WARNING(log, "Find invalid keeper 4lw command {} when initializing, ignore it.", token); } } diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index 0d9eb2544a6..baad8d98e6a 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -55,7 +55,7 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config, if (!keeper_az.empty()) { system_nodes_with_data[keeper_availability_zone_path] = keeper_az; - LOG_INFO(&Poco::Logger::get("KeeperContext"), "Initialize the KeeperContext with availability zone: '{}'", keeper_az); + LOG_INFO(getLogger("KeeperContext"), "Initialize the KeeperContext with availability zone: '{}'", keeper_az); } } @@ -88,7 +88,7 @@ bool diskValidator(const Poco::Util::AbstractConfiguration & config, const std:: supported_disk_types.end(), [&](const auto supported_type) { return disk_type != supported_type; })) { - LOG_INFO(&Poco::Logger::get("KeeperContext"), "Disk type '{}' is not supported for Keeper", disk_type); + LOG_INFO(getLogger("KeeperContext"), "Disk type '{}' is not supported for Keeper", disk_type); return false; } @@ -374,7 +374,7 @@ void KeeperContext::initializeFeatureFlags(const Poco::Util::AbstractConfigurati system_nodes_with_data[keeper_api_feature_flags_path] = feature_flags.getFeatureFlags(); } - feature_flags.logFlags(&Poco::Logger::get("KeeperContext")); + feature_flags.logFlags(getLogger("KeeperContext")); } void KeeperContext::updateKeeperMemorySoftLimit(const Poco::Util::AbstractConfiguration & config) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 8bf48f23f50..35bc953a705 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -94,7 +94,7 @@ bool checkIfRequestIncreaseMem(const Coordination::ZooKeeperRequestPtr & request KeeperDispatcher::KeeperDispatcher() : responses_queue(std::numeric_limits::max()) , configuration_and_settings(std::make_shared()) - , log(&Poco::Logger::get("KeeperDispatcher")) + , log(getLogger("KeeperDispatcher")) {} void KeeperDispatcher::requestThread() diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 9c487e7f0f9..db41fb2ea26 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -70,7 +70,7 @@ private: KeeperConfigurationAndSettingsPtr configuration_and_settings; - Poco::Logger * log; + LoggerPtr log; /// Counter for new session_id requests. std::atomic internal_session_id_counter{0}; diff --git a/src/Coordination/KeeperFeatureFlags.cpp b/src/Coordination/KeeperFeatureFlags.cpp index d0cd1c86b55..2aad6cbed32 100644 --- a/src/Coordination/KeeperFeatureFlags.cpp +++ b/src/Coordination/KeeperFeatureFlags.cpp @@ -80,7 +80,7 @@ const std::string & KeeperFeatureFlags::getFeatureFlags() const return feature_flags; } -void KeeperFeatureFlags::logFlags(Poco::Logger * log) const +void KeeperFeatureFlags::logFlags(LoggerPtr log) const { for (const auto & [feature_flag, feature_flag_name] : magic_enum::enum_entries()) { diff --git a/src/Coordination/KeeperFeatureFlags.h b/src/Coordination/KeeperFeatureFlags.h index 4db972fa2a0..4e26ca60736 100644 --- a/src/Coordination/KeeperFeatureFlags.h +++ b/src/Coordination/KeeperFeatureFlags.h @@ -32,7 +32,7 @@ public: void enableFeatureFlag(KeeperFeatureFlag feature); void disableFeatureFlag(KeeperFeatureFlag feature); - void logFlags(Poco::Logger * log) const; + void logFlags(LoggerPtr log) const; private: std::string feature_flags; }; diff --git a/src/Coordination/KeeperLogStore.cpp b/src/Coordination/KeeperLogStore.cpp index 8cff3419afc..ce7c715237e 100644 --- a/src/Coordination/KeeperLogStore.cpp +++ b/src/Coordination/KeeperLogStore.cpp @@ -7,7 +7,7 @@ namespace DB { KeeperLogStore::KeeperLogStore(LogFileSettings log_file_settings, FlushSettings flush_settings, KeeperContextPtr keeper_context) - : log(&Poco::Logger::get("KeeperLogStore")), changelog(log, log_file_settings, flush_settings, keeper_context) + : log(getLogger("KeeperLogStore")), changelog(log, log_file_settings, flush_settings, keeper_context) { if (log_file_settings.force_sync) LOG_INFO(log, "force_sync enabled"); diff --git a/src/Coordination/KeeperLogStore.h b/src/Coordination/KeeperLogStore.h index de9205241bd..aa277f19d88 100644 --- a/src/Coordination/KeeperLogStore.h +++ b/src/Coordination/KeeperLogStore.h @@ -74,7 +74,7 @@ public: private: mutable std::mutex changelog_lock; - Poco::Logger * log; + LoggerPtr log; Changelog changelog TSA_GUARDED_BY(changelog_lock); }; diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 45619ab38a1..722b1303cc8 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -92,7 +92,7 @@ std::string checkAndGetSuperdigest(const String & user_and_digest) return user_and_digest; } -int32_t getValueOrMaxInt32AndLogWarning(uint64_t value, const std::string & name, Poco::Logger * log) +int32_t getValueOrMaxInt32AndLogWarning(uint64_t value, const std::string & name, LoggerPtr log) { if (value > std::numeric_limits::max()) { @@ -120,7 +120,7 @@ KeeperServer::KeeperServer( KeeperStateMachine::CommitCallback commit_callback) : server_id(configuration_and_settings_->server_id) , coordination_settings(configuration_and_settings_->coordination_settings) - , log(&Poco::Logger::get("KeeperServer")) + , log(getLogger("KeeperServer")) , is_recovering(config.getBool("keeper_server.force_recovery", false)) , keeper_context{std::move(keeper_context_)} , create_snapshot_on_exit(config.getBool("keeper_server.create_snapshot_on_exit", true)) diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index 8c657ab28a7..ef298df3efc 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -48,7 +48,7 @@ private: nuraft::ptr last_local_config; - Poco::Logger * log; + LoggerPtr log; /// Callback func which is called by NuRaft on all internal events. /// Used to determine the moment when raft is ready to server new requests diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index ee5935015e4..f53b8031712 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -383,7 +383,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial { if (keeper_context->ignoreSystemPathOnStartup() || keeper_context->getServerState() != KeeperContext::Phase::INIT) { - LOG_ERROR(&Poco::Logger::get("KeeperSnapshotManager"), "{}. Ignoring it", error_msg); + LOG_ERROR(getLogger("KeeperSnapshotManager"), "{}. Ignoring it", error_msg); continue; } else @@ -399,7 +399,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial { if (keeper_context->ignoreSystemPathOnStartup() || keeper_context->getServerState() != KeeperContext::Phase::INIT) { - LOG_ERROR(&Poco::Logger::get("KeeperSnapshotManager"), "{}. Ignoring it", error_msg); + LOG_ERROR(getLogger("KeeperSnapshotManager"), "{}. Ignoring it", error_msg); node = KeeperStorage::Node{}; } else @@ -437,7 +437,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial { #ifdef NDEBUG /// TODO (alesapin) remove this, it should be always CORRUPTED_DATA. - LOG_ERROR(&Poco::Logger::get("KeeperSnapshotManager"), "Children counter in stat.numChildren {}" + LOG_ERROR(getLogger("KeeperSnapshotManager"), "Children counter in stat.numChildren {}" " is different from actual children size {} for node {}", itr.value.stat.numChildren, itr.value.getChildren().size(), itr.key); #else throw Exception(ErrorCodes::LOGICAL_ERROR, "Children counter in stat.numChildren {}" @@ -594,7 +594,7 @@ KeeperSnapshotManager::KeeperSnapshotManager( if (!inserted) LOG_WARNING( - &Poco::Logger::get("KeeperSnapshotManager"), + getLogger("KeeperSnapshotManager"), "Found another snapshots with last log idx {}, will use snapshot from disk {}", snapshot_up_to, disk->getName()); diff --git a/src/Coordination/KeeperSnapshotManager.h b/src/Coordination/KeeperSnapshotManager.h index 6096ba318da..48a66e79cd8 100644 --- a/src/Coordination/KeeperSnapshotManager.h +++ b/src/Coordination/KeeperSnapshotManager.h @@ -188,7 +188,7 @@ private: KeeperContextPtr keeper_context; - Poco::Logger * log = &Poco::Logger::get("KeeperSnapshotManager"); + LoggerPtr log = getLogger("KeeperSnapshotManager"); }; /// Keeper create snapshots in background thread. KeeperStateMachine just create diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index 716184e07d0..0337a564660 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -43,7 +43,7 @@ struct KeeperSnapshotManagerS3::S3Configuration KeeperSnapshotManagerS3::KeeperSnapshotManagerS3() : snapshots_s3_queue(std::numeric_limits::max()) - , log(&Poco::Logger::get("KeeperSnapshotManagerS3")) + , log(getLogger("KeeperSnapshotManagerS3")) , uuid(UUIDHelpers::generateV4()) {} diff --git a/src/Coordination/KeeperSnapshotManagerS3.h b/src/Coordination/KeeperSnapshotManagerS3.h index e17cf5a1cfb..d03deb60c1a 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.h +++ b/src/Coordination/KeeperSnapshotManagerS3.h @@ -45,7 +45,7 @@ private: std::atomic shutdown_called{false}; - Poco::Logger * log; + LoggerPtr log; UUID uuid; diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 39a2347ff80..8d50f0a76b1 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -59,7 +59,7 @@ KeeperStateMachine::KeeperStateMachine( , snapshots_queue(snapshots_queue_) , min_request_size_to_cache(coordination_settings_->min_request_size_for_cache) , last_committed_idx(0) - , log(&Poco::Logger::get("KeeperStateMachine")) + , log(getLogger("KeeperStateMachine")) , superdigest(superdigest_) , keeper_context(keeper_context_) , snapshot_manager_s3(snapshot_manager_s3_) @@ -144,7 +144,7 @@ void assertDigest( if (!KeeperStorage::checkDigest(first, second)) { LOG_FATAL( - &Poco::Logger::get("KeeperStateMachine"), + getLogger("KeeperStateMachine"), "Digest for nodes is not matching after {} request of type '{}'.\nExpected digest - {}, actual digest - {} (digest " "{}). Keeper will terminate to avoid inconsistencies.\nExtra information about the request:\n{}", committing ? "committing" : "preprocessing", @@ -679,7 +679,7 @@ void KeeperStateMachine::save_logical_snp_obj( } } -static int bufferFromFile(Poco::Logger * log, const std::string & path, nuraft::ptr & data_out) +static int bufferFromFile(LoggerPtr log, const std::string & path, nuraft::ptr & data_out) { if (path.empty() || !std::filesystem::exists(path)) { diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index aad5d3aafd4..b11cd53c00e 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -173,7 +173,7 @@ private: /// Last committed Raft log number. std::atomic last_committed_idx; - Poco::Logger * log; + LoggerPtr log; /// Cluster config for our quorum. /// It's a copy of config stored in StateManager, but here diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index efe8a0cb2bd..4fbb9b52e6e 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -227,7 +227,7 @@ KeeperStateManager::KeeperStateManager(int server_id_, const std::string & host, keeper_context_)) , server_state_file_name("state") , keeper_context(keeper_context_) - , logger(&Poco::Logger::get("KeeperStateManager")) + , logger(getLogger("KeeperStateManager")) { auto peer_config = nuraft::cs_new(my_server_id, host + ":" + std::to_string(port)); configuration_wrapper.cluster_config = nuraft::cs_new(); @@ -262,7 +262,7 @@ KeeperStateManager::KeeperStateManager( keeper_context_)) , server_state_file_name(server_state_file_name_) , keeper_context(keeper_context_) - , logger(&Poco::Logger::get("KeeperStateManager")) + , logger(getLogger("KeeperStateManager")) { } @@ -495,7 +495,7 @@ ClusterUpdateActions KeeperStateManager::getRaftConfigurationDiff( if (old_endpoint != server_config->get_endpoint()) { LOG_WARNING( - &Poco::Logger::get("RaftConfiguration"), + getLogger("RaftConfiguration"), "Config will be ignored because a server with ID {} is already present in the cluster on a different endpoint ({}). " "The endpoint of the current servers should not be changed. For servers on a new endpoint, please use a new ID.", new_id, diff --git a/src/Coordination/KeeperStateManager.h b/src/Coordination/KeeperStateManager.h index fd05261ac6c..02dd6b2ff53 100644 --- a/src/Coordination/KeeperStateManager.h +++ b/src/Coordination/KeeperStateManager.h @@ -128,7 +128,7 @@ private: KeeperContextPtr keeper_context; - Poco::Logger * logger; + LoggerPtr logger; public: /// Parse configuration from xml config. diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index c128d7c2f98..992d4ca8a95 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -609,7 +609,7 @@ namespace [[noreturn]] void onStorageInconsistency() { LOG_ERROR( - &Poco::Logger::get("KeeperStorage"), + getLogger("KeeperStorage"), "Inconsistency found between uncommitted and committed data. Keeper will terminate to avoid undefined behaviour."); std::terminate(); } @@ -887,7 +887,7 @@ void handleSystemNodeModification(const KeeperContext & keeper_context, std::str "If you still want to ignore it, you can set 'keeper_server.ignore_system_path_on_startup' to true.", error_msg); - LOG_ERROR(&Poco::Logger::get("KeeperStorage"), fmt::runtime(error_msg)); + LOG_ERROR(getLogger("KeeperStorage"), fmt::runtime(error_msg)); } } @@ -2381,7 +2381,7 @@ void KeeperStorage::rollbackRequest(int64_t rollback_zxid, bool allow_missing) } catch (...) { - LOG_FATAL(&Poco::Logger::get("KeeperStorage"), "Failed to rollback log. Terminating to avoid inconsistencies"); + LOG_FATAL(getLogger("KeeperStorage"), "Failed to rollback log. Terminating to avoid inconsistencies"); std::terminate(); } } diff --git a/src/Coordination/LoggerWrapper.h b/src/Coordination/LoggerWrapper.h index d092a8d4440..d08c42b6868 100644 --- a/src/Coordination/LoggerWrapper.h +++ b/src/Coordination/LoggerWrapper.h @@ -26,7 +26,7 @@ private: public: LoggerWrapper(const std::string & name, LogsLevel level_) - : log(&Poco::Logger::get(name)) + : log(getLogger(name)) , level(level_) { log->setLevel(static_cast(LEVELS.at(level))); @@ -57,7 +57,7 @@ public: } private: - Poco::Logger * log; + LoggerPtr log; std::atomic level; }; diff --git a/src/Coordination/ZooKeeperDataReader.cpp b/src/Coordination/ZooKeeperDataReader.cpp index b55ebef327f..6b9d5f7c8eb 100644 --- a/src/Coordination/ZooKeeperDataReader.cpp +++ b/src/Coordination/ZooKeeperDataReader.cpp @@ -90,7 +90,7 @@ void deserializeACLMap(KeeperStorage & storage, ReadBuffer & in) } } -int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, Poco::Logger * log) +int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, LoggerPtr log) { int64_t max_zxid = 0; std::string path; @@ -146,7 +146,7 @@ int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, Poco::L return max_zxid; } -void deserializeKeeperStorageFromSnapshot(KeeperStorage & storage, const std::string & snapshot_path, Poco::Logger * log) +void deserializeKeeperStorageFromSnapshot(KeeperStorage & storage, const std::string & snapshot_path, LoggerPtr log) { LOG_INFO(log, "Deserializing storage snapshot {}", snapshot_path); int64_t zxid = getZxidFromName(snapshot_path); @@ -185,7 +185,7 @@ void deserializeKeeperStorageFromSnapshot(KeeperStorage & storage, const std::st LOG_INFO(log, "Finished, snapshot ZXID {}", storage.zxid); } -void deserializeKeeperStorageFromSnapshotsDir(KeeperStorage & storage, const std::string & path, Poco::Logger * log) +void deserializeKeeperStorageFromSnapshotsDir(KeeperStorage & storage, const std::string & path, LoggerPtr log) { namespace fs = std::filesystem; std::map existing_snapshots; @@ -473,7 +473,7 @@ bool hasErrorsInMultiRequest(Coordination::ZooKeeperRequestPtr request) } -bool deserializeTxn(KeeperStorage & storage, ReadBuffer & in, Poco::Logger * /*log*/) +bool deserializeTxn(KeeperStorage & storage, ReadBuffer & in, LoggerPtr /*log*/) { int64_t checksum; Coordination::read(checksum, in); @@ -528,7 +528,7 @@ bool deserializeTxn(KeeperStorage & storage, ReadBuffer & in, Poco::Logger * /*l return true; } -void deserializeLogAndApplyToStorage(KeeperStorage & storage, const std::string & log_path, Poco::Logger * log) +void deserializeLogAndApplyToStorage(KeeperStorage & storage, const std::string & log_path, LoggerPtr log) { ReadBufferFromFile reader(log_path); @@ -552,7 +552,7 @@ void deserializeLogAndApplyToStorage(KeeperStorage & storage, const std::string LOG_INFO(log, "Finished {} deserialization, totally read {} records", log_path, counter); } -void deserializeLogsAndApplyToStorage(KeeperStorage & storage, const std::string & path, Poco::Logger * log) +void deserializeLogsAndApplyToStorage(KeeperStorage & storage, const std::string & path, LoggerPtr log) { namespace fs = std::filesystem; std::map existing_logs; diff --git a/src/Coordination/ZooKeeperDataReader.h b/src/Coordination/ZooKeeperDataReader.h index 8fd86ba99e2..648dc95adcf 100644 --- a/src/Coordination/ZooKeeperDataReader.h +++ b/src/Coordination/ZooKeeperDataReader.h @@ -5,12 +5,12 @@ namespace DB { -void deserializeKeeperStorageFromSnapshot(KeeperStorage & storage, const std::string & snapshot_path, Poco::Logger * log); +void deserializeKeeperStorageFromSnapshot(KeeperStorage & storage, const std::string & snapshot_path, LoggerPtr log); -void deserializeKeeperStorageFromSnapshotsDir(KeeperStorage & storage, const std::string & path, Poco::Logger * log); +void deserializeKeeperStorageFromSnapshotsDir(KeeperStorage & storage, const std::string & path, LoggerPtr log); -void deserializeLogAndApplyToStorage(KeeperStorage & storage, const std::string & log_path, Poco::Logger * log); +void deserializeLogAndApplyToStorage(KeeperStorage & storage, const std::string & log_path, LoggerPtr log); -void deserializeLogsAndApplyToStorage(KeeperStorage & storage, const std::string & path, Poco::Logger * log); +void deserializeLogsAndApplyToStorage(KeeperStorage & storage, const std::string & path, LoggerPtr log); } diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index c981085359e..59a550177a4 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -66,7 +66,7 @@ class CoordinationTest : public ::testing::TestWithParam { protected: DB::KeeperContextPtr keeper_context = std::make_shared(true); - Poco::Logger * log{&Poco::Logger::get("CoordinationTest")}; + LoggerPtr log{getLogger("CoordinationTest")}; void SetUp() override { @@ -1101,7 +1101,7 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2) } /// Truncating only some entries from the end -/// For compressed logs we have no reliable way of knowing how many log entries were lost +/// For compressed logs we have no reliable way of knowing how many log entries were lost /// after we truncate some bytes from the end TEST_F(CoordinationTest, ChangelogTestReadAfterBrokenTruncate3) { @@ -1801,7 +1801,7 @@ void testLogAndStateMachine( = [&snapshot_created](bool & ret, nuraft::ptr & /*exception*/) { snapshot_created = ret; - LOG_INFO(&Poco::Logger::get("CoordinationTest"), "Snapshot finished"); + LOG_INFO(getLogger("CoordinationTest"), "Snapshot finished"); }; state_machine->create_snapshot(s, when_done); diff --git a/src/Core/BackgroundSchedulePool.cpp b/src/Core/BackgroundSchedulePool.cpp index fa892bc3c84..4facdeb4631 100644 --- a/src/Core/BackgroundSchedulePool.cpp +++ b/src/Core/BackgroundSchedulePool.cpp @@ -116,7 +116,7 @@ void BackgroundSchedulePoolTaskInfo::execute() static constexpr UInt64 slow_execution_threshold_ms = 200; if (milliseconds >= slow_execution_threshold_ms) - LOG_TRACE(&Poco::Logger::get(log_name), "Execution took {} ms.", milliseconds); + LOG_TRACE(getLogger(log_name), "Execution took {} ms.", milliseconds); { std::lock_guard lock_schedule(schedule_mutex); @@ -160,7 +160,7 @@ BackgroundSchedulePool::BackgroundSchedulePool(size_t size_, CurrentMetrics::Met , size_metric(size_metric_, size_) , thread_name(thread_name_) { - LOG_INFO(&Poco::Logger::get("BackgroundSchedulePool/" + thread_name), "Create BackgroundSchedulePool with {} threads", size_); + LOG_INFO(getLogger("BackgroundSchedulePool/" + thread_name), "Create BackgroundSchedulePool with {} threads", size_); threads.resize(size_); @@ -174,7 +174,7 @@ BackgroundSchedulePool::BackgroundSchedulePool(size_t size_, CurrentMetrics::Met catch (...) { LOG_FATAL( - &Poco::Logger::get("BackgroundSchedulePool/" + thread_name), + getLogger("BackgroundSchedulePool/" + thread_name), "Couldn't get {} threads from global thread pool: {}", size_, getCurrentExceptionCode() == DB::ErrorCodes::CANNOT_SCHEDULE_TASK @@ -192,7 +192,7 @@ void BackgroundSchedulePool::increaseThreadsCount(size_t new_threads_count) if (new_threads_count < old_threads_count) { - LOG_WARNING(&Poco::Logger::get("BackgroundSchedulePool/" + thread_name), + LOG_WARNING(getLogger("BackgroundSchedulePool/" + thread_name), "Tried to increase the number of threads but the new threads count ({}) is not greater than old one ({})", new_threads_count, old_threads_count); return; } @@ -219,7 +219,7 @@ BackgroundSchedulePool::~BackgroundSchedulePool() tasks_cond_var.notify_all(); delayed_tasks_cond_var.notify_all(); - LOG_TRACE(&Poco::Logger::get("BackgroundSchedulePool/" + thread_name), "Waiting for threads to finish."); + LOG_TRACE(getLogger("BackgroundSchedulePool/" + thread_name), "Waiting for threads to finish."); delayed_thread->join(); for (auto & thread : threads) diff --git a/src/Core/BaseSettings.cpp b/src/Core/BaseSettings.cpp index 72a8070e652..a7e1ab99af7 100644 --- a/src/Core/BaseSettings.cpp +++ b/src/Core/BaseSettings.cpp @@ -47,8 +47,7 @@ void BaseSettingsHelpers::throwSettingNotFound(std::string_view name) void BaseSettingsHelpers::warningSettingNotFound(std::string_view name) { - static auto * log = &Poco::Logger::get("Settings"); - LOG_WARNING(log, "Unknown setting {}, skipping", name); + LOG_WARNING(getLogger("Settings"), "Unknown setting {}, skipping", name); } } diff --git a/src/Core/MySQL/Authentication.cpp b/src/Core/MySQL/Authentication.cpp index 2c10bd88722..ac6ed70dbb5 100644 --- a/src/Core/MySQL/Authentication.cpp +++ b/src/Core/MySQL/Authentication.cpp @@ -102,7 +102,7 @@ void Native41::authenticate( #if USE_SSL -Sha256Password::Sha256Password(RSA & public_key_, RSA & private_key_, Poco::Logger * log_) +Sha256Password::Sha256Password(RSA & public_key_, RSA & private_key_, LoggerPtr log_) : public_key(public_key_), private_key(private_key_), log(log_) { /** Native authentication sent 20 bytes + '\0' character = 21 bytes. diff --git a/src/Core/MySQL/Authentication.h b/src/Core/MySQL/Authentication.h index ee6aaac02bc..3179fa20f59 100644 --- a/src/Core/MySQL/Authentication.h +++ b/src/Core/MySQL/Authentication.h @@ -61,7 +61,7 @@ private: class Sha256Password : public IPlugin { public: - Sha256Password(RSA & public_key_, RSA & private_key_, Poco::Logger * log_); + Sha256Password(RSA & public_key_, RSA & private_key_, LoggerPtr log_); String getName() override { return "sha256_password"; } @@ -74,7 +74,7 @@ public: private: RSA & public_key; RSA & private_key; - Poco::Logger * log; + LoggerPtr log; String scramble; }; #endif diff --git a/src/Core/PostgreSQL/Connection.cpp b/src/Core/PostgreSQL/Connection.cpp index 5a589a80d02..eea24dd6940 100644 --- a/src/Core/PostgreSQL/Connection.cpp +++ b/src/Core/PostgreSQL/Connection.cpp @@ -9,7 +9,7 @@ namespace postgres Connection::Connection(const ConnectionInfo & connection_info_, bool replication_, size_t num_tries_) : connection_info(connection_info_), replication(replication_), num_tries(num_tries_) - , log(&Poco::Logger::get("PostgreSQLReplicaConnection")) + , log(getLogger("PostgreSQLReplicaConnection")) { if (replication) connection_info = {fmt::format("{} replication=database", connection_info.connection_string), connection_info.host_port}; @@ -65,7 +65,7 @@ void Connection::updateConnection() if (replication) connection->set_variable("default_transaction_isolation", "'repeatable read'"); - LOG_DEBUG(&Poco::Logger::get("PostgreSQLConnection"), "New connection to {}", connection_info.host_port); + LOG_DEBUG(getLogger("PostgreSQLConnection"), "New connection to {}", connection_info.host_port); } void Connection::connect() diff --git a/src/Core/PostgreSQL/Connection.h b/src/Core/PostgreSQL/Connection.h index efc10b6ed20..5e0aa0983d5 100644 --- a/src/Core/PostgreSQL/Connection.h +++ b/src/Core/PostgreSQL/Connection.h @@ -6,6 +6,7 @@ #include #include +#include #include /** Methods to work with PostgreSQL connection object. @@ -61,7 +62,7 @@ private: bool replication; size_t num_tries; - Poco::Logger * log; + LoggerPtr log; }; using ConnectionPtr = std::unique_ptr; diff --git a/src/Core/PostgreSQL/PoolWithFailover.cpp b/src/Core/PostgreSQL/PoolWithFailover.cpp index 3655681c515..a034c50094d 100644 --- a/src/Core/PostgreSQL/PoolWithFailover.cpp +++ b/src/Core/PostgreSQL/PoolWithFailover.cpp @@ -32,7 +32,7 @@ PoolWithFailover::PoolWithFailover( , max_tries(max_tries_) , auto_close_connection(auto_close_connection_) { - LOG_TRACE(&Poco::Logger::get("PostgreSQLConnectionPool"), "PostgreSQL connection pool size: {}, connection wait timeout: {}, max failover tries: {}", + LOG_TRACE(getLogger("PostgreSQLConnectionPool"), "PostgreSQL connection pool size: {}, connection wait timeout: {}, max failover tries: {}", pool_size, pool_wait_timeout, max_tries_); for (const auto & [priority, configurations] : configurations_by_priority) @@ -56,13 +56,13 @@ PoolWithFailover::PoolWithFailover( , max_tries(max_tries_) , auto_close_connection(auto_close_connection_) { - LOG_TRACE(&Poco::Logger::get("PostgreSQLConnectionPool"), "PostgreSQL connection pool size: {}, connection wait timeout: {}, max failover tries: {}", + LOG_TRACE(getLogger("PostgreSQLConnectionPool"), "PostgreSQL connection pool size: {}, connection wait timeout: {}, max failover tries: {}", pool_size, pool_wait_timeout, max_tries_); /// Replicas have the same priority, but traversed replicas are moved to the end of the queue. for (const auto & [host, port] : configuration.addresses) { - LOG_DEBUG(&Poco::Logger::get("PostgreSQLPoolWithFailover"), "Adding address host: {}, port: {} to connection pool", host, port); + LOG_DEBUG(getLogger("PostgreSQLPoolWithFailover"), "Adding address host: {}, port: {} to connection pool", host, port); auto connection_string = formatConnectionString(configuration.database, host, port, configuration.username, configuration.password); replicas_with_priority[0].emplace_back(connection_string, pool_size); } diff --git a/src/Core/PostgreSQL/PoolWithFailover.h b/src/Core/PostgreSQL/PoolWithFailover.h index bf3782afba4..3c538fc3dea 100644 --- a/src/Core/PostgreSQL/PoolWithFailover.h +++ b/src/Core/PostgreSQL/PoolWithFailover.h @@ -62,7 +62,7 @@ private: size_t max_tries; bool auto_close_connection; std::mutex mutex; - Poco::Logger * log = &Poco::Logger::get("PostgreSQLConnectionPool"); + LoggerPtr log = getLogger("PostgreSQLConnectionPool"); }; using PoolWithFailoverPtr = std::shared_ptr; diff --git a/src/Core/PostgreSQLProtocol.h b/src/Core/PostgreSQLProtocol.h index b0d7646a5f7..7630fbb0b23 100644 --- a/src/Core/PostgreSQLProtocol.h +++ b/src/Core/PostgreSQLProtocol.h @@ -872,7 +872,7 @@ public: class AuthenticationManager { private: - Poco::Logger * log = &Poco::Logger::get("AuthenticationManager"); + LoggerPtr log = getLogger("AuthenticationManager"); std::unordered_map> type_to_method = {}; public: diff --git a/src/Core/ServerUUID.h b/src/Core/ServerUUID.h index 36bbf0e6315..b5ea17426cb 100644 --- a/src/Core/ServerUUID.h +++ b/src/Core/ServerUUID.h @@ -1,12 +1,10 @@ #pragma once + #include +#include #include namespace fs = std::filesystem; -namespace Poco -{ - class Logger; -} namespace DB { diff --git a/src/Core/SettingsQuirks.cpp b/src/Core/SettingsQuirks.cpp index 1a79c23d955..24dcd43a09c 100644 --- a/src/Core/SettingsQuirks.cpp +++ b/src/Core/SettingsQuirks.cpp @@ -17,7 +17,7 @@ namespace /// /// [1]: https://git.kernel.org/pub/scm/linux/kernel/git/torvalds/linux.git/commit/?id=339ddb53d373 /// [2]: https://git.kernel.org/pub/scm/linux/kernel/git/torvalds/linux.git/commit/?id=0c54a6a44bf3 -bool nestedEpollWorks(Poco::Logger * log) +bool nestedEpollWorks(LoggerPtr log) { if (Poco::Environment::os() != POCO_OS_LINUX) return true; @@ -48,7 +48,7 @@ namespace DB { /// Update some settings defaults to avoid some known issues. -void applySettingsQuirks(Settings & settings, Poco::Logger * log) +void applySettingsQuirks(Settings & settings, LoggerPtr log) { if (!nestedEpollWorks(log)) { diff --git a/src/Core/SettingsQuirks.h b/src/Core/SettingsQuirks.h index 38def8eebf2..f6b2a4e33fa 100644 --- a/src/Core/SettingsQuirks.h +++ b/src/Core/SettingsQuirks.h @@ -1,9 +1,6 @@ #pragma once -namespace Poco -{ -class Logger; -} +#include namespace DB { @@ -11,6 +8,6 @@ namespace DB struct Settings; /// Update some settings defaults to avoid some known issues. -void applySettingsQuirks(Settings & settings, Poco::Logger * log = nullptr); +void applySettingsQuirks(Settings & settings, LoggerPtr log = nullptr); } diff --git a/src/Core/SortDescription.cpp b/src/Core/SortDescription.cpp index 9ba7df8ef24..9edc79a1ff1 100644 --- a/src/Core/SortDescription.cpp +++ b/src/Core/SortDescription.cpp @@ -108,10 +108,9 @@ static std::string getSortDescriptionDump(const SortDescription & description, c return buffer.str(); } -static Poco::Logger * getLogger() +static LoggerPtr getLogger() { - static Poco::Logger & logger = Poco::Logger::get("SortDescription"); - return &logger; + return ::getLogger("SortDescription"); } void compileSortDescriptionIfNeeded(SortDescription & description, const DataTypes & sort_description_types, bool increase_compile_attempts) diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index b7685159f98..289a41bb75e 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -210,7 +210,7 @@ public: static constexpr int SanitizerTrap = -3; explicit SignalListener(BaseDaemon & daemon_) - : log(&Poco::Logger::get("BaseDaemon")) + : log(getLogger("BaseDaemon")) , daemon(daemon_) { } @@ -295,7 +295,7 @@ public: } private: - Poco::Logger * log; + LoggerPtr log; BaseDaemon & daemon; void onTerminate(std::string_view message, UInt32 thread_num) const diff --git a/src/Daemon/SentryWriter.cpp b/src/Daemon/SentryWriter.cpp index 2050d503879..ebfd18abeee 100644 --- a/src/Daemon/SentryWriter.cpp +++ b/src/Daemon/SentryWriter.cpp @@ -68,7 +68,7 @@ void SentryWriter::initialize(Poco::Util::LayeredConfiguration & config) { bool enabled = false; bool debug = config.getBool("send_crash_reports.debug", false); - auto * logger = &Poco::Logger::get("SentryWriter"); + auto logger = getLogger("SentryWriter"); if (config.getBool("send_crash_reports.enabled", false)) { @@ -140,7 +140,7 @@ void SentryWriter::shutdown() void SentryWriter::onFault(int sig, const std::string & error_message, const StackTrace & stack_trace) { - auto * logger = &Poco::Logger::get("SentryWriter"); + auto logger = getLogger("SentryWriter"); if (initialized) { sentry_value_t event = sentry_value_new_message_event(SENTRY_LEVEL_FATAL, "fault", error_message.c_str()); diff --git a/src/Databases/DatabaseDictionary.cpp b/src/Databases/DatabaseDictionary.cpp index e2e0d52cd88..9a65c7a46ef 100644 --- a/src/Databases/DatabaseDictionary.cpp +++ b/src/Databases/DatabaseDictionary.cpp @@ -51,7 +51,7 @@ namespace DatabaseDictionary::DatabaseDictionary(const String & name_, ContextPtr context_) : IDatabase(name_), WithContext(context_->getGlobalContext()) - , log(&Poco::Logger::get("DatabaseDictionary(" + database_name + ")")) + , log(getLogger("DatabaseDictionary(" + database_name + ")")) { } diff --git a/src/Databases/DatabaseDictionary.h b/src/Databases/DatabaseDictionary.h index 425d048aa65..469801d183e 100644 --- a/src/Databases/DatabaseDictionary.h +++ b/src/Databases/DatabaseDictionary.h @@ -48,7 +48,7 @@ protected: ASTPtr getCreateTableQueryImpl(const String & table_name, ContextPtr context, bool throw_on_error) const override; private: - Poco::Logger * log; + LoggerPtr log; Tables listTables(const FilterByNameFunction & filter_by_name) const; }; diff --git a/src/Databases/DatabaseFilesystem.cpp b/src/Databases/DatabaseFilesystem.cpp index 5564f1d07cf..55ae60469ed 100644 --- a/src/Databases/DatabaseFilesystem.cpp +++ b/src/Databases/DatabaseFilesystem.cpp @@ -32,7 +32,7 @@ namespace ErrorCodes } DatabaseFilesystem::DatabaseFilesystem(const String & name_, const String & path_, ContextPtr context_) - : IDatabase(name_), WithContext(context_->getGlobalContext()), path(path_), log(&Poco::Logger::get("DatabaseFileSystem(" + name_ + ")")) + : IDatabase(name_), WithContext(context_->getGlobalContext()), path(path_), log(getLogger("DatabaseFileSystem(" + name_ + ")")) { bool is_local = context_->getApplicationType() == Context::ApplicationType::LOCAL; fs::path user_files_path = is_local ? "" : fs::canonical(getContext()->getUserFilesPath()); diff --git a/src/Databases/DatabaseFilesystem.h b/src/Databases/DatabaseFilesystem.h index b72891b9a5c..3338aa28c21 100644 --- a/src/Databases/DatabaseFilesystem.h +++ b/src/Databases/DatabaseFilesystem.h @@ -61,7 +61,7 @@ protected: private: String path; mutable Tables loaded_tables TSA_GUARDED_BY(mutex); - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/src/Databases/DatabaseHDFS.cpp b/src/Databases/DatabaseHDFS.cpp index 9d0395e4217..3a1e6b16ccf 100644 --- a/src/Databases/DatabaseHDFS.cpp +++ b/src/Databases/DatabaseHDFS.cpp @@ -45,7 +45,7 @@ DatabaseHDFS::DatabaseHDFS(const String & name_, const String & source_url, Cont : IDatabase(name_) , WithContext(context_->getGlobalContext()) , source(source_url) - , log(&Poco::Logger::get("DatabaseHDFS(" + name_ + ")")) + , log(getLogger("DatabaseHDFS(" + name_ + ")")) { if (!source.empty()) { diff --git a/src/Databases/DatabaseHDFS.h b/src/Databases/DatabaseHDFS.h index 957b2080135..b586a912e16 100644 --- a/src/Databases/DatabaseHDFS.h +++ b/src/Databases/DatabaseHDFS.h @@ -60,7 +60,7 @@ private: const String source; mutable Tables loaded_tables TSA_GUARDED_BY(mutex); - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 12b0dc07799..67ecd7be66a 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -660,7 +660,7 @@ void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const Iterat } ASTPtr DatabaseOnDisk::parseQueryFromMetadata( - Poco::Logger * logger, + LoggerPtr logger, ContextPtr local_context, const String & metadata_file_path, bool throw_on_error /*= true*/, diff --git a/src/Databases/DatabaseOnDisk.h b/src/Databases/DatabaseOnDisk.h index 59c2c27068e..b20b754b727 100644 --- a/src/Databases/DatabaseOnDisk.h +++ b/src/Databases/DatabaseOnDisk.h @@ -68,7 +68,7 @@ public: String getTableDataPath(const ASTCreateQuery & query) const override { return getTableDataPath(query.getTable()); } String getMetadataPath() const override { return metadata_path; } - static ASTPtr parseQueryFromMetadata(Poco::Logger * log, ContextPtr context, const String & metadata_file_path, bool throw_on_error = true, bool remove_empty = false); + static ASTPtr parseQueryFromMetadata(LoggerPtr log, ContextPtr context, const String & metadata_file_path, bool throw_on_error = true, bool remove_empty = false); /// will throw when the table we want to attach already exists (in active / detached / detached permanently form) void checkMetadataFilenameAvailability(const String & to_table_name) const override; diff --git a/src/Databases/DatabaseS3.cpp b/src/Databases/DatabaseS3.cpp index 1721b0e9e97..d2ca5a05ea4 100644 --- a/src/Databases/DatabaseS3.cpp +++ b/src/Databases/DatabaseS3.cpp @@ -49,7 +49,7 @@ DatabaseS3::DatabaseS3(const String & name_, const Configuration& config_, Conte : IDatabase(name_) , WithContext(context_->getGlobalContext()) , config(config_) - , log(&Poco::Logger::get("DatabaseS3(" + name_ + ")")) + , log(getLogger("DatabaseS3(" + name_ + ")")) { } diff --git a/src/Databases/DatabaseS3.h b/src/Databases/DatabaseS3.h index 8297ae4e02d..5e7375dbd58 100644 --- a/src/Databases/DatabaseS3.h +++ b/src/Databases/DatabaseS3.h @@ -73,7 +73,7 @@ private: const Configuration config; mutable Tables loaded_tables TSA_GUARDED_BY(mutex); - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index bda48737621..963cf0064df 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -197,7 +197,7 @@ void cleanupObjectDefinitionFromTemporaryFlags(ASTCreateQuery & query) DatabaseWithOwnTablesBase::DatabaseWithOwnTablesBase(const String & name_, const String & logger, ContextPtr context_) - : IDatabase(name_), WithContext(context_->getGlobalContext()), log(&Poco::Logger::get(logger)) + : IDatabase(name_), WithContext(context_->getGlobalContext()), log(getLogger(logger)) { } diff --git a/src/Databases/DatabasesCommon.h b/src/Databases/DatabasesCommon.h index fc67596d3de..4e9d967c11a 100644 --- a/src/Databases/DatabasesCommon.h +++ b/src/Databases/DatabasesCommon.h @@ -45,7 +45,7 @@ public: protected: Tables tables TSA_GUARDED_BY(mutex); - Poco::Logger * log; + LoggerPtr log; DatabaseWithOwnTablesBase(const String & name_, const String & logger, ContextPtr context); diff --git a/src/Databases/DatabasesOverlay.cpp b/src/Databases/DatabasesOverlay.cpp index b44a9798072..8cea3441698 100644 --- a/src/Databases/DatabasesOverlay.cpp +++ b/src/Databases/DatabasesOverlay.cpp @@ -17,7 +17,7 @@ namespace ErrorCodes } DatabasesOverlay::DatabasesOverlay(const String & name_, ContextPtr context_) - : IDatabase(name_), WithContext(context_->getGlobalContext()), log(&Poco::Logger::get("DatabaseOverlay(" + name_ + ")")) + : IDatabase(name_), WithContext(context_->getGlobalContext()), log(getLogger("DatabaseOverlay(" + name_ + ")")) { } diff --git a/src/Databases/DatabasesOverlay.h b/src/Databases/DatabasesOverlay.h index 0f31bbd6a47..b58df506f70 100644 --- a/src/Databases/DatabasesOverlay.h +++ b/src/Databases/DatabasesOverlay.h @@ -60,7 +60,7 @@ public: protected: std::vector databases; - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 5834fb96dc6..2656835f912 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -116,7 +116,7 @@ static BlockIO tryToExecuteQuery(const String & query_to_execute, ContextMutable catch (...) { tryLogCurrentException( - &Poco::Logger::get("MaterializedMySQLSyncThread(" + database + ")"), + getLogger("MaterializedMySQLSyncThread(" + database + ")"), "Query " + query_to_execute + " wasn't finished successfully"); throw; } @@ -255,7 +255,7 @@ MaterializedMySQLSyncThread::MaterializedMySQLSyncThread( const MySQLReplication::BinlogClientPtr & binlog_client_, MaterializedMySQLSettings * settings_) : WithContext(context_->getGlobalContext()) - , log(&Poco::Logger::get("MaterializedMySQLSyncThread")) + , log(getLogger("MaterializedMySQLSyncThread")) , database_name(database_name_) , mysql_database_name(mysql_database_name_) , pool(std::move(pool_)) /// NOLINT @@ -504,7 +504,7 @@ static inline void dumpDataForTables( StreamSettings mysql_input_stream_settings(context->getSettingsRef()); String mysql_select_all_query = "SELECT " + rewriteMysqlQueryColumn(connection, mysql_database_name, table_name, context->getSettingsRef()) + " FROM " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(table_name); - LOG_INFO(&Poco::Logger::get("MaterializedMySQLSyncThread(" + database_name + ")"), "mysql_select_all_query is {}", mysql_select_all_query); + LOG_INFO(getLogger("MaterializedMySQLSyncThread(" + database_name + ")"), "mysql_select_all_query is {}", mysql_select_all_query); auto input = std::make_unique(connection, mysql_select_all_query, pipeline.getHeader(), mysql_input_stream_settings); auto counting = std::make_shared(pipeline.getHeader()); Pipe pipe(std::move(input)); @@ -516,7 +516,7 @@ static inline void dumpDataForTables( executor.execute(); const Progress & progress = counting->getProgress(); - LOG_INFO(&Poco::Logger::get("MaterializedMySQLSyncThread(" + database_name + ")"), + LOG_INFO(getLogger("MaterializedMySQLSyncThread(" + database_name + ")"), "Materialize MySQL step 1: dump {}, {} rows, {} in {} sec., {} rows/sec., {}/sec." , table_name, formatReadableQuantity(progress.written_rows), formatReadableSizeWithBinarySuffix(progress.written_bytes) , watch.elapsedSeconds(), formatReadableQuantity(static_cast(progress.written_rows / watch.elapsedSeconds())) diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.h b/src/Databases/MySQL/MaterializedMySQLSyncThread.h index 004a4d67d32..03e558bfd68 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.h +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.h @@ -56,7 +56,7 @@ public: void assertMySQLAvailable(); private: - Poco::Logger * log; + LoggerPtr log; String database_name; String mysql_database_name; diff --git a/src/Databases/MySQL/MySQLBinlogClient.cpp b/src/Databases/MySQL/MySQLBinlogClient.cpp index e7d707f76ce..94e01673e88 100644 --- a/src/Databases/MySQL/MySQLBinlogClient.cpp +++ b/src/Databases/MySQL/MySQLBinlogClient.cpp @@ -17,7 +17,7 @@ BinlogClient::BinlogClient(const BinlogFactoryPtr & factory_, , binlog_client_name(name) , max_bytes_in_buffer(max_bytes_in_buffer_) , max_flush_ms(max_flush_ms_) - , logger(&Poco::Logger::get("BinlogClient(" + name + ")")) + , logger(getLogger("BinlogClient(" + name + ")")) { } diff --git a/src/Databases/MySQL/MySQLBinlogClient.h b/src/Databases/MySQL/MySQLBinlogClient.h index b76934d08cf..a45b885d87b 100644 --- a/src/Databases/MySQL/MySQLBinlogClient.h +++ b/src/Databases/MySQL/MySQLBinlogClient.h @@ -48,7 +48,7 @@ private: std::vector dispatchers; String binlog_checksum; mutable std::mutex mutex; - Poco::Logger * logger = nullptr; + LoggerPtr logger = nullptr; int dispatchers_count = 0; }; diff --git a/src/Databases/MySQL/MySQLBinlogEventsDispatcher.cpp b/src/Databases/MySQL/MySQLBinlogEventsDispatcher.cpp index 4af307f9c0f..d027d4b2192 100644 --- a/src/Databases/MySQL/MySQLBinlogEventsDispatcher.cpp +++ b/src/Databases/MySQL/MySQLBinlogEventsDispatcher.cpp @@ -19,7 +19,7 @@ public: , mysql_database_names(mysql_database_names_) , max_bytes(max_bytes_) , max_waiting_ms(max_waiting_ms_) - , logger(&Poco::Logger::get("BinlogFromDispatcher(" + name + ")")) + , logger(getLogger("BinlogFromDispatcher(" + name + ")")) { } @@ -65,7 +65,7 @@ private: std::condition_variable cv; bool is_cancelled = false; - Poco::Logger * logger = nullptr; + LoggerPtr logger = nullptr; std::exception_ptr exception; }; @@ -84,7 +84,7 @@ BinlogEventsDispatcher::BinlogEventsDispatcher(const String & logger_name_, size : logger_name(logger_name_) , max_bytes_in_buffer(max_bytes_in_buffer_) , max_flush_ms(max_flush_ms_) - , logger(&Poco::Logger::get("BinlogEventsDispatcher(" + logger_name + ")")) + , logger(getLogger("BinlogEventsDispatcher(" + logger_name + ")")) , dispatching_thread(std::make_unique([this]() { dispatchEvents(); })) { } diff --git a/src/Databases/MySQL/MySQLBinlogEventsDispatcher.h b/src/Databases/MySQL/MySQLBinlogEventsDispatcher.h index 43379697015..324deba3617 100644 --- a/src/Databases/MySQL/MySQLBinlogEventsDispatcher.h +++ b/src/Databases/MySQL/MySQLBinlogEventsDispatcher.h @@ -110,7 +110,7 @@ private: const String logger_name; const size_t max_bytes_in_buffer = 0; const UInt64 max_flush_ms = 0; - Poco::Logger * logger = nullptr; + LoggerPtr logger = nullptr; BinlogPtr binlog_read_from; diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 1fe5c078581..b07b203f786 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -57,7 +57,7 @@ DatabasePostgreSQL::DatabasePostgreSQL( , configuration(configuration_) , pool(std::move(pool_)) , cache_tables(cache_tables_) - , log(&Poco::Logger::get("DatabasePostgreSQL(" + dbname_ + ")")) + , log(getLogger("DatabasePostgreSQL(" + dbname_ + ")")) { fs::create_directories(metadata_path); cleaner_task = getContext()->getSchedulePool().createTask("PostgreSQLCleanerTask", [this]{ removeOutdatedTables(); }); @@ -531,7 +531,7 @@ void registerDatabasePostgreSQL(DatabaseFactory & factory) else { use_table_cache = safeGetLiteralValue(engine_args[4], engine_name); - LOG_WARNING(&Poco::Logger::get("DatabaseFactory"), "A deprecated syntax of PostgreSQL database engine is used"); + LOG_WARNING(getLogger("DatabaseFactory"), "A deprecated syntax of PostgreSQL database engine is used"); is_deprecated_syntax = true; } } diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.h b/src/Databases/PostgreSQL/DatabasePostgreSQL.h index d731e06649b..3ba7333c98e 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.h @@ -73,7 +73,7 @@ private: mutable Tables cached_tables; std::unordered_set detached_or_dropped; BackgroundSchedulePool::TaskHolder cleaner_task; - Poco::Logger * log; + LoggerPtr log; String getTableNameForLogs(const String & table_name) const; diff --git a/src/Databases/SQLite/DatabaseSQLite.cpp b/src/Databases/SQLite/DatabaseSQLite.cpp index 605a354bd7e..b3d5288cdf7 100644 --- a/src/Databases/SQLite/DatabaseSQLite.cpp +++ b/src/Databases/SQLite/DatabaseSQLite.cpp @@ -33,7 +33,7 @@ DatabaseSQLite::DatabaseSQLite( , WithContext(context_->getGlobalContext()) , database_engine_define(database_engine_define_->clone()) , database_path(database_path_) - , log(&Poco::Logger::get("DatabaseSQLite")) + , log(getLogger("DatabaseSQLite")) { sqlite_db = openSQLiteDB(database_path_, context_, !is_attach_); } diff --git a/src/Databases/SQLite/DatabaseSQLite.h b/src/Databases/SQLite/DatabaseSQLite.h index a89fbc32c3d..e5e93bbc8ce 100644 --- a/src/Databases/SQLite/DatabaseSQLite.h +++ b/src/Databases/SQLite/DatabaseSQLite.h @@ -50,7 +50,7 @@ private: mutable SQLitePtr sqlite_db; - Poco::Logger * log; + LoggerPtr log; bool checkSQLiteTable(const String & table_name) const; diff --git a/src/Databases/SQLite/SQLiteUtils.cpp b/src/Databases/SQLite/SQLiteUtils.cpp index 19b8662707b..eeea04476d3 100644 --- a/src/Databases/SQLite/SQLiteUtils.cpp +++ b/src/Databases/SQLite/SQLiteUtils.cpp @@ -21,7 +21,7 @@ void processSQLiteError(const String & message, bool throw_on_error) if (throw_on_error) throw Exception::createDeprecated(message, ErrorCodes::PATH_ACCESS_DENIED); else - LOG_ERROR(&Poco::Logger::get("SQLiteEngine"), fmt::runtime(message)); + LOG_ERROR(getLogger("SQLiteEngine"), fmt::runtime(message)); } String validateSQLiteDatabasePath(const String & path, const String & user_files_path, bool need_check, bool throw_on_error) @@ -54,7 +54,7 @@ SQLitePtr openSQLiteDB(const String & path, ContextPtr context, bool throw_on_er return nullptr; if (!fs::exists(database_path)) - LOG_DEBUG(&Poco::Logger::get("SQLite"), "SQLite database path {} does not exist, will create an empty SQLite database", database_path); + LOG_DEBUG(getLogger("SQLite"), "SQLite database path {} does not exist, will create an empty SQLite database", database_path); sqlite3 * tmp_sqlite_db = nullptr; int status; diff --git a/src/Databases/TablesDependencyGraph.cpp b/src/Databases/TablesDependencyGraph.cpp index 16404c6870f..6b9e202d900 100644 --- a/src/Databases/TablesDependencyGraph.cpp +++ b/src/Databases/TablesDependencyGraph.cpp @@ -720,10 +720,10 @@ void TablesDependencyGraph::log() const } -Poco::Logger * TablesDependencyGraph::getLogger() const +LoggerPtr TablesDependencyGraph::getLogger() const { if (!logger) - logger = &Poco::Logger::get(name_for_logging); + logger = ::getLogger(name_for_logging); return logger; } diff --git a/src/Databases/TablesDependencyGraph.h b/src/Databases/TablesDependencyGraph.h index 50be3bbf969..f0553cef321 100644 --- a/src/Databases/TablesDependencyGraph.h +++ b/src/Databases/TablesDependencyGraph.h @@ -163,7 +163,7 @@ private: mutable bool levels_calculated = false; const String name_for_logging; - mutable Poco::Logger * logger = nullptr; + mutable LoggerPtr logger = nullptr; Node * findNode(const StorageID & table_id) const; Node * addOrUpdateNode(const StorageID & table_id); @@ -175,7 +175,7 @@ private: void setNeedRecalculateLevels() const; const NodesSortedByLevel & getNodesSortedByLevel() const; - Poco::Logger * getLogger() const; + LoggerPtr getLogger() const; }; } diff --git a/src/Databases/TablesLoader.cpp b/src/Databases/TablesLoader.cpp index f1b5c4377fe..48745ff91c2 100644 --- a/src/Databases/TablesLoader.cpp +++ b/src/Databases/TablesLoader.cpp @@ -29,7 +29,7 @@ TablesLoader::TablesLoader(ContextMutablePtr global_context_, Databases database , async_loader(global_context->getAsyncLoader()) { metadata.default_database = global_context->getCurrentDatabase(); - log = &Poco::Logger::get("TablesLoader"); + log = getLogger("TablesLoader"); } LoadTaskPtrs TablesLoader::loadTablesAsync(LoadJobSet load_after) diff --git a/src/Databases/TablesLoader.h b/src/Databases/TablesLoader.h index 038aa35895f..26b5777f1a9 100644 --- a/src/Databases/TablesLoader.h +++ b/src/Databases/TablesLoader.h @@ -73,7 +73,7 @@ private: TablesDependencyGraph referential_dependencies; TablesDependencyGraph loading_dependencies; TablesDependencyGraph all_loading_dependencies; - Poco::Logger * log; + LoggerPtr log; std::atomic tables_processed{0}; AtomicStopwatch stopwatch; diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index b40a60e0915..000f0ef5b4c 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -63,7 +63,7 @@ CacheDictionary::CacheDictionary( update(unit_to_update); }) , configuration(configuration_) - , log(&Poco::Logger::get("ExternalDictionaries")) + , log(getLogger("ExternalDictionaries")) , rnd_engine(randomSeed()) { if (!source_ptr->supportsSelectiveLoad()) diff --git a/src/Dictionaries/CacheDictionary.h b/src/Dictionaries/CacheDictionary.h index 66efb4a85a5..aae86a83f12 100644 --- a/src/Dictionaries/CacheDictionary.h +++ b/src/Dictionaries/CacheDictionary.h @@ -202,7 +202,7 @@ private: const CacheDictionaryConfiguration configuration; - Poco::Logger * log; + LoggerPtr log; mutable pcg64 rnd_engine; diff --git a/src/Dictionaries/CassandraDictionarySource.cpp b/src/Dictionaries/CassandraDictionarySource.cpp index e0cf2483b3d..b3bf288ef5a 100644 --- a/src/Dictionaries/CassandraDictionarySource.cpp +++ b/src/Dictionaries/CassandraDictionarySource.cpp @@ -105,7 +105,7 @@ CassandraDictionarySource::CassandraDictionarySource( const DictionaryStructure & dict_struct_, const Configuration & configuration_, const Block & sample_block_) - : log(&Poco::Logger::get("CassandraDictionarySource")) + : log(getLogger("CassandraDictionarySource")) , dict_struct(dict_struct_) , configuration(configuration_) , sample_block(sample_block_) diff --git a/src/Dictionaries/CassandraDictionarySource.h b/src/Dictionaries/CassandraDictionarySource.h index 2591b33c638..3700642fc5b 100644 --- a/src/Dictionaries/CassandraDictionarySource.h +++ b/src/Dictionaries/CassandraDictionarySource.h @@ -77,7 +77,7 @@ private: void maybeAllowFiltering(String & query) const; CassSessionShared getSession(); - Poco::Logger * log; + LoggerPtr log; const DictionaryStructure dict_struct; const Configuration configuration; Block sample_block; diff --git a/src/Dictionaries/CassandraHelpers.cpp b/src/Dictionaries/CassandraHelpers.cpp index e93b3fe8d49..4c569d00957 100644 --- a/src/Dictionaries/CassandraHelpers.cpp +++ b/src/Dictionaries/CassandraHelpers.cpp @@ -47,7 +47,7 @@ void setupCassandraDriverLibraryLogging(CassLogLevel level) { std::call_once(setup_logging_flag, [level]() { - Poco::Logger * logger = &Poco::Logger::get("CassandraDriverLibrary"); + Poco::Logger * logger = getRawLogger("CassandraDriverLibrary"); cass_log_set_level(level); if (level != CASS_LOG_DISABLED) cass_log_set_callback(cassandraLogCallback, logger); diff --git a/src/Dictionaries/ClickHouseDictionarySource.h b/src/Dictionaries/ClickHouseDictionarySource.h index cfb6a0bcd37..3357514eab2 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.h +++ b/src/Dictionaries/ClickHouseDictionarySource.h @@ -85,7 +85,7 @@ private: ContextMutablePtr context; ConnectionPoolWithFailoverPtr pool; std::string load_all_query; - Poco::Logger * log = &Poco::Logger::get("ClickHouseDictionarySource"); + LoggerPtr log = getLogger("ClickHouseDictionarySource"); /// RegExpTreeDictionary is the only dictionary whose structure of attributions differ from the input block. /// For now we need to modify sample_block in the ctor of RegExpTreeDictionary. diff --git a/src/Dictionaries/DictionaryFactory.cpp b/src/Dictionaries/DictionaryFactory.cpp index f6102d7c657..a566fb27de4 100644 --- a/src/Dictionaries/DictionaryFactory.cpp +++ b/src/Dictionaries/DictionaryFactory.cpp @@ -46,7 +46,7 @@ DictionaryPtr DictionaryFactory::create( DictionarySourcePtr source_ptr = DictionarySourceFactory::instance().create( name, config, config_prefix + ".source", dict_struct, global_context, config.getString(config_prefix + ".database", ""), created_from_ddl); - LOG_TRACE(&Poco::Logger::get("DictionaryFactory"), "Created dictionary source '{}' for dictionary '{}'", source_ptr->toString(), name); + LOG_TRACE(getLogger("DictionaryFactory"), "Created dictionary source '{}' for dictionary '{}'", source_ptr->toString(), name); const auto & layout_type = keys.front(); diff --git a/src/Dictionaries/DictionarySourceFactory.cpp b/src/Dictionaries/DictionarySourceFactory.cpp index 5ae4bb5a439..eedf6967c13 100644 --- a/src/Dictionaries/DictionarySourceFactory.cpp +++ b/src/Dictionaries/DictionarySourceFactory.cpp @@ -65,7 +65,7 @@ namespace } -DictionarySourceFactory::DictionarySourceFactory() : log(&Poco::Logger::get("DictionarySourceFactory")) +DictionarySourceFactory::DictionarySourceFactory() : log(getLogger("DictionarySourceFactory")) { } diff --git a/src/Dictionaries/DictionarySourceFactory.h b/src/Dictionaries/DictionarySourceFactory.h index 4c867db4ea1..a9007230047 100644 --- a/src/Dictionaries/DictionarySourceFactory.h +++ b/src/Dictionaries/DictionarySourceFactory.h @@ -59,7 +59,7 @@ private: using SourceRegistry = std::unordered_map; SourceRegistry registered_sources; - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/src/Dictionaries/DirectDictionary.cpp b/src/Dictionaries/DirectDictionary.cpp index 64c7eb14024..5bfcb22c23b 100644 --- a/src/Dictionaries/DirectDictionary.cpp +++ b/src/Dictionaries/DirectDictionary.cpp @@ -118,7 +118,7 @@ Columns DirectDictionary::getColumns( block_key_columns.clear(); } - LOG_DEBUG(&Poco::Logger::get("DirectDictionary"), "read {} blocks with {} rows from pipeline in {} ms", + LOG_DEBUG(getLogger("DirectDictionary"), "read {} blocks with {} rows from pipeline in {} ms", block_num, rows_num, watch.elapsedMilliseconds()); Field value_to_insert; @@ -353,7 +353,7 @@ Pipe DirectDictionary::getSourcePipe( pipe = Pipe(std::make_shared>(std::move(pipeline))); } - LOG_DEBUG(&Poco::Logger::get("DirectDictionary"), "building pipeline for loading keys done in {} ms", watch.elapsedMilliseconds()); + LOG_DEBUG(getLogger("DirectDictionary"), "building pipeline for loading keys done in {} ms", watch.elapsedMilliseconds()); return pipe; } diff --git a/src/Dictionaries/Embedded/RegionsHierarchies.cpp b/src/Dictionaries/Embedded/RegionsHierarchies.cpp index c3c62bcc83c..3f1222fff3f 100644 --- a/src/Dictionaries/Embedded/RegionsHierarchies.cpp +++ b/src/Dictionaries/Embedded/RegionsHierarchies.cpp @@ -8,7 +8,7 @@ namespace DB RegionsHierarchies::RegionsHierarchies(IRegionsHierarchiesDataProviderPtr data_provider) { - Poco::Logger * log = &Poco::Logger::get("RegionsHierarchies"); + LoggerPtr log = getLogger("RegionsHierarchies"); LOG_DEBUG(log, "Adding default regions hierarchy"); data.emplace("", data_provider->getDefaultHierarchySource()); diff --git a/src/Dictionaries/Embedded/RegionsHierarchy.cpp b/src/Dictionaries/Embedded/RegionsHierarchy.cpp index 23f4c250a23..a59f6fcd0e7 100644 --- a/src/Dictionaries/Embedded/RegionsHierarchy.cpp +++ b/src/Dictionaries/Embedded/RegionsHierarchy.cpp @@ -23,7 +23,7 @@ RegionsHierarchy::RegionsHierarchy(IRegionsHierarchyDataSourcePtr data_source_) void RegionsHierarchy::reload() { - Poco::Logger * log = &Poco::Logger::get("RegionsHierarchy"); + LoggerPtr log = getLogger("RegionsHierarchy"); if (!data_source->isModified()) return; diff --git a/src/Dictionaries/Embedded/RegionsNames.cpp b/src/Dictionaries/Embedded/RegionsNames.cpp index 847dfe99b10..c89bacc7ec9 100644 --- a/src/Dictionaries/Embedded/RegionsNames.cpp +++ b/src/Dictionaries/Embedded/RegionsNames.cpp @@ -42,7 +42,7 @@ std::string RegionsNames::dumpSupportedLanguagesNames() void RegionsNames::reload() { - Poco::Logger * log = &Poco::Logger::get("RegionsNames"); + LoggerPtr log = getLogger("RegionsNames"); LOG_DEBUG(log, "Reloading regions names"); RegionID max_region_id = 0; diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index f1acd610274..6b9f97a6d5c 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -71,7 +71,7 @@ ExecutableDictionarySource::ExecutableDictionarySource( Block & sample_block_, std::shared_ptr coordinator_, ContextPtr context_) - : log(&Poco::Logger::get("ExecutableDictionarySource")) + : log(getLogger("ExecutableDictionarySource")) , dict_struct(dict_struct_) , configuration(configuration_) , sample_block(sample_block_) @@ -93,7 +93,7 @@ ExecutableDictionarySource::ExecutableDictionarySource( } ExecutableDictionarySource::ExecutableDictionarySource(const ExecutableDictionarySource & other) - : log(&Poco::Logger::get("ExecutableDictionarySource")) + : log(getLogger("ExecutableDictionarySource")) , update_time(other.update_time) , dict_struct(other.dict_struct) , configuration(other.configuration) diff --git a/src/Dictionaries/ExecutableDictionarySource.h b/src/Dictionaries/ExecutableDictionarySource.h index c7067a62893..eb936434218 100644 --- a/src/Dictionaries/ExecutableDictionarySource.h +++ b/src/Dictionaries/ExecutableDictionarySource.h @@ -63,7 +63,7 @@ public: QueryPipeline getStreamForBlock(const Block & block); private: - Poco::Logger * log; + LoggerPtr log; time_t update_time = 0; const DictionaryStructure dict_struct; const Configuration configuration; diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index d28c73c9c52..d8111afdc19 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -40,7 +40,7 @@ ExecutablePoolDictionarySource::ExecutablePoolDictionarySource( , sample_block(sample_block_) , coordinator(std::move(coordinator_)) , context(context_) - , log(&Poco::Logger::get("ExecutablePoolDictionarySource")) + , log(getLogger("ExecutablePoolDictionarySource")) { /// Remove keys from sample_block for implicit_key dictionary because /// these columns will not be returned from source @@ -64,7 +64,7 @@ ExecutablePoolDictionarySource::ExecutablePoolDictionarySource(const ExecutableP , sample_block(other.sample_block) , coordinator(other.coordinator) , context(Context::createCopy(other.context)) - , log(&Poco::Logger::get("ExecutablePoolDictionarySource")) + , log(getLogger("ExecutablePoolDictionarySource")) { } diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.h b/src/Dictionaries/ExecutablePoolDictionarySource.h index e8cc6e83406..752d8ea2757 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.h +++ b/src/Dictionaries/ExecutablePoolDictionarySource.h @@ -72,7 +72,7 @@ private: Block sample_block; std::shared_ptr coordinator; ContextPtr context; - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/src/Dictionaries/FileDictionarySource.cpp b/src/Dictionaries/FileDictionarySource.cpp index 86287971428..16a4ecaee75 100644 --- a/src/Dictionaries/FileDictionarySource.cpp +++ b/src/Dictionaries/FileDictionarySource.cpp @@ -48,7 +48,7 @@ FileDictionarySource::FileDictionarySource(const FileDictionarySource & other) QueryPipeline FileDictionarySource::loadAll() { - LOG_TRACE(&Poco::Logger::get("FileDictionary"), "loadAll {}", toString()); + LOG_TRACE(getLogger("FileDictionary"), "loadAll {}", toString()); auto in_ptr = std::make_unique(filepath); auto source = context->getInputFormat(format, *in_ptr, sample_block, max_block_size); source->addBuffer(std::move(in_ptr)); diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index 689593a969e..bf42b7931ed 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -32,7 +32,7 @@ HTTPDictionarySource::HTTPDictionarySource( const Poco::Net::HTTPBasicCredentials & credentials_, Block & sample_block_, ContextPtr context_) - : log(&Poco::Logger::get("HTTPDictionarySource")) + : log(getLogger("HTTPDictionarySource")) , update_time(std::chrono::system_clock::from_time_t(0)) , dict_struct(dict_struct_) , configuration(configuration_) @@ -45,7 +45,7 @@ HTTPDictionarySource::HTTPDictionarySource( } HTTPDictionarySource::HTTPDictionarySource(const HTTPDictionarySource & other) - : log(&Poco::Logger::get("HTTPDictionarySource")) + : log(getLogger("HTTPDictionarySource")) , update_time(other.update_time) , dict_struct(other.dict_struct) , configuration(other.configuration) diff --git a/src/Dictionaries/HTTPDictionarySource.h b/src/Dictionaries/HTTPDictionarySource.h index e22aacd89f1..414372fe7ac 100644 --- a/src/Dictionaries/HTTPDictionarySource.h +++ b/src/Dictionaries/HTTPDictionarySource.h @@ -66,7 +66,7 @@ private: // wrap buffer using encoding from made request QueryPipeline createWrappedBuffer(std::unique_ptr http_buffer); - Poco::Logger * log; + LoggerPtr log; LocalDateTime getLastModification() const; diff --git a/src/Dictionaries/HashedArrayDictionary.cpp b/src/Dictionaries/HashedArrayDictionary.cpp index 4c9ff8abe80..648ecb681fc 100644 --- a/src/Dictionaries/HashedArrayDictionary.cpp +++ b/src/Dictionaries/HashedArrayDictionary.cpp @@ -32,7 +32,7 @@ HashedArrayDictionary::HashedArrayDictionary( const HashedArrayDictionaryStorageConfiguration & configuration_, BlockPtr update_field_loaded_block_) : IDictionary(dict_id_) - , log(&Poco::Logger::get("HashedArrayDictionary")) + , log(getLogger("HashedArrayDictionary")) , dict_struct(dict_struct_) , source_ptr(std::move(source_ptr_)) , configuration(configuration_) @@ -822,7 +822,7 @@ void HashedArrayDictionary::loadData() if (parallel_loader) parallel_loader->finish(); - LOG_DEBUG(&Poco::Logger::get("HashedArrayDictionary"), + LOG_DEBUG(getLogger("HashedArrayDictionary"), "Finished {}reading {} blocks with {} rows from pipeline in {:.2f} sec and inserted into hashtable in {:.2f} sec", configuration.use_async_executor ? "asynchronous " : "", total_blocks, total_rows, pull_time_microseconds / 1000000.0, process_time_microseconds / 1000000.0); diff --git a/src/Dictionaries/HashedArrayDictionary.h b/src/Dictionaries/HashedArrayDictionary.h index 606008ce921..86b21443e18 100644 --- a/src/Dictionaries/HashedArrayDictionary.h +++ b/src/Dictionaries/HashedArrayDictionary.h @@ -244,7 +244,7 @@ private: void resize(size_t total_rows); - Poco::Logger * log; + LoggerPtr log; const DictionaryStructure dict_struct; const DictionarySourcePtr source_ptr; diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index 8009ffab80a..0b8419dd242 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -251,7 +251,7 @@ private: void resize(size_t added_rows); - Poco::Logger * log; + LoggerPtr log; const DictionaryStructure dict_struct; const DictionarySourcePtr source_ptr; @@ -293,7 +293,7 @@ HashedDictionary::HashedDictionary( const HashedDictionaryConfiguration & configuration_, BlockPtr update_field_loaded_block_) : IDictionary(dict_id_) - , log(&Poco::Logger::get("HashedDictionary")) + , log(getLogger("HashedDictionary")) , dict_struct(dict_struct_) , source_ptr(std::move(source_ptr_)) , configuration(configuration_) diff --git a/src/Dictionaries/IPAddressDictionary.cpp b/src/Dictionaries/IPAddressDictionary.cpp index 2e3c09c67c5..98ba95f0053 100644 --- a/src/Dictionaries/IPAddressDictionary.cpp +++ b/src/Dictionaries/IPAddressDictionary.cpp @@ -205,7 +205,7 @@ IPAddressDictionary::IPAddressDictionary( , source_ptr{std::move(source_ptr_)} , configuration(configuration_) , access_to_key_from_attributes(dict_struct_.access_to_key_from_attributes) - , logger(&Poco::Logger::get("IPAddressDictionary")) + , logger(getLogger("IPAddressDictionary")) { createAttributes(); loadData(); diff --git a/src/Dictionaries/IPAddressDictionary.h b/src/Dictionaries/IPAddressDictionary.h index c5b9287c2b5..d758e23043d 100644 --- a/src/Dictionaries/IPAddressDictionary.h +++ b/src/Dictionaries/IPAddressDictionary.h @@ -234,7 +234,7 @@ private: mutable std::atomic query_count{0}; mutable std::atomic found_count{0}; - Poco::Logger * logger; + LoggerPtr logger; }; } diff --git a/src/Dictionaries/LibraryDictionarySource.cpp b/src/Dictionaries/LibraryDictionarySource.cpp index 7eb4d803fe8..f6f104ca11d 100644 --- a/src/Dictionaries/LibraryDictionarySource.cpp +++ b/src/Dictionaries/LibraryDictionarySource.cpp @@ -30,7 +30,7 @@ LibraryDictionarySource::LibraryDictionarySource( Block & sample_block_, ContextPtr context_, bool created_from_ddl) - : log(&Poco::Logger::get("LibraryDictionarySource")) + : log(getLogger("LibraryDictionarySource")) , dict_struct{dict_struct_} , config_prefix{config_prefix_} , path{config.getString(config_prefix + ".path", "")} @@ -78,7 +78,7 @@ LibraryDictionarySource::~LibraryDictionarySource() LibraryDictionarySource::LibraryDictionarySource(const LibraryDictionarySource & other) - : log(&Poco::Logger::get("LibraryDictionarySource")) + : log(getLogger("LibraryDictionarySource")) , dict_struct{other.dict_struct} , config_prefix{other.config_prefix} , path{other.path} diff --git a/src/Dictionaries/LibraryDictionarySource.h b/src/Dictionaries/LibraryDictionarySource.h index 57ab9976a3b..04a3d838577 100644 --- a/src/Dictionaries/LibraryDictionarySource.h +++ b/src/Dictionaries/LibraryDictionarySource.h @@ -75,7 +75,7 @@ private: static Field getDictID() { return UUIDHelpers::generateV4(); } - Poco::Logger * log; + LoggerPtr log; const DictionaryStructure dict_struct; const std::string config_prefix; diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index e61409e2b54..9a84512fde6 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -173,7 +173,7 @@ MySQLDictionarySource::MySQLDictionarySource( mysqlxx::PoolWithFailoverPtr pool_, const Block & sample_block_, const StreamSettings & settings_) - : log(&Poco::Logger::get("MySQLDictionarySource")) + : log(getLogger("MySQLDictionarySource")) , update_time(std::chrono::system_clock::from_time_t(0)) , dict_struct(dict_struct_) , configuration(configuration_) @@ -187,7 +187,7 @@ MySQLDictionarySource::MySQLDictionarySource( /// copy-constructor is provided in order to support cloneability MySQLDictionarySource::MySQLDictionarySource(const MySQLDictionarySource & other) - : log(&Poco::Logger::get("MySQLDictionarySource")) + : log(getLogger("MySQLDictionarySource")) , update_time(other.update_time) , dict_struct(other.dict_struct) , configuration(other.configuration) diff --git a/src/Dictionaries/MySQLDictionarySource.h b/src/Dictionaries/MySQLDictionarySource.h index 1d43ebfe2ba..d9eea3f3e26 100644 --- a/src/Dictionaries/MySQLDictionarySource.h +++ b/src/Dictionaries/MySQLDictionarySource.h @@ -82,7 +82,7 @@ private: // execute invalidate_query. expects single cell in result std::string doInvalidateQuery(const std::string & request) const; - Poco::Logger * log; + LoggerPtr log; std::chrono::time_point update_time; const DictionaryStructure dict_struct; diff --git a/src/Dictionaries/NullDictionarySource.cpp b/src/Dictionaries/NullDictionarySource.cpp index 45dcc77f93d..2d5656e1335 100644 --- a/src/Dictionaries/NullDictionarySource.cpp +++ b/src/Dictionaries/NullDictionarySource.cpp @@ -20,7 +20,7 @@ NullDictionarySource::NullDictionarySource(const NullDictionarySource & other) : QueryPipeline NullDictionarySource::loadAll() { - LOG_TRACE(&Poco::Logger::get("NullDictionarySource"), "loadAll {}", toString()); + LOG_TRACE(getLogger("NullDictionarySource"), "loadAll {}", toString()); return QueryPipeline(std::make_shared(sample_block)); } diff --git a/src/Dictionaries/PolygonDictionaryUtils.cpp b/src/Dictionaries/PolygonDictionaryUtils.cpp index 2af97d3fc6f..8f060fe7b8d 100644 --- a/src/Dictionaries/PolygonDictionaryUtils.cpp +++ b/src/Dictionaries/PolygonDictionaryUtils.cpp @@ -69,7 +69,7 @@ const FinalCellWithSlabs * FinalCellWithSlabs::find(Coord, Coord) const SlabsPolygonIndex::SlabsPolygonIndex( const std::vector & polygons) - : log(&Poco::Logger::get("SlabsPolygonIndex")), + : log(getLogger("SlabsPolygonIndex")), sorted_x(uniqueX(polygons)) { indexBuild(polygons); diff --git a/src/Dictionaries/PolygonDictionaryUtils.h b/src/Dictionaries/PolygonDictionaryUtils.h index 5268cb93f78..0acf0d23e5e 100644 --- a/src/Dictionaries/PolygonDictionaryUtils.h +++ b/src/Dictionaries/PolygonDictionaryUtils.h @@ -83,7 +83,7 @@ private: /** Auxiliary function for adding ring to the index */ void indexAddRing(const Ring & ring, size_t polygon_id); - Poco::Logger * log; + LoggerPtr log; /** Sorted distinct coordinates of all vertices */ std::vector sorted_x; diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index 8ec78308392..c7401386e40 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -57,7 +57,7 @@ PostgreSQLDictionarySource::PostgreSQLDictionarySource( , configuration(configuration_) , pool(std::move(pool_)) , sample_block(sample_block_) - , log(&Poco::Logger::get("PostgreSQLDictionarySource")) + , log(getLogger("PostgreSQLDictionarySource")) , query_builder(makeExternalQueryBuilder(dict_struct, configuration.schema, configuration.table, configuration.query, configuration.where)) , load_all_query(query_builder.composeLoadAllQuery()) { @@ -70,7 +70,7 @@ PostgreSQLDictionarySource::PostgreSQLDictionarySource(const PostgreSQLDictionar , configuration(other.configuration) , pool(other.pool) , sample_block(other.sample_block) - , log(&Poco::Logger::get("PostgreSQLDictionarySource")) + , log(getLogger("PostgreSQLDictionarySource")) , query_builder(makeExternalQueryBuilder(dict_struct, configuration.schema, configuration.table, configuration.query, configuration.where)) , load_all_query(query_builder.composeLoadAllQuery()) , update_time(other.update_time) diff --git a/src/Dictionaries/PostgreSQLDictionarySource.h b/src/Dictionaries/PostgreSQLDictionarySource.h index 1305333458b..3070184ab3d 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.h +++ b/src/Dictionaries/PostgreSQLDictionarySource.h @@ -61,7 +61,7 @@ private: const Configuration configuration; postgres::PoolWithFailoverPtr pool; Block sample_block; - Poco::Logger * log; + LoggerPtr log; ExternalQueryBuilder query_builder; const std::string load_all_query; std::chrono::time_point update_time; diff --git a/src/Dictionaries/RegExpTreeDictionary.cpp b/src/Dictionaries/RegExpTreeDictionary.cpp index bbd101d55aa..a9532737276 100644 --- a/src/Dictionaries/RegExpTreeDictionary.cpp +++ b/src/Dictionaries/RegExpTreeDictionary.cpp @@ -139,7 +139,7 @@ struct RegExpTreeDictionary::RegexTreeNode std::unordered_map attributes; }; -std::vector createStringPieces(const String & value, int num_captures, const String & regex, Poco::Logger * logger) +std::vector createStringPieces(const String & value, int num_captures, const String & regex, LoggerPtr logger) { std::vector result; String literal; @@ -401,7 +401,7 @@ RegExpTreeDictionary::RegExpTreeDictionary( use_vectorscan(use_vectorscan_), flag_case_insensitive(flag_case_insensitive_), flag_dotall(flag_dotall_), - logger(&Poco::Logger::get("RegExpTreeDictionary")) + logger(getLogger("RegExpTreeDictionary")) { if (auto * ch_source = typeid_cast(source_ptr.get())) { diff --git a/src/Dictionaries/RegExpTreeDictionary.h b/src/Dictionaries/RegExpTreeDictionary.h index 6597584ed45..78b7f441d34 100644 --- a/src/Dictionaries/RegExpTreeDictionary.h +++ b/src/Dictionaries/RegExpTreeDictionary.h @@ -208,7 +208,7 @@ private: MultiRegexps::DataBasePtr origin_db; #endif - Poco::Logger * logger; + LoggerPtr logger; }; } diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index 080f7db96be..70fe889a8ea 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -67,7 +67,7 @@ XDBCDictionarySource::XDBCDictionarySource( ContextPtr context_, const BridgeHelperPtr bridge_) : WithContext(context_->getGlobalContext()) - , log(&Poco::Logger::get(bridge_->getName() + "DictionarySource")) + , log(getLogger(bridge_->getName() + "DictionarySource")) , update_time(std::chrono::system_clock::from_time_t(0)) , dict_struct(dict_struct_) , configuration(configuration_) @@ -86,7 +86,7 @@ XDBCDictionarySource::XDBCDictionarySource( /// copy-constructor is provided in order to support cloneability XDBCDictionarySource::XDBCDictionarySource(const XDBCDictionarySource & other) : WithContext(other.getContext()) - , log(&Poco::Logger::get(other.bridge_helper->getName() + "DictionarySource")) + , log(getLogger(other.bridge_helper->getName() + "DictionarySource")) , update_time(other.update_time) , dict_struct(other.dict_struct) , configuration(other.configuration) diff --git a/src/Dictionaries/XDBCDictionarySource.h b/src/Dictionaries/XDBCDictionarySource.h index 8ca2e172aa6..6011563c522 100644 --- a/src/Dictionaries/XDBCDictionarySource.h +++ b/src/Dictionaries/XDBCDictionarySource.h @@ -76,7 +76,7 @@ private: QueryPipeline loadFromQuery(const Poco::URI & url, const Block & required_sample_block, const std::string & query) const; - Poco::Logger * log; + LoggerPtr log; std::chrono::time_point update_time; const DictionaryStructure dict_struct; diff --git a/src/Dictionaries/YAMLRegExpTreeDictionarySource.cpp b/src/Dictionaries/YAMLRegExpTreeDictionarySource.cpp index 118d0f6a0f3..f1591943a12 100644 --- a/src/Dictionaries/YAMLRegExpTreeDictionarySource.cpp +++ b/src/Dictionaries/YAMLRegExpTreeDictionarySource.cpp @@ -284,7 +284,7 @@ Block parseYAMLAsRegExpTree(const YAML::Node & node, const String & key_name, co YAMLRegExpTreeDictionarySource::YAMLRegExpTreeDictionarySource( const String & filepath_, const DictionaryStructure & dict_struct, ContextPtr context_, bool created_from_ddl) - : filepath(filepath_), structure(dict_struct), context(context_), logger(&Poco::Logger::get(kYAMLRegExpTreeDictionarySource)) + : filepath(filepath_), structure(dict_struct), context(context_), logger(getLogger(kYAMLRegExpTreeDictionarySource)) { key_name = (*structure.key)[0].name; diff --git a/src/Dictionaries/YAMLRegExpTreeDictionarySource.h b/src/Dictionaries/YAMLRegExpTreeDictionarySource.h index f5dd9b7d186..041cbca81c6 100644 --- a/src/Dictionaries/YAMLRegExpTreeDictionarySource.h +++ b/src/Dictionaries/YAMLRegExpTreeDictionarySource.h @@ -64,7 +64,7 @@ private: ContextPtr context; - Poco::Logger * logger; + LoggerPtr logger; Poco::Timestamp last_modification; Poco::Timestamp getLastModification() const; diff --git a/src/Dictionaries/registerHashedDictionary.cpp b/src/Dictionaries/registerHashedDictionary.cpp index f511cad04b0..303ca8c577e 100644 --- a/src/Dictionaries/registerHashedDictionary.cpp +++ b/src/Dictionaries/registerHashedDictionary.cpp @@ -50,7 +50,7 @@ void registerDictionaryHashed(DictionaryFactory & factory) const std::string dictionary_layout_prefix = ".layout." + dictionary_layout_name; const bool preallocate = config.getBool(config_prefix + dictionary_layout_prefix + ".preallocate", false); if (preallocate) - LOG_WARNING(&Poco::Logger::get("HashedDictionary"), "'prellocate' attribute is obsolete, consider looking at 'shards'"); + LOG_WARNING(getLogger("HashedDictionary"), "'prellocate' attribute is obsolete, consider looking at 'shards'"); Int64 shards = config.getInt(config_prefix + dictionary_layout_prefix + ".shards", 1); if (shards <= 0 || shards > 128) diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 07c2beb002d..f5d67d37b07 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -106,7 +106,7 @@ public: if (disk->reserved_bytes < size) { disk->reserved_bytes = 0; - LOG_ERROR(&Poco::Logger::get("DiskLocal"), "Unbalanced reservations size for disk '{}'.", disk->getName()); + LOG_ERROR(getLogger("DiskLocal"), "Unbalanced reservations size for disk '{}'.", disk->getName()); } else { @@ -114,7 +114,7 @@ public: } if (disk->reservation_count == 0) - LOG_ERROR(&Poco::Logger::get("DiskLocal"), "Unbalanced reservation count for disk '{}'.", disk->getName()); + LOG_ERROR(getLogger("DiskLocal"), "Unbalanced reservation count for disk '{}'.", disk->getName()); else --disk->reservation_count; } @@ -475,7 +475,7 @@ DiskLocal::DiskLocal(const String & name_, const String & path_, UInt64 keep_fre : IDisk(name_, config, config_prefix) , disk_path(path_) , keep_free_space_bytes(keep_free_space_bytes_) - , logger(&Poco::Logger::get("DiskLocal")) + , logger(getLogger("DiskLocal")) , data_source_description(getLocalDataSourceDescription(disk_path)) { } @@ -494,7 +494,7 @@ DiskLocal::DiskLocal(const String & name_, const String & path_) : IDisk(name_) , disk_path(path_) , keep_free_space_bytes(0) - , logger(&Poco::Logger::get("DiskLocal")) + , logger(getLogger("DiskLocal")) , data_source_description(getLocalDataSourceDescription(disk_path)) { } diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index affce5a847e..b9703019c19 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -153,7 +153,7 @@ private: const String disk_path; const String disk_checker_path = ".disk_checker_file"; std::atomic keep_free_space_bytes; - Poco::Logger * logger; + LoggerPtr logger; DataSourceDescription data_source_description; UInt64 reserved_bytes = 0; diff --git a/src/Disks/DiskLocalCheckThread.cpp b/src/Disks/DiskLocalCheckThread.cpp index 87fcc0d1cf5..e95c614336b 100644 --- a/src/Disks/DiskLocalCheckThread.cpp +++ b/src/Disks/DiskLocalCheckThread.cpp @@ -13,7 +13,7 @@ DiskLocalCheckThread::DiskLocalCheckThread(DiskLocal * disk_, ContextPtr context : WithContext(context_) , disk(std::move(disk_)) , check_period_ms(local_disk_check_period_ms) - , log(&Poco::Logger::get(fmt::format("DiskLocalCheckThread({})", disk->getName()))) + , log(getLogger(fmt::format("DiskLocalCheckThread({})", disk->getName()))) { task = getContext()->getSchedulePool().createTask(log->name(), [this] { run(); }); } diff --git a/src/Disks/DiskLocalCheckThread.h b/src/Disks/DiskLocalCheckThread.h index eb688d599ca..046b7553136 100644 --- a/src/Disks/DiskLocalCheckThread.h +++ b/src/Disks/DiskLocalCheckThread.h @@ -29,7 +29,7 @@ private: DiskLocal * disk; size_t check_period_ms; - Poco::Logger * log; + LoggerPtr log; std::atomic need_stop{false}; BackgroundSchedulePool::TaskHolder task; diff --git a/src/Disks/DiskSelector.cpp b/src/Disks/DiskSelector.cpp index dad1c728560..a9260a249dd 100644 --- a/src/Disks/DiskSelector.cpp +++ b/src/Disks/DiskSelector.cpp @@ -124,7 +124,7 @@ DiskSelectorPtr DiskSelector::updateFromConfig( if (num_disks_removed_from_config > 0) { LOG_WARNING( - &Poco::Logger::get("DiskSelector"), + getLogger("DiskSelector"), "{} disappeared from configuration, this change will be applied after restart of ClickHouse", warning.str()); } diff --git a/src/Disks/IDisk.cpp b/src/Disks/IDisk.cpp index 5426f8d0904..066acc250a2 100644 --- a/src/Disks/IDisk.cpp +++ b/src/Disks/IDisk.cpp @@ -33,7 +33,7 @@ void IDisk::copyFile( /// NOLINT const std::function & cancellation_hook ) { - LOG_DEBUG(&Poco::Logger::get("IDisk"), "Copying from {} (path: {}) {} to {} (path: {}) {}.", + LOG_DEBUG(getLogger("IDisk"), "Copying from {} (path: {}) {} to {} (path: {}) {}.", getName(), getPath(), from_file_path, to_disk.getName(), to_disk.getPath(), to_file_path); auto in = readFile(from_file_path, read_settings); @@ -194,7 +194,7 @@ void IDisk::startup(ContextPtr context, bool skip_access_check) { if (isReadOnly()) { - LOG_DEBUG(&Poco::Logger::get("IDisk"), + LOG_DEBUG(getLogger("IDisk"), "Skip access check for disk {} (read-only disk).", getName()); } diff --git a/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp b/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp index 1952d8ae253..8eecd0d99d1 100644 --- a/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp +++ b/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp @@ -55,7 +55,7 @@ AsynchronousBoundedReadBuffer::AsynchronousBoundedReadBuffer( , prefetch_buffer(chooseBufferSizeForRemoteReading(read_settings, impl->getFileSize())) , query_id(CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() != nullptr ? CurrentThread::getQueryId() : "") , current_reader_id(getRandomASCIIString(8)) - , log(&Poco::Logger::get("AsynchronousBoundedReadBuffer")) + , log(getLogger("AsynchronousBoundedReadBuffer")) , async_read_counters(async_read_counters_) , prefetches_log(prefetches_log_) { diff --git a/src/Disks/IO/AsynchronousBoundedReadBuffer.h b/src/Disks/IO/AsynchronousBoundedReadBuffer.h index c43b08ce2b0..e5030f37b1d 100644 --- a/src/Disks/IO/AsynchronousBoundedReadBuffer.h +++ b/src/Disks/IO/AsynchronousBoundedReadBuffer.h @@ -67,7 +67,7 @@ private: const std::string query_id; const std::string current_reader_id; - Poco::Logger * log; + LoggerPtr log; AsyncReadCountersPtr async_read_counters; FilesystemReadPrefetchesLogPtr prefetches_log; diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 2b169bd9c7d..7ce3d58dcd8 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -60,9 +60,9 @@ CachedOnDiskReadBufferFromFile::CachedOnDiskReadBufferFromFile( std::shared_ptr cache_log_) : ReadBufferFromFileBase(use_external_buffer_ ? 0 : settings_.remote_fs_buffer_size, nullptr, 0, file_size_) #ifdef ABORT_ON_LOGICAL_ERROR - , log(&Poco::Logger::get(fmt::format("CachedOnDiskReadBufferFromFile({})", cache_key_))) + , log(getLogger(fmt::format("CachedOnDiskReadBufferFromFile({})", cache_key_))) #else - , log(&Poco::Logger::get("CachedOnDiskReadBufferFromFile")) + , log(getLogger("CachedOnDiskReadBufferFromFile")) #endif , cache_key(cache_key_) , source_file_path(source_file_path_) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h index e2ea5dce3c0..74fb6220af2 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h @@ -105,7 +105,7 @@ private: bool nextFileSegmentsBatch(); - Poco::Logger * log; + LoggerPtr log; FileCacheKey cache_key; String source_file_path; diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp index a97f25e80e5..25c8ab1c4ae 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp @@ -31,7 +31,7 @@ FileSegmentRangeWriter::FileSegmentRangeWriter( const String & source_path_) : cache(cache_) , key(key_) - , log(&Poco::Logger::get("FileSegmentRangeWriter")) + , log(getLogger("FileSegmentRangeWriter")) , cache_log(cache_log_) , query_id(query_id_) , source_path(source_path_) @@ -205,7 +205,7 @@ CachedOnDiskWriteBufferFromFile::CachedOnDiskWriteBufferFromFile( const FileCacheUserInfo & user_, std::shared_ptr cache_log_) : WriteBufferFromFileDecorator(std::move(impl_)) - , log(&Poco::Logger::get("CachedOnDiskWriteBufferFromFile")) + , log(getLogger("CachedOnDiskWriteBufferFromFile")) , cache(cache_) , source_path(source_path_) , key(key_) diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h index 2a4e78f44c6..59e0c76ca3d 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h @@ -54,7 +54,7 @@ private: FileCache * cache; FileSegment::Key key; - Poco::Logger * log; + LoggerPtr log; std::shared_ptr cache_log; String query_id; String source_path; @@ -93,7 +93,7 @@ public: private: void cacheData(char * data, size_t size, bool throw_on_error); - Poco::Logger * log; + LoggerPtr log; FileCachePtr cache; String source_path; diff --git a/src/Disks/IO/IOUringReader.cpp b/src/Disks/IO/IOUringReader.cpp index 4c9f665093d..90a4d285ecb 100644 --- a/src/Disks/IO/IOUringReader.cpp +++ b/src/Disks/IO/IOUringReader.cpp @@ -46,7 +46,7 @@ namespace ErrorCodes } IOUringReader::IOUringReader(uint32_t entries_) - : log(&Poco::Logger::get("IOUringReader")) + : log(getLogger("IOUringReader")) { struct io_uring_probe * probe = io_uring_get_probe(); if (!probe) diff --git a/src/Disks/IO/IOUringReader.h b/src/Disks/IO/IOUringReader.h index b038b3acf7d..2504dd73a6b 100644 --- a/src/Disks/IO/IOUringReader.h +++ b/src/Disks/IO/IOUringReader.h @@ -73,7 +73,7 @@ private: return promise.get_future(); } - const Poco::Logger * log; + const LoggerPtr log; public: IOUringReader(uint32_t entries_); diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.h b/src/Disks/IO/ReadBufferFromAzureBlobStorage.h index 4e21f543653..68058b84a2f 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.h +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.h @@ -73,7 +73,7 @@ private: char * data_ptr; size_t data_capacity; - Poco::Logger * log = &Poco::Logger::get("ReadBufferFromAzureBlobStorage"); + LoggerPtr log = getLogger("ReadBufferFromAzureBlobStorage"); }; } diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 63bacaa99e9..923aab5c343 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -56,7 +56,7 @@ ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather( , query_id(CurrentThread::getQueryId()) , use_external_buffer(use_external_buffer_) , with_cache(withCache(settings)) - , log(&Poco::Logger::get("ReadBufferFromRemoteFSGather")) + , log(getLogger("ReadBufferFromRemoteFSGather")) { if (!blobs_to_read.empty()) current_object = blobs_to_read.front(); diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h index c5886dea603..93ded9fefb3 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.h @@ -82,7 +82,7 @@ private: size_t current_buf_idx = 0; std::unique_ptr current_buf; - Poco::Logger * log; + LoggerPtr log; }; size_t chooseBufferSizeForRemoteReading(const DB::ReadSettings & settings, size_t file_size); diff --git a/src/Disks/IO/ReadBufferFromWebServer.cpp b/src/Disks/IO/ReadBufferFromWebServer.cpp index 7a602856183..cc872392738 100644 --- a/src/Disks/IO/ReadBufferFromWebServer.cpp +++ b/src/Disks/IO/ReadBufferFromWebServer.cpp @@ -27,7 +27,7 @@ ReadBufferFromWebServer::ReadBufferFromWebServer( bool use_external_buffer_, size_t read_until_position_) : ReadBufferFromFileBase(settings_.remote_fs_buffer_size, nullptr, 0) - , log(&Poco::Logger::get("ReadBufferFromWebServer")) + , log(getLogger("ReadBufferFromWebServer")) , context(context_) , url(url_) , buf_size(settings_.remote_fs_buffer_size) diff --git a/src/Disks/IO/ReadBufferFromWebServer.h b/src/Disks/IO/ReadBufferFromWebServer.h index b4edf16b095..68ad752bbdb 100644 --- a/src/Disks/IO/ReadBufferFromWebServer.h +++ b/src/Disks/IO/ReadBufferFromWebServer.h @@ -41,7 +41,7 @@ public: private: std::unique_ptr initialize(); - Poco::Logger * log; + LoggerPtr log; ContextPtr context; const String url; diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index a2e84edf45f..d281c3dfdc2 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -27,7 +27,7 @@ WriteBufferFromAzureBlobStorage::WriteBufferFromAzureBlobStorage( size_t buf_size_, const WriteSettings & write_settings_) : WriteBufferFromFileBase(buf_size_, nullptr, 0) - , log(&Poco::Logger::get("WriteBufferFromAzureBlobStorage")) + , log(getLogger("WriteBufferFromAzureBlobStorage")) , max_single_part_upload_size(max_single_part_upload_size_) , blob_path(blob_path_) , write_settings(write_settings_) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h index f1be81922e1..5e4f97b0a08 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h @@ -45,7 +45,7 @@ private: void execWithRetry(std::function func, size_t num_tries, size_t cost = 0); void uploadBlock(const char * data, size_t size); - Poco::Logger * log; + LoggerPtr log; const size_t max_single_part_upload_size; const std::string blob_path; diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 93fed1e61bf..05bf2281842 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -96,7 +96,7 @@ AzureObjectStorage::AzureObjectStorage( : name(name_) , client(std::move(client_)) , settings(std::move(settings_)) - , log(&Poco::Logger::get("AzureObjectStorage")) + , log(getLogger("AzureObjectStorage")) { } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 85213a3c24f..a05eb824b91 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -134,7 +134,7 @@ private: MultiVersion client; MultiVersion settings; - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp index d4aba197ae4..1444f4c9c76 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp @@ -24,7 +24,7 @@ CachedObjectStorage::CachedObjectStorage( , cache(cache_) , cache_settings(cache_settings_) , cache_config_name(cache_config_name_) - , log(&Poco::Logger::get(getName())) + , log(getLogger(getName())) { cache->initialize(); } diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index 028b0f7c510..7b231b68805 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -129,7 +129,7 @@ private: FileCachePtr cache; FileCacheSettings cache_settings; std::string cache_config_name; - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/src/Disks/ObjectStorages/Cached/registerDiskCache.cpp b/src/Disks/ObjectStorages/Cached/registerDiskCache.cpp index 5d788f2641a..6e0453f5f02 100644 --- a/src/Disks/ObjectStorages/Cached/registerDiskCache.cpp +++ b/src/Disks/ObjectStorages/Cached/registerDiskCache.cpp @@ -114,7 +114,7 @@ void registerDiskCache(DiskFactory & factory, bool /* global_skip_access_check * disk_object_storage->wrapWithCache(cache, file_cache_settings, name); LOG_INFO( - &Poco::Logger::get("DiskCache"), + getLogger("DiskCache"), "Registered cached disk (`{}`) with structure: {}", name, assert_cast(disk_object_storage.get())->getStructure()); diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 3c39fa2a8ff..9c4132f433f 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -67,7 +67,7 @@ DiskObjectStorage::DiskObjectStorage( const String & config_prefix) : IDisk(name_, config, config_prefix) , object_key_prefix(object_key_prefix_) - , log (&Poco::Logger::get("DiskObjectStorage(" + name + ")")) + , log(getLogger("DiskObjectStorage(" + name + ")")) , metadata_storage(std::move(metadata_storage_)) , object_storage(std::move(object_storage_)) , send_metadata(config.getBool(config_prefix + ".send_metadata", false)) diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.h b/src/Disks/ObjectStorages/DiskObjectStorage.h index 983af483b8a..e1576509713 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.h +++ b/src/Disks/ObjectStorages/DiskObjectStorage.h @@ -219,7 +219,7 @@ private: String getWriteResourceName() const; const String object_key_prefix; - Poco::Logger * log; + LoggerPtr log; MetadataStoragePtr metadata_storage; ObjectStoragePtr object_storage; diff --git a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp index 881f7a46c16..19b8b51384f 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp @@ -104,7 +104,7 @@ void DiskObjectStorageMetadata::serialize(WriteBuffer & buf, bool sync) const if (version == VERSION_FULL_OBJECT_KEY && !storage_metadata_write_full_object_key) { - Poco::Logger * logger = &Poco::Logger::get("DiskObjectStorageMetadata"); + LoggerPtr logger = getLogger("DiskObjectStorageMetadata"); LOG_WARNING( logger, "Metadata file {} is written with VERSION_FULL_OBJECT_KEY version" @@ -192,7 +192,7 @@ void DiskObjectStorageMetadata::addObject(ObjectStorageKey key, size_t size) bool storage_metadata_write_full_object_key = getWriteFullObjectKeySetting(); if (!storage_metadata_write_full_object_key) { - Poco::Logger * logger = &Poco::Logger::get("DiskObjectStorageMetadata"); + LoggerPtr logger = getLogger("DiskObjectStorageMetadata"); LOG_WARNING( logger, "Metadata file {} has at least one key {} without fixed common key prefix." diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp index f39e16828c5..de7a71e8dc1 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp @@ -242,7 +242,7 @@ struct RemoveManyObjectStorageOperation final : public IDiskObjectStorageOperati || e.code() == ErrorCodes::CANNOT_OPEN_FILE) { LOG_DEBUG( - &Poco::Logger::get("RemoveManyObjectStorageOperation"), + getLogger("RemoveManyObjectStorageOperation"), "Can't read metadata because of an exception. Just remove it from the filesystem. Path: {}, exception: {}", metadata_storage.getPath() + path, e.message()); @@ -276,7 +276,7 @@ struct RemoveManyObjectStorageOperation final : public IDiskObjectStorageOperati if (!keep_all_batch_data) { LOG_DEBUG( - &Poco::Logger::get("RemoveManyObjectStorageOperation"), + getLogger("RemoveManyObjectStorageOperation"), "metadata and objects were removed for [{}], " "only metadata were removed for [{}].", boost::algorithm::join(paths_removed_with_objects, ", "), @@ -345,7 +345,7 @@ struct RemoveRecursiveObjectStorageOperation final : public IDiskObjectStorageOp || e.code() == ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED) { LOG_DEBUG( - &Poco::Logger::get("RemoveRecursiveObjectStorageOperation"), + getLogger("RemoveRecursiveObjectStorageOperation"), "Can't read metadata because of an exception. Just remove it from the filesystem. Path: {}, exception: {}", metadata_storage.getPath() + path_to_remove, e.message()); @@ -399,7 +399,7 @@ struct RemoveRecursiveObjectStorageOperation final : public IDiskObjectStorageOp object_storage.removeObjectsIfExist(remove_from_remote); LOG_DEBUG( - &Poco::Logger::get("RemoveRecursiveObjectStorageOperation"), + getLogger("RemoveRecursiveObjectStorageOperation"), "Recursively remove path {}: " "metadata and objects were removed for [{}], " "only metadata were removed for [{}].", @@ -905,7 +905,7 @@ void DiskObjectStorageTransaction::commit() catch (...) { tryLogCurrentException( - &Poco::Logger::get("DiskObjectStorageTransaction"), + getLogger("DiskObjectStorageTransaction"), fmt::format("An error occurred while executing transaction's operation #{} ({})", i, operations_to_execute[i]->getInfoForLog())); for (int64_t j = i; j >= 0; --j) @@ -917,7 +917,7 @@ void DiskObjectStorageTransaction::commit() catch (...) { tryLogCurrentException( - &Poco::Logger::get("DiskObjectStorageTransaction"), + getLogger("DiskObjectStorageTransaction"), fmt::format("An error occurred while undoing transaction's operation #{}", i)); throw; diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index b3c1c3202a5..02700b358e0 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -25,7 +25,7 @@ namespace ErrorCodes LocalObjectStorage::LocalObjectStorage(String key_prefix_) : key_prefix(std::move(key_prefix_)) - , log(&Poco::Logger::get("LocalObjectStorage")) + , log(getLogger("LocalObjectStorage")) { if (auto block_device_id = tryGetBlockDeviceId("/"); block_device_id.has_value()) description = *block_device_id; diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.h b/src/Disks/ObjectStorages/Local/LocalObjectStorage.h index 522e73b415d..ed5f8c1f537 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.h +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.h @@ -90,7 +90,7 @@ public: private: String key_prefix; - Poco::Logger * log; + LoggerPtr log; std::string description; }; diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 49943554384..4cc49288af6 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -482,7 +482,7 @@ void S3ObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT /// If authentication/permissions error occurs then fallthrough to copy with buffer. if (exc.getS3ErrorCode() != Aws::S3::S3Errors::ACCESS_DENIED) throw; - LOG_WARNING(&Poco::Logger::get("S3ObjectStorage"), + LOG_WARNING(getLogger("S3ObjectStorage"), "S3-server-side copy object from the disk {} to the disk {} can not be performed: {}\n", getName(), dest_s3->getName(), exc.what()); } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index 820d4977f98..ab0fa5bed68 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -59,7 +59,7 @@ private: , client(std::move(client_)) , s3_settings(std::move(s3_settings_)) , s3_capabilities(s3_capabilities_) - , log(&Poco::Logger::get(logger_name)) + , log(getLogger(logger_name)) { } @@ -179,7 +179,7 @@ private: MultiVersion s3_settings; S3Capabilities s3_capabilities; - Poco::Logger * log; + LoggerPtr log; }; /// Do not encode keys, store as-is, and do not require separate disk for metadata. diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp index ff4216a83da..0223c24973e 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp @@ -119,7 +119,7 @@ WebObjectStorage::WebObjectStorage( ContextPtr context_) : WithContext(context_->getGlobalContext()) , url(url_) - , log(&Poco::Logger::get("WebObjectStorage")) + , log(getLogger("WebObjectStorage")) { } @@ -130,7 +130,7 @@ bool WebObjectStorage::exists(const StoredObject & object) const bool WebObjectStorage::exists(const std::string & path) const { - LOG_TRACE(&Poco::Logger::get("DiskWeb"), "Checking existence of path: {}", path); + LOG_TRACE(getLogger("DiskWeb"), "Checking existence of path: {}", path); return tryGetFileInfo(path) != nullptr; } diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.h b/src/Disks/ObjectStorages/Web/WebObjectStorage.h index 9688873f0c4..a285742c66d 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.h +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.h @@ -158,7 +158,7 @@ private: loadFiles(const String & path, const std::unique_lock &) const; const String url; - Poco::Logger * log; + LoggerPtr log; size_t min_bytes_for_seek; }; diff --git a/src/Disks/StoragePolicy.cpp b/src/Disks/StoragePolicy.cpp index 6cf22cbaa1b..13bd74ceaeb 100644 --- a/src/Disks/StoragePolicy.cpp +++ b/src/Disks/StoragePolicy.cpp @@ -41,7 +41,7 @@ StoragePolicy::StoragePolicy( const String & config_prefix, DiskSelectorPtr disks) : name(std::move(name_)) - , log(&Poco::Logger::get("StoragePolicy (" + name + ")")) + , log(getLogger("StoragePolicy (" + name + ")")) { Poco::Util::AbstractConfiguration::Keys keys; String volumes_prefix = config_prefix + ".volumes"; @@ -96,7 +96,7 @@ StoragePolicy::StoragePolicy(String name_, Volumes volumes_, double move_factor_ : volumes(std::move(volumes_)) , name(std::move(name_)) , move_factor(move_factor_) - , log(&Poco::Logger::get("StoragePolicy (" + name + ")")) + , log(getLogger("StoragePolicy (" + name + ")")) { if (volumes.empty()) throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Storage policy {} must contain at least one Volume.", backQuote(name)); @@ -418,7 +418,7 @@ StoragePolicySelector::StoragePolicySelector( */ policies.emplace(name, std::make_shared(name, config, config_prefix + "." + name, disks)); - LOG_INFO(&Poco::Logger::get("StoragePolicySelector"), "Storage policy {} loaded", backQuote(name)); + LOG_INFO(getLogger("StoragePolicySelector"), "Storage policy {} loaded", backQuote(name)); } /// Add default policy if it isn't explicitly specified. diff --git a/src/Disks/StoragePolicy.h b/src/Disks/StoragePolicy.h index d210d8c1e2f..501e033abc3 100644 --- a/src/Disks/StoragePolicy.h +++ b/src/Disks/StoragePolicy.h @@ -105,7 +105,7 @@ private: void buildVolumeIndices(); - Poco::Logger * log; + LoggerPtr log; }; diff --git a/src/Disks/TemporaryFileOnDisk.cpp b/src/Disks/TemporaryFileOnDisk.cpp index 06d7da4af58..92219a7f25f 100644 --- a/src/Disks/TemporaryFileOnDisk.cpp +++ b/src/Disks/TemporaryFileOnDisk.cpp @@ -59,7 +59,7 @@ TemporaryFileOnDisk::~TemporaryFileOnDisk() if (!disk->exists(relative_path)) { - LOG_WARNING(&Poco::Logger::get("TemporaryFileOnDisk"), "Temporary path '{}' does not exist in '{}'", relative_path, disk->getPath()); + LOG_WARNING(getLogger("TemporaryFileOnDisk"), "Temporary path '{}' does not exist in '{}'", relative_path, disk->getPath()); return; } diff --git a/src/Disks/VolumeJBOD.cpp b/src/Disks/VolumeJBOD.cpp index 682a167bf5f..e437684b802 100644 --- a/src/Disks/VolumeJBOD.cpp +++ b/src/Disks/VolumeJBOD.cpp @@ -21,7 +21,7 @@ VolumeJBOD::VolumeJBOD( : IVolume(name_, config, config_prefix, disk_selector) , disks_by_size(disks.begin(), disks.end()) { - Poco::Logger * logger = &Poco::Logger::get("StorageConfiguration"); + LoggerPtr logger = getLogger("StorageConfiguration"); auto has_max_bytes = config.has(config_prefix + ".max_data_part_size_bytes"); auto has_max_ratio = config.has(config_prefix + ".max_data_part_size_ratio"); diff --git a/src/Disks/getOrCreateDiskFromAST.cpp b/src/Disks/getOrCreateDiskFromAST.cpp index ab2fb5e7f8b..7b2762613b6 100644 --- a/src/Disks/getOrCreateDiskFromAST.cpp +++ b/src/Disks/getOrCreateDiskFromAST.cpp @@ -114,7 +114,7 @@ std::string getOrCreateDiskFromDiskAST(const ASTPtr & disk_function, ContextPtr FlattenDiskConfigurationVisitor{data}.visit(ast); auto disk_name = assert_cast(*ast).value.get(); - LOG_TRACE(&Poco::Logger::get("getOrCreateDiskFromDiskAST"), "Result disk name: {}", disk_name); + LOG_TRACE(getLogger("getOrCreateDiskFromDiskAST"), "Result disk name: {}", disk_name); return disk_name; } diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 608f9433d6f..62cbadec4f4 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -395,7 +395,7 @@ std::unique_ptr FormatFactory::wrapReadBufferIfNeeded( { parallel_read = false; LOG_TRACE( - &Poco::Logger::get("FormatFactory"), + getLogger("FormatFactory"), "Failed to setup ParallelReadBuffer because of an exception:\n{}.\n" "Falling back to the single-threaded buffer", e.displayText()); @@ -405,7 +405,7 @@ std::unique_ptr FormatFactory::wrapReadBufferIfNeeded( if (parallel_read) { LOG_TRACE( - &Poco::Logger::get("FormatFactory"), + getLogger("FormatFactory"), "Using ParallelReadBuffer with {} workers with chunks of {} bytes", max_download_threads, settings.max_download_buffer_size); diff --git a/src/Formats/ProtobufSerializer.cpp b/src/Formats/ProtobufSerializer.cpp index dd37c25719c..c0d0713e254 100644 --- a/src/Formats/ProtobufSerializer.cpp +++ b/src/Formats/ProtobufSerializer.cpp @@ -3045,7 +3045,7 @@ namespace { *root_serializer_ptr = message_serializer.get(); #if 0 - LOG_INFO(&Poco::Logger::get("ProtobufSerializer"), "Serialization tree:\n{}", get_root_desc_function(0)); + LOG_INFO(getLogger("ProtobufSerializer"), "Serialization tree:\n{}", get_root_desc_function(0)); #endif return message_serializer; } @@ -3054,7 +3054,7 @@ namespace auto envelope_serializer = std::make_unique(std::move(message_serializer), reader_or_writer); *root_serializer_ptr = envelope_serializer.get(); #if 0 - LOG_INFO(&Poco::Logger::get("ProtobufSerializer"), "Serialization tree:\n{}", get_root_desc_function(0)); + LOG_INFO(getLogger("ProtobufSerializer"), "Serialization tree:\n{}", get_root_desc_function(0)); #endif return envelope_serializer; } diff --git a/src/Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.cpp b/src/Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.cpp index f3c9f511ef6..db98f88e53b 100644 --- a/src/Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.cpp +++ b/src/Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.cpp @@ -95,7 +95,7 @@ namespace } ExternalUserDefinedExecutableFunctionsLoader::ExternalUserDefinedExecutableFunctionsLoader(ContextPtr global_context_) - : ExternalLoader("external user defined function", &Poco::Logger::get("ExternalUserDefinedExecutableFunctionsLoader")) + : ExternalLoader("external user defined function", getLogger("ExternalUserDefinedExecutableFunctionsLoader")) , WithContext(global_context_) { setConfigSettings({"function", "name", "database", "uuid"}); diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.cpp b/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.cpp index 271c464e79a..34946db7d9e 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.cpp @@ -54,7 +54,7 @@ namespace UserDefinedSQLObjectsDiskStorage::UserDefinedSQLObjectsDiskStorage(const ContextPtr & global_context_, const String & dir_path_) : global_context(global_context_) , dir_path{makeDirectoryPathCanonical(dir_path_)} - , log{&Poco::Logger::get("UserDefinedSQLObjectsLoaderFromDisk")} + , log{getLogger("UserDefinedSQLObjectsLoaderFromDisk")} { createDirectory(); } diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.h b/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.h index f0986dbda72..ae0cbd0c589 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.h +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.h @@ -44,7 +44,7 @@ private: ContextPtr global_context; String dir_path; - Poco::Logger * log; + LoggerPtr log; std::atomic objects_loaded = false; }; diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.cpp b/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.cpp index 6e5a5338437..c43b223ffeb 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.cpp @@ -53,7 +53,7 @@ UserDefinedSQLObjectsZooKeeperStorage::UserDefinedSQLObjectsZooKeeperStorage( , zookeeper_getter{[global_context_]() { return global_context_->getZooKeeper(); }} , zookeeper_path{zookeeper_path_} , watch_queue{std::make_shared>>(std::numeric_limits::max())} - , log{&Poco::Logger::get("UserDefinedSQLObjectsLoaderFromZooKeeper")} + , log{getLogger("UserDefinedSQLObjectsLoaderFromZooKeeper")} { if (zookeeper_path.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "ZooKeeper path must be non-empty"); diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.h b/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.h index 9f41763c59c..61002be2bfd 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.h +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.h @@ -80,7 +80,7 @@ private: using UserDefinedSQLObjectTypeAndName = std::pair; std::shared_ptr> watch_queue; - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/src/Functions/logTrace.cpp b/src/Functions/logTrace.cpp index 55f387cbfeb..923ea9fd70e 100644 --- a/src/Functions/logTrace.cpp +++ b/src/Functions/logTrace.cpp @@ -46,7 +46,7 @@ namespace throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be Constant string", getName()); - static auto * log = &Poco::Logger::get("FunctionLogTrace"); + static auto log = getLogger("FunctionLogTrace"); LOG_TRACE(log, fmt::runtime(message)); return DataTypeUInt8().createColumnConst(input_rows_count, 0); diff --git a/src/IO/Archives/ZipArchiveWriter.cpp b/src/IO/Archives/ZipArchiveWriter.cpp index 785a5005f87..8cb4a2e0bd6 100644 --- a/src/IO/Archives/ZipArchiveWriter.cpp +++ b/src/IO/Archives/ZipArchiveWriter.cpp @@ -246,7 +246,7 @@ ZipArchiveWriter::~ZipArchiveWriter() /// However it is suspicious to destroy instance without finalization at the green path. if (!std::uncaught_exceptions() && std::current_exception() == nullptr) { - Poco::Logger * log = &Poco::Logger::get("ZipArchiveWriter"); + LoggerPtr log = getLogger("ZipArchiveWriter"); LOG_ERROR(log, "ZipArchiveWriter is not finalized when destructor is called. " "The zip archive might not be written at all or might be truncated. " diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index cce394c67c9..c4468a1b896 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -70,7 +70,7 @@ namespace static_assert(std::has_virtual_destructor_v, "The base class must have a virtual destructor"); public: - HTTPSessionAdapter(const std::string & host, UInt16 port) : Session(host, port), log{&Poco::Logger::get("HTTPSessionAdapter")} { } + HTTPSessionAdapter(const std::string & host, UInt16 port) : Session(host, port), log{getLogger("HTTPSessionAdapter")} { } ~HTTPSessionAdapter() override = default; protected: @@ -132,7 +132,7 @@ namespace } } } - Poco::Logger * log; + LoggerPtr log; }; bool isHTTPS(const Poco::URI & uri) @@ -223,7 +223,7 @@ namespace bool wait_on_pool_size_limit) : Base( static_cast(max_pool_size_), - &Poco::Logger::get("HTTPSessionPool"), + getLogger("HTTPSessionPool"), wait_on_pool_size_limit ? BehaviourOnLimit::Wait : BehaviourOnLimit::AllocateNewBypassingPool) , host(host_) , port(port_) diff --git a/src/IO/ParallelReadBuffer.cpp b/src/IO/ParallelReadBuffer.cpp index 8d73f221748..cdeb8a18635 100644 --- a/src/IO/ParallelReadBuffer.cpp +++ b/src/IO/ParallelReadBuffer.cpp @@ -50,7 +50,7 @@ ParallelReadBuffer::ParallelReadBuffer( , file_size(file_size_) , range_step(std::max(1ul, range_step_)) { - LOG_TRACE(&Poco::Logger::get("ParallelReadBuffer"), "Parallel reading is used"); + LOG_TRACE(getLogger("ParallelReadBuffer"), "Parallel reading is used"); try { diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index 101e25f8b43..f28c23a71d7 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -39,7 +39,7 @@ private: std::optional read_result; std::unique_ptr impl; - Poco::Logger * log = &Poco::Logger::get("ReadBufferFromS3"); + LoggerPtr log = getLogger("ReadBufferFromS3"); public: ReadBufferFromS3( diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index b2c4a53bd9b..bf5c426f803 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -265,7 +265,7 @@ ReadWriteBufferFromHTTPBase::ReadWriteBufferFromHTTPBase( , file_info(file_info_) , http_skip_not_found_url(http_skip_not_found_url_) , settings {settings_} - , log(&Poco::Logger::get("ReadWriteBufferFromHTTP")) + , log(getLogger("ReadWriteBufferFromHTTP")) , proxy_config(proxy_config_) { if (settings.http_max_tries <= 0 || settings.http_retry_initial_backoff_ms <= 0 diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index 29c0804bb28..63ca3e0417c 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -109,7 +109,7 @@ namespace detail bool http_skip_not_found_url; ReadSettings settings; - Poco::Logger * log; + LoggerPtr log; ProxyConfiguration proxy_config; diff --git a/src/IO/S3/AWSLogger.cpp b/src/IO/S3/AWSLogger.cpp index d6162823aee..dcdba7753b2 100644 --- a/src/IO/S3/AWSLogger.cpp +++ b/src/IO/S3/AWSLogger.cpp @@ -41,7 +41,7 @@ AWSLogger::AWSLogger(bool enable_s3_requests_logging_) : enable_s3_requests_logging(enable_s3_requests_logging_) { for (auto [tag, name] : S3_LOGGER_TAG_NAMES) - tag_loggers[tag] = &Poco::Logger::get(name); + tag_loggers[tag] = getLogger(name); default_logger = tag_loggers[S3_LOGGER_TAG_NAMES[0][0]]; } diff --git a/src/IO/S3/AWSLogger.h b/src/IO/S3/AWSLogger.h index fdb6eed1f86..a4987f17c0d 100644 --- a/src/IO/S3/AWSLogger.h +++ b/src/IO/S3/AWSLogger.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace Poco { class Logger; } @@ -29,9 +30,9 @@ public: void Flush() final {} private: - Poco::Logger * default_logger; + LoggerPtr default_logger; bool enable_s3_requests_logging; - std::unordered_map tag_loggers; + std::unordered_map tag_loggers; }; } diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 64259ce5a76..7f0ede72740 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -184,7 +184,7 @@ Client::Client( , client_settings(client_settings_) , max_redirects(max_redirects_) , sse_kms_config(std::move(sse_kms_config_)) - , log(&Poco::Logger::get("S3Client")) + , log(getLogger("S3Client")) { auto * endpoint_provider = dynamic_cast(accessEndpointProvider().get()); endpoint_provider->GetBuiltInParameters().GetParameter("Region").GetString(explicit_region); @@ -234,7 +234,7 @@ Client::Client( , provider_type(other.provider_type) , max_redirects(other.max_redirects) , sse_kms_config(other.sse_kms_config) - , log(&Poco::Logger::get("S3Client")) + , log(getLogger("S3Client")) { cache = std::make_shared(*other.cache); ClientCacheRegistry::instance().registerClient(cache); @@ -854,7 +854,7 @@ void ClientCacheRegistry::clearCacheForAll() } else { - LOG_INFO(&Poco::Logger::get("ClientCacheRegistry"), "Deleting leftover S3 client cache"); + LOG_INFO(getLogger("ClientCacheRegistry"), "Deleting leftover S3 client cache"); it = client_caches.erase(it); } } diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index 677b739fd39..8da21bd2c2c 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -281,7 +281,7 @@ private: const ServerSideEncryptionKMSConfig sse_kms_config; - Poco::Logger * log; + LoggerPtr log; }; class ClientFactory diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index b0b33244015..e64f54b99ad 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -76,7 +76,7 @@ constexpr int AVAILABILITY_ZONE_REQUEST_TIMEOUT_SECONDS = 3; AWSEC2MetadataClient::AWSEC2MetadataClient(const Aws::Client::ClientConfiguration & client_configuration, const char * endpoint_) : Aws::Internal::AWSHttpResourceClient(client_configuration) , endpoint(endpoint_) - , logger(&Poco::Logger::get("AWSEC2InstanceProfileConfigLoader")) + , logger(getLogger("AWSEC2InstanceProfileConfigLoader")) { } @@ -200,7 +200,7 @@ Aws::String AWSEC2MetadataClient::getCurrentRegion() const static Aws::String getAWSMetadataEndpoint() { - auto * logger = &Poco::Logger::get("AWSEC2InstanceProfileConfigLoader"); + auto logger = getLogger("AWSEC2InstanceProfileConfigLoader"); Aws::String ec2_metadata_service_endpoint = Aws::Environment::GetEnv("AWS_EC2_METADATA_SERVICE_ENDPOINT"); if (ec2_metadata_service_endpoint.empty()) { @@ -285,7 +285,7 @@ String getGCPAvailabilityZoneOrException() String getRunningAvailabilityZone() { - LOG_INFO(&Poco::Logger::get("Application"), "Trying to detect the availability zone."); + LOG_INFO(getLogger("Application"), "Trying to detect the availability zone."); try { return AWSEC2MetadataClient::getAvailabilityZoneOrException(); @@ -310,7 +310,7 @@ String getRunningAvailabilityZone() AWSEC2InstanceProfileConfigLoader::AWSEC2InstanceProfileConfigLoader(const std::shared_ptr & client_, bool use_secure_pull_) : client(client_) , use_secure_pull(use_secure_pull_) - , logger(&Poco::Logger::get("AWSEC2InstanceProfileConfigLoader")) + , logger(getLogger("AWSEC2InstanceProfileConfigLoader")) { } @@ -352,7 +352,7 @@ bool AWSEC2InstanceProfileConfigLoader::LoadInternal() AWSInstanceProfileCredentialsProvider::AWSInstanceProfileCredentialsProvider(const std::shared_ptr & config_loader) : ec2_metadata_config_loader(config_loader) , load_frequency_ms(Aws::Auth::REFRESH_THRESHOLD) - , logger(&Poco::Logger::get("AWSInstanceProfileCredentialsProvider")) + , logger(getLogger("AWSInstanceProfileCredentialsProvider")) { LOG_INFO(logger, "Creating Instance with injected EC2MetadataClient and refresh rate."); } @@ -396,7 +396,7 @@ void AWSInstanceProfileCredentialsProvider::refreshIfExpired() AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider::AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider( DB::S3::PocoHTTPClientConfiguration & aws_client_configuration, uint64_t expiration_window_seconds_) - : logger(&Poco::Logger::get("AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider")) + : logger(getLogger("AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider")) , expiration_window_seconds(expiration_window_seconds_) { // check environment variables @@ -529,7 +529,7 @@ SSOCredentialsProvider::SSOCredentialsProvider(DB::S3::PocoHTTPClientConfigurati : profile_to_use(Aws::Auth::GetConfigProfileName()) , aws_client_configuration(std::move(aws_client_configuration_)) , expiration_window_seconds(expiration_window_seconds_) - , logger(&Poco::Logger::get(SSO_CREDENTIALS_PROVIDER_LOG_TAG)) + , logger(getLogger(SSO_CREDENTIALS_PROVIDER_LOG_TAG)) { LOG_TRACE(logger, "Setting sso credentials provider to read config from {}", profile_to_use); } @@ -659,7 +659,7 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( const Aws::Auth::AWSCredentials & credentials, CredentialsConfiguration credentials_configuration) { - auto * logger = &Poco::Logger::get("S3CredentialsProviderChain"); + auto logger = getLogger("S3CredentialsProviderChain"); /// we don't provide any credentials to avoid signing if (credentials_configuration.no_sign_request) diff --git a/src/IO/S3/Credentials.h b/src/IO/S3/Credentials.h index 5e83ea30798..34dc0c1d2bd 100644 --- a/src/IO/S3/Credentials.h +++ b/src/IO/S3/Credentials.h @@ -70,7 +70,7 @@ private: const Aws::String endpoint; mutable std::recursive_mutex token_mutex; mutable Aws::String token; - Poco::Logger * logger; + LoggerPtr logger; }; std::shared_ptr InitEC2MetadataClient(const Aws::Client::ClientConfiguration & client_configuration); @@ -88,7 +88,7 @@ protected: private: std::shared_ptr client; bool use_secure_pull; - Poco::Logger * logger; + LoggerPtr logger; }; class AWSInstanceProfileCredentialsProvider : public Aws::Auth::AWSCredentialsProvider @@ -107,7 +107,7 @@ private: std::shared_ptr ec2_metadata_config_loader; Int64 load_frequency_ms; - Poco::Logger * logger; + LoggerPtr logger; }; class AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider : public Aws::Auth::AWSCredentialsProvider @@ -133,7 +133,7 @@ private: Aws::String session_name; Aws::String token; bool initialized = false; - Poco::Logger * logger; + LoggerPtr logger; uint64_t expiration_window_seconds; }; @@ -163,7 +163,7 @@ private: DB::S3::PocoHTTPClientConfiguration aws_client_configuration; uint64_t expiration_window_seconds; - Poco::Logger * logger; + LoggerPtr logger; void Reload() override; void refreshIfExpired(); diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 946bd74dcb5..21acdfd69f2 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -345,7 +345,7 @@ void PocoHTTPClient::makeRequestInternalImpl( { using SessionPtr = std::conditional_t; - Poco::Logger * log = &Poco::Logger::get("AWSClient"); + LoggerPtr log = getLogger("AWSClient"); auto uri = request.GetUri().GetURIString(); auto method = getMethod(request); diff --git a/src/IO/S3/copyS3File.cpp b/src/IO/S3/copyS3File.cpp index 830377622ef..98024e74f8e 100644 --- a/src/IO/S3/copyS3File.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -61,7 +61,7 @@ namespace ThreadPoolCallbackRunner schedule_, bool for_disk_s3_, BlobStorageLogWriterPtr blob_storage_log_, - const Poco::Logger * log_) + const LoggerPtr log_) : client_ptr(client_ptr_) , dest_bucket(dest_bucket_) , dest_key(dest_key_) @@ -87,7 +87,7 @@ namespace ThreadPoolCallbackRunner schedule; bool for_disk_s3; BlobStorageLogWriterPtr blob_storage_log; - const Poco::Logger * log; + const LoggerPtr log; /// Represents a task uploading a single part. /// Keep this struct small because there can be thousands of parts. @@ -475,7 +475,7 @@ namespace ThreadPoolCallbackRunner schedule_, bool for_disk_s3_, BlobStorageLogWriterPtr blob_storage_log_) - : UploadHelper(client_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, for_disk_s3_, blob_storage_log_, &Poco::Logger::get("copyDataToS3File")) + : UploadHelper(client_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, for_disk_s3_, blob_storage_log_, getLogger("copyDataToS3File")) , create_read_buffer(create_read_buffer_) , offset(offset_) , size(size_) @@ -658,7 +658,7 @@ namespace ThreadPoolCallbackRunner schedule_, bool for_disk_s3_, BlobStorageLogWriterPtr blob_storage_log_) - : UploadHelper(client_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, for_disk_s3_, blob_storage_log_, &Poco::Logger::get("copyS3File")) + : UploadHelper(client_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, for_disk_s3_, blob_storage_log_, getLogger("copyS3File")) , src_bucket(src_bucket_) , src_key(src_key_) , offset(src_offset_) diff --git a/src/IO/WriteBuffer.cpp b/src/IO/WriteBuffer.cpp index 61fdd31e16a..bcc7445486e 100644 --- a/src/IO/WriteBuffer.cpp +++ b/src/IO/WriteBuffer.cpp @@ -17,7 +17,7 @@ WriteBuffer::~WriteBuffer() /// However it is suspicious to destroy instance without finalization at the green path if (!std::uncaught_exceptions() && std::current_exception() == nullptr) { - Poco::Logger * log = &Poco::Logger::get("WriteBuffer"); + LoggerPtr log = getLogger("WriteBuffer"); LOG_ERROR( log, "WriteBuffer is not finalized when destructor is called. " diff --git a/src/IO/WriteBufferFromEncryptedFile.h b/src/IO/WriteBufferFromEncryptedFile.h index c6edcf76533..2b59bb468d1 100644 --- a/src/IO/WriteBufferFromEncryptedFile.h +++ b/src/IO/WriteBufferFromEncryptedFile.h @@ -40,7 +40,7 @@ private: FileEncryption::Encryptor encryptor; - Poco::Logger * log = &Poco::Logger::get("WriteBufferFromEncryptedFile"); + LoggerPtr log = getLogger("WriteBufferFromEncryptedFile"); }; } diff --git a/src/IO/WriteBufferFromHTTP.cpp b/src/IO/WriteBufferFromHTTP.cpp index 3b2721e3bff..8ddcbc03b84 100644 --- a/src/IO/WriteBufferFromHTTP.cpp +++ b/src/IO/WriteBufferFromHTTP.cpp @@ -33,7 +33,7 @@ WriteBufferFromHTTP::WriteBufferFromHTTP( for (const auto & header: additional_headers) request.add(header.name, header.value); - LOG_TRACE((&Poco::Logger::get("WriteBufferToHTTP")), "Sending request to {}", uri.toString()); + LOG_TRACE((getLogger("WriteBufferToHTTP")), "Sending request to {}", uri.toString()); ostr = &session->sendRequest(request); } diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index 191e522c59a..230f39b074e 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -91,7 +91,7 @@ private: const WriteSettings write_settings; const std::shared_ptr client_ptr; const std::optional> object_metadata; - Poco::Logger * log = &Poco::Logger::get("WriteBufferFromS3"); + LoggerPtr log = getLogger("WriteBufferFromS3"); LogSeriesLimiterPtr limitedLog = std::make_shared(log, 1, 5); IBufferAllocationPolicyPtr buffer_allocation_policy; diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 4171818d3e6..331cd991ea1 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -105,7 +105,7 @@ public: if (const auto hint = cache->get(params.key)) { LOG_TRACE( - &Poco::Logger::get("Aggregator"), + getLogger("Aggregator"), "An entry for key={} found in cache: sum_of_sizes={}, median_size={}", params.key, hint->sum_of_sizes, @@ -129,7 +129,7 @@ public: || hint->median_size < median_size) { LOG_TRACE( - &Poco::Logger::get("Aggregator"), + getLogger("Aggregator"), "Statistics updated for key={}: new sum_of_sizes={}, median_size={}", params.key, sum_of_sizes, @@ -229,7 +229,7 @@ void initDataVariantsWithSizeHint( /// But we will also work with the big (i.e. not so cache friendly) HT from the beginning which may result in a slight slowdown. /// So let's just do nothing. LOG_TRACE( - &Poco::Logger::get("Aggregator"), + getLogger("Aggregator"), "No space were preallocated in hash tables because 'max_size_to_preallocate_for_aggregation' has too small value: {}, " "should be at least {}", stats_collecting_params.max_size_to_preallocate_for_aggregation, diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index f4aa78043ca..109bd0dd939 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1293,7 +1293,7 @@ private: /// How many RAM were used to process the query before processing the first block. Int64 memory_usage_before_aggregation = 0; - Poco::Logger * log = &Poco::Logger::get("Aggregator"); + LoggerPtr log = getLogger("Aggregator"); /// For external aggregation. TemporaryDataOnDiskPtr tmp_data; diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 63ee62cdef4..8206c31624c 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -515,7 +515,7 @@ try SCOPE_EXIT(CurrentMetrics::sub(CurrentMetrics::PendingAsyncInsert, data->entries.size())); - const auto * log = &Poco::Logger::get("AsynchronousInsertQueue"); + const auto log = getLogger("AsynchronousInsertQueue"); const auto & insert_query = assert_cast(*key.query); auto insert_context = Context::createCopy(global_context); @@ -732,7 +732,7 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing( const std::list & entries, const Block & header, const ContextPtr & insert_context, - const Poco::Logger * logger, + const LoggerPtr logger, LogFunc && add_to_async_insert_log) { size_t total_rows = 0; diff --git a/src/Interpreters/AsynchronousInsertQueue.h b/src/Interpreters/AsynchronousInsertQueue.h index 99394d0fb14..f4bfdbd38a5 100644 --- a/src/Interpreters/AsynchronousInsertQueue.h +++ b/src/Interpreters/AsynchronousInsertQueue.h @@ -214,7 +214,7 @@ private: /// Uses async_insert_busy_timeout_ms and processBatchDeadlines() std::vector dump_by_first_update_threads; - Poco::Logger * log = &Poco::Logger::get("AsynchronousInsertQueue"); + LoggerPtr log = getLogger("AsynchronousInsertQueue"); PushResult pushDataChunk(ASTPtr query, DataChunk chunk, ContextPtr query_context); void preprocessInsertQuery(const ASTPtr & query, const ContextPtr & query_context); @@ -230,7 +230,7 @@ private: const std::list & entries, const Block & header, const ContextPtr & insert_context, - const Poco::Logger * logger, + const LoggerPtr logger, LogFunc && add_to_async_insert_log); template diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 9c29a9c4a47..d242544f787 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -85,7 +85,7 @@ FileCache::FileCache(const std::string & cache_name, const FileCacheSettings & s , boundary_alignment(settings.boundary_alignment) , load_metadata_threads(settings.load_metadata_threads) , write_cache_per_user_directory(settings.write_cache_per_user_id_directory) - , log(&Poco::Logger::get("FileCache(" + cache_name + ")")) + , log(getLogger("FileCache(" + cache_name + ")")) , metadata(settings.base_path, settings.background_download_queue_size_limit, settings.background_download_threads, write_cache_per_user_directory) { if (settings.cache_policy == "LRU") diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 64e03b73968..2de2f347999 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -193,7 +193,7 @@ private: size_t load_metadata_threads; const bool write_cache_per_user_directory; - Poco::Logger * log; + LoggerPtr log; std::exception_ptr init_exception; std::atomic is_initialized = false; diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 8e5d1fd757f..9866d95f5ef 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -65,9 +65,9 @@ FileSegment::FileSegment( , queue_iterator(queue_iterator_) , cache(cache_) #ifdef ABORT_ON_LOGICAL_ERROR - , log(&Poco::Logger::get(fmt::format("FileSegment({}) : {}", key_.toString(), range().toString()))) + , log(getLogger(fmt::format("FileSegment({}) : {}", key_.toString(), range().toString()))) #else - , log(&Poco::Logger::get("FileSegment")) + , log(getLogger("FileSegment")) #endif { /// On creation, file segment state can be EMPTY, DOWNLOADED, DOWNLOADING. diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index cb718bcdd2e..9a2e243131b 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -269,7 +269,7 @@ private: FileCache * cache; std::condition_variable cv; - Poco::Logger * log; + LoggerPtr log; std::atomic hits_count = 0; /// cache hits. std::atomic ref_count = 0; /// Used for getting snapshot state diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index 998b11723d8..a74a4b8b621 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -71,7 +71,7 @@ private: friend class SLRUFileCachePriority; LRUQueue queue; - Poco::Logger * log = &Poco::Logger::get("LRUFileCachePriority"); + LoggerPtr log = getLogger("LRUFileCachePriority"); StatePtr state; void updateElementsCount(int64_t num); diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 953413a8ef2..b6f04807ad8 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -153,7 +153,7 @@ std::string KeyMetadata::getFileSegmentPath(const FileSegment & file_segment) co return cache_metadata->getFileSegmentPath(key, file_segment.offset(), file_segment.getKind(), user); } -Poco::Logger * KeyMetadata::logger() const +LoggerPtr KeyMetadata::logger() const { return cache_metadata->log; } @@ -167,7 +167,7 @@ CacheMetadata::CacheMetadata( , cleanup_queue(std::make_shared()) , download_queue(std::make_shared(background_download_queue_size_limit_)) , write_cache_per_user_directory(write_cache_per_user_directory_) - , log(&Poco::Logger::get("CacheMetadata")) + , log(getLogger("CacheMetadata")) , download_threads_num(background_download_threads_) { } diff --git a/src/Interpreters/Cache/Metadata.h b/src/Interpreters/Cache/Metadata.h index 3003ad74e18..c02127cdef3 100644 --- a/src/Interpreters/Cache/Metadata.h +++ b/src/Interpreters/Cache/Metadata.h @@ -99,7 +99,7 @@ private: std::atomic created_base_directory = false; LockedKeyPtr lockNoStateCheck(); - Poco::Logger * logger() const; + LoggerPtr logger() const; bool addToDownloadQueue(FileSegmentPtr file_segment); void addToCleanupQueue(); }; @@ -177,7 +177,7 @@ private: const DownloadQueuePtr download_queue; const bool write_cache_per_user_directory; - Poco::Logger * log; + LoggerPtr log; mutable std::shared_mutex key_prefix_directory_mutex; struct MetadataBucket : public std::unordered_map diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index 2dd4887dd20..c574f3085e3 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -156,7 +156,7 @@ public: Cache::MappedPtr query_result TSA_GUARDED_BY(mutex) = std::make_shared(); std::atomic skip_insert = false; bool was_finalized = false; - Poco::Logger * logger = &Poco::Logger::get("QueryCache"); + LoggerPtr logger = getLogger("QueryCache"); Writer(Cache & cache_, const Key & key_, size_t max_entry_size_in_bytes_, size_t max_entry_size_in_rows_, @@ -183,7 +183,7 @@ public: std::unique_ptr source_from_chunks; std::unique_ptr source_from_chunks_totals; std::unique_ptr source_from_chunks_extremes; - Poco::Logger * logger = &Poco::Logger::get("QueryCache"); + LoggerPtr logger = getLogger("QueryCache"); friend class QueryCache; /// for createReader() }; diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.h b/src/Interpreters/Cache/SLRUFileCachePriority.h index e949979ae61..d97fa80a6c7 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.h +++ b/src/Interpreters/Cache/SLRUFileCachePriority.h @@ -58,7 +58,7 @@ private: double size_ratio; LRUFileCachePriority protected_queue; LRUFileCachePriority probationary_queue; - Poco::Logger * log = &Poco::Logger::get("SLRUFileCachePriority"); + LoggerPtr log = getLogger("SLRUFileCachePriority"); void increasePriority(SLRUIterator & iterator, const CacheGuard::Lock & lock); }; diff --git a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp index bf5b8712fb7..751e78a0a2d 100644 --- a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp +++ b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp @@ -74,7 +74,7 @@ void WriteBufferToFileSegment::nextImpl() } catch (...) { - LOG_WARNING(&Poco::Logger::get("WriteBufferToFileSegment"), "Failed to write to the underlying buffer ({})", file_segment->getInfoForLog()); + LOG_WARNING(getLogger("WriteBufferToFileSegment"), "Failed to write to the underlying buffer ({})", file_segment->getInfoForLog()); throw; } diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index d0b00056cb4..52b74597c4b 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -116,7 +116,7 @@ ClusterDiscovery::ClusterDiscovery( const String & config_prefix) : context(Context::createCopy(context_)) , current_node_name(toString(ServerUUID::get())) - , log(&Poco::Logger::get("ClusterDiscovery")) + , log(getLogger("ClusterDiscovery")) { LOG_DEBUG(log, "Cluster discovery is enabled"); @@ -553,7 +553,7 @@ bool ClusterDiscovery::NodeInfo::parse(const String & data, NodeInfo & result) else { LOG_ERROR( - &Poco::Logger::get("ClusterDiscovery"), + getLogger("ClusterDiscovery"), "Unsupported version '{}' of data in zk node '{}'", ver, data.size() < 1024 ? data : "[data too long]"); } @@ -561,7 +561,7 @@ bool ClusterDiscovery::NodeInfo::parse(const String & data, NodeInfo & result) catch (Poco::Exception & e) { LOG_WARNING( - &Poco::Logger::get("ClusterDiscovery"), + getLogger("ClusterDiscovery"), "Can't parse '{}' from node: {}", data.size() < 1024 ? data : "[data too long]", e.displayText()); return false; diff --git a/src/Interpreters/ClusterDiscovery.h b/src/Interpreters/ClusterDiscovery.h index 8083fb6db41..756ed3d8d9b 100644 --- a/src/Interpreters/ClusterDiscovery.h +++ b/src/Interpreters/ClusterDiscovery.h @@ -152,7 +152,7 @@ private: bool is_initialized = false; ThreadFromGlobalPool main_thread; - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index f8a070a6fde..f0592735caf 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -175,7 +175,7 @@ void SelectStreamFactory::createForShard( ProfileEvents::increment(ProfileEvents::DistributedConnectionMissingTable); if (shard_info.hasRemoteConnections()) { - LOG_WARNING(&Poco::Logger::get("ClusterProxy::SelectStreamFactory"), + LOG_WARNING(getLogger("ClusterProxy::SelectStreamFactory"), "There is no table {} on local replica of shard {}, will try remote replicas.", main_table.getNameForLogs(), shard_info.shard_num); emplace_remote_stream(); @@ -213,7 +213,7 @@ void SelectStreamFactory::createForShard( /// If we reached this point, local replica is stale. ProfileEvents::increment(ProfileEvents::DistributedConnectionStaleReplica); - LOG_WARNING(&Poco::Logger::get("ClusterProxy::SelectStreamFactory"), "Local replica of shard {} is stale (delay: {}s.)", shard_info.shard_num, local_delay); + LOG_WARNING(getLogger("ClusterProxy::SelectStreamFactory"), "Local replica of shard {} is stale (delay: {}s.)", shard_info.shard_num, local_delay); if (!settings.fallback_to_stale_replicas_for_distributed_queries) { diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 5865e669e47..35451e1d774 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -42,7 +42,7 @@ ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, const Settings & settings, const StorageID & main_table, ASTPtr additional_filter_ast, - Poco::Logger * log, + LoggerPtr log, const DistributedSettings * distributed_settings) { Settings new_settings = settings; @@ -202,7 +202,7 @@ void executeQuery( const StorageID & main_table, const ASTPtr & table_func_ptr, SelectStreamFactory & stream_factory, - Poco::Logger * log, + LoggerPtr log, const ASTPtr & query_ast, ContextPtr context, const SelectQueryInfo & query_info, @@ -347,14 +347,14 @@ void executeQueryWithParallelReplicas( if (settings.use_hedged_requests.changed) { LOG_WARNING( - &Poco::Logger::get("executeQueryWithParallelReplicas"), + getLogger("executeQueryWithParallelReplicas"), "Setting 'use_hedged_requests' explicitly with enabled 'allow_experimental_parallel_reading_from_replicas' has no effect. " "Hedged connections are not used for parallel reading from replicas"); } else { LOG_INFO( - &Poco::Logger::get("executeQueryWithParallelReplicas"), + getLogger("executeQueryWithParallelReplicas"), "Disabling 'use_hedged_requests' in favor of 'allow_experimental_parallel_reading_from_replicas'. Hedged connections are " "not used for parallel reading from replicas"); } @@ -390,7 +390,7 @@ void executeQueryWithParallelReplicas( chassert(shard_count == not_optimized_cluster->getShardsAddresses().size()); - LOG_DEBUG(&Poco::Logger::get("executeQueryWithParallelReplicas"), "Parallel replicas query in shard scope: shard_num={} cluster={}", + LOG_DEBUG(getLogger("executeQueryWithParallelReplicas"), "Parallel replicas query in shard scope: shard_num={} cluster={}", shard_num, not_optimized_cluster->getName()); // get cluster for shard specified by shard_num @@ -417,7 +417,7 @@ void executeQueryWithParallelReplicas( getThrottler(new_context), std::move(scalars), std::move(external_tables), - &Poco::Logger::get("ReadFromParallelRemoteReplicasStep"), + getLogger("ReadFromParallelRemoteReplicasStep"), std::move(storage_limits)); query_plan.addStep(std::move(read_from_remote)); diff --git a/src/Interpreters/ClusterProxy/executeQuery.h b/src/Interpreters/ClusterProxy/executeQuery.h index a19ece0bbdc..bbc3c6c9e49 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.h +++ b/src/Interpreters/ClusterProxy/executeQuery.h @@ -43,7 +43,7 @@ ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, const Settings & settings, const StorageID & main_table, ASTPtr additional_filter_ast = nullptr, - Poco::Logger * log = nullptr, + LoggerPtr log = nullptr, const DistributedSettings * distributed_settings = nullptr); using AdditionalShardFilterGenerator = std::function; @@ -57,7 +57,7 @@ void executeQuery( const StorageID & main_table, const ASTPtr & table_func_ptr, SelectStreamFactory & stream_factory, - Poco::Logger * log, + LoggerPtr log, const ASTPtr & query_ast, ContextPtr context, const SelectQueryInfo & query_info, diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 9778a02f1fa..2472fbbe596 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -198,7 +198,7 @@ namespace ErrorCodes */ struct ContextSharedPart : boost::noncopyable { - Poco::Logger * log = &Poco::Logger::get("Context"); + LoggerPtr log = getLogger("Context"); /// For access of most of shared objects. mutable ContextSharedMutex mutex; @@ -1011,7 +1011,7 @@ void Context::setFilesystemCacheUser(const String & user) shared->filesystem_cache_user = user; } -static void setupTmpPath(Poco::Logger * log, const std::string & path) +static void setupTmpPath(LoggerPtr log, const std::string & path) try { LOG_DEBUG(log, "Setting up {} to store temporary data in it", path); @@ -4292,7 +4292,7 @@ void Context::setDefaultProfiles(const Poco::Util::AbstractConfiguration & confi shared->system_profile_name = config.getString("system_profile", shared->default_profile_name); setCurrentProfile(shared->system_profile_name); - applySettingsQuirks(settings, &Poco::Logger::get("SettingsQuirks")); + applySettingsQuirks(settings, getLogger("SettingsQuirks")); shared->buffer_profile_name = config.getString("buffer_profile", shared->system_profile_name); buffer_context = Context::createCopy(shared_from_this()); diff --git a/src/Interpreters/CrossToInnerJoinVisitor.cpp b/src/Interpreters/CrossToInnerJoinVisitor.cpp index 005450c2a2c..42af164f4ad 100644 --- a/src/Interpreters/CrossToInnerJoinVisitor.cpp +++ b/src/Interpreters/CrossToInnerJoinVisitor.cpp @@ -249,7 +249,7 @@ void CrossToInnerJoinMatcher::visit(ASTSelectQuery & select, ASTPtr &, Data & da ASTPtr on_expr = makeOnExpression(expr_it->second); if (rewritten = joined.rewriteCrossToInner(on_expr); rewritten) { - LOG_DEBUG(&Poco::Logger::get("CrossToInnerJoin"), "Rewritten '{}' to '{}'", query_before, queryToString(*joined.tableJoin())); + LOG_DEBUG(getLogger("CrossToInnerJoin"), "Rewritten '{}' to '{}'", query_before, queryToString(*joined.tableJoin())); } } diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index d418be51cc5..90eec421abf 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -215,7 +215,7 @@ ContextMutablePtr DDLTaskBase::makeQueryContext(ContextPtr from_context, const Z } -bool DDLTask::findCurrentHostID(ContextPtr global_context, Poco::Logger * log, const ZooKeeperPtr & zookeeper, const std::optional & config_host_name) +bool DDLTask::findCurrentHostID(ContextPtr global_context, LoggerPtr log, const ZooKeeperPtr & zookeeper, const std::optional & config_host_name) { bool host_in_hostlist = false; std::exception_ptr first_exception = nullptr; @@ -312,7 +312,7 @@ bool DDLTask::findCurrentHostID(ContextPtr global_context, Poco::Logger * log, c return host_in_hostlist; } -void DDLTask::setClusterInfo(ContextPtr context, Poco::Logger * log) +void DDLTask::setClusterInfo(ContextPtr context, LoggerPtr log) { auto * query_on_cluster = dynamic_cast(query.get()); if (!query_on_cluster) diff --git a/src/Interpreters/DDLTask.h b/src/Interpreters/DDLTask.h index bc45b46bf0f..5a8a5bfb184 100644 --- a/src/Interpreters/DDLTask.h +++ b/src/Interpreters/DDLTask.h @@ -146,9 +146,9 @@ struct DDLTask : public DDLTaskBase { DDLTask(const String & name, const String & path) : DDLTaskBase(name, path) {} - bool findCurrentHostID(ContextPtr global_context, Poco::Logger * log, const ZooKeeperPtr & zookeeper, const std::optional & config_host_name); + bool findCurrentHostID(ContextPtr global_context, LoggerPtr log, const ZooKeeperPtr & zookeeper, const std::optional & config_host_name); - void setClusterInfo(ContextPtr context, Poco::Logger * log); + void setClusterInfo(ContextPtr context, LoggerPtr log); String getShardID() const override; diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index c0611dfaf7d..fd807d54eff 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -79,7 +79,7 @@ DDLWorker::DDLWorker( const CurrentMetrics::Metric * max_entry_metric_, const CurrentMetrics::Metric * max_pushed_entry_metric_) : context(Context::createCopy(context_)) - , log(&Poco::Logger::get(logger_name)) + , log(getLogger(logger_name)) , pool_size(pool_size_) , max_entry_metric(max_entry_metric_) , max_pushed_entry_metric(max_pushed_entry_metric_) diff --git a/src/Interpreters/DDLWorker.h b/src/Interpreters/DDLWorker.h index adc9a491d81..9eb6606e73c 100644 --- a/src/Interpreters/DDLWorker.h +++ b/src/Interpreters/DDLWorker.h @@ -151,7 +151,7 @@ protected: void runCleanupThread(); ContextMutablePtr context; - Poco::Logger * log; + LoggerPtr log; std::optional config_host_name; /// host_name from config diff --git a/src/Interpreters/DNSCacheUpdater.cpp b/src/Interpreters/DNSCacheUpdater.cpp index c6a38d85acb..4769395acaa 100644 --- a/src/Interpreters/DNSCacheUpdater.cpp +++ b/src/Interpreters/DNSCacheUpdater.cpp @@ -24,7 +24,7 @@ void DNSCacheUpdater::run() /// Reload cluster config if IP of any host has been changed since last update. if (resolver.updateCache(max_consecutive_failures)) { - LOG_INFO(&Poco::Logger::get("DNSCacheUpdater"), "IPs of some hosts have been changed. Will reload cluster config."); + LOG_INFO(getLogger("DNSCacheUpdater"), "IPs of some hosts have been changed. Will reload cluster config."); try { getContext()->reloadClusterConfig(); @@ -45,7 +45,7 @@ void DNSCacheUpdater::run() void DNSCacheUpdater::start() { - LOG_INFO(&Poco::Logger::get("DNSCacheUpdater"), "Update period {} seconds", update_period_seconds); + LOG_INFO(getLogger("DNSCacheUpdater"), "Update period {} seconds", update_period_seconds); task_handle->activateAndSchedule(); } diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 138ec588202..87985d1d12b 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -827,7 +827,7 @@ DatabaseCatalog::DatabaseCatalog(ContextMutablePtr global_context_) , referential_dependencies{"ReferentialDeps"} , loading_dependencies{"LoadingDeps"} , view_dependencies{"ViewDeps"} - , log(&Poco::Logger::get("DatabaseCatalog")) + , log(getLogger("DatabaseCatalog")) , first_async_drop_in_queue(tables_marked_dropped.end()) { } diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 19882b0b828..beb73e3ef96 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -318,7 +318,7 @@ private: /// View dependencies between a source table and its view. TablesDependencyGraph view_dependencies TSA_GUARDED_BY(databases_mutex); - Poco::Logger * log; + LoggerPtr log; std::atomic_bool is_shutting_down = false; diff --git a/src/Interpreters/DirectJoin.cpp b/src/Interpreters/DirectJoin.cpp index 431f216436d..3255b56b3be 100644 --- a/src/Interpreters/DirectJoin.cpp +++ b/src/Interpreters/DirectJoin.cpp @@ -67,7 +67,7 @@ DirectKeyValueJoin::DirectKeyValueJoin(std::shared_ptr table_join_, : table_join(table_join_) , storage(storage_) , right_sample_block(right_sample_block_) - , log(&Poco::Logger::get("DirectKeyValueJoin")) + , log(getLogger("DirectKeyValueJoin")) { if (!table_join->oneDisjunct() || table_join->getOnlyClause().key_names_left.size() != 1 || diff --git a/src/Interpreters/DirectJoin.h b/src/Interpreters/DirectJoin.h index 5f664314818..ef8d12a1b8f 100644 --- a/src/Interpreters/DirectJoin.h +++ b/src/Interpreters/DirectJoin.h @@ -60,7 +60,7 @@ private: Block right_sample_block; Block right_sample_block_with_storage_column_names; Block sample_block_with_columns_to_add; - Poco::Logger * log; + LoggerPtr log; }; diff --git a/src/Interpreters/EmbeddedDictionaries.cpp b/src/Interpreters/EmbeddedDictionaries.cpp index 6c0ccce66b5..1435d16cb07 100644 --- a/src/Interpreters/EmbeddedDictionaries.cpp +++ b/src/Interpreters/EmbeddedDictionaries.cpp @@ -125,7 +125,7 @@ EmbeddedDictionaries::EmbeddedDictionaries( ContextPtr context_, const bool throw_on_error) : WithContext(context_) - , log(&Poco::Logger::get("EmbeddedDictionaries")) + , log(getLogger("EmbeddedDictionaries")) , geo_dictionaries_loader(std::move(geo_dictionaries_loader_)) , reload_period(getContext()->getConfigRef().getInt("builtin_dictionaries_reload_interval", 3600)) { diff --git a/src/Interpreters/EmbeddedDictionaries.h b/src/Interpreters/EmbeddedDictionaries.h index e71098636fe..b537146e92d 100644 --- a/src/Interpreters/EmbeddedDictionaries.h +++ b/src/Interpreters/EmbeddedDictionaries.h @@ -24,7 +24,7 @@ class GeoDictionariesLoader; class EmbeddedDictionaries : WithContext { private: - Poco::Logger * log; + LoggerPtr log; MultiVersion regions_hierarchies; MultiVersion regions_names; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 5c628436d60..fefbd67bfc1 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -120,7 +120,7 @@ bool allowEarlyConstantFolding(const ActionsDAG & actions, const Settings & sett return true; } -Poco::Logger * getLogger() { return &Poco::Logger::get("ExpressionAnalyzer"); } +LoggerPtr getLogger() { return ::getLogger("ExpressionAnalyzer"); } } diff --git a/src/Interpreters/ExpressionJIT.cpp b/src/Interpreters/ExpressionJIT.cpp index 0eacb598fbe..16275b23053 100644 --- a/src/Interpreters/ExpressionJIT.cpp +++ b/src/Interpreters/ExpressionJIT.cpp @@ -38,10 +38,9 @@ static CHJIT & getJITInstance() return jit; } -static Poco::Logger * getLogger() +static LoggerPtr getLogger() { - static Poco::Logger & logger = Poco::Logger::get("ExpressionJIT"); - return &logger; + return ::getLogger("ExpressionJIT"); } class CompiledFunctionHolder : public CompiledExpressionCacheEntry diff --git a/src/Interpreters/ExternalDictionariesLoader.cpp b/src/Interpreters/ExternalDictionariesLoader.cpp index 46171c95cb0..74984de0064 100644 --- a/src/Interpreters/ExternalDictionariesLoader.cpp +++ b/src/Interpreters/ExternalDictionariesLoader.cpp @@ -22,7 +22,7 @@ namespace ErrorCodes /// Must not acquire Context lock in constructor to avoid possibility of deadlocks. ExternalDictionariesLoader::ExternalDictionariesLoader(ContextPtr global_context_) - : ExternalLoader("external dictionary", &Poco::Logger::get("ExternalDictionariesLoader")) + : ExternalLoader("external dictionary", getLogger("ExternalDictionariesLoader")) , WithContext(global_context_) { setConfigSettings({"dictionary", "name", "database", "uuid"}); diff --git a/src/Interpreters/ExternalLoader.cpp b/src/Interpreters/ExternalLoader.cpp index 56d480d8735..36664cbd06f 100644 --- a/src/Interpreters/ExternalLoader.cpp +++ b/src/Interpreters/ExternalLoader.cpp @@ -95,7 +95,7 @@ namespace class ExternalLoader::LoadablesConfigReader : private boost::noncopyable { public: - LoadablesConfigReader(const String & type_name_, Poco::Logger * log_) + LoadablesConfigReader(const String & type_name_, LoggerPtr log_) : type_name(type_name_), log(log_) { } @@ -377,7 +377,7 @@ private: } const String type_name; - Poco::Logger * log; + LoggerPtr log; std::mutex mutex; ExternalLoaderConfigSettings settings; @@ -401,7 +401,7 @@ public: LoadingDispatcher( const CreateObjectFunction & create_object_function_, const String & type_name_, - Poco::Logger * log_) + LoggerPtr log_) : create_object(create_object_function_) , type_name(type_name_) , log(log_) @@ -1193,7 +1193,7 @@ private: const CreateObjectFunction create_object; const String type_name; - Poco::Logger * log; + LoggerPtr log; mutable std::mutex mutex; std::condition_variable event; @@ -1273,7 +1273,7 @@ private: }; -ExternalLoader::ExternalLoader(const String & type_name_, Poco::Logger * log_) +ExternalLoader::ExternalLoader(const String & type_name_, LoggerPtr log_) : config_files_reader(std::make_unique(type_name_, log_)) , loading_dispatcher(std::make_unique( [this](auto && a, auto && b, auto && c) { return createObject(a, b, c); }, diff --git a/src/Interpreters/ExternalLoader.h b/src/Interpreters/ExternalLoader.h index 49b4ea77e0d..a5d83bdab50 100644 --- a/src/Interpreters/ExternalLoader.h +++ b/src/Interpreters/ExternalLoader.h @@ -8,6 +8,7 @@ #include #include #include +#include #include namespace Poco { class Logger; } @@ -84,7 +85,7 @@ public: template static constexpr bool is_vector_load_result_type = std::is_same_v || std::is_same_v; - ExternalLoader(const String & type_name_, Poco::Logger * log); + ExternalLoader(const String & type_name_, LoggerPtr log); virtual ~ExternalLoader(); /// Adds a repository which will be used to read configurations from. @@ -230,7 +231,7 @@ private: std::unique_ptr periodic_updater; const String type_name; - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/src/Interpreters/FullSortingMergeJoin.h b/src/Interpreters/FullSortingMergeJoin.h index 3fc9f8920ed..7688d44f7a9 100644 --- a/src/Interpreters/FullSortingMergeJoin.h +++ b/src/Interpreters/FullSortingMergeJoin.h @@ -25,7 +25,7 @@ public: : table_join(table_join_) , right_sample_block(right_sample_block_) { - LOG_TRACE(&Poco::Logger::get("FullSortingMergeJoin"), "Will use full sorting merge join"); + LOG_TRACE(getLogger("FullSortingMergeJoin"), "Will use full sorting merge join"); } std::string getName() const override { return "FullSortingMergeJoin"; } diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 26d666a8913..5fb92a68a29 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -121,7 +121,7 @@ class GraceHashJoin::FileBucket : boost::noncopyable public: using BucketLock = std::unique_lock; - explicit FileBucket(size_t bucket_index_, TemporaryFileStream & left_file_, TemporaryFileStream & right_file_, Poco::Logger * log_) + explicit FileBucket(size_t bucket_index_, TemporaryFileStream & left_file_, TemporaryFileStream & right_file_, LoggerPtr log_) : idx{bucket_index_} , left_file{left_file_} , right_file{right_file_} @@ -223,7 +223,7 @@ private: std::atomic state; - Poco::Logger * log; + LoggerPtr log; }; namespace @@ -261,7 +261,7 @@ GraceHashJoin::GraceHashJoin( const Block & right_sample_block_, TemporaryDataOnDiskScopePtr tmp_data_, bool any_take_last_row_) - : log{&Poco::Logger::get("GraceHashJoin")} + : log{getLogger("GraceHashJoin")} , context{context_} , table_join{std::move(table_join_)} , left_sample_block{left_sample_block_} @@ -403,7 +403,7 @@ void GraceHashJoin::addBuckets(const size_t bucket_count) catch (...) { LOG_ERROR( - &Poco::Logger::get("GraceHashJoin"), + getLogger("GraceHashJoin"), "Can't create bucket {} due to error: {}", current_size + i, getCurrentExceptionMessage(false)); diff --git a/src/Interpreters/GraceHashJoin.h b/src/Interpreters/GraceHashJoin.h index 2cadeee10b9..ff396683230 100644 --- a/src/Interpreters/GraceHashJoin.h +++ b/src/Interpreters/GraceHashJoin.h @@ -120,7 +120,7 @@ private: /// Structure block to store in the HashJoin according to sample_block. Block prepareRightBlock(const Block & block); - Poco::Logger * log; + LoggerPtr log; ContextPtr context; std::shared_ptr table_join; Block left_sample_block; diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 467cc4c2531..33dc178ca00 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -245,7 +245,7 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s , right_sample_block(right_sample_block_) , max_joined_block_rows(table_join->maxJoinedBlockRows()) , instance_log_id(!instance_id_.empty() ? "(" + instance_id_ + ") " : "") - , log(&Poco::Logger::get("HashJoin")) + , log(getLogger("HashJoin")) { LOG_TRACE(log, "{}Keys: {}, datatype: {}, kind: {}, strictness: {}, right header: {}", instance_log_id, TableJoin::formatClauses(table_join->getClauses(), true), data->type, kind, strictness, right_sample_block.dumpStructure()); diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 17f003adc4b..29bb9070009 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -446,7 +446,7 @@ private: /// Several instances can be created, for example, in GraceHashJoin to handle different buckets String instance_log_id; - Poco::Logger * log; + LoggerPtr log; /// Should be set via setLock to protect hash table from modification from StorageJoin /// If set HashJoin instance is not available for modification (addBlockToJoin) diff --git a/src/Interpreters/InternalTextLogsQueue.cpp b/src/Interpreters/InternalTextLogsQueue.cpp index 3be58a11beb..ca8461937ac 100644 --- a/src/Interpreters/InternalTextLogsQueue.cpp +++ b/src/Interpreters/InternalTextLogsQueue.cpp @@ -43,7 +43,7 @@ void InternalTextLogsQueue::pushBlock(Block && log_block) if (blocksHaveEqualStructure(sample_block, log_block)) (void)(emplace(log_block.mutateColumns())); else - LOG_WARNING(&Poco::Logger::get("InternalTextLogsQueue"), "Log block have different structure"); + LOG_WARNING(getLogger("InternalTextLogsQueue"), "Log block have different structure"); } std::string_view InternalTextLogsQueue::getPriorityName(int priority) diff --git a/src/Interpreters/InterpreterCheckQuery.cpp b/src/Interpreters/InterpreterCheckQuery.cpp index 0cc4afd62f2..98a281bd5ad 100644 --- a/src/Interpreters/InterpreterCheckQuery.cpp +++ b/src/Interpreters/InterpreterCheckQuery.cpp @@ -149,7 +149,7 @@ private: class TableCheckSource : public ISource { public: - TableCheckSource(Strings databases_, ContextPtr context_, Poco::Logger * log_) + TableCheckSource(Strings databases_, ContextPtr context_, LoggerPtr log_) : ISource(getSingleValueBlock(0)) , databases(databases_) , context(context_) @@ -157,7 +157,7 @@ public: { } - TableCheckSource(std::shared_ptr table_check_task_, Poco::Logger * log_) + TableCheckSource(std::shared_ptr table_check_task_, LoggerPtr log_) : ISource(getSingleValueBlock(0)) , table_check_task(table_check_task_) , log(log_) @@ -260,14 +260,14 @@ private: ContextPtr context; - Poco::Logger * log; + LoggerPtr log; }; /// Receives TableCheckTask and returns CheckResult converted to sinle-row chunk class TableCheckWorkerProcessor : public ISimpleTransform { public: - TableCheckWorkerProcessor(bool with_table_name_, Poco::Logger * log_) + TableCheckWorkerProcessor(bool with_table_name_, LoggerPtr log_) : ISimpleTransform(getSingleValueBlock(0), getHeaderForCheckResult(with_table_name_), true) , with_table_name(with_table_name_) , log(log_) @@ -308,7 +308,7 @@ private: /// If true, then output will contain columns with database and table names bool with_table_name; - Poco::Logger * log; + LoggerPtr log; }; /// Accumulates all results and returns single value diff --git a/src/Interpreters/InterpreterCheckQuery.h b/src/Interpreters/InterpreterCheckQuery.h index 5ffd1d4298f..4bba3ed780c 100644 --- a/src/Interpreters/InterpreterCheckQuery.h +++ b/src/Interpreters/InterpreterCheckQuery.h @@ -19,7 +19,7 @@ public: private: ASTPtr query_ptr; - Poco::Logger * log = &Poco::Logger::get("InterpreterCheckQuery"); + LoggerPtr log = getLogger("InterpreterCheckQuery"); }; } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index d002cc6d980..9ce1c856622 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1214,7 +1214,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) } else if (create.attach && !create.attach_short_syntax && getContext()->getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY) { - auto * log = &Poco::Logger::get("InterpreterCreateQuery"); + auto log = getLogger("InterpreterCreateQuery"); LOG_WARNING(log, "ATTACH TABLE query with full table definition is not recommended: " "use either ATTACH TABLE {}; to attach existing table " "or CREATE TABLE {} ; to create new table " @@ -1455,7 +1455,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, /// so the existing directory probably contains some leftovers from previous unsuccessful attempts to create the table fs::path trash_path = fs::path{getContext()->getPath()} / "trash" / data_path / getHexUIntLowercase(thread_local_rng()); - LOG_WARNING(&Poco::Logger::get("InterpreterCreateQuery"), "Directory for {} data {} already exists. Will move it to {}", + LOG_WARNING(getLogger("InterpreterCreateQuery"), "Directory for {} data {} already exists. Will move it to {}", Poco::toLower(storage_name), String(data_path), trash_path); fs::create_directories(trash_path.parent_path()); renameNoReplace(full_data_path, trash_path); diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index 5efffdaa194..3431cd5e568 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -161,7 +161,7 @@ public: if (curr_process.processed) continue; - LOG_DEBUG(&Poco::Logger::get("KillQuery"), "Will kill query {} (synchronously)", curr_process.query_id); + LOG_DEBUG(getLogger("KillQuery"), "Will kill query {} (synchronously)", curr_process.query_id); auto code = process_list.sendCancelToQuery(curr_process.query_id, curr_process.user, true); @@ -229,7 +229,7 @@ BlockIO InterpreterKillQueryQuery::execute() for (const auto & query_desc : queries_to_stop) { if (!query.test) - LOG_DEBUG(&Poco::Logger::get("KillQuery"), "Will kill query {} (asynchronously)", query_desc.query_id); + LOG_DEBUG(getLogger("KillQuery"), "Will kill query {} (asynchronously)", query_desc.query_id); auto code = (query.test) ? CancellationCode::Unknown : process_list.sendCancelToQuery(query_desc.query_id, query_desc.user, true); insertResultRow(query_desc.source_num, code, processes_block, header, res_columns); } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 4e4e25617e4..2d994483ba8 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -381,7 +381,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( : IInterpreterUnionOrSelectQuery(options_.modify_inplace ? query_ptr_ : query_ptr_->clone(), context_, options_) , storage(storage_) , input_pipe(std::move(input_pipe_)) - , log(&Poco::Logger::get("InterpreterSelectQuery")) + , log(getLogger("InterpreterSelectQuery")) , metadata_snapshot(metadata_snapshot_) , prepared_sets(prepared_sets_) { diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index fbb53d71755..c307e457649 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -253,7 +253,7 @@ private: /// Used when we read from prepared input, not table or subquery. std::optional input_pipe; - Poco::Logger * log; + LoggerPtr log; StorageMetadataPtr metadata_snapshot; StorageSnapshotPtr storage_snapshot; diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 8a242cee213..9a80553f149 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -221,7 +221,7 @@ void InterpreterSystemQuery::startStopAction(StorageActionBlockType action_type, void InterpreterSystemQuery::startStopActionInDatabase(StorageActionBlockType action_type, bool start, const String & database_name, const DatabasePtr & database, - const ContextPtr & local_context, Poco::Logger * log) + const ContextPtr & local_context, LoggerPtr log) { auto manager = local_context->getActionLocksManager(); auto access = local_context->getAccess(); @@ -251,7 +251,7 @@ void InterpreterSystemQuery::startStopActionInDatabase(StorageActionBlockType ac InterpreterSystemQuery::InterpreterSystemQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) - : WithMutableContext(context_), query_ptr(query_ptr_->clone()), log(&Poco::Logger::get("InterpreterSystemQuery")) + : WithMutableContext(context_), query_ptr(query_ptr_->clone()), log(getLogger("InterpreterSystemQuery")) { } diff --git a/src/Interpreters/InterpreterSystemQuery.h b/src/Interpreters/InterpreterSystemQuery.h index 89de7402b4d..1419c430aca 100644 --- a/src/Interpreters/InterpreterSystemQuery.h +++ b/src/Interpreters/InterpreterSystemQuery.h @@ -43,11 +43,11 @@ public: static void startStopActionInDatabase(StorageActionBlockType action_type, bool start, const String & database_name, const DatabasePtr & database, - const ContextPtr & local_context, Poco::Logger * log); + const ContextPtr & local_context, LoggerPtr log); private: ASTPtr query_ptr; - Poco::Logger * log = nullptr; + LoggerPtr log = nullptr; StorageID table_id = StorageID::createEmpty(); /// Will be set up if query contains table name VolumePtr volume_ptr; diff --git a/src/Interpreters/InterserverCredentials.cpp b/src/Interpreters/InterserverCredentials.cpp index 094b58789a8..c344732a262 100644 --- a/src/Interpreters/InterserverCredentials.cpp +++ b/src/Interpreters/InterserverCredentials.cpp @@ -35,7 +35,7 @@ InterserverCredentials::CurrentCredentials InterserverCredentials::parseCredenti const Poco::Util::AbstractConfiguration & config, const std::string & root_tag) { - auto * log = &Poco::Logger::get("InterserverCredentials"); + auto log = getLogger("InterserverCredentials"); CurrentCredentials store; store.emplace_back(current_user_, current_password_); if (config.getBool(root_tag + ".allow_empty", false)) diff --git a/src/Interpreters/JoinedTables.cpp b/src/Interpreters/JoinedTables.cpp index c104af770f0..9be8bf178a1 100644 --- a/src/Interpreters/JoinedTables.cpp +++ b/src/Interpreters/JoinedTables.cpp @@ -335,12 +335,12 @@ std::shared_ptr JoinedTables::makeTableJoin(const ASTSelectQuery & se auto dictionary = dictionary_helper.getDictionary(dictionary_name); if (!dictionary) { - LOG_TRACE(&Poco::Logger::get("JoinedTables"), "Can't use dictionary join: dictionary '{}' was not found", dictionary_name); + LOG_TRACE(getLogger("JoinedTables"), "Can't use dictionary join: dictionary '{}' was not found", dictionary_name); return nullptr; } if (dictionary->getSpecialKeyType() == DictionarySpecialKeyType::Range) { - LOG_TRACE(&Poco::Logger::get("JoinedTables"), "Can't use dictionary join: dictionary '{}' is a range dictionary", dictionary_name); + LOG_TRACE(getLogger("JoinedTables"), "Can't use dictionary join: dictionary '{}' is a range dictionary", dictionary_name); return nullptr; } diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 4a80e1a3c56..901c82029ee 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -492,7 +492,7 @@ MergeJoin::MergeJoin(std::shared_ptr table_join_, const Block & right , max_joined_block_rows(table_join->maxJoinedBlockRows()) , max_rows_in_right_block(table_join->maxRowsInRightBlock()) , max_files_to_merge(table_join->maxFilesToMerge()) - , log(&Poco::Logger::get("MergeJoin")) + , log(getLogger("MergeJoin")) { switch (table_join->strictness()) { diff --git a/src/Interpreters/MergeJoin.h b/src/Interpreters/MergeJoin.h index 98fae1d419f..4486c134d51 100644 --- a/src/Interpreters/MergeJoin.h +++ b/src/Interpreters/MergeJoin.h @@ -117,7 +117,7 @@ private: Names lowcard_right_keys; - Poco::Logger * log; + LoggerPtr log; void changeLeftColumns(Block & block, MutableColumns && columns) const; void addRightColumns(Block & block, MutableColumns && columns); diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 86cd2d84fa3..cc447dfef24 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -1284,7 +1284,7 @@ void MutationsInterpreter::Source::read( plan, *data, storage_snapshot, part, std::move(virtual_columns.columns_to_read), apply_deleted_mask_, filter, context_, - &Poco::Logger::get("MutationsInterpreter")); + getLogger("MutationsInterpreter")); virtual_columns.addVirtuals(plan); } diff --git a/src/Interpreters/PartLog.cpp b/src/Interpreters/PartLog.cpp index 9819b8e3ec4..a7f20a06785 100644 --- a/src/Interpreters/PartLog.cpp +++ b/src/Interpreters/PartLog.cpp @@ -271,7 +271,7 @@ bool PartLog::addNewParts( } catch (...) { - tryLogCurrentException(part_log ? part_log->log : &Poco::Logger::get("PartLog"), __PRETTY_FUNCTION__); + tryLogCurrentException(part_log ? part_log->log : getLogger("PartLog"), __PRETTY_FUNCTION__); return false; } diff --git a/src/Interpreters/PasteJoin.h b/src/Interpreters/PasteJoin.h index df7bb2f280c..f87a7021551 100644 --- a/src/Interpreters/PasteJoin.h +++ b/src/Interpreters/PasteJoin.h @@ -24,7 +24,7 @@ public: : table_join(table_join_) , right_sample_block(right_sample_block_) { - LOG_TRACE(&Poco::Logger::get("PasteJoin"), "Will use paste join"); + LOG_TRACE(getLogger("PasteJoin"), "Will use paste join"); } std::string getName() const override { return "PasteJoin"; } diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 2b84b7655b3..5b3b87114ae 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -86,7 +86,7 @@ ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr q if (!is_unlimited_query && max_size && processes.size() >= max_size) { if (queue_max_wait_ms) - LOG_WARNING(&Poco::Logger::get("ProcessList"), "Too many simultaneous queries, will wait {} ms.", queue_max_wait_ms); + LOG_WARNING(getLogger("ProcessList"), "Too many simultaneous queries, will wait {} ms.", queue_max_wait_ms); if (!queue_max_wait_ms || !have_space.wait_for(lock, std::chrono::milliseconds(queue_max_wait_ms), [&]{ return processes.size() < max_size; })) throw Exception(ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES, "Too many simultaneous queries. Maximum: {}", max_size); } @@ -295,7 +295,7 @@ ProcessListEntry::~ProcessListEntry() auto user_process_list_it = parent.user_to_queries.find(user); if (user_process_list_it == parent.user_to_queries.end()) { - LOG_ERROR(&Poco::Logger::get("ProcessList"), "Logical error: cannot find user in ProcessList"); + LOG_ERROR(getLogger("ProcessList"), "Logical error: cannot find user in ProcessList"); std::terminate(); } @@ -323,7 +323,7 @@ ProcessListEntry::~ProcessListEntry() if (!found) { - LOG_ERROR(&Poco::Logger::get("ProcessList"), "Logical error: cannot find query by query_id and pointer to ProcessListElement in ProcessListForUser"); + LOG_ERROR(getLogger("ProcessList"), "Logical error: cannot find query by query_id and pointer to ProcessListElement in ProcessListForUser"); std::terminate(); } diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index d2f9fe8b325..533f33033e3 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -265,7 +265,7 @@ private: ThreadFromGlobalPool thread; bool quit = false; - Poco::Logger * log = &Poco::Logger::get("NamedSessionsStorage"); + LoggerPtr log = getLogger("NamedSessionsStorage"); }; @@ -282,7 +282,7 @@ void Session::shutdownNamedSessions() Session::Session(const ContextPtr & global_context_, ClientInfo::Interface interface_, bool is_secure, const std::string & certificate) : auth_id(UUIDHelpers::generateV4()), global_context(global_context_), - log(&Poco::Logger::get(String{magic_enum::enum_name(interface_)} + "-Session")) + log(getLogger(String{magic_enum::enum_name(interface_)} + "-Session")) { prepared_client_info.emplace(); prepared_client_info->interface = interface_; diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 75e1414b8cb..cde000d89fa 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -123,7 +123,7 @@ private: /// to set when creating a session context SettingsChanges settings_from_auth_server; - Poco::Logger * log = nullptr; + LoggerPtr log = nullptr; }; } diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index 7136b090c42..5a65d40d89f 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -33,7 +33,7 @@ public: /// store all set elements in explicit form. /// This is needed for subsequent use for index. Set(const SizeLimits & limits_, size_t max_elements_to_fill_, bool transform_null_in_) - : log(&Poco::Logger::get("Set")), + : log(getLogger("Set")), limits(limits_), max_elements_to_fill(max_elements_to_fill_), transform_null_in(transform_null_in_), cast_cache(std::make_unique()) {} @@ -114,7 +114,7 @@ private: /// Types for set_elements. DataTypes set_elements_types; - Poco::Logger * log; + LoggerPtr log; /// Limitations on the maximum size of the set SizeLimits limits; diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 954368db312..2fb782befa1 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -125,13 +125,13 @@ std::shared_ptr createSystemLog( { if (!config.has(config_prefix)) { - LOG_DEBUG(&Poco::Logger::get("SystemLog"), + LOG_DEBUG(getLogger("SystemLog"), "Not creating {}.{} since corresponding section '{}' is missing from config", default_database_name, default_table_name, config_prefix); return {}; } - LOG_DEBUG(&Poco::Logger::get("SystemLog"), + LOG_DEBUG(getLogger("SystemLog"), "Creating {}.{} from {}", default_database_name, default_table_name, config_prefix); SystemLogSettings log_settings; @@ -143,7 +143,7 @@ std::shared_ptr createSystemLog( { /// System tables must be loaded before other tables, but loading order is undefined for all databases except `system` LOG_ERROR( - &Poco::Logger::get("SystemLog"), + getLogger("SystemLog"), "Custom database name for a system table specified in config." " Table `{}` will be created in `system` database instead of `{}`", log_settings.queue_settings.table, @@ -395,7 +395,7 @@ SystemLog::SystemLog( std::shared_ptr> queue_) : Base(settings_.queue_settings, queue_) , WithContext(context_) - , log(&Poco::Logger::get("SystemLog (" + settings_.queue_settings.database + "." + settings_.queue_settings.table + ")")) + , log(getLogger("SystemLog (" + settings_.queue_settings.database + "." + settings_.queue_settings.table + ")")) , table_id(settings_.queue_settings.database, settings_.queue_settings.table) , storage_def(settings_.engine) , create_query(serializeAST(*getCreateTableQuery())) diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 8c357e43be9..c296b91e24a 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -131,7 +131,7 @@ public: void stopFlushThread() override; protected: - Poco::Logger * log; + LoggerPtr log; using ISystemLog::is_shutdown; using ISystemLog::saving_thread; diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index efe3fd7f740..e9fa224df11 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -683,7 +683,7 @@ void TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig if (!left_type_map.empty() || !right_type_map.empty()) { LOG_TRACE( - &Poco::Logger::get("TableJoin"), + getLogger("TableJoin"), "Infer supertype for joined columns. Left: [{}], Right: [{}]", formatTypeMap(left_type_map, left_types), formatTypeMap(right_type_map, right_types)); @@ -876,7 +876,7 @@ static void addJoinConditionWithAnd(ASTPtr & current_cond, const ASTPtr & new_co void TableJoin::addJoinCondition(const ASTPtr & ast, bool is_left) { auto & cond_ast = is_left ? clauses.back().on_filter_condition_left : clauses.back().on_filter_condition_right; - LOG_TRACE(&Poco::Logger::get("TableJoin"), "Adding join condition for {} table: {} -> {}", + LOG_TRACE(getLogger("TableJoin"), "Adding join condition for {} table: {} -> {}", (is_left ? "left" : "right"), ast ? queryToString(ast) : "NULL", cond_ast ? queryToString(cond_ast) : "NULL"); addJoinConditionWithAnd(cond_ast, ast); } diff --git a/src/Interpreters/TemporaryDataOnDisk.cpp b/src/Interpreters/TemporaryDataOnDisk.cpp index 3ad72af95f9..a48e7d8e040 100644 --- a/src/Interpreters/TemporaryDataOnDisk.cpp +++ b/src/Interpreters/TemporaryDataOnDisk.cpp @@ -223,7 +223,7 @@ struct TemporaryFileStream::InputReader , in_compressed_buf(in_file_buf) , in_reader(in_compressed_buf, header_, DBMS_TCP_PROTOCOL_VERSION) { - LOG_TEST(&Poco::Logger::get("TemporaryFileStream"), "Reading {} from {}", header_.dumpStructure(), path); + LOG_TEST(getLogger("TemporaryFileStream"), "Reading {} from {}", header_.dumpStructure(), path); } explicit InputReader(const String & path, size_t size = 0) @@ -231,7 +231,7 @@ struct TemporaryFileStream::InputReader , in_compressed_buf(in_file_buf) , in_reader(in_compressed_buf, DBMS_TCP_PROTOCOL_VERSION) { - LOG_TEST(&Poco::Logger::get("TemporaryFileStream"), "Reading from {}", path); + LOG_TEST(getLogger("TemporaryFileStream"), "Reading from {}", path); } Block read() @@ -250,7 +250,7 @@ TemporaryFileStream::TemporaryFileStream(TemporaryFileOnDiskHolder file_, const , file(std::move(file_)) , out_writer(std::make_unique(std::make_unique(file->getAbsolutePath()), header)) { - LOG_TEST(&Poco::Logger::get("TemporaryFileStream"), "Writing to temporary file {}", file->getAbsolutePath()); + LOG_TEST(getLogger("TemporaryFileStream"), "Writing to temporary file {}", file->getAbsolutePath()); } TemporaryFileStream::TemporaryFileStream(FileSegmentsHolderPtr segments_, const Block & header_, TemporaryDataOnDisk * parent_) @@ -262,7 +262,7 @@ TemporaryFileStream::TemporaryFileStream(FileSegmentsHolderPtr segments_, const throw Exception(ErrorCodes::LOGICAL_ERROR, "TemporaryFileStream can be created only from single segment"); auto out_buf = std::make_unique(&segment_holder->front()); - LOG_TEST(&Poco::Logger::get("TemporaryFileStream"), "Writing to temporary file {}", out_buf->getFileName()); + LOG_TEST(getLogger("TemporaryFileStream"), "Writing to temporary file {}", out_buf->getFileName()); out_writer = std::make_unique(std::move(out_buf), header); } diff --git a/src/Interpreters/TraceCollector.cpp b/src/Interpreters/TraceCollector.cpp index 30fbe26d038..1fe11be6090 100644 --- a/src/Interpreters/TraceCollector.cpp +++ b/src/Interpreters/TraceCollector.cpp @@ -65,7 +65,7 @@ TraceCollector::~TraceCollector() if (thread.joinable()) thread.join(); else - LOG_ERROR(&Poco::Logger::get("TraceCollector"), "TraceCollector thread is malformed and cannot be joined"); + LOG_ERROR(getLogger("TraceCollector"), "TraceCollector thread is malformed and cannot be joined"); } diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index a86f6110a84..96c69536c9a 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -21,7 +21,7 @@ namespace ErrorCodes extern const int UNKNOWN_STATUS_OF_TRANSACTION; } -static void tryWriteEventToSystemLog(Poco::Logger * log, ContextPtr context, +static void tryWriteEventToSystemLog(LoggerPtr log, ContextPtr context, TransactionsInfoLogElement::Type type, const TransactionID & tid, CSN csn = Tx::UnknownCSN) try { @@ -44,7 +44,7 @@ catch (...) TransactionLog::TransactionLog() : global_context(Context::getGlobalContextInstance()) - , log(&Poco::Logger::get("TransactionLog")) + , log(getLogger("TransactionLog")) , zookeeper_path(global_context->getConfigRef().getString("transaction_log.zookeeper_path", "/clickhouse/txn")) , zookeeper_path_log(zookeeper_path + "/log") , fault_probability_before_commit(global_context->getConfigRef().getDouble("transaction_log.fault_probability_before_commit", 0)) diff --git a/src/Interpreters/TransactionLog.h b/src/Interpreters/TransactionLog.h index 6e8777d8519..58847553dfd 100644 --- a/src/Interpreters/TransactionLog.h +++ b/src/Interpreters/TransactionLog.h @@ -154,7 +154,7 @@ private: CSN getCSNImpl(const TIDHash & tid_hash, const std::atomic * failback_with_strict_load_csn = nullptr) const; const ContextPtr global_context; - Poco::Logger * const log; + LoggerPtr const log; /// The newest snapshot available for reading std::atomic latest_snapshot; diff --git a/src/Interpreters/TransactionVersionMetadata.cpp b/src/Interpreters/TransactionVersionMetadata.cpp index 01735a798b9..7bedca5d5c7 100644 --- a/src/Interpreters/TransactionVersionMetadata.cpp +++ b/src/Interpreters/TransactionVersionMetadata.cpp @@ -23,7 +23,7 @@ namespace ErrorCodes VersionMetadata::VersionMetadata() { /// It would be better to make it static, but static loggers do not work for some reason (initialization order?) - log = &Poco::Logger::get("VersionMetadata"); + log = getLogger("VersionMetadata"); } /// It can be used for introspection purposes only diff --git a/src/Interpreters/TransactionVersionMetadata.h b/src/Interpreters/TransactionVersionMetadata.h index 18ac445cc29..4309975d195 100644 --- a/src/Interpreters/TransactionVersionMetadata.h +++ b/src/Interpreters/TransactionVersionMetadata.h @@ -72,7 +72,7 @@ struct VersionMetadata String toString(bool one_line = true) const; - Poco::Logger * log; + LoggerPtr log; VersionMetadata(); }; diff --git a/src/Interpreters/TransactionsInfoLog.cpp b/src/Interpreters/TransactionsInfoLog.cpp index e893be814ca..4a413439671 100644 --- a/src/Interpreters/TransactionsInfoLog.cpp +++ b/src/Interpreters/TransactionsInfoLog.cpp @@ -92,7 +92,7 @@ void TransactionsInfoLogElement::appendToBlock(MutableColumns & columns) const } -void tryWriteEventToSystemLog(Poco::Logger * log, +void tryWriteEventToSystemLog(LoggerPtr log, TransactionsInfoLogElement::Type type, const TransactionID & tid, const TransactionInfoContext & context) try diff --git a/src/Interpreters/TransactionsInfoLog.h b/src/Interpreters/TransactionsInfoLog.h index 0a607704e74..009d1b67474 100644 --- a/src/Interpreters/TransactionsInfoLog.h +++ b/src/Interpreters/TransactionsInfoLog.h @@ -54,7 +54,7 @@ class TransactionsInfoLog : public SystemLog }; -void tryWriteEventToSystemLog(Poco::Logger * log, TransactionsInfoLogElement::Type type, +void tryWriteEventToSystemLog(LoggerPtr log, TransactionsInfoLogElement::Type type, const TransactionID & tid, const TransactionInfoContext & context); } diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 6ed3ff2f1e6..ecd021328e7 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -642,13 +642,13 @@ bool tryJoinOnConst(TableJoin & analyzed_join, const ASTPtr & on_expression, Con if (eval_const_res.value()) { /// JOIN ON 1 == 1 - LOG_DEBUG(&Poco::Logger::get("TreeRewriter"), "Join on constant executed as cross join"); + LOG_DEBUG(getLogger("TreeRewriter"), "Join on constant executed as cross join"); analyzed_join.resetToCross(); } else { /// JOIN ON 1 != 1 - LOG_DEBUG(&Poco::Logger::get("TreeRewriter"), "Join on constant executed as empty join"); + LOG_DEBUG(getLogger("TreeRewriter"), "Join on constant executed as empty join"); analyzed_join.resetKeys(); } return true; diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index 6b6054fdae3..6122ec6180a 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -221,7 +221,7 @@ private: String node_path; ContextPtr context; Stopwatch watch; - Poco::Logger * log; + LoggerPtr log; NameSet waiting_hosts; /// hosts from task host list NameSet finished_hosts; /// finished hosts from host list @@ -309,7 +309,7 @@ DDLQueryStatusSource::DDLQueryStatusSource( , node_path(zk_node_path) , context(context_) , watch(CLOCK_MONOTONIC_COARSE) - , log(&Poco::Logger::get("DDLQueryStatusSource")) + , log(getLogger("DDLQueryStatusSource")) { auto output_mode = context->getSettingsRef().distributed_ddl_output_mode; throw_on_timeout = output_mode == DistributedDDLOutputMode::THROW || output_mode == DistributedDDLOutputMode::THROW_ONLY_ACTIVE @@ -382,7 +382,7 @@ Chunk DDLQueryStatusSource::generateChunkWithUnfinishedHosts() const return Chunk(std::move(columns), unfinished_hosts.size()); } -static NameSet getOfflineHosts(const String & node_path, const NameSet & hosts_to_wait, const ZooKeeperPtr & zookeeper, Poco::Logger * log) +static NameSet getOfflineHosts(const String & node_path, const NameSet & hosts_to_wait, const ZooKeeperPtr & zookeeper, LoggerPtr log) { fs::path replicas_path; if (node_path.ends_with('/')) @@ -470,7 +470,7 @@ Chunk DDLQueryStatusSource::generate() { auto retries_ctl = ZooKeeperRetriesControl( - "executeDDLQueryOnCluster", &Poco::Logger::get("DDLQueryStatusSource"), getRetriesInfo(), context->getProcessListElement()); + "executeDDLQueryOnCluster", getLogger("DDLQueryStatusSource"), getRetriesInfo(), context->getProcessListElement()); retries_ctl.retryLoop([&]() { auto zookeeper = context->getZooKeeper(); @@ -540,7 +540,7 @@ Chunk DDLQueryStatusSource::generate() auto retries_ctl = ZooKeeperRetriesControl( "executeDDLQueryOnCluster", - &Poco::Logger::get("DDLQueryStatusSource"), + getLogger("DDLQueryStatusSource"), getRetriesInfo(), context->getProcessListElement()); retries_ctl.retryLoop([&]() diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 533d58aaa8f..a377d2e0b97 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -119,7 +119,7 @@ static void logQuery(const String & query, ContextPtr context, bool internal, Qu { if (internal) { - LOG_DEBUG(&Poco::Logger::get("executeQuery"), "(internal) {} (stage: {})", toOneLineQuery(query), QueryProcessingStage::toString(stage)); + LOG_DEBUG(getLogger("executeQuery"), "(internal) {} (stage: {})", toOneLineQuery(query), QueryProcessingStage::toString(stage)); } else { @@ -142,7 +142,7 @@ static void logQuery(const String & query, ContextPtr context, bool internal, Qu if (auto txn = context->getCurrentTransaction()) transaction_info = fmt::format(" (TID: {}, TIDH: {})", txn->tid, txn->tid.getHash()); - LOG_DEBUG(&Poco::Logger::get("executeQuery"), "(from {}{}{}){}{} {} (stage: {})", + LOG_DEBUG(getLogger("executeQuery"), "(from {}{}{}){}{} {} (stage: {})", client_info.current_address.toString(), (current_user != "default" ? ", user: " + current_user : ""), (!initial_query_id.empty() && current_query_id != initial_query_id ? ", initial_query_id: " + initial_query_id : std::string()), @@ -153,7 +153,7 @@ static void logQuery(const String & query, ContextPtr context, bool internal, Qu if (client_info.client_trace_context.trace_id != UUID()) { - LOG_TRACE(&Poco::Logger::get("executeQuery"), + LOG_TRACE(getLogger("executeQuery"), "OpenTelemetry traceparent '{}'", client_info.client_trace_context.composeTraceparentHeader()); } @@ -207,9 +207,9 @@ static void logException(ContextPtr context, QueryLogElement & elem, bool log_er elem.stack_trace); if (log_error) - LOG_ERROR(&Poco::Logger::get("executeQuery"), message); + LOG_ERROR(getLogger("executeQuery"), message); else - LOG_INFO(&Poco::Logger::get("executeQuery"), message); + LOG_INFO(getLogger("executeQuery"), message); } static void @@ -396,7 +396,7 @@ void logQueryFinish( double elapsed_seconds = static_cast(info.elapsed_microseconds) / 1000000.0; double rows_per_second = static_cast(elem.read_rows) / elapsed_seconds; LOG_DEBUG( - &Poco::Logger::get("executeQuery"), + getLogger("executeQuery"), "Read {} rows, {} in {} sec., {} rows/sec., {}/sec.", elem.read_rows, ReadableSize(elem.read_bytes), @@ -660,7 +660,7 @@ static std::tuple executeQueryImpl( /// we still have enough span logs for the execution of external queries. std::shared_ptr query_span = internal ? nullptr : std::make_shared("query"); if (query_span && query_span->trace_id != UUID{}) - LOG_TRACE(&Poco::Logger::get("executeQuery"), "Query span trace_id for opentelemetry log: {}", query_span->trace_id); + LOG_TRACE(getLogger("executeQuery"), "Query span trace_id for opentelemetry log: {}", query_span->trace_id); auto query_start_time = std::chrono::system_clock::now(); @@ -925,7 +925,7 @@ static std::tuple executeQueryImpl( bool async_insert = false; auto * queue = context->getAsynchronousInsertQueue(); - auto * logger = &Poco::Logger::get("executeQuery"); + auto logger = getLogger("executeQuery"); if (insert_query && async_insert_enabled) { @@ -1131,7 +1131,7 @@ static std::tuple executeQueryImpl( const size_t num_query_runs = query_cache->recordQueryRun(key); if (num_query_runs <= settings.query_cache_min_query_runs) { - LOG_TRACE(&Poco::Logger::get("QueryCache"), + LOG_TRACE(getLogger("QueryCache"), "Skipped insert because the query ran {} times but the minimum required number of query runs to cache the query result is {}", num_query_runs, settings.query_cache_min_query_runs); } @@ -1387,7 +1387,7 @@ void executeQuery( catch (const DB::Exception & e) { /// Ignore this exception and report the original one - LOG_WARNING(&Poco::Logger::get("executeQuery"), getExceptionMessageAndPattern(e, true)); + LOG_WARNING(getLogger("executeQuery"), getExceptionMessageAndPattern(e, true)); } } }; diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index a2d2c56c710..0b7a6dc92b0 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -156,7 +156,7 @@ static void checkIncompleteOrdinaryToAtomicConversion(ContextPtr context, const LoadTaskPtrs loadMetadata(ContextMutablePtr context, const String & default_database_name, bool async_load_databases) { - Poco::Logger * log = &Poco::Logger::get("loadMetadata"); + LoggerPtr log = getLogger("loadMetadata"); String path = context->getPath() + "metadata"; @@ -290,7 +290,7 @@ static void loadSystemDatabaseImpl(ContextMutablePtr context, const String & dat } } -static void convertOrdinaryDatabaseToAtomic(Poco::Logger * log, ContextMutablePtr context, const DatabasePtr & database, +static void convertOrdinaryDatabaseToAtomic(LoggerPtr log, ContextMutablePtr context, const DatabasePtr & database, const String & name, const String tmp_name) { /// It's kind of C++ script that creates temporary database with Atomic engine, @@ -369,7 +369,7 @@ static void convertOrdinaryDatabaseToAtomic(Poco::Logger * log, ContextMutablePt /// Can be called only during server startup when there are no queries from users. static void maybeConvertOrdinaryDatabaseToAtomic(ContextMutablePtr context, const String & database_name, LoadTaskPtrs * startup_tasks = nullptr) { - Poco::Logger * log = &Poco::Logger::get("loadMetadata"); + LoggerPtr log = getLogger("loadMetadata"); auto database = DatabaseCatalog::instance().getDatabase(database_name); if (!database) @@ -482,7 +482,7 @@ void convertDatabasesEnginesIfNeed(const LoadTaskPtrs & load_metadata, ContextMu if (!fs::exists(convert_flag_path)) return; - LOG_INFO(&Poco::Logger::get("loadMetadata"), "Found convert_ordinary_to_atomic file in flags directory, " + LOG_INFO(getLogger("loadMetadata"), "Found convert_ordinary_to_atomic file in flags directory, " "will try to convert all Ordinary databases to Atomic"); // Wait for all table to be loaded and started @@ -492,7 +492,7 @@ void convertDatabasesEnginesIfNeed(const LoadTaskPtrs & load_metadata, ContextMu if (name != DatabaseCatalog::SYSTEM_DATABASE) maybeConvertOrdinaryDatabaseToAtomic(context, name); - LOG_INFO(&Poco::Logger::get("loadMetadata"), "Conversion finished, removing convert_ordinary_to_atomic flag"); + LOG_INFO(getLogger("loadMetadata"), "Conversion finished, removing convert_ordinary_to_atomic flag"); fs::remove(convert_flag_path); } diff --git a/src/Interpreters/removeOnClusterClauseIfNeeded.cpp b/src/Interpreters/removeOnClusterClauseIfNeeded.cpp index f8df03ed830..44167fe7242 100644 --- a/src/Interpreters/removeOnClusterClauseIfNeeded.cpp +++ b/src/Interpreters/removeOnClusterClauseIfNeeded.cpp @@ -52,7 +52,7 @@ ASTPtr removeOnClusterClauseIfNeeded(const ASTPtr & query, ContextPtr context, c && context->getSettings().ignore_on_cluster_for_replicated_access_entities_queries && context->getAccessControl().containsStorage(ReplicatedAccessStorage::STORAGE_TYPE))) { - LOG_DEBUG(&Poco::Logger::get("removeOnClusterClauseIfNeeded"), "ON CLUSTER clause was ignored for query {}", query->getID()); + LOG_DEBUG(getLogger("removeOnClusterClauseIfNeeded"), "ON CLUSTER clause was ignored for query {}", query->getID()); return query_on_cluster->getRewrittenASTWithoutOnCluster(params); } diff --git a/src/Parsers/DumpASTNode.h b/src/Parsers/DumpASTNode.h index 60fcece5590..5efc0e018f4 100644 --- a/src/Parsers/DumpASTNode.h +++ b/src/Parsers/DumpASTNode.h @@ -165,7 +165,7 @@ public: : log(nullptr) { if constexpr (_enable) - log = &Poco::Logger::get("AST"); + log = getLogger("AST"); } ~DebugASTLog() @@ -177,7 +177,7 @@ public: WriteBuffer * stream() { return (_enable ? &buf : nullptr); } private: - Poco::Logger * log; + LoggerPtr log; WriteBufferFromOwnString buf; }; diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index f2def571325..b55f0e44038 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1196,7 +1196,7 @@ void Planner::buildQueryPlanIfNeeded() if (query_plan.isInitialized()) return; - LOG_TRACE(&Poco::Logger::get("Planner"), "Query {} to stage {}{}", + LOG_TRACE(getLogger("Planner"), "Query {} to stage {}{}", query_tree->formatConvertedASTForErrorMessage(), QueryProcessingStage::toString(select_query_options.to_stage), select_query_options.only_analyze ? " only analyze" : ""); @@ -1355,7 +1355,7 @@ void Planner::buildPlanForQueryNode() auto & mutable_context = planner_context->getMutableQueryContext(); mutable_context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); - LOG_DEBUG(&Poco::Logger::get("Planner"), "Disabling parallel replicas to execute a query with IN with subquery"); + LOG_DEBUG(getLogger("Planner"), "Disabling parallel replicas to execute a query with IN with subquery"); } } @@ -1382,7 +1382,7 @@ void Planner::buildPlanForQueryNode() else { LOG_DEBUG( - &Poco::Logger::get("Planner"), + getLogger("Planner"), "FINAL modifier is not supported with parallel replicas. Query will be executed without using them."); auto & mutable_context = planner_context->getMutableQueryContext(); mutable_context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); @@ -1401,7 +1401,7 @@ void Planner::buildPlanForQueryNode() else { LOG_DEBUG( - &Poco::Logger::get("Planner"), + getLogger("Planner"), "JOINs are not supported with parallel replicas. Query will be executed without using them."); auto & mutable_context = planner_context->getMutableQueryContext(); @@ -1422,7 +1422,7 @@ void Planner::buildPlanForQueryNode() query_plan = std::move(join_tree_query_plan.query_plan); used_row_policies = std::move(join_tree_query_plan.used_row_policies); - LOG_TRACE(&Poco::Logger::get("Planner"), "Query {} from stage {} to stage {}{}", + LOG_TRACE(getLogger("Planner"), "Query {} from stage {} to stage {}{}", query_tree->formatConvertedASTForErrorMessage(), QueryProcessingStage::toString(from_stage), QueryProcessingStage::toString(select_query_options.to_stage), diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 552f25d7035..ab25f6d2423 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -276,7 +276,7 @@ bool applyTrivialCountIfPossible( /// The query could use trivial count if it didn't use parallel replicas, so let's disable it query_context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); query_context->setSetting("max_parallel_replicas", UInt64{0}); - LOG_TRACE(&Poco::Logger::get("Planner"), "Disabling parallel replicas to be able to use a trivial count optimization"); + LOG_TRACE(getLogger("Planner"), "Disabling parallel replicas to be able to use a trivial count optimization"); } @@ -478,7 +478,7 @@ FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage, "(setting 'max_parallel_replcias'), but the table does not have custom_key defined for it " " or it's invalid (setting 'parallel_replicas_custom_key')"); - LOG_TRACE(&Poco::Logger::get("Planner"), "Processing query on a replica using custom_key '{}'", settings.parallel_replicas_custom_key.value); + LOG_TRACE(getLogger("Planner"), "Processing query on a replica using custom_key '{}'", settings.parallel_replicas_custom_key.value); auto parallel_replicas_custom_filter_ast = getCustomKeyFilterForParallelReplica( settings.parallel_replicas_count, @@ -725,7 +725,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres size_t number_of_replicas_to_use = rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica; LOG_TRACE( - &Poco::Logger::get("Planner"), + getLogger("Planner"), "Estimated {} rows to read. It is enough work for {} parallel replicas", rows_to_read, number_of_replicas_to_use); @@ -735,12 +735,12 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres planner_context->getMutableQueryContext()->setSetting( "allow_experimental_parallel_reading_from_replicas", Field(0)); planner_context->getMutableQueryContext()->setSetting("max_parallel_replicas", UInt64{0}); - LOG_DEBUG(&Poco::Logger::get("Planner"), "Disabling parallel replicas because there aren't enough rows to read"); + LOG_DEBUG(getLogger("Planner"), "Disabling parallel replicas because there aren't enough rows to read"); } else if (number_of_replicas_to_use < settings.max_parallel_replicas) { planner_context->getMutableQueryContext()->setSetting("max_parallel_replicas", number_of_replicas_to_use); - LOG_DEBUG(&Poco::Logger::get("Planner"), "Reducing the number of replicas to use to {}", number_of_replicas_to_use); + LOG_DEBUG(getLogger("Planner"), "Reducing the number of replicas to use to {}", number_of_replicas_to_use); } } diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 9b249d21a24..94ee249106a 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -388,8 +388,8 @@ JoinClausesAndActions buildJoinClausesAndActions(//const ColumnsWithTypeAndName ActionsDAGPtr left_join_actions = std::make_shared(left_table_expression_columns); ActionsDAGPtr right_join_actions = std::make_shared(right_table_expression_columns); - // LOG_TRACE(&Poco::Logger::get("Planner"), "buildJoinClausesAndActions cols {} ", left_join_actions->dumpDAG()); - // LOG_TRACE(&Poco::Logger::get("Planner"), "buildJoinClausesAndActions cols {} ", right_join_actions->dumpDAG()); + // LOG_TRACE(getLogger("Planner"), "buildJoinClausesAndActions cols {} ", left_join_actions->dumpDAG()); + // LOG_TRACE(getLogger("Planner"), "buildJoinClausesAndActions cols {} ", right_join_actions->dumpDAG()); /** In ActionsDAG if input node has constant representation additional constant column is added. * That way we cannot simply check that node has INPUT type during resolution of expression join table side. @@ -411,8 +411,8 @@ JoinClausesAndActions buildJoinClausesAndActions(//const ColumnsWithTypeAndName * ON (t1.id = t2.id) AND 1 != 1 AND (t1.value >= t1.value); */ auto join_expression = join_node.getJoinExpression(); - // LOG_TRACE(&Poco::Logger::get("Planner"), "buildJoinClausesAndActions expr {} ", join_expression->formatConvertedASTForErrorMessage()); - // LOG_TRACE(&Poco::Logger::get("Planner"), "buildJoinClausesAndActions expr {} ", join_expression->dumpTree()); + // LOG_TRACE(getLogger("Planner"), "buildJoinClausesAndActions expr {} ", join_expression->formatConvertedASTForErrorMessage()); + // LOG_TRACE(getLogger("Planner"), "buildJoinClausesAndActions expr {} ", join_expression->dumpTree()); auto * constant_join_expression = join_expression->as(); diff --git a/src/Processors/Executors/PipelineExecutor.h b/src/Processors/Executors/PipelineExecutor.h index dee12dad282..862a460f0ed 100644 --- a/src/Processors/Executors/PipelineExecutor.h +++ b/src/Processors/Executors/PipelineExecutor.h @@ -83,7 +83,7 @@ private: std::atomic_bool cancelled = false; std::atomic_bool cancelled_reading = false; - Poco::Logger * log = &Poco::Logger::get("PipelineExecutor"); + LoggerPtr log = getLogger("PipelineExecutor"); /// Now it's used to check if query was killed. QueryStatusPtr process_list_element; diff --git a/src/Processors/Formats/IRowInputFormat.cpp b/src/Processors/Formats/IRowInputFormat.cpp index 5f27fa78c55..8c317a34a9d 100644 --- a/src/Processors/Formats/IRowInputFormat.cpp +++ b/src/Processors/Formats/IRowInputFormat.cpp @@ -230,7 +230,7 @@ Chunk IRowInputFormat::read() { if (num_errors && (params.allow_errors_num > 0 || params.allow_errors_ratio > 0)) { - Poco::Logger * log = &Poco::Logger::get("IRowInputFormat"); + LoggerPtr log = getLogger("IRowInputFormat"); LOG_DEBUG(log, "Skipped {} rows with errors while reading the input stream", num_errors); } diff --git a/src/Processors/Formats/ISchemaReader.cpp b/src/Processors/Formats/ISchemaReader.cpp index 26c632b83dc..79b7ca17a5a 100644 --- a/src/Processors/Formats/ISchemaReader.cpp +++ b/src/Processors/Formats/ISchemaReader.cpp @@ -91,7 +91,7 @@ void IIRowSchemaReader::setContext(ContextPtr & context) } else { - LOG_WARNING(&Poco::Logger::get("IIRowSchemaReader"), "Couldn't parse schema inference hints: {}. This setting will be ignored", hints_parsing_error); + LOG_WARNING(getLogger("IIRowSchemaReader"), "Couldn't parse schema inference hints: {}. This setting will be ignored", hints_parsing_error); } } diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 5722c660071..8dc8fa516dc 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -984,7 +984,7 @@ private: try { Poco::URI url(base_url, base_url.getPath() + "/schemas/ids/" + std::to_string(id)); - LOG_TRACE((&Poco::Logger::get("AvroConfluentRowInputFormat")), "Fetching schema id = {} from url {}", id, url.toString()); + LOG_TRACE((getLogger("AvroConfluentRowInputFormat")), "Fetching schema id = {} from url {}", id, url.toString()); /// One second for connect/send/receive. Just in case. auto timeouts = ConnectionTimeouts() @@ -1029,7 +1029,7 @@ private: markSessionForReuse(session); auto schema = json_body->getValue("schema"); - LOG_TRACE((&Poco::Logger::get("AvroConfluentRowInputFormat")), "Successfully fetched schema id = {}\n{}", id, schema); + LOG_TRACE((getLogger("AvroConfluentRowInputFormat")), "Successfully fetched schema id = {}\n{}", id, schema); return avro::compileJsonSchemaFromString(schema); } catch (const Exception &) diff --git a/src/Processors/Formats/Impl/DWARFBlockInputFormat.cpp b/src/Processors/Formats/Impl/DWARFBlockInputFormat.cpp index 43ef2521032..b655e892d3b 100644 --- a/src/Processors/Formats/Impl/DWARFBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/DWARFBlockInputFormat.cpp @@ -198,7 +198,7 @@ void DWARFBlockInputFormat::initializeIfNeeded() if (elf.has_value()) return; - LOG_DEBUG(&Poco::Logger::get("DWARF"), "Opening ELF"); + LOG_DEBUG(getLogger("DWARF"), "Opening ELF"); initELF(); if (is_stopped) return; @@ -209,7 +209,7 @@ void DWARFBlockInputFormat::initializeIfNeeded() auto abbrev_section = elf->findSectionByName(".debug_abbrev"); if (!abbrev_section.has_value()) throw Exception(ErrorCodes::CANNOT_PARSE_ELF, "No .debug_abbrev section"); - LOG_DEBUG(&Poco::Logger::get("DWARF"), ".debug_abbrev is {:.3f} MiB, .debug_info is {:.3f} MiB", abbrev_section->size() * 1. / (1 << 20), info_section->size() * 1. / (1 << 20)); + LOG_DEBUG(getLogger("DWARF"), ".debug_abbrev is {:.3f} MiB, .debug_info is {:.3f} MiB", abbrev_section->size() * 1. / (1 << 20), info_section->size() * 1. / (1 << 20)); /// (The StringRef points into Elf's mmap of the whole file, or into file_contents.) extractor.emplace(llvm::StringRef(info_section->begin(), info_section->size()), /*IsLittleEndian*/ true, /*AddressSize*/ 8); @@ -237,7 +237,7 @@ void DWARFBlockInputFormat::initializeIfNeeded() for (std::unique_ptr & unit : dwarf_context->info_section_units()) units_queue.emplace_back(unit.get()); - LOG_DEBUG(&Poco::Logger::get("DWARF"), "{} units, reading in {} threads", units_queue.size(), num_threads); + LOG_DEBUG(getLogger("DWARF"), "{} units, reading in {} threads", units_queue.size(), num_threads); pool.emplace(CurrentMetrics::DWARFReaderThreads, CurrentMetrics::DWARFReaderThreadsActive, CurrentMetrics::DWARFReaderThreadsScheduled, num_threads); for (size_t i = 0; i < num_threads; ++i) @@ -782,7 +782,7 @@ void DWARFBlockInputFormat::parseFilenameTable(UnitState & unit, uint64_t offset auto error = prologue.parse(*debug_line_extractor, &offset, /*RecoverableErrorHandler*/ [&](auto e) { if (++seen_debug_line_warnings < 10) - LOG_INFO(&Poco::Logger::get("DWARF"), "Parsing error: {}", llvm::toString(std::move(e))); + LOG_INFO(getLogger("DWARF"), "Parsing error: {}", llvm::toString(std::move(e))); }, *dwarf_context, unit.dwarf_unit); if (error) diff --git a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h index c2f08479730..50a73646359 100644 --- a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h +++ b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h @@ -84,7 +84,7 @@ public: , pool(CurrentMetrics::ParallelFormattingOutputFormatThreads, CurrentMetrics::ParallelFormattingOutputFormatThreadsActive, CurrentMetrics::ParallelFormattingOutputFormatThreadsScheduled, params.max_threads_for_parallel_formatting) { - LOG_TEST(&Poco::Logger::get("ParallelFormattingOutputFormat"), "Parallel formatting is being used"); + LOG_TEST(getLogger("ParallelFormattingOutputFormat"), "Parallel formatting is being used"); NullWriteBuffer buf; save_totals_and_extremes_in_statistics = internal_formatter_creator(buf)->areTotalsAndExtremesUsedInFinalize(); diff --git a/src/Processors/Formats/Impl/ParallelParsingInputFormat.h b/src/Processors/Formats/Impl/ParallelParsingInputFormat.h index ff97afa8348..c4736ceea3a 100644 --- a/src/Processors/Formats/Impl/ParallelParsingInputFormat.h +++ b/src/Processors/Formats/Impl/ParallelParsingInputFormat.h @@ -111,7 +111,7 @@ public: // bump into reader thread on wraparound. processing_units.resize(params.max_threads + 2); - LOG_TRACE(&Poco::Logger::get("ParallelParsingInputFormat"), "Parallel parsing is used"); + LOG_TRACE(getLogger("ParallelParsingInputFormat"), "Parallel parsing is used"); } ~ParallelParsingInputFormat() override diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index aa193ffd36a..3e61bfbc794 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -492,7 +492,7 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx &found_in_cache, delimiter); - LOG_TEST(&Poco::Logger::get("ValuesBlockInputFormat"), "Will use an expression template to parse column {}: {}", + LOG_TEST(getLogger("ValuesBlockInputFormat"), "Will use an expression template to parse column {}: {}", column_idx, structure->dumpTemplate()); templates[column_idx].emplace(structure); diff --git a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp index 14325223602..8948cee217c 100644 --- a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp @@ -28,7 +28,7 @@ CollapsingSortedAlgorithm::CollapsingSortedAlgorithm( bool only_positive_sign_, size_t max_block_size_rows_, size_t max_block_size_bytes_, - Poco::Logger * log_, + LoggerPtr log_, WriteBuffer * out_row_sources_buf_, bool use_average_block_sizes) : IMergingAlgorithmWithSharedChunks(header_, num_inputs, std::move(description_), out_row_sources_buf_, max_row_refs) diff --git a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h index 28bb87cb394..be1a3a3bf33 100644 --- a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h @@ -34,7 +34,7 @@ public: bool only_positive_sign_, /// For select final. Skip rows with sum(sign) < 0. size_t max_block_size_rows_, size_t max_block_size_bytes_, - Poco::Logger * log_, + LoggerPtr log_, WriteBuffer * out_row_sources_buf_ = nullptr, bool use_average_block_sizes = false); @@ -64,7 +64,7 @@ private: PODArray current_row_sources; /// Sources of rows with the current primary key size_t count_incorrect_data = 0; /// To prevent too many error messages from writing to the log. - Poco::Logger * log; + LoggerPtr log; void reportIncorrectData(); void insertRow(RowRef & row); diff --git a/src/Processors/Merges/Algorithms/RowRef.h b/src/Processors/Merges/Algorithms/RowRef.h index 81969cd1988..ee64224d44d 100644 --- a/src/Processors/Merges/Algorithms/RowRef.h +++ b/src/Processors/Merges/Algorithms/RowRef.h @@ -86,7 +86,7 @@ public: { if (free_chunks.size() != chunks.size()) { - LOG_ERROR(&Poco::Logger::get("SharedChunkAllocator"), "SharedChunkAllocator was destroyed before RowRef was released. StackTrace: {}", StackTrace().toString()); + LOG_ERROR(getLogger("SharedChunkAllocator"), "SharedChunkAllocator was destroyed before RowRef was released. StackTrace: {}", StackTrace().toString()); return; } @@ -103,7 +103,7 @@ private: /// This may happen if allocator was removed before chunks. /// Log message and exit, because we don't want to throw exception in destructor. - LOG_ERROR(&Poco::Logger::get("SharedChunkAllocator"), "SharedChunkAllocator was destroyed before RowRef was released. StackTrace: {}", StackTrace().toString()); + LOG_ERROR(getLogger("SharedChunkAllocator"), "SharedChunkAllocator was destroyed before RowRef was released. StackTrace: {}", StackTrace().toString()); return; } diff --git a/src/Processors/Merges/CollapsingSortedTransform.h b/src/Processors/Merges/CollapsingSortedTransform.h index b0cb6bc6d62..4479ac82f66 100644 --- a/src/Processors/Merges/CollapsingSortedTransform.h +++ b/src/Processors/Merges/CollapsingSortedTransform.h @@ -29,7 +29,7 @@ public: only_positive_sign, max_block_size_rows, max_block_size_bytes, - &Poco::Logger::get("CollapsingSortedTransform"), + getLogger("CollapsingSortedTransform"), out_row_sources_buf_, use_average_block_sizes) { diff --git a/src/Processors/Merges/MergingSortedTransform.cpp b/src/Processors/Merges/MergingSortedTransform.cpp index 62275f37857..338b1ff7935 100644 --- a/src/Processors/Merges/MergingSortedTransform.cpp +++ b/src/Processors/Merges/MergingSortedTransform.cpp @@ -53,7 +53,7 @@ void MergingSortedTransform::onFinish() const auto & merged_data = algorithm.getMergedData(); - auto * log = &Poco::Logger::get("MergingSortedTransform"); + auto log = getLogger("MergingSortedTransform"); double seconds = total_stopwatch.elapsedSeconds(); diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h index 023901dba02..70ae4c6156e 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h @@ -54,7 +54,7 @@ private: JoinTableSide position; - Poco::Logger * log = &Poco::Logger::get("CreateSetAndFilterOnTheFlyStep"); + LoggerPtr log = getLogger("CreateSetAndFilterOnTheFlyStep"); }; } diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 3b31a809f9d..e71bcc5602a 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -383,7 +383,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes const size_t updated_steps = tryAddNewFilterStep(parent_node, nodes, split_filter, can_remove_filter, child_idx); if (updated_steps > 0) { - LOG_DEBUG(&Poco::Logger::get("QueryPlanOptimizations"), "Pushed down filter {} to the {} side of join", split_filter_column_name, kind); + LOG_DEBUG(getLogger("QueryPlanOptimizations"), "Pushed down filter {} to the {} side of join", split_filter_column_name, kind); } return updated_steps; }; diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index 5c5171d4296..7902b36f80e 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -164,7 +164,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) storage.getConditionEstimatorByPredicate(read_from_merge_tree->getQueryInfo(), storage_snapshot, context), queried_columns, storage.supportedPrewhereColumns(), - &Poco::Logger::get("QueryPlanOptimizePrewhere")}; + getLogger("QueryPlanOptimizePrewhere")}; auto optimize_result = where_optimizer.optimize(filter_step->getExpression(), filter_step->getFilterColumnName(), diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index c3e651154ae..534716cc60e 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -123,7 +123,7 @@ std::optional matchAggregateFunctions( if (it == projection_aggregate_functions.end()) { // LOG_TRACE( - // &Poco::Logger::get("optimizeUseProjections"), + // getLogger("optimizeUseProjections"), // "Cannot match agg func {} by name {}", // aggregate.column_name, aggregate.function->getName()); @@ -151,7 +151,7 @@ std::optional matchAggregateFunctions( /// not match. if (!candidate.function->getStateType()->equals(*aggregate.function->getStateType())) { - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot match agg func {} vs {} by state {} vs {}", + // LOG_TRACE(getLogger("optimizeUseProjections"), "Cannot match agg func {} vs {} by state {} vs {}", // aggregate.column_name, candidate.column_name, // candidate.function->getStateType()->getName(), aggregate.function->getStateType()->getName()); continue; @@ -194,7 +194,7 @@ std::optional matchAggregateFunctions( if (mt == matches.end()) { // LOG_TRACE( - // &Poco::Logger::get("optimizeUseProjections"), + // getLogger("optimizeUseProjections"), // "Cannot match agg func {} vs {} : can't match arg {} vs {} : no node in map", // aggregate.column_name, candidate.column_name, query_name, proj_name); @@ -205,7 +205,7 @@ std::optional matchAggregateFunctions( if (node_match.node != proj_node || node_match.monotonicity) { // LOG_TRACE( - // &Poco::Logger::get("optimizeUseProjections"), + // getLogger("optimizeUseProjections"), // "Cannot match agg func {} vs {} : can't match arg {} vs {} : no match or monotonicity", // aggregate.column_name, candidate.column_name, query_name, proj_name); @@ -285,7 +285,7 @@ ActionsDAGPtr analyzeAggregateProjection( // for (const auto & [node, match] : matches) // { - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Match {} {} -> {} {} (with monotonicity : {})", + // LOG_TRACE(getLogger("optimizeUseProjections"), "Match {} {} -> {} {} (with monotonicity : {})", // static_cast(node), node->result_name, // static_cast(match.node), (match.node ? match.node->result_name : ""), match.monotonicity != std::nullopt); // } @@ -379,7 +379,7 @@ ActionsDAGPtr analyzeAggregateProjection( /// Not a match and there is no matched child. if (frame.node->type == ActionsDAG::ActionType::INPUT) { - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot find match for {}", frame.node->result_name); + // LOG_TRACE(getLogger("optimizeUseProjections"), "Cannot find match for {}", frame.node->result_name); return {}; } @@ -389,7 +389,7 @@ ActionsDAGPtr analyzeAggregateProjection( } } - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Folding actions by projection"); + // LOG_TRACE(getLogger("optimizeUseProjections"), "Folding actions by projection"); auto proj_dag = query.dag->foldActionsByProjection(new_inputs, query_key_nodes); appendAggregateFunctions(*proj_dag, aggregates, *matched_aggregates); @@ -453,7 +453,7 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( if (!can_use_minmax_projection && agg_projections.empty()) return candidates; - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Has agg projection"); + // LOG_TRACE(getLogger("optimizeUseProjections"), "Has agg projection"); QueryDAG dag; if (!dag.build(*node.children.front())) @@ -461,22 +461,22 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( auto query_index = buildDAGIndex(*dag.dag); - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Query DAG: {}", dag.dag->dumpDAG()); + // LOG_TRACE(getLogger("optimizeUseProjections"), "Query DAG: {}", dag.dag->dumpDAG()); candidates.has_filter = dag.filter_node; if (can_use_minmax_projection) { const auto * projection = &*(metadata->minmax_count_projection); - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try projection {}", projection->name); + // LOG_TRACE(getLogger("optimizeUseProjections"), "Try projection {}", projection->name); auto info = getAggregatingProjectionInfo(*projection, context, metadata, key_virtual_columns); - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection DAG {}", info.before_aggregation->dumpDAG()); + // LOG_TRACE(getLogger("optimizeUseProjections"), "Projection DAG {}", info.before_aggregation->dumpDAG()); if (auto proj_dag = analyzeAggregateProjection(info, dag, query_index, keys, aggregates)) { - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG()); + // LOG_TRACE(getLogger("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG()); AggregateProjectionCandidate candidate{.info = std::move(info), .dag = std::move(proj_dag)}; - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection sample block {}", sample_block.dumpStructure()); + // LOG_TRACE(getLogger("optimizeUseProjections"), "Projection sample block {}", sample_block.dumpStructure()); auto block = reading.getMergeTreeData().getMinMaxCountProjectionBlock( metadata, candidate.dag->getRequiredColumnsNames(), @@ -485,7 +485,7 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( max_added_blocks.get(), context); - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection sample block 2 {}", block.dumpStructure()); + // LOG_TRACE(getLogger("optimizeUseProjections"), "Projection sample block 2 {}", block.dumpStructure()); // minmax_count_projection cannot be used when there is no data to process, because // it will produce incorrect result during constant aggregation. @@ -518,12 +518,12 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( candidates.real.reserve(agg_projections.size()); for (const auto * projection : agg_projections) { - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Try projection {}", projection->name); + // LOG_TRACE(getLogger("optimizeUseProjections"), "Try projection {}", projection->name); auto info = getAggregatingProjectionInfo(*projection, context, metadata, key_virtual_columns); - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection DAG {}", info.before_aggregation->dumpDAG()); + // LOG_TRACE(getLogger("optimizeUseProjections"), "Projection DAG {}", info.before_aggregation->dumpDAG()); if (auto proj_dag = analyzeAggregateProjection(info, dag, query_index, keys, aggregates)) { - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG()); + // LOG_TRACE(getLogger("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG()); AggregateProjectionCandidate candidate{.info = std::move(info), .dag = std::move(proj_dag)}; candidate.projection = projection; candidates.real.emplace_back(std::move(candidate)); @@ -650,7 +650,7 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & /// Add reading from projection step. if (candidates.minmax_projection) { - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Minmax proj block {}", + // LOG_TRACE(getLogger("optimizeUseProjections"), "Minmax proj block {}", // candidates.minmax_projection->block.dumpStructure()); Pipe pipe(std::make_shared(std::move(candidates.minmax_projection->block))); @@ -712,7 +712,7 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & }); } - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection reading header {}", + // LOG_TRACE(getLogger("optimizeUseProjections"), "Projection reading header {}", // projection_reading->getOutputStream().header.dumpStructure()); projection_reading->setStepDescription(best_candidate->projection->name); diff --git a/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp b/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp index a6029d673e3..232d3118612 100644 --- a/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp +++ b/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp @@ -39,14 +39,14 @@ namespace else ss << value; - LOG_DEBUG(&Poco::Logger::get("redundantDistinct"), "{}{}{}", key, separator, ss.str()); + LOG_DEBUG(getLogger("redundantDistinct"), "{}{}{}", key, separator, ss.str()); } } void logActionsDAG(const String & prefix, const ActionsDAGPtr & actions) { if constexpr (debug_logging_enabled) - LOG_DEBUG(&Poco::Logger::get("redundantDistinct"), "{} :\n{}", prefix, actions->dumpDAG()); + LOG_DEBUG(getLogger("redundantDistinct"), "{} :\n{}", prefix, actions->dumpDAG()); } using DistinctColumns = std::set; diff --git a/src/Processors/QueryPlan/QueryPlanVisitor.h b/src/Processors/QueryPlan/QueryPlanVisitor.h index 0f265216649..aed1a2b2249 100644 --- a/src/Processors/QueryPlan/QueryPlanVisitor.h +++ b/src/Processors/QueryPlan/QueryPlanVisitor.h @@ -99,7 +99,7 @@ protected: { const IQueryPlanStep * current_step = node->step.get(); LOG_DEBUG( - &Poco::Logger::get("QueryPlanVisitor"), + getLogger("QueryPlanVisitor"), "{}: {}: {}", prefix, getStepId(current_step), diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 62d4538cb84..0465ff54f5a 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1281,7 +1281,7 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( data, real_column_names, sample_factor_column_queried, - log.get(), + log, indexes); } @@ -1476,7 +1476,7 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( const MergeTreeData & data, const Names & real_column_names, bool sample_factor_column_queried, - Poco::Logger * log, + LoggerPtr log, std::optional & indexes) { auto updated_query_info_with_filter_dag = query_info; @@ -1508,7 +1508,7 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( const MergeTreeData & data, const Names & real_column_names, bool sample_factor_column_queried, - Poco::Logger * log, + LoggerPtr log, std::optional & indexes) { AnalysisResult result; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 3f2ac7ea931..fdeaff57279 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -168,7 +168,7 @@ public: const MergeTreeData & data, const Names & real_column_names, bool sample_factor_column_queried, - Poco::Logger * log, + LoggerPtr log, std::optional & indexes); AnalysisResultPtr selectRangesToRead( @@ -217,7 +217,7 @@ private: const MergeTreeData & data, const Names & real_column_names, bool sample_factor_column_queried, - Poco::Logger * log, + LoggerPtr log, std::optional & indexes); int getSortDirection() const diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 4bbba4cfa30..4dd79903965 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -102,7 +102,7 @@ ReadFromRemote::ReadFromRemote( ThrottlerPtr throttler_, Scalars scalars_, Tables external_tables_, - Poco::Logger * log_, + LoggerPtr log_, UInt32 shard_count_, std::shared_ptr storage_limits_, const String & cluster_name_) @@ -172,7 +172,7 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStream catch (const Exception & ex) { if (ex.code() == ErrorCodes::ALL_CONNECTION_TRIES_FAILED) - LOG_WARNING(&Poco::Logger::get("ClusterProxy::SelectStreamFactory"), + LOG_WARNING(getLogger("ClusterProxy::SelectStreamFactory"), "Connections to remote replicas of local shard {} failed, will use stale local replica", my_shard.shard_info.shard_num); else throw; @@ -361,7 +361,7 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( ThrottlerPtr throttler_, Scalars scalars_, Tables external_tables_, - Poco::Logger * log_, + LoggerPtr log_, std::shared_ptr storage_limits_) : ISourceStep(DataStream{.header = std::move(header_)}) , cluster(cluster_) @@ -402,7 +402,7 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder size_t all_replicas_count = current_settings.max_parallel_replicas; if (all_replicas_count > cluster->getShardsInfo().size()) { - LOG_INFO(&Poco::Logger::get("ReadFromParallelRemoteReplicasStep"), + LOG_INFO(getLogger("ReadFromParallelRemoteReplicasStep"), "The number of replicas requested ({}) is bigger than the real number available in the cluster ({}). "\ "Will use the latter number to execute the query.", current_settings.max_parallel_replicas, cluster->getShardsInfo().size()); all_replicas_count = cluster->getShardsInfo().size(); diff --git a/src/Processors/QueryPlan/ReadFromRemote.h b/src/Processors/QueryPlan/ReadFromRemote.h index 82ef45d6bbf..f853a12910b 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.h +++ b/src/Processors/QueryPlan/ReadFromRemote.h @@ -35,7 +35,7 @@ public: ThrottlerPtr throttler_, Scalars scalars_, Tables external_tables_, - Poco::Logger * log_, + LoggerPtr log_, UInt32 shard_count_, std::shared_ptr storage_limits_, const String & cluster_name_); @@ -57,7 +57,7 @@ private: Scalars scalars; Tables external_tables; std::shared_ptr storage_limits; - Poco::Logger * log; + LoggerPtr log; UInt32 shard_count; const String cluster_name; std::optional priority_func_factory; @@ -80,7 +80,7 @@ public: ThrottlerPtr throttler_, Scalars scalars_, Tables external_tables_, - Poco::Logger * log_, + LoggerPtr log_, std::shared_ptr storage_limits_); String getName() const override { return "ReadFromRemoteParallelReplicas"; } @@ -103,7 +103,7 @@ private: Scalars scalars; Tables external_tables; std::shared_ptr storage_limits; - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/src/Processors/Sources/MySQLSource.cpp b/src/Processors/Sources/MySQLSource.cpp index 81225d1cdf2..be691fd5b2e 100644 --- a/src/Processors/Sources/MySQLSource.cpp +++ b/src/Processors/Sources/MySQLSource.cpp @@ -58,7 +58,7 @@ MySQLSource::MySQLSource( const Block & sample_block, const StreamSettings & settings_) : ISource(sample_block.cloneEmpty()) - , log(&Poco::Logger::get("MySQLSource")) + , log(getLogger("MySQLSource")) , connection{std::make_unique(entry, query_str)} , settings{std::make_unique(settings_)} { @@ -69,7 +69,7 @@ MySQLSource::MySQLSource( /// For descendant MySQLWithFailoverSource MySQLSource::MySQLSource(const Block &sample_block_, const StreamSettings & settings_) : ISource(sample_block_.cloneEmpty()) - , log(&Poco::Logger::get("MySQLSource")) + , log(getLogger("MySQLSource")) , settings(std::make_unique(settings_)) { description.init(sample_block_); diff --git a/src/Processors/Sources/MySQLSource.h b/src/Processors/Sources/MySQLSource.h index c4d447886c0..fc26ffa3645 100644 --- a/src/Processors/Sources/MySQLSource.h +++ b/src/Processors/Sources/MySQLSource.h @@ -50,7 +50,7 @@ protected: mysqlxx::UseQueryResult result; }; - Poco::Logger * log; + LoggerPtr log; std::unique_ptr connection; const std::unique_ptr settings; diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index 1f23292c6b3..55eaf67eb3b 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -158,7 +158,7 @@ public: throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Executable generates stderr: {}", str); else if (stderr_reaction == ExternalCommandStderrReaction::LOG) LOG_WARNING( - &::Poco::Logger::get("TimeoutReadBufferFromFileDescriptor"), "Executable generates stderr: {}", str); + getLogger("TimeoutReadBufferFromFileDescriptor"), "Executable generates stderr: {}", str); else if (stderr_reaction == ExternalCommandStderrReaction::LOG_FIRST) { res = std::min(ssize_t(stderr_result_buf.reserve()), res); @@ -217,7 +217,7 @@ public: stderr_result.reserve(stderr_result_buf.size()); stderr_result.append(stderr_result_buf.begin(), stderr_result_buf.end()); LOG_WARNING( - &::Poco::Logger::get("ShellCommandSource"), + getLogger("ShellCommandSource"), "Executable generates stderr at the {}: {}", stderr_reaction == ExternalCommandStderrReaction::LOG_FIRST ? "beginning" : "end", stderr_result); diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.h b/src/Processors/Transforms/AggregatingInOrderTransform.h index af63ac61c3c..5d50e97f552 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.h +++ b/src/Processors/Transforms/AggregatingInOrderTransform.h @@ -83,7 +83,7 @@ private: Chunk current_chunk; Chunk to_push_chunk; - Poco::Logger * log = &Poco::Logger::get("AggregatingInOrderTransform"); + LoggerPtr log = getLogger("AggregatingInOrderTransform"); }; diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index 91fdf479ffb..e05528afdc7 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -180,7 +180,7 @@ private: Processors processors; AggregatingTransformParamsPtr params; - Poco::Logger * log = &Poco::Logger::get("AggregatingTransform"); + LoggerPtr log = getLogger("AggregatingTransform"); ColumnRawPtrs key_columns; Aggregator::AggregateColumns aggregate_columns; diff --git a/src/Processors/Transforms/ColumnGathererTransform.cpp b/src/Processors/Transforms/ColumnGathererTransform.cpp index 7c2b93faa91..d7f52a538e1 100644 --- a/src/Processors/Transforms/ColumnGathererTransform.cpp +++ b/src/Processors/Transforms/ColumnGathererTransform.cpp @@ -128,7 +128,7 @@ ColumnGathererTransform::ColumnGathererTransform( : IMergingTransform( num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, /*always_read_till_end_=*/ false, num_inputs, row_sources_buf_, block_preferred_size_) - , log(&Poco::Logger::get("ColumnGathererStream")) + , log(getLogger("ColumnGathererStream")) { if (header.columns() != 1) throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "Header should have 1 column, but contains {}", diff --git a/src/Processors/Transforms/ColumnGathererTransform.h b/src/Processors/Transforms/ColumnGathererTransform.h index b5bbbff9aca..885cb3f81ba 100644 --- a/src/Processors/Transforms/ColumnGathererTransform.h +++ b/src/Processors/Transforms/ColumnGathererTransform.h @@ -120,7 +120,7 @@ protected: void onFinish() override; UInt64 elapsed_ns = 0; - Poco::Logger * log; + LoggerPtr log; }; diff --git a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h index d214a310a8c..0f5dab06fc9 100644 --- a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h +++ b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h @@ -67,7 +67,7 @@ private: /// Set to fill SetWithStatePtr set; - Poco::Logger * log = &Poco::Logger::get("CreatingSetsOnTheFlyTransform"); + LoggerPtr log = getLogger("CreatingSetsOnTheFlyTransform"); }; /* @@ -108,7 +108,7 @@ private: size_t result_rows = 0; } stat; - Poco::Logger * log = &Poco::Logger::get("FilterBySetOnTheFlyTransform"); + LoggerPtr log = getLogger("FilterBySetOnTheFlyTransform"); }; } diff --git a/src/Processors/Transforms/CreatingSetsTransform.h b/src/Processors/Transforms/CreatingSetsTransform.h index d1ec7dcbca7..74dcd829b4d 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.h +++ b/src/Processors/Transforms/CreatingSetsTransform.h @@ -63,7 +63,7 @@ private: size_t bytes_to_transfer = 0; using Logger = Poco::Logger; - Poco::Logger * log = &Poco::Logger::get("CreatingSetsTransform"); + LoggerPtr log = getLogger("CreatingSetsTransform"); bool is_initialized = false; diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index b725c3e1a5f..aaa98e96803 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -28,7 +28,7 @@ void logDebug(String key, const T & value, const char * separator = " : ") else ss << value; - LOG_DEBUG(&Poco::Logger::get("FillingTransform"), "{}{}{}", key, separator, ss.str()); + LOG_DEBUG(getLogger("FillingTransform"), "{}{}{}", key, separator, ss.str()); } } diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index 4e7868ea1c2..0c0a86ce270 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -14,12 +14,12 @@ namespace ErrorCodes Block JoiningTransform::transformHeader(Block header, const JoinPtr & join) { - LOG_DEBUG(&Poco::Logger::get("JoiningTransform"), "Before join block: '{}'", header.dumpStructure()); + LOG_DEBUG(getLogger("JoiningTransform"), "Before join block: '{}'", header.dumpStructure()); join->checkTypesOfKeys(header); join->initialize(header); ExtraBlockPtr tmp; join->joinBlock(header, tmp); - LOG_DEBUG(&Poco::Logger::get("JoiningTransform"), "After join block: '{}'", header.dumpStructure()); + LOG_DEBUG(getLogger("JoiningTransform"), "After join block: '{}'", header.dumpStructure()); return header; } diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 15c88244cbd..2d313d4ea5c 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -273,7 +273,7 @@ MergeJoinAlgorithm::MergeJoinAlgorithm( size_t max_block_size_) : table_join(table_join_) , max_block_size(max_block_size_) - , log(&Poco::Logger::get("MergeJoinAlgorithm")) + , log(getLogger("MergeJoinAlgorithm")) { if (input_headers.size() != 2) throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeJoinAlgorithm requires exactly two inputs"); @@ -860,7 +860,7 @@ MergeJoinTransform::MergeJoinTransform( /* always_read_till_end_= */ false, /* empty_chunk_on_finish_= */ true, table_join, input_headers, max_block_size) - , log(&Poco::Logger::get("MergeJoinTransform")) + , log(getLogger("MergeJoinTransform")) { LOG_TRACE(log, "Use MergeJoinTransform"); } diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index eb45169a2b0..793de00db40 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -269,7 +269,7 @@ private: Statistic stat; - Poco::Logger * log; + LoggerPtr log; }; class MergeJoinTransform final : public IMergingTransform @@ -289,7 +289,7 @@ public: protected: void onFinish() override; - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/src/Processors/Transforms/MergeSortingTransform.cpp b/src/Processors/Transforms/MergeSortingTransform.cpp index de77711d129..64d84ea4b00 100644 --- a/src/Processors/Transforms/MergeSortingTransform.cpp +++ b/src/Processors/Transforms/MergeSortingTransform.cpp @@ -30,7 +30,7 @@ namespace DB class BufferingToFileTransform : public IAccumulatingTransform { public: - BufferingToFileTransform(const Block & header, TemporaryFileStream & tmp_stream_, Poco::Logger * log_) + BufferingToFileTransform(const Block & header, TemporaryFileStream & tmp_stream_, LoggerPtr log_) : IAccumulatingTransform(header, header) , tmp_stream(tmp_stream_) , log(log_) @@ -73,7 +73,7 @@ public: private: TemporaryFileStream & tmp_stream; - Poco::Logger * log; + LoggerPtr log; }; MergeSortingTransform::MergeSortingTransform( diff --git a/src/Processors/Transforms/MergeSortingTransform.h b/src/Processors/Transforms/MergeSortingTransform.h index e8c180b6903..4478d5a07e8 100644 --- a/src/Processors/Transforms/MergeSortingTransform.h +++ b/src/Processors/Transforms/MergeSortingTransform.h @@ -50,7 +50,7 @@ private: size_t sum_rows_in_blocks = 0; size_t sum_bytes_in_blocks = 0; - Poco::Logger * log = &Poco::Logger::get("MergeSortingTransform"); + LoggerPtr log = getLogger("MergeSortingTransform"); /// If remerge doesn't save memory at least several times, mark it as useless and don't do it anymore. bool remerge_is_useful = true; diff --git a/src/Processors/Transforms/MergingAggregatedTransform.h b/src/Processors/Transforms/MergingAggregatedTransform.h index 73e0d8cd013..ade76b2f304 100644 --- a/src/Processors/Transforms/MergingAggregatedTransform.h +++ b/src/Processors/Transforms/MergingAggregatedTransform.h @@ -21,7 +21,7 @@ protected: private: AggregatingTransformParamsPtr params; - Poco::Logger * log = &Poco::Logger::get("MergingAggregatedTransform"); + LoggerPtr log = getLogger("MergingAggregatedTransform"); size_t max_threads; AggregatedDataVariants data_variants; diff --git a/src/Processors/Transforms/PasteJoinTransform.cpp b/src/Processors/Transforms/PasteJoinTransform.cpp index ff3e2fb85e5..d2fa7eed256 100644 --- a/src/Processors/Transforms/PasteJoinTransform.cpp +++ b/src/Processors/Transforms/PasteJoinTransform.cpp @@ -33,7 +33,7 @@ PasteJoinAlgorithm::PasteJoinAlgorithm( size_t max_block_size_) : table_join(table_join_) , max_block_size(max_block_size_) - , log(&Poco::Logger::get("PasteJoinAlgorithm")) + , log(getLogger("PasteJoinAlgorithm")) { if (input_headers.size() != 2) throw Exception(ErrorCodes::LOGICAL_ERROR, "PasteJoinAlgorithm requires exactly two inputs"); @@ -117,7 +117,7 @@ PasteJoinTransform::PasteJoinTransform( /* always_read_till_end_= */ false, /* empty_chunk_on_finish_= */ true, table_join, input_headers, max_block_size) - , log(&Poco::Logger::get("PasteJoinTransform")) + , log(getLogger("PasteJoinTransform")) { LOG_TRACE(log, "Use PasteJoinTransform"); } diff --git a/src/Processors/Transforms/PasteJoinTransform.h b/src/Processors/Transforms/PasteJoinTransform.h index 7ecf70e18dc..04cb5486cd5 100644 --- a/src/Processors/Transforms/PasteJoinTransform.h +++ b/src/Processors/Transforms/PasteJoinTransform.h @@ -61,7 +61,7 @@ private: Statistic stat; - Poco::Logger * log; + LoggerPtr log; UInt64 last_used_row[2] = {0, 0}; }; @@ -82,7 +82,7 @@ public: protected: void onFinish() override; - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/src/Processors/Transforms/TTLCalcTransform.cpp b/src/Processors/Transforms/TTLCalcTransform.cpp index 31fb61239ef..2b4ed96d4e3 100644 --- a/src/Processors/Transforms/TTLCalcTransform.cpp +++ b/src/Processors/Transforms/TTLCalcTransform.cpp @@ -13,7 +13,7 @@ TTLCalcTransform::TTLCalcTransform( bool force_) : IAccumulatingTransform(header_, header_) , data_part(data_part_) - , log(&Poco::Logger::get(storage_.getLogName() + " (TTLCalcTransform)")) + , log(getLogger(storage_.getLogName() + " (TTLCalcTransform)")) { auto old_ttl_infos = data_part->ttl_infos; diff --git a/src/Processors/Transforms/TTLCalcTransform.h b/src/Processors/Transforms/TTLCalcTransform.h index 495879400dc..baa31c01c52 100644 --- a/src/Processors/Transforms/TTLCalcTransform.h +++ b/src/Processors/Transforms/TTLCalcTransform.h @@ -38,7 +38,7 @@ private: /// ttl_infos and empty_columns are updating while reading const MergeTreeData::MutableDataPartPtr & data_part; - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/src/Processors/Transforms/TTLTransform.cpp b/src/Processors/Transforms/TTLTransform.cpp index 7cde86098c7..db9326f9acf 100644 --- a/src/Processors/Transforms/TTLTransform.cpp +++ b/src/Processors/Transforms/TTLTransform.cpp @@ -25,7 +25,7 @@ TTLTransform::TTLTransform( bool force_) : IAccumulatingTransform(header_, header_) , data_part(data_part_) - , log(&Poco::Logger::get(storage_.getLogName() + " (TTLTransform)")) + , log(getLogger(storage_.getLogName() + " (TTLTransform)")) { auto old_ttl_infos = data_part->ttl_infos; diff --git a/src/Processors/Transforms/TTLTransform.h b/src/Processors/Transforms/TTLTransform.h index 3f0dffd1998..3606db7f4c2 100644 --- a/src/Processors/Transforms/TTLTransform.h +++ b/src/Processors/Transforms/TTLTransform.h @@ -42,7 +42,7 @@ private: /// ttl_infos and empty_columns are updating while reading const MergeTreeData::MutableDataPartPtr & data_part; - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 71d652e74d0..960cc019001 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -267,7 +267,7 @@ Chain buildPushingToViewsChain( if (view == nullptr) { LOG_WARNING( - &Poco::Logger::get("PushingToViews"), "Trying to access table {} but it doesn't exist", view_id.getFullTableName()); + getLogger("PushingToViews"), "Trying to access table {} but it doesn't exist", view_id.getFullTableName()); continue; } @@ -310,7 +310,7 @@ Chain buildPushingToViewsChain( // In case the materialized view is dropped/detached at this point, we register a warning and ignore it assert(materialized_view->is_dropped || materialized_view->is_detached); LOG_WARNING( - &Poco::Logger::get("PushingToViews"), "Trying to access table {} but it doesn't exist", view_id.getFullTableName()); + getLogger("PushingToViews"), "Trying to access table {} but it doesn't exist", view_id.getFullTableName()); continue; } @@ -341,7 +341,7 @@ Chain buildPushingToViewsChain( /// It may happen if materialize view query was changed and it doesn't depend on this source table anymore. /// See setting `allow_experimental_alter_materialized_view_structure` LOG_DEBUG( - &Poco::Logger::get("PushingToViews"), "Table '{}' is not a source for view '{}' anymore, current source is '{}'", + getLogger("PushingToViews"), "Table '{}' is not a source for view '{}' anymore, current source is '{}'", select_query.select_table_id.getFullTableName(), view_id.getFullTableName(), table_id); continue; } @@ -835,14 +835,14 @@ void FinalizingViewsTransform::work() /// Exception will be ignored, it is saved here for the system.query_views_log if (materialized_views_ignore_errors) - tryLogException(view.exception, &Poco::Logger::get("PushingToViews"), "Cannot push to the storage, ignoring the error"); + tryLogException(view.exception, getLogger("PushingToViews"), "Cannot push to the storage, ignoring the error"); } else { view.runtime_stats->setStatus(QueryViewsLogElement::ViewStatus::QUERY_FINISH); LOG_TRACE( - &Poco::Logger::get("PushingToViews"), + getLogger("PushingToViews"), "Pushing ({}) from {} to {} took {} ms.", views_data->max_threads <= 1 ? "sequentially" : ("parallel " + std::to_string(views_data->max_threads)), views_data->source_storage_id.getNameForLogs(), diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index 5a8ccc2592b..444f1258f3e 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -186,7 +186,7 @@ public: void setMainTable(StorageID main_table_) { main_table = std::move(main_table_); } - void setLogger(Poco::Logger * logger) { log = logger; } + void setLogger(LoggerPtr logger) { log = logger; } const Block & getHeader() const { return header; } @@ -283,7 +283,7 @@ private: PoolMode pool_mode = PoolMode::GET_MANY; StorageID main_table = StorageID::createEmpty(); - Poco::Logger * log = nullptr; + LoggerPtr log = nullptr; GetPriorityForLoadBalancing::Func priority_func; diff --git a/src/Server/CertificateReloader.cpp b/src/Server/CertificateReloader.cpp index 8795d4807de..c974f450c9a 100644 --- a/src/Server/CertificateReloader.cpp +++ b/src/Server/CertificateReloader.cpp @@ -105,7 +105,7 @@ CertificateReloader::Data::Data(std::string cert_path, std::string key_path, std } -bool CertificateReloader::File::changeIfModified(std::string new_path, Poco::Logger * logger) +bool CertificateReloader::File::changeIfModified(std::string new_path, LoggerPtr logger) { std::error_code ec; std::filesystem::file_time_type new_modification_time = std::filesystem::last_write_time(new_path, ec); diff --git a/src/Server/CertificateReloader.h b/src/Server/CertificateReloader.h index 9f04179b8d6..028914e682f 100644 --- a/src/Server/CertificateReloader.h +++ b/src/Server/CertificateReloader.h @@ -14,6 +14,7 @@ #include #include #include +#include namespace DB @@ -51,7 +52,7 @@ public: private: CertificateReloader() = default; - Poco::Logger * log = &Poco::Logger::get("CertificateReloader"); + LoggerPtr log = getLogger("CertificateReloader"); struct File { @@ -61,7 +62,7 @@ private: std::string path; std::filesystem::file_time_type modification_time; - bool changeIfModified(std::string new_path, Poco::Logger * logger); + bool changeIfModified(std::string new_path, LoggerPtr logger); }; File cert_file{"certificate"}; diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 6bb6ba139ad..f31a8d6feb5 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -76,7 +76,7 @@ namespace static std::once_flag once_flag; std::call_once(once_flag, [&config] { - static Poco::Logger * logger = &Poco::Logger::get("grpc"); + static LoggerPtr logger = getLogger("grpc"); gpr_set_log_function([](gpr_log_func_args* args) { if (args->severity == GPR_LOG_SEVERITY_DEBUG) @@ -614,7 +614,7 @@ namespace class Call { public: - Call(CallType call_type_, std::unique_ptr responder_, IServer & iserver_, Poco::Logger * log_); + Call(CallType call_type_, std::unique_ptr responder_, IServer & iserver_, LoggerPtr log_); ~Call(); void start(const std::function & on_finish_call_callback); @@ -656,7 +656,7 @@ namespace const CallType call_type; std::unique_ptr responder; IServer & iserver; - Poco::Logger * log = nullptr; + LoggerPtr log = nullptr; std::optional session; ContextMutablePtr query_context; @@ -718,7 +718,7 @@ namespace }; // NOLINTEND(clang-analyzer-optin.performance.Padding) - Call::Call(CallType call_type_, std::unique_ptr responder_, IServer & iserver_, Poco::Logger * log_) + Call::Call(CallType call_type_, std::unique_ptr responder_, IServer & iserver_, LoggerPtr log_) : call_type(call_type_), responder(std::move(responder_)), iserver(iserver_), log(log_) { } @@ -1843,7 +1843,7 @@ private: GRPCServer::GRPCServer(IServer & iserver_, const Poco::Net::SocketAddress & address_to_listen_) : iserver(iserver_) , address_to_listen(address_to_listen_) - , log(&Poco::Logger::get("GRPCServer")) + , log(getLogger("GRPCServer")) , runner(std::make_unique(*this)) {} diff --git a/src/Server/GRPCServer.h b/src/Server/GRPCServer.h index 359a2506e95..a9c8161298f 100644 --- a/src/Server/GRPCServer.h +++ b/src/Server/GRPCServer.h @@ -5,6 +5,7 @@ #if USE_GRPC #include #include +#include #include "clickhouse_grpc.grpc.pb.h" namespace Poco { class Logger; } @@ -46,7 +47,7 @@ private: IServer & iserver; const Poco::Net::SocketAddress address_to_listen; - Poco::Logger * log; + LoggerPtr log; GRPCService grpc_service; std::unique_ptr grpc_server; std::unique_ptr queue; diff --git a/src/Server/HTTP/HTTPServerRequest.cpp b/src/Server/HTTP/HTTPServerRequest.cpp index 4a6e85ba0fb..9db02eac220 100644 --- a/src/Server/HTTP/HTTPServerRequest.cpp +++ b/src/Server/HTTP/HTTPServerRequest.cpp @@ -65,7 +65,7 @@ HTTPServerRequest::HTTPServerRequest(HTTPContextPtr context, HTTPServerResponse { stream = std::move(in); if (!startsWith(getContentType(), "multipart/form-data")) - LOG_WARNING(LogFrequencyLimiter(&Poco::Logger::get("HTTPServerRequest"), 10), "Got an HTTP request with no content length " + LOG_WARNING(LogFrequencyLimiter(getLogger("HTTPServerRequest"), 10), "Got an HTTP request with no content length " "and no chunked/multipart encoding, it may be impossible to distinguish graceful EOF from abnormal connection loss"); } else diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index bdc8e7d59c9..72e7c5552f8 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -137,7 +137,7 @@ bool tryAddHttpOptionHeadersFromConfig(HTTPServerResponse & response, const Poco { /// If there is empty header name, it will not be processed and message about it will be in logs if (config.getString("http_options_response." + config_key + ".name", "").empty()) - LOG_WARNING(&Poco::Logger::get("processOptionsRequest"), "Empty header was found in config. It will not be processed."); + LOG_WARNING(getLogger("processOptionsRequest"), "Empty header was found in config. It will not be processed."); else response.add(config.getString("http_options_response." + config_key + ".name", ""), config.getString("http_options_response." + config_key + ".value", "")); @@ -328,7 +328,7 @@ void HTTPHandler::pushDelayedResults(Output & used_output) HTTPHandler::HTTPHandler(IServer & server_, const std::string & name, const std::optional & content_type_override_) : server(server_) - , log(&Poco::Logger::get(name)) + , log(getLogger(name)) , default_settings(server.context()->getSettingsRef()) , content_type_override(content_type_override_) { diff --git a/src/Server/HTTPHandler.h b/src/Server/HTTPHandler.h index 0b623fe5f65..fa2d0dae199 100644 --- a/src/Server/HTTPHandler.h +++ b/src/Server/HTTPHandler.h @@ -100,7 +100,7 @@ private: }; IServer & server; - Poco::Logger * log; + LoggerPtr log; /// It is the name of the server that will be sent in an http-header X-ClickHouse-Server-Display-Name. String server_display_name; diff --git a/src/Server/HTTPRequestHandlerFactoryMain.cpp b/src/Server/HTTPRequestHandlerFactoryMain.cpp index 5481bcd5083..48c2ab21468 100644 --- a/src/Server/HTTPRequestHandlerFactoryMain.cpp +++ b/src/Server/HTTPRequestHandlerFactoryMain.cpp @@ -7,7 +7,7 @@ namespace DB { HTTPRequestHandlerFactoryMain::HTTPRequestHandlerFactoryMain(const std::string & name_) - : log(&Poco::Logger::get(name_)), name(name_) + : log(getLogger(name_)), name(name_) { } diff --git a/src/Server/HTTPRequestHandlerFactoryMain.h b/src/Server/HTTPRequestHandlerFactoryMain.h index 07b278d831c..1075b7d1d60 100644 --- a/src/Server/HTTPRequestHandlerFactoryMain.h +++ b/src/Server/HTTPRequestHandlerFactoryMain.h @@ -21,7 +21,7 @@ public: std::unique_ptr createRequestHandler(const HTTPServerRequest & request) override; private: - Poco::Logger * log; + LoggerPtr log; std::string name; HTTPPathHints hints; diff --git a/src/Server/InterserverIOHTTPHandler.h b/src/Server/InterserverIOHTTPHandler.h index 66042ad3d1d..226a06f5a45 100644 --- a/src/Server/InterserverIOHTTPHandler.h +++ b/src/Server/InterserverIOHTTPHandler.h @@ -26,7 +26,7 @@ class InterserverIOHTTPHandler : public HTTPRequestHandler public: explicit InterserverIOHTTPHandler(IServer & server_) : server(server_) - , log(&Poco::Logger::get("InterserverIOHTTPHandler")) + , log(getLogger("InterserverIOHTTPHandler")) { } @@ -39,7 +39,7 @@ private: }; IServer & server; - Poco::Logger * log; + LoggerPtr log; CurrentMetrics::Increment metric_increment{CurrentMetrics::InterserverConnection}; diff --git a/src/Server/KeeperReadinessHandler.cpp b/src/Server/KeeperReadinessHandler.cpp index de6edd199d7..c973be040c8 100644 --- a/src/Server/KeeperReadinessHandler.cpp +++ b/src/Server/KeeperReadinessHandler.cpp @@ -63,7 +63,7 @@ void KeeperReadinessHandler::handleRequest(HTTPServerRequest & /*request*/, HTTP } catch (...) { - LOG_ERROR((&Poco::Logger::get("KeeperReadinessHandler")), "Cannot send exception to client"); + LOG_ERROR((getLogger("KeeperReadinessHandler")), "Cannot send exception to client"); } } } diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 76b84f0ce6e..6709cd298e5 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -220,7 +220,7 @@ KeeperTCPHandler::KeeperTCPHandler( Poco::Timespan send_timeout_, const Poco::Net::StreamSocket & socket_) : Poco::Net::TCPServerConnection(socket_) - , log(&Poco::Logger::get("KeeperTCPHandler")) + , log(getLogger("KeeperTCPHandler")) , keeper_dispatcher(keeper_dispatcher_) , operation_timeout( 0, diff --git a/src/Server/KeeperTCPHandler.h b/src/Server/KeeperTCPHandler.h index adb1baa084f..c1c522eee89 100644 --- a/src/Server/KeeperTCPHandler.h +++ b/src/Server/KeeperTCPHandler.h @@ -63,7 +63,7 @@ public: ~KeeperTCPHandler() override; private: - Poco::Logger * log; + LoggerPtr log; std::shared_ptr keeper_dispatcher; Poco::Timespan operation_timeout; Poco::Timespan min_session_timeout; diff --git a/src/Server/KeeperTCPHandlerFactory.h b/src/Server/KeeperTCPHandlerFactory.h index 36f284442c6..239bf8b5524 100644 --- a/src/Server/KeeperTCPHandlerFactory.h +++ b/src/Server/KeeperTCPHandlerFactory.h @@ -17,7 +17,7 @@ class KeeperTCPHandlerFactory : public TCPServerConnectionFactory private: ConfigGetter config_getter; std::shared_ptr keeper_dispatcher; - Poco::Logger * log; + LoggerPtr log; Poco::Timespan receive_timeout; Poco::Timespan send_timeout; @@ -37,7 +37,7 @@ public: bool secure) : config_getter(config_getter_) , keeper_dispatcher(keeper_dispatcher_) - , log(&Poco::Logger::get(std::string{"KeeperTCP"} + (secure ? "S" : "") + "HandlerFactory")) + , log(getLogger(std::string{"KeeperTCP"} + (secure ? "S" : "") + "HandlerFactory")) , receive_timeout(/* seconds = */ receive_timeout_seconds, /* microseconds = */ 0) , send_timeout(/* seconds = */ send_timeout_seconds, /* microseconds = */ 0) { diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index 969eb24d126..c159a09c874 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -78,7 +78,7 @@ MySQLHandler::MySQLHandler( : Poco::Net::TCPServerConnection(socket_) , server(server_) , tcp_server(tcp_server_) - , log(&Poco::Logger::get("MySQLHandler")) + , log(getLogger("MySQLHandler")) , connection_id(connection_id_) , auth_plugin(new MySQLProtocol::Authentication::Native41()) , read_event(read_event_) diff --git a/src/Server/MySQLHandler.h b/src/Server/MySQLHandler.h index 36d63ebca84..867a90a6205 100644 --- a/src/Server/MySQLHandler.h +++ b/src/Server/MySQLHandler.h @@ -81,7 +81,7 @@ protected: IServer & server; TCPServer & tcp_server; - Poco::Logger * log; + LoggerPtr log; uint32_t connection_id = 0; uint32_t server_capabilities = 0; diff --git a/src/Server/MySQLHandlerFactory.cpp b/src/Server/MySQLHandlerFactory.cpp index 79234c647aa..1dd43e6dab2 100644 --- a/src/Server/MySQLHandlerFactory.cpp +++ b/src/Server/MySQLHandlerFactory.cpp @@ -23,7 +23,7 @@ namespace ErrorCodes MySQLHandlerFactory::MySQLHandlerFactory(IServer & server_, const ProfileEvents::Event & read_event_, const ProfileEvents::Event & write_event_) : server(server_) - , log(&Poco::Logger::get("MySQLHandlerFactory")) + , log(getLogger("MySQLHandlerFactory")) , read_event(read_event_) , write_event(write_event_) { diff --git a/src/Server/MySQLHandlerFactory.h b/src/Server/MySQLHandlerFactory.h index 307ee3b2f0d..4108269d738 100644 --- a/src/Server/MySQLHandlerFactory.h +++ b/src/Server/MySQLHandlerFactory.h @@ -20,7 +20,7 @@ class MySQLHandlerFactory : public TCPServerConnectionFactory { private: IServer & server; - Poco::Logger * log; + LoggerPtr log; #if USE_SSL struct RSADeleter diff --git a/src/Server/PostgreSQLHandler.h b/src/Server/PostgreSQLHandler.h index 57b91a0ad04..1c23d896415 100644 --- a/src/Server/PostgreSQLHandler.h +++ b/src/Server/PostgreSQLHandler.h @@ -40,7 +40,7 @@ public: void run() final; private: - Poco::Logger * log = &Poco::Logger::get("PostgreSQLHandler"); + LoggerPtr log = getLogger("PostgreSQLHandler"); IServer & server; TCPServer & tcp_server; diff --git a/src/Server/PostgreSQLHandlerFactory.cpp b/src/Server/PostgreSQLHandlerFactory.cpp index 096bbbdcda9..29eb7567976 100644 --- a/src/Server/PostgreSQLHandlerFactory.cpp +++ b/src/Server/PostgreSQLHandlerFactory.cpp @@ -7,7 +7,7 @@ namespace DB PostgreSQLHandlerFactory::PostgreSQLHandlerFactory(IServer & server_, const ProfileEvents::Event & read_event_, const ProfileEvents::Event & write_event_) : server(server_) - , log(&Poco::Logger::get("PostgreSQLHandlerFactory")) + , log(getLogger("PostgreSQLHandlerFactory")) , read_event(read_event_) , write_event(write_event_) { diff --git a/src/Server/PostgreSQLHandlerFactory.h b/src/Server/PostgreSQLHandlerFactory.h index e5f762fca6d..43674306ff6 100644 --- a/src/Server/PostgreSQLHandlerFactory.h +++ b/src/Server/PostgreSQLHandlerFactory.h @@ -14,7 +14,7 @@ class PostgreSQLHandlerFactory : public TCPServerConnectionFactory { private: IServer & server; - Poco::Logger * log; + LoggerPtr log; ProfileEvents::Event read_event; ProfileEvents::Event write_event; diff --git a/src/Server/ProxyV1Handler.h b/src/Server/ProxyV1Handler.h index b50c2acbc55..a044b9a966b 100644 --- a/src/Server/ProxyV1Handler.h +++ b/src/Server/ProxyV1Handler.h @@ -13,7 +13,7 @@ class ProxyV1Handler : public Poco::Net::TCPServerConnection using StreamSocket = Poco::Net::StreamSocket; public: explicit ProxyV1Handler(const StreamSocket & socket, IServer & server_, const std::string & conf_name_, TCPProtocolStackData & stack_data_) - : Poco::Net::TCPServerConnection(socket), log(&Poco::Logger::get("ProxyV1Handler")), server(server_), conf_name(conf_name_), stack_data(stack_data_) {} + : Poco::Net::TCPServerConnection(socket), log(getLogger("ProxyV1Handler")), server(server_), conf_name(conf_name_), stack_data(stack_data_) {} void run() override; @@ -21,7 +21,7 @@ protected: bool readWord(int max_len, std::string & word, bool & eol); private: - Poco::Logger * log; + LoggerPtr log; IServer & server; std::string conf_name; TCPProtocolStackData & stack_data; diff --git a/src/Server/ProxyV1HandlerFactory.h b/src/Server/ProxyV1HandlerFactory.h index 028596d745d..0398c8c1ccf 100644 --- a/src/Server/ProxyV1HandlerFactory.h +++ b/src/Server/ProxyV1HandlerFactory.h @@ -16,7 +16,7 @@ class ProxyV1HandlerFactory : public TCPServerConnectionFactory { private: IServer & server; - Poco::Logger * log; + LoggerPtr log; std::string conf_name; class DummyTCPHandler : public Poco::Net::TCPServerConnection @@ -28,7 +28,7 @@ private: public: explicit ProxyV1HandlerFactory(IServer & server_, const std::string & conf_name_) - : server(server_), log(&Poco::Logger::get("ProxyV1HandlerFactory")), conf_name(conf_name_) + : server(server_), log(getLogger("ProxyV1HandlerFactory")), conf_name(conf_name_) { } diff --git a/src/Server/ReplicasStatusHandler.cpp b/src/Server/ReplicasStatusHandler.cpp index 07f3b67b6a7..91c6bd722d3 100644 --- a/src/Server/ReplicasStatusHandler.cpp +++ b/src/Server/ReplicasStatusHandler.cpp @@ -118,7 +118,7 @@ void ReplicasStatusHandler::handleRequest(HTTPServerRequest & request, HTTPServe } catch (...) { - LOG_ERROR((&Poco::Logger::get("ReplicasStatusHandler")), "Cannot send exception to client"); + LOG_ERROR((getLogger("ReplicasStatusHandler")), "Cannot send exception to client"); } } } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index fa7206eeaac..ec6b374518d 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -189,7 +189,7 @@ TCPHandler::TCPHandler(IServer & server_, TCPServer & tcp_server_, const Poco::N , server(server_) , tcp_server(tcp_server_) , parse_proxy_protocol(parse_proxy_protocol_) - , log(&Poco::Logger::get("TCPHandler")) + , log(getLogger("TCPHandler")) , read_event(read_event_) , write_event(write_event_) , server_display_name(std::move(server_display_name_)) @@ -200,7 +200,7 @@ TCPHandler::TCPHandler(IServer & server_, TCPServer & tcp_server_, const Poco::N : Poco::Net::TCPServerConnection(socket_) , server(server_) , tcp_server(tcp_server_) - , log(&Poco::Logger::get("TCPHandler")) + , log(getLogger("TCPHandler")) , forwarded_for(stack_data.forwarded_for) , certificate(stack_data.certificate) , read_event(read_event_) diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index 4eb84ee5eee..26cecf46662 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -160,7 +160,7 @@ private: IServer & server; TCPServer & tcp_server; bool parse_proxy_protocol = false; - Poco::Logger * log; + LoggerPtr log; String forwarded_for; String certificate; diff --git a/src/Server/TCPHandlerFactory.h b/src/Server/TCPHandlerFactory.h index 3eb032f4250..d65c9898b23 100644 --- a/src/Server/TCPHandlerFactory.h +++ b/src/Server/TCPHandlerFactory.h @@ -18,7 +18,7 @@ class TCPHandlerFactory : public TCPServerConnectionFactory private: IServer & server; bool parse_proxy_protocol = false; - Poco::Logger * log; + LoggerPtr log; std::string server_display_name; ProfileEvents::Event read_event; @@ -38,7 +38,7 @@ public: */ TCPHandlerFactory(IServer & server_, bool secure_, bool parse_proxy_protocol_, const ProfileEvents::Event & read_event_ = ProfileEvents::end(), const ProfileEvents::Event & write_event_ = ProfileEvents::end()) : server(server_), parse_proxy_protocol(parse_proxy_protocol_) - , log(&Poco::Logger::get(std::string("TCP") + (secure_ ? "S" : "") + "HandlerFactory")) + , log(getLogger(std::string("TCP") + (secure_ ? "S" : "") + "HandlerFactory")) , read_event(read_event_) , write_event(write_event_) { diff --git a/src/Server/TCPProtocolStackFactory.h b/src/Server/TCPProtocolStackFactory.h index 7373e6e1c4e..b76bb8d72fd 100644 --- a/src/Server/TCPProtocolStackFactory.h +++ b/src/Server/TCPProtocolStackFactory.h @@ -23,7 +23,7 @@ class TCPProtocolStackFactory : public TCPServerConnectionFactory { private: IServer & server [[maybe_unused]]; - Poco::Logger * log; + LoggerPtr log; std::string conf_name; std::vector stack; AllowedClientHosts allowed_client_hosts; @@ -38,7 +38,7 @@ private: public: template explicit TCPProtocolStackFactory(IServer & server_, const std::string & conf_name_, T... factory) - : server(server_), log(&Poco::Logger::get("TCPProtocolStackFactory")), conf_name(conf_name_), stack({factory...}) + : server(server_), log(getLogger("TCPProtocolStackFactory")), conf_name(conf_name_), stack({factory...}) { const auto & config = server.config(); /// Fill list of allowed hosts. diff --git a/src/Server/TLSHandlerFactory.h b/src/Server/TLSHandlerFactory.h index 9e3002d2971..19602c7d25e 100644 --- a/src/Server/TLSHandlerFactory.h +++ b/src/Server/TLSHandlerFactory.h @@ -19,7 +19,7 @@ class TLSHandlerFactory : public TCPServerConnectionFactory { private: IServer & server; - Poco::Logger * log; + LoggerPtr log; std::string conf_name; class DummyTCPHandler : public Poco::Net::TCPServerConnection @@ -31,7 +31,7 @@ private: public: explicit TLSHandlerFactory(IServer & server_, const std::string & conf_name_) - : server(server_), log(&Poco::Logger::get("TLSHandlerFactory")), conf_name(conf_name_) + : server(server_), log(getLogger("TLSHandlerFactory")), conf_name(conf_name_) { } diff --git a/src/Storages/Cache/ExternalDataSourceCache.h b/src/Storages/Cache/ExternalDataSourceCache.h index 937801c4767..a5dea2f63db 100644 --- a/src/Storages/Cache/ExternalDataSourceCache.h +++ b/src/Storages/Cache/ExternalDataSourceCache.h @@ -91,7 +91,7 @@ private: std::mutex mutex; std::unique_ptr lru_caches; - Poco::Logger * log = &Poco::Logger::get("ExternalDataSourceCache"); + LoggerPtr log = getLogger("ExternalDataSourceCache"); String calculateLocalPath(IRemoteFileMetadataPtr meta) const; diff --git a/src/Storages/Cache/RemoteCacheController.cpp b/src/Storages/Cache/RemoteCacheController.cpp index b72f5336ea4..403d0c8e43b 100644 --- a/src/Storages/Cache/RemoteCacheController.cpp +++ b/src/Storages/Cache/RemoteCacheController.cpp @@ -20,7 +20,7 @@ namespace ErrorCodes std::shared_ptr RemoteCacheController::recover(const std::filesystem::path & local_path_) { - auto * log = &Poco::Logger::get("RemoteCacheController"); + auto log = getLogger("RemoteCacheController"); if (!std::filesystem::exists(local_path_ / "data.bin")) { diff --git a/src/Storages/Cache/RemoteCacheController.h b/src/Storages/Cache/RemoteCacheController.h index fafe363bbd4..782a6b89519 100644 --- a/src/Storages/Cache/RemoteCacheController.h +++ b/src/Storages/Cache/RemoteCacheController.h @@ -116,7 +116,7 @@ private: //std::shared_ptr remote_read_buffer; std::unique_ptr data_file_writer; - Poco::Logger * log = &Poco::Logger::get("RemoteCacheController"); + LoggerPtr log = getLogger("RemoteCacheController"); }; using RemoteCacheControllerPtr = std::shared_ptr; diff --git a/src/Storages/DataLakes/DeltaLakeMetadataParser.cpp b/src/Storages/DataLakes/DeltaLakeMetadataParser.cpp index b8bffb267e5..3584f137225 100644 --- a/src/Storages/DataLakes/DeltaLakeMetadataParser.cpp +++ b/src/Storages/DataLakes/DeltaLakeMetadataParser.cpp @@ -314,7 +314,7 @@ struct DeltaLakeMetadataParser::Impl return version; } - Poco::Logger * log = &Poco::Logger::get("DeltaLakeMetadataParser"); + LoggerPtr log = getLogger("DeltaLakeMetadataParser"); }; diff --git a/src/Storages/DataLakes/HudiMetadataParser.cpp b/src/Storages/DataLakes/HudiMetadataParser.cpp index 78d69c83989..699dfe8fda0 100644 --- a/src/Storages/DataLakes/HudiMetadataParser.cpp +++ b/src/Storages/DataLakes/HudiMetadataParser.cpp @@ -50,7 +50,7 @@ struct HudiMetadataParser::Impl */ Strings processMetadataFiles(const Configuration & configuration) { - auto * log = &Poco::Logger::get("HudiMetadataParser"); + auto log = getLogger("HudiMetadataParser"); const auto keys = MetadataReadHelper::listFiles(configuration, "", Poco::toLower(configuration.format)); diff --git a/src/Storages/DataLakes/IStorageDataLake.h b/src/Storages/DataLakes/IStorageDataLake.h index d87b1272ba6..db3f835494f 100644 --- a/src/Storages/DataLakes/IStorageDataLake.h +++ b/src/Storages/DataLakes/IStorageDataLake.h @@ -25,7 +25,7 @@ public: explicit IStorageDataLake(const Configuration & configuration_, ContextPtr context_, bool attach, Args && ...args) : Storage(getConfigurationForDataRead(configuration_, context_, {}, attach), context_, std::forward(args)...) , base_configuration(configuration_) - , log(&Poco::Logger::get(getName())) {} // NOLINT(clang-analyzer-optin.cplusplus.VirtualCall) + , log(getLogger(getName())) {} // NOLINT(clang-analyzer-optin.cplusplus.VirtualCall) template static StoragePtr create(const Configuration & configuration_, ContextPtr context_, bool attach, Args && ...args) @@ -78,7 +78,7 @@ private: configuration.keys = keys; LOG_TRACE( - &Poco::Logger::get("DataLake"), + getLogger("DataLake"), "New configuration path: {}, keys: {}", configuration.getPath(), fmt::join(configuration.keys, ", ")); @@ -112,7 +112,7 @@ private: Configuration base_configuration; std::mutex configuration_update_mutex; - Poco::Logger * log; + LoggerPtr log; }; diff --git a/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp b/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp index e0c7e26a2e1..e01a9a831c0 100644 --- a/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp +++ b/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp @@ -58,7 +58,7 @@ IcebergMetadata::IcebergMetadata( , manifest_list_file(std::move(manifest_list_file_)) , current_schema_id(current_schema_id_) , schema(std::move(schema_)) - , log(&Poco::Logger::get("IcebergMetadata")) + , log(getLogger("IcebergMetadata")) { } @@ -375,7 +375,7 @@ std::pair getMetadataFileAndVersion(const StorageS3::Configuratio std::unique_ptr parseIcebergMetadata(const StorageS3::Configuration & configuration, ContextPtr context_) { const auto [metadata_version, metadata_file_path] = getMetadataFileAndVersion(configuration); - LOG_DEBUG(&Poco::Logger::get("IcebergMetadata"), "Parse metadata {}", metadata_file_path); + LOG_DEBUG(getLogger("IcebergMetadata"), "Parse metadata {}", metadata_file_path); auto buf = S3DataLakeMetadataReadHelper::createReadBuffer(metadata_file_path, context_, configuration); String json_str; readJSONObjectPossiblyInvalid(json_str, *buf); diff --git a/src/Storages/DataLakes/Iceberg/IcebergMetadata.h b/src/Storages/DataLakes/Iceberg/IcebergMetadata.h index d42ad84f472..3e6a2ec3415 100644 --- a/src/Storages/DataLakes/Iceberg/IcebergMetadata.h +++ b/src/Storages/DataLakes/Iceberg/IcebergMetadata.h @@ -84,7 +84,7 @@ private: Int32 current_schema_id; NamesAndTypesList schema; Strings data_files; - Poco::Logger * log; + LoggerPtr log; }; diff --git a/src/Storages/DataLakes/S3MetadataReader.cpp b/src/Storages/DataLakes/S3MetadataReader.cpp index ac472c190e4..d66e21550a3 100644 --- a/src/Storages/DataLakes/S3MetadataReader.cpp +++ b/src/Storages/DataLakes/S3MetadataReader.cpp @@ -77,7 +77,7 @@ std::vector S3DataLakeMetadataReadHelper::listFiles( is_finished = !outcome.GetResult().GetIsTruncated(); } - LOG_TRACE(&Poco::Logger::get("S3DataLakeMetadataReadHelper"), "Listed {} files", res.size()); + LOG_TRACE(getLogger("S3DataLakeMetadataReadHelper"), "Listed {} files", res.size()); return res; } diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index 26fa489a63d..4e01cb2c6cf 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -60,7 +60,7 @@ namespace { template -ConnectionPoolPtrs createPoolsForAddresses(const Cluster::Addresses & addresses, PoolFactory && factory, Poco::Logger * log) +ConnectionPoolPtrs createPoolsForAddresses(const Cluster::Addresses & addresses, PoolFactory && factory, LoggerPtr log) { ConnectionPoolPtrs pools; @@ -121,7 +121,7 @@ DistributedAsyncInsertDirectoryQueue::DistributedAsyncInsertDirectoryQueue( , default_sleep_time(storage.getDistributedSettingsRef().background_insert_sleep_time_ms.totalMilliseconds()) , sleep_time(default_sleep_time) , max_sleep_time(storage.getDistributedSettingsRef().background_insert_max_sleep_time_ms.totalMilliseconds()) - , log(&Poco::Logger::get(getLoggerName())) + , log(getLogger(getLoggerName())) , monitor_blocker(monitor_blocker_) , metric_pending_bytes(CurrentMetrics::DistributedBytesToInsert, 0) , metric_pending_files(CurrentMetrics::DistributedFilesToInsert, 0) diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.h b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.h index 8bbd99c786a..f7d7553851a 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.h +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.h @@ -145,7 +145,7 @@ private: const std::chrono::milliseconds max_sleep_time; std::chrono::time_point last_decrease_time {std::chrono::system_clock::now()}; std::mutex mutex; - Poco::Logger * log; + LoggerPtr log; ActionBlocker & monitor_blocker; BackgroundSchedulePoolTaskHolder task_handle; diff --git a/src/Storages/Distributed/DistributedAsyncInsertHeader.cpp b/src/Storages/Distributed/DistributedAsyncInsertHeader.cpp index a8ed89e66f1..cfcee4dc8a2 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertHeader.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertHeader.cpp @@ -18,7 +18,7 @@ namespace ErrorCodes extern const int CHECKSUM_DOESNT_MATCH; } -DistributedAsyncInsertHeader DistributedAsyncInsertHeader::read(ReadBufferFromFile & in, Poco::Logger * log) +DistributedAsyncInsertHeader DistributedAsyncInsertHeader::read(ReadBufferFromFile & in, LoggerPtr log) { DistributedAsyncInsertHeader distributed_header; diff --git a/src/Storages/Distributed/DistributedAsyncInsertHeader.h b/src/Storages/Distributed/DistributedAsyncInsertHeader.h index a7330fa5ef1..fb4b4696463 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertHeader.h +++ b/src/Storages/Distributed/DistributedAsyncInsertHeader.h @@ -38,7 +38,7 @@ struct DistributedAsyncInsertHeader std::string block_header_string; Block block_header; - static DistributedAsyncInsertHeader read(ReadBufferFromFile & in, Poco::Logger * log); + static DistributedAsyncInsertHeader read(ReadBufferFromFile & in, LoggerPtr log); OpenTelemetry::TracingContextHolderPtr createTracingContextHolder(const char * function, std::shared_ptr open_telemetry_span_log) const; }; diff --git a/src/Storages/Distributed/DistributedAsyncInsertHelpers.cpp b/src/Storages/Distributed/DistributedAsyncInsertHelpers.cpp index 98073ba1e08..a9bdef31711 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertHelpers.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertHelpers.cpp @@ -72,7 +72,7 @@ void writeRemoteConvert( RemoteInserter & remote, bool compression_expected, ReadBufferFromFile & in, - Poco::Logger * log) + LoggerPtr log) { if (!remote.getHeader()) { diff --git a/src/Storages/Distributed/DistributedAsyncInsertHelpers.h b/src/Storages/Distributed/DistributedAsyncInsertHelpers.h index 9543450418c..202d9ff6fff 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertHelpers.h +++ b/src/Storages/Distributed/DistributedAsyncInsertHelpers.h @@ -1,9 +1,7 @@ #pragma once -namespace Poco -{ -class Logger; -} +#include + namespace DB { @@ -30,6 +28,6 @@ void writeRemoteConvert( RemoteInserter & remote, bool compression_expected, ReadBufferFromFile & in, - Poco::Logger * log); + LoggerPtr log); } diff --git a/src/Storages/Distributed/DistributedAsyncInsertSource.cpp b/src/Storages/Distributed/DistributedAsyncInsertSource.cpp index 7992636ac11..33e53da2857 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertSource.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertSource.cpp @@ -10,7 +10,7 @@ namespace DB struct DistributedAsyncInsertSource::Data { - Poco::Logger * log = nullptr; + LoggerPtr log = nullptr; ReadBufferFromFile in; CompressedReadBuffer decompressing_in; @@ -19,7 +19,7 @@ struct DistributedAsyncInsertSource::Data Block first_block; explicit Data(const String & file_name) - : log(&Poco::Logger::get("DistributedAsyncInsertSource")) + : log(getLogger("DistributedAsyncInsertSource")) , in(file_name) , decompressing_in(in) , block_in(decompressing_in, DistributedAsyncInsertHeader::read(in, log).revision) diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index 650539ef1e9..1efa98d0c13 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -62,7 +62,7 @@ namespace ErrorCodes extern const int ABORTED; } -static Block adoptBlock(const Block & header, const Block & block, Poco::Logger * log) +static Block adoptBlock(const Block & header, const Block & block, LoggerPtr log) { if (blocksHaveEqualStructure(header, block)) return block; @@ -84,7 +84,7 @@ static Block adoptBlock(const Block & header, const Block & block, Poco::Logger } -static void writeBlockConvert(PushingPipelineExecutor & executor, const Block & block, size_t repeats, Poco::Logger * log) +static void writeBlockConvert(PushingPipelineExecutor & executor, const Block & block, size_t repeats, LoggerPtr log) { Block adopted_block = adoptBlock(executor.getHeader(), block, log); for (size_t i = 0; i < repeats; ++i) @@ -126,7 +126,7 @@ DistributedSink::DistributedSink( , insert_timeout(insert_timeout_) , main_table(main_table_) , columns_to_send(columns_to_send_.begin(), columns_to_send_.end()) - , log(&Poco::Logger::get("DistributedSink")) + , log(getLogger("DistributedSink")) { const auto & settings = context->getSettingsRef(); if (settings.max_distributed_depth && context->getClientInfo().distributed_depth >= settings.max_distributed_depth) diff --git a/src/Storages/Distributed/DistributedSink.h b/src/Storages/Distributed/DistributedSink.h index 1bb4419e1a5..654c1db354f 100644 --- a/src/Storages/Distributed/DistributedSink.h +++ b/src/Storages/Distributed/DistributedSink.h @@ -152,7 +152,7 @@ private: std::atomic finished_jobs_count{0}; - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/src/Storages/FileLog/FileLogConsumer.cpp b/src/Storages/FileLog/FileLogConsumer.cpp index bfe081c7bad..1bd3026ab8c 100644 --- a/src/Storages/FileLog/FileLogConsumer.cpp +++ b/src/Storages/FileLog/FileLogConsumer.cpp @@ -22,7 +22,7 @@ FileLogConsumer::FileLogConsumer( ContextPtr context_, size_t stream_number_, size_t max_streams_number_) - : log(&Poco::Logger::get("FileLogConsumer " + toString(stream_number_))) + : log(getLogger("FileLogConsumer " + toString(stream_number_))) , storage(storage_) , batch_size(max_batch_size) , poll_timeout(poll_timeout_) diff --git a/src/Storages/FileLog/FileLogConsumer.h b/src/Storages/FileLog/FileLogConsumer.h index b19f3a9350b..e44bfeb1806 100644 --- a/src/Storages/FileLog/FileLogConsumer.h +++ b/src/Storages/FileLog/FileLogConsumer.h @@ -42,7 +42,7 @@ private: BufferStatus buffer_status = BufferStatus::INIT; - Poco::Logger * log; + LoggerPtr log; StorageFileLog & storage; diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index 9d488616e85..844b31fd7c9 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -6,7 +6,7 @@ namespace DB FileLogDirectoryWatcher::FileLogDirectoryWatcher(const std::string & path_, StorageFileLog & storage_, ContextPtr context_) : path(path_) , storage(storage_) - , log(&Poco::Logger::get("FileLogDirectoryWatcher(" + path + ")")) + , log(getLogger("FileLogDirectoryWatcher(" + path + ")")) , dw(std::make_unique(*this, path, context_)) { } diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index 9b7afcf8e12..1cf3697c7c0 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -65,7 +65,7 @@ private: /// accessed in thread created by dw. Events events; - Poco::Logger * log; + LoggerPtr log; std::mutex mutex; diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index ef776a3d313..9c7648ef658 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -139,7 +139,7 @@ StorageFileLog::StorageFileLog( , path(path_) , metadata_base_path(std::filesystem::path(metadata_base_path_) / "metadata") , format_name(format_name_) - , log(&Poco::Logger::get("StorageFileLog (" + table_id_.table_name + ")")) + , log(getLogger("StorageFileLog (" + table_id_.table_name + ")")) , disk(getContext()->getStoragePolicy("default")->getDisks().at(0)) , milliseconds_to_wait(filelog_settings->poll_directory_watch_events_backoff_init.totalMilliseconds()) { diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 33442d8b33b..cc5815a1cef 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -149,7 +149,7 @@ private: FileInfos file_infos; const String format_name; - Poco::Logger * log; + LoggerPtr log; DiskPtr disk; diff --git a/src/Storages/Freeze.cpp b/src/Storages/Freeze.cpp index b9642ec7907..a5a5a07c9a1 100644 --- a/src/Storages/Freeze.cpp +++ b/src/Storages/Freeze.cpp @@ -76,7 +76,7 @@ bool FreezeMetaData::load(DiskPtr data_disk, const String & path) readIntText(version, buffer); if (version < 1 || version > 2) { - LOG_ERROR(&Poco::Logger::get("FreezeMetaData"), "Unknown frozen metadata version: {}", version); + LOG_ERROR(getLogger("FreezeMetaData"), "Unknown frozen metadata version: {}", version); return false; } DB::assertChar('\n', buffer); diff --git a/src/Storages/Freeze.h b/src/Storages/Freeze.h index a64be7465dd..5775653aaea 100644 --- a/src/Storages/Freeze.h +++ b/src/Storages/Freeze.h @@ -38,7 +38,7 @@ public: private: ContextPtr local_context; zkutil::ZooKeeperPtr zookeeper; - Poco::Logger * log = &Poco::Logger::get("Unfreezer"); + LoggerPtr log = getLogger("Unfreezer"); static constexpr std::string_view backup_directory_prefix = "shadow"; static bool removeFreezedPart(DiskPtr disk, const String & path, const String & part_name, ContextPtr local_context, zkutil::ZooKeeperPtr zookeeper); }; diff --git a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp index 553473fcc9e..65df2c020ba 100644 --- a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp @@ -44,7 +44,7 @@ AsynchronousReadBufferFromHDFS::AsynchronousReadBufferFromHDFS( , prefetch_buffer(settings_.remote_fs_buffer_size) , read_until_position(impl->getFileSize()) , use_prefetch(settings_.remote_fs_prefetch) - , log(&Poco::Logger::get("AsynchronousReadBufferFromHDFS")) + , log(getLogger("AsynchronousReadBufferFromHDFS")) { ProfileEvents::increment(ProfileEvents::RemoteFSBuffers); } diff --git a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h index 9c01bd6e434..1d3e8b8e3e9 100644 --- a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h +++ b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h @@ -62,7 +62,7 @@ private: std::optional read_until_position; bool use_prefetch; - Poco::Logger * log; + LoggerPtr log; /// Metrics to profile prefetch Stopwatch interval_watch; diff --git a/src/Storages/HDFS/HDFSCommon.cpp b/src/Storages/HDFS/HDFSCommon.cpp index 12b32b740de..f9a55a1285a 100644 --- a/src/Storages/HDFS/HDFSCommon.cpp +++ b/src/Storages/HDFS/HDFSCommon.cpp @@ -55,7 +55,7 @@ void HDFSBuilderWrapper::loadFromConfig( need_kinit = true; hadoop_kerberos_keytab = config.getString(key_path); #else // USE_KRB5 - LOG_WARNING(&Poco::Logger::get("HDFSClient"), "hadoop_kerberos_keytab parameter is ignored because ClickHouse was built without support of krb5 library."); + LOG_WARNING(getLogger("HDFSClient"), "hadoop_kerberos_keytab parameter is ignored because ClickHouse was built without support of krb5 library."); #endif // USE_KRB5 continue; } @@ -66,7 +66,7 @@ void HDFSBuilderWrapper::loadFromConfig( hadoop_kerberos_principal = config.getString(key_path); hdfsBuilderSetPrincipal(hdfs_builder, hadoop_kerberos_principal.c_str()); #else // USE_KRB5 - LOG_WARNING(&Poco::Logger::get("HDFSClient"), "hadoop_kerberos_principal parameter is ignored because ClickHouse was built without support of krb5 library."); + LOG_WARNING(getLogger("HDFSClient"), "hadoop_kerberos_principal parameter is ignored because ClickHouse was built without support of krb5 library."); #endif // USE_KRB5 continue; } @@ -81,7 +81,7 @@ void HDFSBuilderWrapper::loadFromConfig( hadoop_security_kerberos_ticket_cache_path = config.getString(key_path); // standard param - pass further #else // USE_KRB5 - LOG_WARNING(&Poco::Logger::get("HDFSClient"), "hadoop.security.kerberos.ticket.cache.path parameter is ignored because ClickHouse was built without support of krb5 library."); + LOG_WARNING(getLogger("HDFSClient"), "hadoop.security.kerberos.ticket.cache.path parameter is ignored because ClickHouse was built without support of krb5 library."); #endif // USE_KRB5 } @@ -95,7 +95,7 @@ void HDFSBuilderWrapper::loadFromConfig( #if USE_KRB5 void HDFSBuilderWrapper::runKinit() { - LOG_DEBUG(&Poco::Logger::get("HDFSClient"), "Running KerberosInit"); + LOG_DEBUG(getLogger("HDFSClient"), "Running KerberosInit"); try { kerberosInit(hadoop_kerberos_keytab,hadoop_kerberos_principal,hadoop_security_kerberos_ticket_cache_path); @@ -104,7 +104,7 @@ void HDFSBuilderWrapper::runKinit() { throw Exception(ErrorCodes::KERBEROS_ERROR, "KerberosInit failure: {}", getExceptionMessage(e, false)); } - LOG_DEBUG(&Poco::Logger::get("HDFSClient"), "Finished KerberosInit"); + LOG_DEBUG(getLogger("HDFSClient"), "Finished KerberosInit"); } #endif // USE_KRB5 diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h index f1f0019d3e0..7170763c959 100644 --- a/src/Storages/HDFS/StorageHDFS.h +++ b/src/Storages/HDFS/StorageHDFS.h @@ -105,7 +105,7 @@ private: bool is_path_with_globs; NamesAndTypesList virtual_columns; - Poco::Logger * log = &Poco::Logger::get("StorageHDFS"); + LoggerPtr log = getLogger("StorageHDFS"); }; class PullingPipelineExecutor; diff --git a/src/Storages/HDFS/StorageHDFSCluster.cpp b/src/Storages/HDFS/StorageHDFSCluster.cpp index 2e8129b9845..fad29436102 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.cpp +++ b/src/Storages/HDFS/StorageHDFSCluster.cpp @@ -45,7 +45,7 @@ StorageHDFSCluster::StorageHDFSCluster( const ConstraintsDescription & constraints_, const String & compression_method_, bool structure_argument_was_provided_) - : IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageHDFSCluster (" + table_id_.table_name + ")"), structure_argument_was_provided_) + : IStorageCluster(cluster_name_, table_id_, getLogger("StorageHDFSCluster (" + table_id_.table_name + ")"), structure_argument_was_provided_) , uri(uri_) , format_name(format_name_) , compression_method(compression_method_) diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index 609adcf65c9..b58302f262e 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -25,7 +25,7 @@ static const int hive_metastore_client_recv_timeout_ms = 10000; static const int hive_metastore_client_send_timeout_ms = 10000; ThriftHiveMetastoreClientPool::ThriftHiveMetastoreClientPool(ThriftHiveMetastoreClientBuilder builder_) - : PoolBase(max_hive_metastore_client_connections, &Poco::Logger::get("ThriftHiveMetastoreClientPool")), builder(builder_) + : PoolBase(max_hive_metastore_client_connections, getLogger("ThriftHiveMetastoreClientPool")), builder(builder_) { } diff --git a/src/Storages/Hive/HiveCommon.h b/src/Storages/Hive/HiveCommon.h index e2c19fb1684..0f9d3364ffd 100644 --- a/src/Storages/Hive/HiveCommon.h +++ b/src/Storages/Hive/HiveCommon.h @@ -115,7 +115,7 @@ public: const bool empty_partition_keys; const HiveFilesCachePtr hive_files_cache; - Poco::Logger * log = &Poco::Logger::get("HiveMetastoreClient"); + LoggerPtr log = getLogger("HiveMetastoreClient"); }; @@ -138,7 +138,7 @@ private: CacheBase table_metadata_cache; ThriftHiveMetastoreClientPool client_pool; - Poco::Logger * log = &Poco::Logger::get("HiveMetastoreClient"); + LoggerPtr log = getLogger("HiveMetastoreClient"); }; using HiveMetastoreClientPtr = std::shared_ptr; diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index a9347ac4d99..a7ee8ff9891 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -411,7 +411,7 @@ private: bool generate_chunk_from_metadata{false}; UInt64 current_file_remained_rows = 0; - Poco::Logger * log = &Poco::Logger::get("StorageHive"); + LoggerPtr log = getLogger("StorageHive"); }; @@ -780,7 +780,7 @@ public: HDFSFSPtr fs_, HiveMetastoreClient::HiveTableMetadataPtr hive_table_metadata_, Block sample_block_, - Poco::Logger * log_, + LoggerPtr log_, ContextPtr context_, size_t max_block_size_, size_t num_streams_) @@ -805,7 +805,7 @@ private: HDFSFSPtr fs; HiveMetastoreClient::HiveTableMetadataPtr hive_table_metadata; Block sample_block; - Poco::Logger * log; + LoggerPtr log; ContextPtr context; size_t max_block_size; diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index b0ec96604cc..07440097f7a 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -149,7 +149,7 @@ private: std::shared_ptr storage_settings; - Poco::Logger * log = &Poco::Logger::get("StorageHive"); + LoggerPtr log = getLogger("StorageHive"); }; } diff --git a/src/Storages/IMessageProducer.cpp b/src/Storages/IMessageProducer.cpp index cf314696041..c723ec77b70 100644 --- a/src/Storages/IMessageProducer.cpp +++ b/src/Storages/IMessageProducer.cpp @@ -4,7 +4,7 @@ namespace DB { -IMessageProducer::IMessageProducer(Poco::Logger * log_) : log(log_) +IMessageProducer::IMessageProducer(LoggerPtr log_) : log(log_) { } diff --git a/src/Storages/IMessageProducer.h b/src/Storages/IMessageProducer.h index 12580d5f94a..c769c325191 100644 --- a/src/Storages/IMessageProducer.h +++ b/src/Storages/IMessageProducer.h @@ -16,7 +16,7 @@ namespace DB class IMessageProducer { public: - explicit IMessageProducer(Poco::Logger * log_); + explicit IMessageProducer(LoggerPtr log_); /// Do some preparations. virtual void start(const ContextPtr & context) = 0; @@ -30,14 +30,14 @@ public: virtual ~IMessageProducer() = default; protected: - Poco::Logger * log; + LoggerPtr log; }; /// Implements interface for concurrent message producing. class AsynchronousMessageProducer : public IMessageProducer { public: - explicit AsynchronousMessageProducer(Poco::Logger * log_) : IMessageProducer(log_) {} + explicit AsynchronousMessageProducer(LoggerPtr log_) : IMessageProducer(log_) {} /// Create and schedule task in BackgroundSchedulePool that will produce messages. void start(const ContextPtr & context) override; diff --git a/src/Storages/IStorageCluster.cpp b/src/Storages/IStorageCluster.cpp index 6f42d8f855c..c9eb07bd9d1 100644 --- a/src/Storages/IStorageCluster.cpp +++ b/src/Storages/IStorageCluster.cpp @@ -32,7 +32,7 @@ namespace DB IStorageCluster::IStorageCluster( const String & cluster_name_, const StorageID & table_id_, - Poco::Logger * log_, + LoggerPtr log_, bool structure_argument_was_provided_) : IStorage(table_id_) , log(log_) @@ -54,7 +54,7 @@ public: ASTPtr query_to_send_, QueryProcessingStage::Enum processed_stage_, ClusterPtr cluster_, - Poco::Logger * log_, + LoggerPtr log_, ContextPtr context_) : SourceStepWithFilter(DataStream{.header = std::move(sample_block)}) , storage(std::move(storage_)) @@ -71,7 +71,7 @@ private: ASTPtr query_to_send; QueryProcessingStage::Enum processed_stage; ClusterPtr cluster; - Poco::Logger * log; + LoggerPtr log; ContextPtr context; std::optional extension; diff --git a/src/Storages/IStorageCluster.h b/src/Storages/IStorageCluster.h index b233f20103d..8d93e94be9a 100644 --- a/src/Storages/IStorageCluster.h +++ b/src/Storages/IStorageCluster.h @@ -19,7 +19,7 @@ public: IStorageCluster( const String & cluster_name_, const StorageID & table_id_, - Poco::Logger * log_, + LoggerPtr log_, bool structure_argument_was_provided_); void read( @@ -46,7 +46,7 @@ protected: virtual void addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) = 0; private: - Poco::Logger * log; + LoggerPtr log; String cluster_name; bool structure_argument_was_provided; }; diff --git a/src/Storages/Kafka/KafkaConsumer.cpp b/src/Storages/Kafka/KafkaConsumer.cpp index 40f2897322d..47167e19a38 100644 --- a/src/Storages/Kafka/KafkaConsumer.cpp +++ b/src/Storages/Kafka/KafkaConsumer.cpp @@ -47,7 +47,7 @@ const auto DRAIN_TIMEOUT_MS = 5000ms; KafkaConsumer::KafkaConsumer( - Poco::Logger * log_, + LoggerPtr log_, size_t max_batch_size, size_t poll_timeout_, bool intermediate_commit_, diff --git a/src/Storages/Kafka/KafkaConsumer.h b/src/Storages/Kafka/KafkaConsumer.h index c4dfc56312f..9cc78d42856 100644 --- a/src/Storages/Kafka/KafkaConsumer.h +++ b/src/Storages/Kafka/KafkaConsumer.h @@ -62,7 +62,7 @@ public: }; KafkaConsumer( - Poco::Logger * log_, + LoggerPtr log_, size_t max_batch_size, size_t poll_timeout_, bool intermediate_commit_, @@ -150,7 +150,7 @@ private: std::string rdkafka_stat; ConsumerPtr consumer; - Poco::Logger * log; + LoggerPtr log; const size_t batch_size = 1; const size_t poll_timeout = 0; size_t offsets_stored = 0; diff --git a/src/Storages/Kafka/KafkaProducer.cpp b/src/Storages/Kafka/KafkaProducer.cpp index edbfc76ef93..77676fb010b 100644 --- a/src/Storages/Kafka/KafkaProducer.cpp +++ b/src/Storages/Kafka/KafkaProducer.cpp @@ -18,7 +18,7 @@ namespace DB KafkaProducer::KafkaProducer( ProducerPtr producer_, const std::string & topic_, std::chrono::milliseconds poll_timeout, std::atomic & shutdown_called_, const Block & header) - : IMessageProducer(&Poco::Logger::get("KafkaProducer")) + : IMessageProducer(getLogger("KafkaProducer")) , producer(producer_) , topic(topic_) , timeout(poll_timeout) diff --git a/src/Storages/Kafka/KafkaSource.cpp b/src/Storages/Kafka/KafkaSource.cpp index 1fbd7e2d705..dc62c13f633 100644 --- a/src/Storages/Kafka/KafkaSource.cpp +++ b/src/Storages/Kafka/KafkaSource.cpp @@ -33,7 +33,7 @@ KafkaSource::KafkaSource( const StorageSnapshotPtr & storage_snapshot_, const ContextPtr & context_, const Names & columns, - Poco::Logger * log_, + LoggerPtr log_, size_t max_block_size_, bool commit_in_suffix_) : ISource(storage_snapshot_->getSampleBlockForColumns(columns)) diff --git a/src/Storages/Kafka/KafkaSource.h b/src/Storages/Kafka/KafkaSource.h index 485a8e55b6a..a1b94b15a19 100644 --- a/src/Storages/Kafka/KafkaSource.h +++ b/src/Storages/Kafka/KafkaSource.h @@ -22,7 +22,7 @@ public: const StorageSnapshotPtr & storage_snapshot_, const ContextPtr & context_, const Names & columns, - Poco::Logger * log_, + LoggerPtr log_, size_t max_block_size_, bool commit_in_suffix = false); ~KafkaSource() override; @@ -41,7 +41,7 @@ private: StorageSnapshotPtr storage_snapshot; ContextPtr context; Names column_names; - Poco::Logger * log; + LoggerPtr log; UInt64 max_block_size; KafkaConsumerPtr consumer; diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 522a381700d..aa347fc719d 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -327,7 +327,7 @@ StorageKafka::StorageKafka( , max_rows_per_message(kafka_settings->kafka_max_rows_per_message.value) , schema_name(getContext()->getMacros()->expand(kafka_settings->kafka_schema.value, macros_info)) , num_consumers(kafka_settings->kafka_num_consumers.value) - , log(&Poco::Logger::get("StorageKafka (" + table_id_.table_name + ")")) + , log(getLogger("StorageKafka (" + table_id_.table_name + ")")) , intermediate_commit(kafka_settings->kafka_commit_every_batch.value) , settings_adjustments(createSettingsAdjustments()) , thread_per_consumer(kafka_settings->kafka_thread_per_consumer.value) diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index d370d6018f7..f9a1e3ff6f3 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -101,7 +101,7 @@ private: const size_t max_rows_per_message; const String schema_name; const size_t num_consumers; /// total number of consumers - Poco::Logger * log; + LoggerPtr log; const bool intermediate_commit; const SettingsChanges settings_adjustments; diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 3c116321083..f81225bbee3 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -209,7 +209,7 @@ StorageLiveView::StorageLiveView( live_view_context = Context::createCopy(getContext()); live_view_context->makeQueryContext(); - log = &Poco::Logger::get("StorageLiveView (" + table_id_.database_name + "." + table_id_.table_name + ")"); + log = getLogger("StorageLiveView (" + table_id_.database_name + "." + table_id_.table_name + ")"); StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); diff --git a/src/Storages/LiveView/StorageLiveView.h b/src/Storages/LiveView/StorageLiveView.h index e0566d586ee..6b8780cb81b 100644 --- a/src/Storages/LiveView/StorageLiveView.h +++ b/src/Storages/LiveView/StorageLiveView.h @@ -184,7 +184,7 @@ private: ContextMutablePtr live_view_context; - Poco::Logger * log; + LoggerPtr log; bool is_periodically_refreshed = false; Seconds periodic_live_view_refresh; diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index bc26301e3b9..daf7bd65784 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -27,7 +27,7 @@ namespace ErrorCodes RefreshTask::RefreshTask( const ASTRefreshStrategy & strategy) - : log(&Poco::Logger::get("RefreshTask")) + : log(getLogger("RefreshTask")) , refresh_schedule(strategy) {} diff --git a/src/Storages/MaterializedView/RefreshTask.h b/src/Storages/MaterializedView/RefreshTask.h index 8a062f6f359..78599f4f4b4 100644 --- a/src/Storages/MaterializedView/RefreshTask.h +++ b/src/Storages/MaterializedView/RefreshTask.h @@ -62,7 +62,7 @@ public: void setFakeTime(std::optional t); private: - Poco::Logger * log = nullptr; + LoggerPtr log = nullptr; std::weak_ptr view_to_refresh; /// Protects interrupt_execution and running_executor. diff --git a/src/Storages/MergeTree/AsyncBlockIDsCache.cpp b/src/Storages/MergeTree/AsyncBlockIDsCache.cpp index cc3bc8fc2a8..9d64592ed64 100644 --- a/src/Storages/MergeTree/AsyncBlockIDsCache.cpp +++ b/src/Storages/MergeTree/AsyncBlockIDsCache.cpp @@ -60,7 +60,7 @@ AsyncBlockIDsCache::AsyncBlockIDsCache(TStorage & storage_) , update_wait(storage.getSettings()->async_block_ids_cache_update_wait_ms) , path(storage.getZooKeeperPath() + "/async_blocks") , log_name(storage.getStorageID().getFullTableName() + " (AsyncBlockIDsCache)") - , log(&Poco::Logger::get(log_name)) + , log(getLogger(log_name)) { task = storage.getContext()->getSchedulePool().createTask(log_name, [this]{ update(); }); } diff --git a/src/Storages/MergeTree/AsyncBlockIDsCache.h b/src/Storages/MergeTree/AsyncBlockIDsCache.h index 38c38da0033..bea012f1d32 100644 --- a/src/Storages/MergeTree/AsyncBlockIDsCache.h +++ b/src/Storages/MergeTree/AsyncBlockIDsCache.h @@ -43,7 +43,7 @@ private: BackgroundSchedulePool::TaskHolder task; const String log_name; - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp index f4b92ff8c57..0cb9eb84bf8 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp @@ -57,7 +57,7 @@ std::string DataPartStorageOnDiskBase::getRelativePath() const return fs::path(root_path) / part_dir / ""; } -std::optional DataPartStorageOnDiskBase::getRelativePathForPrefix(Poco::Logger * log, const String & prefix, bool detached, bool broken) const +std::optional DataPartStorageOnDiskBase::getRelativePathForPrefix(LoggerPtr log, const String & prefix, bool detached, bool broken) const { assert(!broken || detached); String res; @@ -471,7 +471,7 @@ MutableDataPartStoragePtr DataPartStorageOnDiskBase::clonePart( const DiskPtr & dst_disk, const ReadSettings & read_settings, const WriteSettings & write_settings, - Poco::Logger * log, + LoggerPtr log, const std::function & cancellation_hook) const { String path_to_clone = fs::path(to) / dir_path / ""; @@ -505,7 +505,7 @@ MutableDataPartStoragePtr DataPartStorageOnDiskBase::clonePart( void DataPartStorageOnDiskBase::rename( std::string new_root_path, std::string new_part_dir, - Poco::Logger * log, + LoggerPtr log, bool remove_new_dir_if_exists, bool fsync_part_dir) { @@ -564,7 +564,7 @@ void DataPartStorageOnDiskBase::remove( const MergeTreeDataPartChecksums & checksums, std::list projections, bool is_temp, - Poco::Logger * log) + LoggerPtr log) { /// NOTE We rename part to delete_tmp_ instead of delete_tmp_ to avoid race condition /// when we try to remove two parts with the same name, but different relative paths, @@ -722,7 +722,7 @@ void DataPartStorageOnDiskBase::clearDirectory( const CanRemoveDescription & can_remove_description, const MergeTreeDataPartChecksums & checksums, bool is_temp, - Poco::Logger * log) + LoggerPtr log) { auto disk = volume->getDisk(); auto [can_remove_shared_data, names_not_to_remove] = can_remove_description; diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h index 339acce5953..52dc850c7fd 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h @@ -25,7 +25,7 @@ public: UInt64 calculateTotalSizeOnDisk() const override; /// Returns path to place detached part in or nullopt if we don't need to detach part (if it already exists and has the same content) - std::optional getRelativePathForPrefix(Poco::Logger * log, const String & prefix, bool detached, bool broken) const override; + std::optional getRelativePathForPrefix(LoggerPtr log, const String & prefix, bool detached, bool broken) const override; /// Returns true if detached part already exists and has the same content (compares checksums.txt and the list of files) bool looksLikeBrokenDetachedPartHasTheSameContent(const String & detached_part_path, std::optional & original_checksums_content, @@ -74,14 +74,14 @@ public: const DiskPtr & dst_disk, const ReadSettings & read_settings, const WriteSettings & write_settings, - Poco::Logger * log, + LoggerPtr log, const std::function & cancellation_hook ) const override; void rename( std::string new_root_path, std::string new_part_dir, - Poco::Logger * log, + LoggerPtr log, bool remove_new_dir_if_exists, bool fsync_part_dir) override; @@ -90,7 +90,7 @@ public: const MergeTreeDataPartChecksums & checksums, std::list projections, bool is_temp, - Poco::Logger * log) override; + LoggerPtr log) override; void changeRootPath(const std::string & from_root, const std::string & to_root) override; void createDirectories() override; @@ -130,7 +130,7 @@ private: const CanRemoveDescription & can_remove_description, const MergeTreeDataPartChecksums & checksums, bool is_temp, - Poco::Logger * log); + LoggerPtr log); /// For names of expected data part files returns the actual names /// of files in filesystem to which data of these files is written. diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index a59f2a356e8..ce70fbe18e5 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -99,7 +99,7 @@ struct ReplicatedFetchReadCallback Service::Service(StorageReplicatedMergeTree & data_) : data(data_) - , log(&Poco::Logger::get(data.getStorageID().getNameForLogs() + " (Replicated PartsService)")) + , log(getLogger(data.getStorageID().getNameForLogs() + " (Replicated PartsService)")) {} std::string Service::getId(const std::string & node_id) const @@ -415,7 +415,7 @@ MergeTreeData::DataPartPtr Service::findPart(const String & name) Fetcher::Fetcher(StorageReplicatedMergeTree & data_) : data(data_) - , log(&Poco::Logger::get(data.getStorageID().getNameForLogs() + " (Fetcher)")) + , log(getLogger(data.getStorageID().getNameForLogs() + " (Fetcher)")) {} std::pair Fetcher::fetchSelectedPart( diff --git a/src/Storages/MergeTree/DataPartsExchange.h b/src/Storages/MergeTree/DataPartsExchange.h index 07939a660a8..8c15dc3cfdb 100644 --- a/src/Storages/MergeTree/DataPartsExchange.h +++ b/src/Storages/MergeTree/DataPartsExchange.h @@ -55,7 +55,7 @@ private: /// StorageReplicatedMergeTree::shutdown() waits for all parts exchange handlers to finish, /// so Service will never access dangling reference to storage StorageReplicatedMergeTree & data; - Poco::Logger * log; + LoggerPtr log; }; /** Client for getting the parts from the table *MergeTree. @@ -137,7 +137,7 @@ private: ThrottlerPtr throttler); StorageReplicatedMergeTree & data; - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp index 5741e11aa22..1ffb5177430 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp @@ -64,7 +64,7 @@ std::optional createEphemeralLockInZooKeeper( { const String & failed_op_path = ops[failed_idx]->getPath(); LOG_DEBUG( - &Poco::Logger::get("createEphemeralLockInZooKeeper"), + getLogger("createEphemeralLockInZooKeeper"), "Deduplication path already exists: deduplication_path={}", failed_op_path); return EphemeralLockInZooKeeper{"", nullptr, "", failed_op_path}; @@ -73,7 +73,7 @@ std::optional createEphemeralLockInZooKeeper( else if (responses[0]->error == Coordination::Error::ZNODEEXISTS) { LOG_DEBUG( - &Poco::Logger::get("createEphemeralLockInZooKeeper"), + getLogger("createEphemeralLockInZooKeeper"), "Deduplication path already exists: deduplication_path={}", deduplication_path); return {}; @@ -119,7 +119,7 @@ EphemeralLockInZooKeeper::~EphemeralLockInZooKeeper() { if (Coordination::isHardwareError(e.code)) LOG_DEBUG( - &Poco::Logger::get("EphemeralLockInZooKeeper"), + getLogger("EphemeralLockInZooKeeper"), "ZooKeeper communication error during unlock: code={} message='{}'", e.code, e.message()); @@ -130,7 +130,7 @@ EphemeralLockInZooKeeper::~EphemeralLockInZooKeeper() /// But it's possible that the multi op request can be executed on server side, and client will not get response due to network issue. /// In such case, assumeUnlocked() will not be called, so we'll get ZNONODE error here since the noded is already deleted LOG_DEBUG( - &Poco::Logger::get("EphemeralLockInZooKeeper"), + getLogger("EphemeralLockInZooKeeper"), "ZooKeeper node was already deleted: code={} message={}", e.code, e.message()); @@ -168,7 +168,7 @@ EphemeralLocksInAllPartitions::EphemeralLocksInAllPartitions( Coordination::Error rc = zookeeper->tryMulti(lock_ops, lock_responses); if (rc == Coordination::Error::ZBADVERSION) { - LOG_TRACE(&Poco::Logger::get("EphemeralLocksInAllPartitions"), "Someone has inserted a block in a new partition while we were creating locks. Retry."); + LOG_TRACE(getLogger("EphemeralLocksInAllPartitions"), "Someone has inserted a block in a new partition while we were creating locks. Retry."); continue; } else if (rc != Coordination::Error::ZOK) diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index afbe91a8a6d..5899ef58cd5 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -151,12 +151,12 @@ public: const MergeTreeDataPartChecksums & checksums, std::list projections, bool is_temp, - Poco::Logger * log) = 0; + LoggerPtr log) = 0; /// Get a name like 'prefix_partdir_tryN' which does not exist in a root dir. /// TODO: remove it. virtual std::optional getRelativePathForPrefix( - Poco::Logger * log, const String & prefix, bool detached, bool broken) const = 0; + LoggerPtr log, const String & prefix, bool detached, bool broken) const = 0; /// Reset part directory, used for in-memory parts. /// TODO: remove it. @@ -263,7 +263,7 @@ public: const DiskPtr & disk, const ReadSettings & read_settings, const WriteSettings & write_settings, - Poco::Logger * log, + LoggerPtr log, const std::function & cancellation_hook ) const = 0; @@ -314,7 +314,7 @@ public: virtual void rename( std::string new_root_path, std::string new_part_dir, - Poco::Logger * log, + LoggerPtr log, bool remove_new_dir_if_exists, bool fsync_part_dir) = 0; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 0c1bcff2c50..87f23b0da2a 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1663,7 +1663,7 @@ try metadata_manager->deleteAll(true); metadata_manager->assertAllDeleted(true); - getDataPartStorage().rename(to.parent_path(), to.filename(), storage.log.get(), remove_new_dir_if_exists, fsync_dir); + getDataPartStorage().rename(to.parent_path(), to.filename(), storage.log, remove_new_dir_if_exists, fsync_dir); metadata_manager->updateAll(true); auto new_projection_root_path = to.string(); @@ -1758,7 +1758,7 @@ void IMergeTreeDataPart::remove() } bool is_temporary_part = is_temp || state == MergeTreeDataPartState::Temporary; - getDataPartStorage().remove(std::move(can_remove_callback), checksums, projection_checksums, is_temporary_part, storage.log.get()); + getDataPartStorage().remove(std::move(can_remove_callback), checksums, projection_checksums, is_temporary_part, storage.log); } std::optional IMergeTreeDataPart::getRelativePathForPrefix(const String & prefix, bool detached, bool broken) const @@ -1775,7 +1775,7 @@ std::optional IMergeTreeDataPart::getRelativePathForPrefix(const String if (detached && parent_part) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot detach projection"); - return getDataPartStorage().getRelativePathForPrefix(storage.log.get(), prefix, detached, broken); + return getDataPartStorage().getRelativePathForPrefix(storage.log, prefix, detached, broken); } std::optional IMergeTreeDataPart::getRelativePathForDetachedPart(const String & prefix, bool broken) const @@ -1841,7 +1841,7 @@ MutableDataPartStoragePtr IMergeTreeDataPart::makeCloneOnDisk( throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not clone data part {} to empty directory.", name); String path_to_clone = fs::path(storage.relative_data_path) / directory_name / ""; - return getDataPartStorage().clonePart(path_to_clone, getDataPartStorage().getPartDirectory(), disk, read_settings, write_settings, storage.log.get(), cancellation_hook); + return getDataPartStorage().clonePart(path_to_clone, getDataPartStorage().getPartDirectory(), disk, read_settings, write_settings, storage.log, cancellation_hook); } UInt64 IMergeTreeDataPart::getIndexSizeFromFile() const diff --git a/src/Storages/MergeTree/InsertBlockInfo.cpp b/src/Storages/MergeTree/InsertBlockInfo.cpp index ac900f8cf09..2de3ae8996a 100644 --- a/src/Storages/MergeTree/InsertBlockInfo.cpp +++ b/src/Storages/MergeTree/InsertBlockInfo.cpp @@ -9,7 +9,7 @@ namespace ErrorCodes } AsyncInsertBlockInfo::AsyncInsertBlockInfo( - Poco::Logger * log_, + LoggerPtr log_, std::vector && block_id_, BlockWithPartition && block_, std::optional && unmerged_block_with_partition_) diff --git a/src/Storages/MergeTree/InsertBlockInfo.h b/src/Storages/MergeTree/InsertBlockInfo.h index 3882373c0fa..7d7ec0c9f29 100644 --- a/src/Storages/MergeTree/InsertBlockInfo.h +++ b/src/Storages/MergeTree/InsertBlockInfo.h @@ -8,7 +8,7 @@ namespace DB struct SyncInsertBlockInfo { SyncInsertBlockInfo( - Poco::Logger * /*log_*/, + LoggerPtr /*log_*/, std::string && block_id_, BlockWithPartition && /*block_*/, std::optional && /*unmerged_block_with_partition_*/) @@ -25,7 +25,7 @@ struct SyncInsertBlockInfo struct AsyncInsertBlockInfo { - Poco::Logger * log; + LoggerPtr log; std::vector block_id; BlockWithPartition block_with_partition; /// Some merging algorithms can mofidy the block which loses the information about the async insert offsets @@ -34,7 +34,7 @@ struct AsyncInsertBlockInfo std::unordered_map> block_id_to_offset_idx; AsyncInsertBlockInfo( - Poco::Logger * log_, + LoggerPtr log_, std::vector && block_id_, BlockWithPartition && block_, std::optional && unmerged_block_with_partition_); diff --git a/src/Storages/MergeTree/LeaderElection.h b/src/Storages/MergeTree/LeaderElection.h index 2e48892563b..3bd486fd54a 100644 --- a/src/Storages/MergeTree/LeaderElection.h +++ b/src/Storages/MergeTree/LeaderElection.h @@ -19,7 +19,7 @@ namespace zkutil * For now, every replica can become leader if there is no leader among replicas with old version. */ -void checkNoOldLeaders(Poco::Logger * log, ZooKeeper & zookeeper, const String path) +void checkNoOldLeaders(LoggerPtr log, ZooKeeper & zookeeper, const String path) { /// Previous versions (before 21.12) used to create ephemeral sequential node path/leader_election- /// Replica with the lexicographically smallest node name becomes leader (before 20.6) or enables multi-leader mode (since 20.6) diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index 23037b1ee7a..ae6e398026d 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -28,7 +28,7 @@ MergeFromLogEntryTask::MergeFromLogEntryTask( StorageReplicatedMergeTree & storage_, IExecutableTask::TaskResultCallback & task_result_callback_) : ReplicatedMergeMutateTaskBase( - &Poco::Logger::get( + getLogger( storage_.getStorageID().getShortName() + "::" + selected_entry_->log_entry->new_part_name + " (MergeFromLogEntryTask)"), storage_, selected_entry_, diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index b2a5796737d..6f5336baaad 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -228,7 +228,7 @@ private: size_t sum_compressed_bytes_upper_bound{0}; bool blocks_are_granules_size{false}; - Poco::Logger * log{&Poco::Logger::get("MergeTask::PrepareStage")}; + LoggerPtr log{getLogger("MergeTask::PrepareStage")}; /// Dependencies for next stages std::list::const_iterator it_name_and_type; @@ -354,7 +354,7 @@ private: MergeTasks tasks_for_projections; MergeTasks::iterator projections_iterator; - Poco::Logger * log{&Poco::Logger::get("MergeTask::MergeProjectionsStage")}; + LoggerPtr log{getLogger("MergeTask::MergeProjectionsStage")}; }; using MergeProjectionsRuntimeContextPtr = std::shared_ptr; diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp index a3f8e02f5eb..8cb0badc19b 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp @@ -144,7 +144,7 @@ bool MergeTreeBackgroundExecutor::trySchedule(ExecutableTaskPtr task) return true; } -void printExceptionWithRespectToAbort(Poco::Logger * log, const String & query_id) +void printExceptionWithRespectToAbort(LoggerPtr log, const String & query_id) { std::exception_ptr ex = std::current_exception(); diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h index 63f75ffc8d9..0ed03293589 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h @@ -307,7 +307,7 @@ private: std::condition_variable has_tasks TSA_GUARDED_BY(mutex); bool shutdown TSA_GUARDED_BY(mutex) = false; std::unique_ptr pool; - Poco::Logger * log = &Poco::Logger::get("MergeTreeBackgroundExecutor"); + LoggerPtr log = getLogger("MergeTreeBackgroundExecutor"); }; extern template class MergeTreeBackgroundExecutor; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ef18a1d46a8..39c113c240e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2114,7 +2114,7 @@ size_t MergeTreeData::clearOldTemporaryDirectories(const String & root_path, siz { /// Actually we don't rely on temporary_directories_lifetime when removing old temporaries directories, /// it's just an extra level of protection just in case we have a bug. - LOG_INFO(LogFrequencyLimiter(log.get(), 10), "{} is in use (by merge/mutation/INSERT) (consider increasing temporary_directories_lifetime setting)", full_path); + LOG_INFO(LogFrequencyLimiter(log, 10), "{} is in use (by merge/mutation/INSERT) (consider increasing temporary_directories_lifetime setting)", full_path); continue; } else if (!disk->exists(it->path())) @@ -8181,7 +8181,7 @@ ReservationPtr MergeTreeData::balancedReservation( } // Record submerging big parts in the tagger to clean them up. - tagger_ptr->emplace(*this, part_name, std::move(covered_parts), log.get()); + tagger_ptr->emplace(*this, part_name, std::move(covered_parts), log); } } } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 7ac46e12e1a..caef247500a 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include #include @@ -1602,10 +1602,10 @@ struct CurrentlySubmergingEmergingTagger MergeTreeData & storage; String emerging_part_name; MergeTreeData::DataPartsVector submerging_parts; - Poco::Logger * log; + LoggerPtr log; CurrentlySubmergingEmergingTagger( - MergeTreeData & storage_, const String & name_, MergeTreeData::DataPartsVector && parts_, Poco::Logger * log_) + MergeTreeData & storage_, const String & name_, MergeTreeData::DataPartsVector && parts_, LoggerPtr log_) : storage(storage_), emerging_part_name(name_), submerging_parts(std::move(parts_)), log(log_) { } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 66f593bbf33..a76d370d057 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -74,7 +74,7 @@ namespace ErrorCodes MergeTreeDataSelectExecutor::MergeTreeDataSelectExecutor(const MergeTreeData & data_) - : data(data_), log(&Poco::Logger::get(data.getLogName() + " (SelectExecutor)")) + : data(data_), log(getLogger(data.getLogName() + " (SelectExecutor)")) { } @@ -83,7 +83,7 @@ size_t MergeTreeDataSelectExecutor::getApproximateTotalRowsToRead( const StorageMetadataPtr & metadata_snapshot, const KeyCondition & key_condition, const Settings & settings, - Poco::Logger * log) + LoggerPtr log) { size_t rows_count = 0; @@ -167,7 +167,7 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( const StorageMetadataPtr & metadata_snapshot, ContextPtr context, bool sample_factor_column_queried, - Poco::Logger * log) + LoggerPtr log) { const Settings & settings = context->getSettingsRef(); /// Sampling. @@ -503,7 +503,7 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( const MergeTreeData & data, const ContextPtr & context, const PartitionIdToMaxBlock * max_block_numbers_to_read, - Poco::Logger * log, + LoggerPtr log, ReadFromMergeTree::IndexStats & index_stats) { chassert(alter_conversions.empty() || parts.size() == alter_conversions.size()); @@ -590,7 +590,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd const std::optional & part_offset_condition, const UsefulSkipIndexes & skip_indexes, const MergeTreeReaderSettings & reader_settings, - Poco::Logger * log, + LoggerPtr log, size_t num_streams, ReadFromMergeTree::IndexStats & index_stats, bool use_skip_indexes) @@ -1082,7 +1082,7 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange( const KeyCondition & key_condition, const std::optional & part_offset_condition, const Settings & settings, - Poco::Logger * log) + LoggerPtr log) { MarkRanges res; @@ -1322,7 +1322,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex( const MergeTreeReaderSettings & reader_settings, MarkCache * mark_cache, UncompressedCache * uncompressed_cache, - Poco::Logger * log) + LoggerPtr log) { if (!index_helper->getDeserializedFormat(part->getDataPartStorage(), index_helper->getFileName())) { @@ -1440,7 +1440,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingMergedIndex( const MergeTreeReaderSettings & reader_settings, MarkCache * mark_cache, UncompressedCache * uncompressed_cache, - Poco::Logger * log) + LoggerPtr log) { for (const auto & index_helper : indices) { @@ -1596,7 +1596,7 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( const PartitionIdToMaxBlock * max_block_numbers_to_read, ContextPtr query_context, PartFilterCounters & counters, - Poco::Logger * log) + LoggerPtr log) { /// process_parts prepare parts that have to be read for the query, /// returns false if duplicated parts' UUID have been met diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 93eac427f77..17975354187 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -71,7 +71,7 @@ public: const KeyCondition & key_condition, const std::optional & part_offset_condition, const Settings & settings, - Poco::Logger * log); + LoggerPtr log); private: const MergeTreeData & data; @@ -83,7 +83,7 @@ private: const StorageMetadataPtr & metadata_snapshot, const KeyCondition & key_condition, const Settings & settings, - Poco::Logger * log); + LoggerPtr log); static MarkRanges filterMarksUsingIndex( MergeTreeIndexPtr index_helper, @@ -94,7 +94,7 @@ private: const MergeTreeReaderSettings & reader_settings, MarkCache * mark_cache, UncompressedCache * uncompressed_cache, - Poco::Logger * log); + LoggerPtr log); static MarkRanges filterMarksUsingMergedIndex( MergeTreeIndices indices, @@ -105,7 +105,7 @@ private: const MergeTreeReaderSettings & reader_settings, MarkCache * mark_cache, UncompressedCache * uncompressed_cache, - Poco::Logger * log); + LoggerPtr log); struct PartFilterCounters { @@ -141,7 +141,7 @@ private: const PartitionIdToMaxBlock * max_block_numbers_to_read, ContextPtr query_context, PartFilterCounters & counters, - Poco::Logger * log); + LoggerPtr log); public: /// For given number rows and bytes, get the number of marks to read. @@ -184,7 +184,7 @@ public: const MergeTreeData & data, const ContextPtr & context, const PartitionIdToMaxBlock * max_block_numbers_to_read, - Poco::Logger * log, + LoggerPtr log, ReadFromMergeTree::IndexStats & index_stats); /// Filter parts using primary key and secondary indexes. @@ -199,7 +199,7 @@ public: const std::optional & part_offset_condition, const UsefulSkipIndexes & skip_indexes, const MergeTreeReaderSettings & reader_settings, - Poco::Logger * log, + LoggerPtr log, size_t num_streams, ReadFromMergeTree::IndexStats & index_stats, bool use_skip_indexes); @@ -216,7 +216,7 @@ public: const StorageMetadataPtr & metadata_snapshot, ContextPtr context, bool sample_factor_column_queried, - Poco::Logger * log); + LoggerPtr log); /// Check query limits: max_partitions_to_read, max_concurrent_queries. /// Also, return QueryIdHolder. If not null, we should keep it until query finishes. diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 25785b8aea0..ce3015c5dcb 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -115,7 +115,7 @@ void buildScatterSelector( if (max_parts && partitions_count >= max_parts && !throw_on_limit) { const auto & client_info = context->getClientInfo(); - Poco::Logger * log = &Poco::Logger::get("MergeTreeDataWriter"); + LoggerPtr log = getLogger("MergeTreeDataWriter"); LOG_WARNING(log, "INSERT query from initial_user {} (query ID: {}) inserted a block " "that created parts in {} partitions. This is being logged " @@ -335,7 +335,7 @@ Block MergeTreeDataWriter::mergeBlock( case MergeTreeData::MergingParams::Collapsing: return std::make_shared( block, 1, sort_description, merging_params.sign_column, - false, block_size + 1, /*block_size_bytes=*/0, &Poco::Logger::get("MergeTreeDataWriter")); + false, block_size + 1, /*block_size_bytes=*/0, getLogger("MergeTreeDataWriter")); case MergeTreeData::MergingParams::Summing: return std::make_shared( block, 1, sort_description, merging_params.columns_to_sum, @@ -591,7 +591,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( auto projection_block = projection.calculate(block, context); if (projection_block.rows()) { - auto proj_temp_part = writeProjectionPart(data, log.get(), projection_block, projection, new_data_part.get()); + auto proj_temp_part = writeProjectionPart(data, log, projection_block, projection, new_data_part.get()); new_data_part->addProjectionPart(projection.name, std::move(proj_temp_part.part)); for (auto & stream : proj_temp_part.streams) temp_part.streams.emplace_back(std::move(stream)); @@ -618,7 +618,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( bool is_temp, IMergeTreeDataPart * parent_part, const MergeTreeData & data, - Poco::Logger * log, + LoggerPtr log, Block block, const ProjectionDescription & projection) { @@ -729,7 +729,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPart( const MergeTreeData & data, - Poco::Logger * log, + LoggerPtr log, Block block, const ProjectionDescription & projection, IMergeTreeDataPart * parent_part) @@ -748,7 +748,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPart( /// projection part merges. MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempProjectionPart( const MergeTreeData & data, - Poco::Logger * log, + LoggerPtr log, Block block, const ProjectionDescription & projection, IMergeTreeDataPart * parent_part, diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.h b/src/Storages/MergeTree/MergeTreeDataWriter.h index aaa0f71eccf..8fb8b82dbe6 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.h +++ b/src/Storages/MergeTree/MergeTreeDataWriter.h @@ -92,7 +92,7 @@ public: /// For insertion. static TemporaryPart writeProjectionPart( const MergeTreeData & data, - Poco::Logger * log, + LoggerPtr log, Block block, const ProjectionDescription & projection, IMergeTreeDataPart * parent_part); @@ -100,7 +100,7 @@ public: /// For mutation: MATERIALIZE PROJECTION. static TemporaryPart writeTempProjectionPart( const MergeTreeData & data, - Poco::Logger * log, + LoggerPtr log, Block block, const ProjectionDescription & projection, IMergeTreeDataPart * parent_part, @@ -127,7 +127,7 @@ private: bool is_temp, IMergeTreeDataPart * parent_part, const MergeTreeData & data, - Poco::Logger * log, + LoggerPtr log, Block block, const ProjectionDescription & projection); diff --git a/src/Storages/MergeTree/MergeTreePartsMover.h b/src/Storages/MergeTree/MergeTreePartsMover.h index b9109e51309..43d8ebdd6d3 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.h +++ b/src/Storages/MergeTree/MergeTreePartsMover.h @@ -48,7 +48,7 @@ public: explicit MergeTreePartsMover(MergeTreeData * data_) : data(data_) - , log(&Poco::Logger::get("MergeTreePartsMover")) + , log(getLogger("MergeTreePartsMover")) { } @@ -81,7 +81,7 @@ public: private: MergeTreeData * data; - Poco::Logger * log; + LoggerPtr log; }; diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index 3f9632637b6..47c2fe07bb4 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -128,7 +128,7 @@ MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool( context_) , WithContext(context_) , prefetch_threadpool(getContext()->getPrefetchThreadpool()) - , log(&Poco::Logger::get("MergeTreePrefetchedReadPool(" + (parts_ranges.empty() ? "" : parts_ranges.front().data_part->storage.getStorageID().getNameForLogs()) + ")")) + , log(getLogger("MergeTreePrefetchedReadPool(" + (parts_ranges.empty() ? "" : parts_ranges.front().data_part->storage.getStorageID().getNameForLogs()) + ")")) { /// Tasks creation might also create a lost of readers - check they do not /// do any time consuming operations in ctor. diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h index 9925d4e2fa4..378034c5eae 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h @@ -122,7 +122,7 @@ private: TasksPerThread per_thread_tasks; std::priority_queue prefetch_queue; /// the smallest on top bool started_prefetches = false; - Poco::Logger * log; + LoggerPtr log; /// A struct which allows to track max number of tasks which were in the /// threadpool simultaneously (similar to CurrentMetrics, but the result diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.h b/src/Storages/MergeTree/MergeTreeRangeReader.h index 04d42138963..79ed18f4d1f 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -231,7 +231,7 @@ public: using RangesInfo = std::vector; - explicit ReadResult(Poco::Logger * log_) : log(log_) {} + explicit ReadResult(LoggerPtr log_) : log(log_) {} static size_t getLastMark(const MergeTreeRangeReader::ReadResult::RangesInfo & ranges); @@ -298,7 +298,7 @@ public: size_t countZeroTails(const IColumn::Filter & filter, NumRows & zero_tails, bool can_read_incomplete_granules) const; static size_t numZerosInTail(const UInt8 * begin, const UInt8 * end); - Poco::Logger * log; + LoggerPtr log; }; ReadResult read(size_t max_rows, MarkRanges & ranges); @@ -325,7 +325,7 @@ private: bool is_initialized = false; Names non_const_virtual_column_names; - Poco::Logger * log = &Poco::Logger::get("MergeTreeRangeReader"); + LoggerPtr log = getLogger("MergeTreeRangeReader"); }; } diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index 3a1af947cae..e45ccad912f 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -108,7 +108,7 @@ private: std::vector threads_tasks; std::set remaining_thread_tasks; - Poco::Logger * log = &Poco::Logger::get("MergeTreeReadPool"); + LoggerPtr log = getLogger("MergeTreeReadPool"); }; } diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h index 7579a892b67..6a548dffe37 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h @@ -34,7 +34,7 @@ private: const CoordinationMode coordination_mode; RangesInDataPartsDescription buffered_ranges; bool no_more_tasks_available{false}; - Poco::Logger * log = &Poco::Logger::get("MergeTreeReadPoolParallelReplicas"); + LoggerPtr log = getLogger("MergeTreeReadPoolParallelReplicas"); }; } diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.h b/src/Storages/MergeTree/MergeTreeSelectProcessor.h index cf1a6313b51..b06ae788e91 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.h @@ -114,7 +114,7 @@ private: /// Should we add part level to produced chunk. Part level is useful for next steps if query has FINAL bool add_part_level = false; - Poco::Logger * log = &Poco::Logger::get("MergeTreeSelectProcessor"); + LoggerPtr log = getLogger("MergeTreeSelectProcessor"); std::atomic is_cancelled{false}; }; diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 82e9f8fd2db..29af7fb4820 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -68,7 +68,7 @@ private: /// Should read using direct IO bool read_with_direct_io; - Poco::Logger * log = &Poco::Logger::get("MergeTreeSequentialSource"); + LoggerPtr log = getLogger("MergeTreeSequentialSource"); std::optional mark_ranges; @@ -318,7 +318,7 @@ public: bool apply_deleted_mask_, ActionsDAGPtr filter_, ContextPtr context_, - Poco::Logger * log_) + LoggerPtr log_) : ISourceStep(DataStream{.header = storage_snapshot_->getSampleBlockForColumns(columns_to_read_)}) , type(type_) , storage(storage_) @@ -381,7 +381,7 @@ private: bool apply_deleted_mask; ActionsDAGPtr filter; ContextPtr context; - Poco::Logger * log; + LoggerPtr log; }; void createReadFromPartStep( @@ -394,7 +394,7 @@ void createReadFromPartStep( bool apply_deleted_mask, ActionsDAGPtr filter, ContextPtr context, - Poco::Logger * log) + LoggerPtr log) { auto reading = std::make_unique(type, storage, storage_snapshot, std::move(data_part), diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.h b/src/Storages/MergeTree/MergeTreeSequentialSource.h index 41def48aab6..a5e36a7726f 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.h +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.h @@ -41,6 +41,6 @@ void createReadFromPartStep( bool apply_deleted_mask, ActionsDAGPtr filter, ContextPtr context, - Poco::Logger * log); + LoggerPtr log); } diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 153930b400d..b42da22239e 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -65,7 +65,7 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def, ContextPtr conte if (ast && isDiskFunction(ast)) { auto disk_name = getOrCreateDiskFromDiskAST(ast, context, is_attach); - LOG_TRACE(&Poco::Logger::get("MergeTreeSettings"), "Created custom disk {}", disk_name); + LOG_TRACE(getLogger("MergeTreeSettings"), "Created custom disk {}", disk_name); value = disk_name; } } diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 0cac051bb2c..3f3dadb3cc5 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -56,7 +56,7 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( const ConditionEstimator & estimator_, const Names & queried_columns_, const std::optional & supported_columns_, - Poco::Logger * log_) + LoggerPtr log_) : estimator(estimator_) , table_columns{collections::map( metadata_snapshot->getColumns().getAllPhysical(), [](const NameAndTypePair & col) { return col.name; })} diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h index 0ef7ac9efff..7a6651210d0 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h @@ -41,7 +41,7 @@ public: const ConditionEstimator & estimator_, const Names & queried_columns_, const std::optional & supported_columns_, - Poco::Logger * log_); + LoggerPtr log_); void optimize(SelectQueryInfo & select_query_info, const ContextPtr & context) const; @@ -156,7 +156,7 @@ private: const std::optional supported_columns; const NameSet sorting_key_names; const NameToIndexMap primary_key_names_positions; - Poco::Logger * log; + LoggerPtr log; std::unordered_map column_sizes; UInt64 total_size_of_queried_columns = 0; }; diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index a8b3df483ed..2236c1a9380 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -36,7 +36,7 @@ MergeTreeWriteAheadLog::MergeTreeWriteAheadLog( , name(name_) , path(storage.getRelativeDataPath() + name_) , pool(storage.getContext()->getSchedulePool()) - , log(&Poco::Logger::get(storage.getLogName() + " (WriteAheadLog)")) + , log(getLogger(storage.getLogName() + " (WriteAheadLog)")) { init(); sync_task = pool.createTask("MergeTreeWriteAheadLog::sync", [this] diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h index 5fb9dd907a1..9550fa6ecee 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h @@ -99,7 +99,7 @@ private: mutable std::mutex write_mutex; - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 8b34c221eec..1d10a1433ef 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -155,7 +155,7 @@ MergedBlockOutputStream::Finalizer MergedBlockOutputStream::finalizePartAsync( for (const auto & name : checksums_to_remove) checksums.files.erase(name); - LOG_TRACE(&Poco::Logger::get("MergedBlockOutputStream"), "filled checksums {}", new_part->getNameWithState()); + LOG_TRACE(getLogger("MergedBlockOutputStream"), "filled checksums {}", new_part->getNameWithState()); for (const auto & [projection_name, projection_part] : new_part->getProjectionParts()) checksums.addFile( diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.h b/src/Storages/MergeTree/MutateFromLogEntryTask.h index 42d8307e948..68c7f464214 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.h +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.h @@ -23,7 +23,7 @@ public: StorageReplicatedMergeTree & storage_, Callback && task_result_callback_) : ReplicatedMergeMutateTaskBase( - &Poco::Logger::get(storage_.getStorageID().getShortName() + "::" + selected_entry_->log_entry->new_part_name + " (MutateFromLogEntryTask)"), + getLogger(storage_.getStorageID().getShortName() + "::" + selected_entry_->log_entry->new_part_name + " (MutateFromLogEntryTask)"), storage_, selected_entry_, task_result_callback_) diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index bf8e879e3d0..0b19aebe36d 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -111,7 +111,7 @@ bool MutatePlainMergeTreeTask::executeStep() if (merge_mutate_entry->txn) merge_mutate_entry->txn->onException(); PreformattedMessage exception_message = getCurrentExceptionMessageAndPattern(/* with_stacktrace */ false); - LOG_ERROR(&Poco::Logger::get("MutatePlainMergeTreeTask"), exception_message); + LOG_ERROR(getLogger("MutatePlainMergeTreeTask"), exception_message); storage.updateMutationEntriesErrors(future_part, false, exception_message.text); write_part_log(ExecutionStatus::fromCurrentException("", true)); tryLogCurrentException(__PRETTY_FUNCTION__); diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index e4070aa8262..6bcdfe34296 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -61,7 +61,7 @@ static void splitAndModifyMutationCommands( const MutationCommands & commands, MutationCommands & for_interpreter, MutationCommands & for_file_renames, - Poco::Logger * log) + LoggerPtr log) { auto part_columns = part->getColumnsDescription(); @@ -896,7 +896,7 @@ struct MutationContext TableLockHolder * holder; MergeListEntry * mutate_entry; - Poco::Logger * log{&Poco::Logger::get("MutateTask")}; + LoggerPtr log{getLogger("MutateTask")}; FutureMergedMutatedPartPtr future_part; MergeTreeData::DataPartPtr source_part; @@ -975,7 +975,7 @@ public: , projection(projection_) , block_num(block_num_) , ctx(ctx_) - , log(&Poco::Logger::get("MergeProjectionPartsTask")) + , log(getLogger("MergeProjectionPartsTask")) { LOG_DEBUG(log, "Selected {} projection_parts from {} to {}", parts.size(), parts.front()->name, parts.back()->name); level_parts[current_level] = std::move(parts); @@ -1079,7 +1079,7 @@ private: size_t & block_num; MutationContextPtr ctx; - Poco::Logger * log; + LoggerPtr log; std::map level_parts; size_t current_level = 0; diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 484a0b37644..abc51bde3fb 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -245,7 +245,7 @@ private: }; std::vector replica_status; - Poco::Logger * log = &Poco::Logger::get("DefaultCoordinator"); + LoggerPtr log = getLogger("DefaultCoordinator"); /// Workflow of a segment: /// 0. `all_parts_to_read` contains all the parts and thus all the segments initially present there (virtually) @@ -835,7 +835,7 @@ public: Parts all_parts_to_read; size_t total_rows_to_read = 0; - Poco::Logger * log = &Poco::Logger::get(fmt::format("{}{}", magic_enum::enum_name(mode), "Coordinator")); + LoggerPtr log = getLogger(fmt::format("{}{}", magic_enum::enum_name(mode), "Coordinator")); }; template diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp index 76b8080f64c..78fcfabb704 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp @@ -20,7 +20,7 @@ PartMovesBetweenShardsOrchestrator::PartMovesBetweenShardsOrchestrator(StorageRe : storage(storage_) , zookeeper_path(storage.zookeeper_path) , logger_name(storage.getStorageID().getFullTableName() + " (PartMovesBetweenShardsOrchestrator)") - , log(&Poco::Logger::get(logger_name)) + , log(getLogger(logger_name)) , entries_znode_path(zookeeper_path + "/part_moves_shard") { /// Schedule pool is not designed for long-running tasks. TODO replace with a separate thread? diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h index af21022953c..abe259c77ab 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h @@ -176,7 +176,7 @@ private: String zookeeper_path; String logger_name; - Poco::Logger * log = nullptr; + LoggerPtr log = nullptr; std::atomic need_stop{false}; BackgroundSchedulePool::TaskHolder task; diff --git a/src/Storages/MergeTree/PartitionPruner.cpp b/src/Storages/MergeTree/PartitionPruner.cpp index 668576f9021..eb51d600da3 100644 --- a/src/Storages/MergeTree/PartitionPruner.cpp +++ b/src/Storages/MergeTree/PartitionPruner.cpp @@ -59,7 +59,7 @@ bool PartitionPruner::canBePruned(const IMergeTreeDataPart & part) const { WriteBufferFromOwnString buf; part.partition.serializeText(part.storage, buf, FormatSettings{}); - LOG_TRACE(&Poco::Logger::get("PartitionPruner"), "Partition {} gets pruned", buf.str()); + LOG_TRACE(getLogger("PartitionPruner"), "Partition {} gets pruned", buf.str()); } } diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h index 18fcacecc9e..2b1fcec62a8 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h @@ -17,7 +17,7 @@ class ReplicatedMergeMutateTaskBase : public IExecutableTask { public: ReplicatedMergeMutateTaskBase( - Poco::Logger * log_, + LoggerPtr log_, StorageReplicatedMergeTree & storage_, ReplicatedMergeTreeQueue::SelectedEntryPtr & selected_entry_, IExecutableTask::TaskResultCallback & task_result_callback_) @@ -66,7 +66,7 @@ protected: ReplicatedMergeTreeQueue::SelectedEntryPtr selected_entry; ReplicatedMergeTreeLogEntry & entry; MergeList::EntryPtr merge_mutate_entry{nullptr}; - Poco::Logger * log; + LoggerPtr log; /// ProfileEvents for current part will be stored here ProfileEvents::Counters profile_counters; ContextMutablePtr task_context; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.cpp index a544ac908a4..336d19692d4 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.cpp @@ -19,7 +19,7 @@ namespace ErrorCodes ReplicatedMergeTreeAttachThread::ReplicatedMergeTreeAttachThread(StorageReplicatedMergeTree & storage_) : storage(storage_) , log_name(storage.getStorageID().getFullTableName() + " (ReplicatedMergeTreeAttachThread)") - , log(&Poco::Logger::get(log_name)) + , log(getLogger(log_name)) { task = storage.getContext()->getSchedulePool().createTask(log_name, [this] { run(); }); const auto storage_settings = storage.getSettings(); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.h index 222b30b519b..250a5ed34d1 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.h @@ -34,7 +34,7 @@ private: BackgroundSchedulePool::TaskHolder task; std::string log_name; - Poco::Logger * log; + LoggerPtr log; std::atomic first_try_done{false}; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index 8daee661c75..67942491ae2 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -24,7 +24,7 @@ namespace ErrorCodes ReplicatedMergeTreeCleanupThread::ReplicatedMergeTreeCleanupThread(StorageReplicatedMergeTree & storage_) : storage(storage_) , log_name(storage.getStorageID().getFullTableName() + " (ReplicatedMergeTreeCleanupThread)") - , log(&Poco::Logger::get(log_name)) + , log(getLogger(log_name)) , sleep_ms(storage.getSettings()->cleanup_delay_period * 1000) { task = storage.getContext()->getSchedulePool().createTask(log_name, [this]{ run(); }); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h index ae9aabdb4e7..5beaef56995 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h @@ -40,7 +40,7 @@ public: private: StorageReplicatedMergeTree & storage; String log_name; - Poco::Logger * log; + LoggerPtr log; BackgroundSchedulePool::TaskHolder task; pcg64 rng{randomSeed()}; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index b1875464725..156c41563ec 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -28,7 +28,7 @@ static const auto PART_CHECK_ERROR_SLEEP_MS = 5 * 1000; ReplicatedMergeTreePartCheckThread::ReplicatedMergeTreePartCheckThread(StorageReplicatedMergeTree & storage_) : storage(storage_) , log_name(storage.getStorageID().getFullTableName() + " (ReplicatedMergeTreePartCheckThread)") - , log(&Poco::Logger::get(log_name)) + , log(getLogger(log_name)) { task = storage.getContext()->getSchedulePool().createTask(log_name, [this] { run(); }); task->schedule(); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h index 68dc6ca3d1d..f2e26b3d324 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h @@ -87,7 +87,7 @@ private: StorageReplicatedMergeTree & storage; String log_name; - Poco::Logger * log; + LoggerPtr log; using StringSet = std::set; struct PartToCheck diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index a3afa8cd88a..8d921bdcb1c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -36,7 +36,7 @@ ReplicatedMergeTreeQueue::ReplicatedMergeTreeQueue(StorageReplicatedMergeTree & zookeeper_path = storage.zookeeper_path; replica_path = storage.replica_path; logger_name = storage.getStorageID().getFullTableName() + " (ReplicatedMergeTreeQueue)"; - log = &Poco::Logger::get(logger_name); + log = getLogger(logger_name); } @@ -2149,7 +2149,7 @@ LocalMergePredicate::LocalMergePredicate(ReplicatedMergeTreeQueue & queue_) template CommittingBlocks BaseMergePredicate::getCommittingBlocks( - zkutil::ZooKeeperPtr & zookeeper, const std::string & zookeeper_path, Poco::Logger * log_) + zkutil::ZooKeeperPtr & zookeeper, const std::string & zookeeper_path, LoggerPtr log_) { CommittingBlocks committing_blocks; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 92201b11d37..84106565dff 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -71,7 +71,7 @@ private: String zookeeper_path; String replica_path; String logger_name; - Poco::Logger * log = nullptr; + LoggerPtr log = nullptr; /// Protects the queue, future_parts and other queue state variables. mutable std::mutex state_mutex; @@ -519,7 +519,7 @@ public: /// This predicate is checked for the first part of each range. bool canMergeSinglePart(const MergeTreeData::DataPartPtr & part, String & out_reason) const; - CommittingBlocks getCommittingBlocks(zkutil::ZooKeeperPtr & zookeeper, const std::string & zookeeper_path, Poco::Logger * log_); + CommittingBlocks getCommittingBlocks(zkutil::ZooKeeperPtr & zookeeper, const std::string & zookeeper_path, LoggerPtr log_); protected: /// A list of partitions that can be used in the merge predicate diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index 579592b0b3e..b79418da791 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -33,7 +33,7 @@ static String generateActiveNodeIdentifier() ReplicatedMergeTreeRestartingThread::ReplicatedMergeTreeRestartingThread(StorageReplicatedMergeTree & storage_) : storage(storage_) , log_name(storage.getStorageID().getFullTableName() + " (ReplicatedMergeTreeRestartingThread)") - , log(&Poco::Logger::get(log_name)) + , log(getLogger(log_name)) , active_node_identifier(generateActiveNodeIdentifier()) { const auto storage_settings = storage.getSettings(); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h index 02103272a1f..01071d80e8b 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h @@ -42,7 +42,7 @@ public: private: StorageReplicatedMergeTree & storage; String log_name; - Poco::Logger * log; + LoggerPtr log; std::atomic need_stop {false}; /// The random data we wrote into `/replicas/me/is_active`. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 73ad595ec20..1fb2393948a 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -58,7 +58,7 @@ struct ReplicatedMergeTreeSinkImpl::DelayedChunk ProfileEvents::Counters part_counters; Partition() = default; - Partition(Poco::Logger * log_, + Partition(LoggerPtr log_, MergeTreeDataWriter::TemporaryPart && temp_part_, UInt64 elapsed_ns_, BlockIDsType && block_id_, @@ -92,7 +92,7 @@ std::vector testSelfDeduplicate(std::vector data, std::vector::DelayedChunk::Partition part( - &Poco::Logger::get("testSelfDeduplicate"), MergeTreeDataWriter::TemporaryPart(), 0, std::move(hashes), std::move(block1), std::nullopt, std::move(profile_counters)); + getLogger("testSelfDeduplicate"), MergeTreeDataWriter::TemporaryPart(), 0, std::move(hashes), std::move(block1), std::nullopt, std::move(profile_counters)); part.filterSelfDuplicate(); @@ -138,7 +138,7 @@ ReplicatedMergeTreeSinkImpl::ReplicatedMergeTreeSinkImpl( , is_attach(is_attach_) , quorum_parallel(quorum_parallel_) , deduplicate(deduplicate_) - , log(&Poco::Logger::get(storage.getLogName() + " (Replicated OutputStream)")) + , log(getLogger(storage.getLogName() + " (Replicated OutputStream)")) , context(context_) , storage_snapshot(storage.getStorageSnapshotWithoutData(metadata_snapshot, context_)) { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h index 4811d93775b..bc23204e7d3 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h @@ -128,7 +128,7 @@ private: bool last_block_is_duplicate = false; UInt64 num_blocks_processed = 0; - Poco::Logger * log; + LoggerPtr log; ContextPtr context; StorageSnapshotPtr storage_snapshot; diff --git a/src/Storages/MergeTree/ZooKeeperRetries.h b/src/Storages/MergeTree/ZooKeeperRetries.h index 92faf80e61b..ecef174c6c7 100644 --- a/src/Storages/MergeTree/ZooKeeperRetries.h +++ b/src/Storages/MergeTree/ZooKeeperRetries.h @@ -30,7 +30,7 @@ struct ZooKeeperRetriesInfo class ZooKeeperRetriesControl { public: - ZooKeeperRetriesControl(std::string name_, Poco::Logger * logger_, ZooKeeperRetriesInfo retries_info_, QueryStatusPtr elem) + ZooKeeperRetriesControl(std::string name_, LoggerPtr logger_, ZooKeeperRetriesInfo retries_info_, QueryStatusPtr elem) : name(std::move(name_)), logger(logger_), retries_info(retries_info_), process_list_element(elem) { } @@ -160,7 +160,7 @@ public: const std::string & getName() const { return name; } - Poco::Logger * getLogger() const { return logger; } + LoggerPtr getLogger() const { return logger; } private: struct KeeperError @@ -263,7 +263,7 @@ private: std::string name; - Poco::Logger * logger = nullptr; + LoggerPtr logger = nullptr; ZooKeeperRetriesInfo retries_info; UInt64 total_failures = 0; UserError user_error; diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 8cf5b6a8894..8ae9b54b6e9 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -338,7 +338,7 @@ IMergeTreeDataPart::Checksums checkDataPart( throw; LOG_DEBUG( - &Poco::Logger::get("checkDataPart"), + getLogger("checkDataPart"), "Will drop cache for data part {} and will check it once again", data_part->name); auto & cache = *FileCacheFactory::instance().getByName(*cache_name)->cache; diff --git a/src/Storages/MessageQueueSink.cpp b/src/Storages/MessageQueueSink.cpp index 1aa19c9ccde..4fb81d69070 100644 --- a/src/Storages/MessageQueueSink.cpp +++ b/src/Storages/MessageQueueSink.cpp @@ -20,7 +20,7 @@ MessageQueueSink::MessageQueueSink( void MessageQueueSink::onStart() { LOG_TEST( - &Poco::Logger::get("MessageQueueSink"), + getLogger("MessageQueueSink"), "Executing startup for MessageQueueSink"); initialize(); diff --git a/src/Storages/NATS/NATSConnection.cpp b/src/Storages/NATS/NATSConnection.cpp index 70b3599aa09..d7ad0cf8219 100644 --- a/src/Storages/NATS/NATSConnection.cpp +++ b/src/Storages/NATS/NATSConnection.cpp @@ -13,7 +13,7 @@ static const auto RETRIES_MAX = 20; static const auto CONNECTED_TO_BUFFER_SIZE = 256; -NATSConnectionManager::NATSConnectionManager(const NATSConfiguration & configuration_, Poco::Logger * log_) +NATSConnectionManager::NATSConnectionManager(const NATSConfiguration & configuration_, LoggerPtr log_) : configuration(configuration_) , log(log_) , event_handler(loop.getLoop(), log) @@ -115,8 +115,8 @@ void NATSConnectionManager::connectImpl() } natsOptions_SetMaxReconnect(options, configuration.max_reconnect); natsOptions_SetReconnectWait(options, configuration.reconnect_wait); - natsOptions_SetDisconnectedCB(options, disconnectedCallback, log); - natsOptions_SetReconnectedCB(options, reconnectedCallback, log); + natsOptions_SetDisconnectedCB(options, disconnectedCallback, log.get()); + natsOptions_SetReconnectedCB(options, reconnectedCallback, log.get()); natsStatus status; { auto lock = event_handler.setThreadLocalLoop(); diff --git a/src/Storages/NATS/NATSConnection.h b/src/Storages/NATS/NATSConnection.h index b49070473b2..c350f395a92 100644 --- a/src/Storages/NATS/NATSConnection.h +++ b/src/Storages/NATS/NATSConnection.h @@ -24,7 +24,7 @@ struct NATSConfiguration class NATSConnectionManager { public: - NATSConnectionManager(const NATSConfiguration & configuration_, Poco::Logger * log_); + NATSConnectionManager(const NATSConfiguration & configuration_, LoggerPtr log_); ~NATSConnectionManager(); bool isConnected(); @@ -54,7 +54,7 @@ private: static void reconnectedCallback(natsConnection * nc, void * log); NATSConfiguration configuration; - Poco::Logger * log; + LoggerPtr log; UVLoop loop; NATSHandler event_handler; diff --git a/src/Storages/NATS/NATSConsumer.cpp b/src/Storages/NATS/NATSConsumer.cpp index c7b40973b72..136cb13ddfa 100644 --- a/src/Storages/NATS/NATSConsumer.cpp +++ b/src/Storages/NATS/NATSConsumer.cpp @@ -21,7 +21,7 @@ NATSConsumer::NATSConsumer( StorageNATS & storage_, std::vector & subjects_, const String & subscribe_queue_name, - Poco::Logger * log_, + LoggerPtr log_, uint32_t queue_size_, const std::atomic & stopped_) : connection(connection_) diff --git a/src/Storages/NATS/NATSConsumer.h b/src/Storages/NATS/NATSConsumer.h index a5470433303..e8d3a849c2a 100644 --- a/src/Storages/NATS/NATSConsumer.h +++ b/src/Storages/NATS/NATSConsumer.h @@ -24,7 +24,7 @@ public: StorageNATS & storage_, std::vector & subjects_, const String & subscribe_queue_name, - Poco::Logger * log_, + LoggerPtr log_, uint32_t queue_size_, const std::atomic & stopped_); @@ -58,7 +58,7 @@ private: StorageNATS & storage; std::vector subscriptions; std::vector subjects; - Poco::Logger * log; + LoggerPtr log; const std::atomic & stopped; bool subscribed = false; diff --git a/src/Storages/NATS/NATSHandler.cpp b/src/Storages/NATS/NATSHandler.cpp index 7006e5633a9..03f1fc1a495 100644 --- a/src/Storages/NATS/NATSHandler.cpp +++ b/src/Storages/NATS/NATSHandler.cpp @@ -12,7 +12,7 @@ namespace DB static const auto MAX_THREAD_WORK_DURATION_MS = 60000; -NATSHandler::NATSHandler(uv_loop_t * loop_, Poco::Logger * log_) : +NATSHandler::NATSHandler(uv_loop_t * loop_, LoggerPtr log_) : loop(loop_), log(log_), loop_running(false), diff --git a/src/Storages/NATS/NATSHandler.h b/src/Storages/NATS/NATSHandler.h index e3894c888a3..6f9ec398cfa 100644 --- a/src/Storages/NATS/NATSHandler.h +++ b/src/Storages/NATS/NATSHandler.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include namespace DB { @@ -23,7 +23,7 @@ using LockPtr = std::unique_ptr>; class NATSHandler { public: - NATSHandler(uv_loop_t * loop_, Poco::Logger * log_); + NATSHandler(uv_loop_t * loop_, LoggerPtr log_); ~NATSHandler(); @@ -47,7 +47,7 @@ public: private: uv_loop_t * loop; natsOptions * opts = nullptr; - Poco::Logger * log; + LoggerPtr log; std::atomic loop_running; std::atomic loop_state; diff --git a/src/Storages/NATS/NATSProducer.cpp b/src/Storages/NATS/NATSProducer.cpp index a8510149baf..fb8abb016f8 100644 --- a/src/Storages/NATS/NATSProducer.cpp +++ b/src/Storages/NATS/NATSProducer.cpp @@ -23,7 +23,7 @@ NATSProducer::NATSProducer( const NATSConfiguration & configuration_, const String & subject_, std::atomic & shutdown_called_, - Poco::Logger * log_) + LoggerPtr log_) : AsynchronousMessageProducer(log_) , connection(configuration_, log_) , subject(subject_) diff --git a/src/Storages/NATS/NATSProducer.h b/src/Storages/NATS/NATSProducer.h index 0303d05969b..6923553a551 100644 --- a/src/Storages/NATS/NATSProducer.h +++ b/src/Storages/NATS/NATSProducer.h @@ -20,7 +20,7 @@ public: const NATSConfiguration & configuration_, const String & subject_, std::atomic & shutdown_called_, - Poco::Logger * log_); + LoggerPtr log_); void produce(const String & message, size_t rows_in_message, const Columns & columns, size_t last_row) override; diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index 9cb1fbd8506..2af9a9f974f 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -59,7 +59,7 @@ StorageNATS::StorageNATS( , schema_name(getContext()->getMacros()->expand(nats_settings->nats_schema)) , num_consumers(nats_settings->nats_num_consumers.value) , max_rows_per_message(nats_settings->nats_max_rows_per_message) - , log(&Poco::Logger::get("StorageNATS (" + table_id_.table_name + ")")) + , log(getLogger("StorageNATS (" + table_id_.table_name + ")")) , semaphore(0, static_cast(num_consumers)) , queue_size(std::max(QUEUE_SIZE, static_cast(getMaxBlockSize()))) , is_attach(is_attach_) diff --git a/src/Storages/NATS/StorageNATS.h b/src/Storages/NATS/StorageNATS.h index 16a162b8500..882119f5cdb 100644 --- a/src/Storages/NATS/StorageNATS.h +++ b/src/Storages/NATS/StorageNATS.h @@ -78,7 +78,7 @@ private: size_t num_consumers; size_t max_rows_per_message; - Poco::Logger * log; + LoggerPtr log; NATSConnectionManagerPtr connection; /// Connection for all consumers NATSConfiguration configuration; diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index b2442109409..f99ebf51792 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -51,7 +51,7 @@ MaterializedPostgreSQLConsumer::MaterializedPostgreSQLConsumer( bool schema_as_a_part_of_table_name_, StorageInfos storages_info_, const String & name_for_logger) - : log(&Poco::Logger::get("PostgreSQLReplicaConsumer(" + name_for_logger + ")")) + : log(getLogger("PostgreSQLReplicaConsumer(" + name_for_logger + ")")) , context(context_) , replication_slot_name(replication_slot_name_) , publication_name(publication_name_) @@ -76,7 +76,7 @@ MaterializedPostgreSQLConsumer::MaterializedPostgreSQLConsumer( } -MaterializedPostgreSQLConsumer::StorageData::StorageData(const StorageInfo & storage_info, Poco::Logger * log_) +MaterializedPostgreSQLConsumer::StorageData::StorageData(const StorageInfo & storage_info, LoggerPtr log_) : storage(storage_info.storage) , table_description(storage_info.storage->getInMemoryMetadataPtr()->getSampleBlock()) , columns_attributes(storage_info.attributes) diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h index 3e95c1cd7de..972c03e50d8 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h @@ -32,7 +32,7 @@ class MaterializedPostgreSQLConsumer private: struct StorageData { - explicit StorageData(const StorageInfo & storage_info, Poco::Logger * log_); + explicit StorageData(const StorageInfo & storage_info, LoggerPtr log_); size_t getColumnsNum() const { return table_description.sample_block.columns(); } @@ -137,7 +137,7 @@ private: return (static_cast(upper_half) << 32) + lower_half; } - Poco::Logger * log; + LoggerPtr log; ContextPtr context; const std::string replication_slot_name, publication_name; diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 43de2069b19..2bb1e2dde0d 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -128,7 +128,7 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( const MaterializedPostgreSQLSettings & replication_settings, bool is_materialized_postgresql_database_) : WithContext(context_->getGlobalContext()) - , log(&Poco::Logger::get("PostgreSQLReplicationHandler")) + , log(getLogger("PostgreSQLReplicationHandler")) , is_attach(is_attach_) , postgres_database(postgres_database_) , postgres_schema(replication_settings.materialized_postgresql_schema) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 5d426b3c512..5c519053d84 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -102,7 +102,7 @@ private: void assertInitialized() const; - Poco::Logger * log; + LoggerPtr log; /// If it is not attach, i.e. a create query, then if publication already exists - always drop it. bool is_attach; diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index 0faf553797a..f13cb820ec3 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -60,7 +60,7 @@ StorageMaterializedPostgreSQL::StorageMaterializedPostgreSQL( std::unique_ptr replication_settings) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) - , log(&Poco::Logger::get("StorageMaterializedPostgreSQL(" + postgres::formatNameForLogs(remote_database_name, remote_table_name_) + ")")) + , log(getLogger("StorageMaterializedPostgreSQL(" + postgres::formatNameForLogs(remote_database_name, remote_table_name_) + ")")) , is_materialized_postgresql_database(false) , has_nested(false) , nested_context(makeNestedTableContext(context_->getGlobalContext())) @@ -101,7 +101,7 @@ StorageMaterializedPostgreSQL::StorageMaterializedPostgreSQL( const String & postgres_table_name) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) - , log(&Poco::Logger::get("StorageMaterializedPostgreSQL(" + postgres::formatNameForLogs(postgres_database_name, postgres_table_name) + ")")) + , log(getLogger("StorageMaterializedPostgreSQL(" + postgres::formatNameForLogs(postgres_database_name, postgres_table_name) + ")")) , is_materialized_postgresql_database(true) , has_nested(false) , nested_context(makeNestedTableContext(context_->getGlobalContext())) @@ -120,7 +120,7 @@ StorageMaterializedPostgreSQL::StorageMaterializedPostgreSQL( const String & postgres_table_name) : IStorage(StorageID(nested_storage_->getStorageID().database_name, nested_storage_->getStorageID().table_name)) , WithContext(context_->getGlobalContext()) - , log(&Poco::Logger::get("StorageMaterializedPostgreSQL(" + postgres::formatNameForLogs(postgres_database_name, postgres_table_name) + ")")) + , log(getLogger("StorageMaterializedPostgreSQL(" + postgres::formatNameForLogs(postgres_database_name, postgres_table_name) + ")")) , is_materialized_postgresql_database(true) , has_nested(true) , nested_context(makeNestedTableContext(context_->getGlobalContext())) @@ -141,7 +141,7 @@ StoragePtr StorageMaterializedPostgreSQL::createTemporary() const auto tmp_storage = DatabaseCatalog::instance().tryGetTable(tmp_table_id, nested_context); if (tmp_storage) { - LOG_TRACE(&Poco::Logger::get("MaterializedPostgreSQLStorage"), "Temporary table {} already exists, dropping", tmp_table_id.getNameForLogs()); + LOG_TRACE(getLogger("MaterializedPostgreSQLStorage"), "Temporary table {} already exists, dropping", tmp_table_id.getNameForLogs()); InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), getContext(), tmp_table_id, /* sync */true); } diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h index bebbb74ddd1..9c9418a8caa 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h @@ -142,7 +142,7 @@ private: String getNestedTableName() const; - Poco::Logger * log; + LoggerPtr log; /// Not nullptr only for single MaterializedPostgreSQL storage, because for MaterializedPostgreSQL /// database engine there is one replication handler for all tables. diff --git a/src/Storages/RabbitMQ/RabbitMQConnection.cpp b/src/Storages/RabbitMQ/RabbitMQConnection.cpp index 13d065774a2..98ceba42676 100644 --- a/src/Storages/RabbitMQ/RabbitMQConnection.cpp +++ b/src/Storages/RabbitMQ/RabbitMQConnection.cpp @@ -11,7 +11,7 @@ static const auto CONNECT_SLEEP = 200; static const auto RETRIES_MAX = 20; -RabbitMQConnection::RabbitMQConnection(const RabbitMQConfiguration & configuration_, Poco::Logger * log_) +RabbitMQConnection::RabbitMQConnection(const RabbitMQConfiguration & configuration_, LoggerPtr log_) : configuration(configuration_) , log(log_) , event_handler(loop.getLoop(), log) diff --git a/src/Storages/RabbitMQ/RabbitMQConnection.h b/src/Storages/RabbitMQ/RabbitMQConnection.h index 698230b16f4..5adb6456194 100644 --- a/src/Storages/RabbitMQ/RabbitMQConnection.h +++ b/src/Storages/RabbitMQ/RabbitMQConnection.h @@ -22,7 +22,7 @@ struct RabbitMQConfiguration class RabbitMQConnection { public: - RabbitMQConnection(const RabbitMQConfiguration & configuration_, Poco::Logger * log_); + RabbitMQConnection(const RabbitMQConfiguration & configuration_, LoggerPtr log_); bool isConnected(); @@ -51,7 +51,7 @@ private: void disconnectImpl(bool immediately = false); RabbitMQConfiguration configuration; - Poco::Logger * log; + LoggerPtr log; UVLoop loop; /// Preserve order of destruction here: diff --git a/src/Storages/RabbitMQ/RabbitMQConsumer.cpp b/src/Storages/RabbitMQ/RabbitMQConsumer.cpp index f6facc04212..1843bebe3c7 100644 --- a/src/Storages/RabbitMQ/RabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/RabbitMQConsumer.cpp @@ -24,7 +24,7 @@ RabbitMQConsumer::RabbitMQConsumer( std::vector & queues_, size_t channel_id_base_, const String & channel_base_, - Poco::Logger * log_, + LoggerPtr log_, uint32_t queue_size_) : event_handler(event_handler_) , queues(queues_) diff --git a/src/Storages/RabbitMQ/RabbitMQConsumer.h b/src/Storages/RabbitMQ/RabbitMQConsumer.h index 89dfa060eec..c78b33bfc7c 100644 --- a/src/Storages/RabbitMQ/RabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/RabbitMQConsumer.h @@ -32,7 +32,7 @@ public: std::vector & queues_, size_t channel_id_base_, const String & channel_base_, - Poco::Logger * log_, + LoggerPtr log_, uint32_t queue_size_); struct CommitInfo @@ -88,7 +88,7 @@ private: const String channel_base; const size_t channel_id_base; - Poco::Logger * log; + LoggerPtr log; std::atomic stopped; String channel_id; diff --git a/src/Storages/RabbitMQ/RabbitMQHandler.cpp b/src/Storages/RabbitMQ/RabbitMQHandler.cpp index 745af0d20e3..be352f26f7b 100644 --- a/src/Storages/RabbitMQ/RabbitMQHandler.cpp +++ b/src/Storages/RabbitMQ/RabbitMQHandler.cpp @@ -8,7 +8,7 @@ namespace DB /* The object of this class is shared between concurrent consumers (who share the same connection == share the same * event loop and handler). */ -RabbitMQHandler::RabbitMQHandler(uv_loop_t * loop_, Poco::Logger * log_) : +RabbitMQHandler::RabbitMQHandler(uv_loop_t * loop_, LoggerPtr log_) : AMQP::LibUvHandler(loop_), loop(loop_), log(log_), diff --git a/src/Storages/RabbitMQ/RabbitMQHandler.h b/src/Storages/RabbitMQ/RabbitMQHandler.h index 4223732a4a0..244692cf800 100644 --- a/src/Storages/RabbitMQ/RabbitMQHandler.h +++ b/src/Storages/RabbitMQ/RabbitMQHandler.h @@ -24,7 +24,7 @@ class RabbitMQHandler : public AMQP::LibUvHandler { public: - RabbitMQHandler(uv_loop_t * loop_, Poco::Logger * log_); + RabbitMQHandler(uv_loop_t * loop_, LoggerPtr log_); void onError(AMQP::TcpConnection * connection, const char * message) override; void onReady(AMQP::TcpConnection * connection) override; @@ -50,7 +50,7 @@ public: private: uv_loop_t * loop; - Poco::Logger * log; + LoggerPtr log; std::atomic connection_running, loop_running; std::atomic loop_state; diff --git a/src/Storages/RabbitMQ/RabbitMQProducer.cpp b/src/Storages/RabbitMQ/RabbitMQProducer.cpp index 246569060d0..7ad83213b9b 100644 --- a/src/Storages/RabbitMQ/RabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/RabbitMQProducer.cpp @@ -31,7 +31,7 @@ RabbitMQProducer::RabbitMQProducer( const size_t channel_id_base_, const bool persistent_, std::atomic & shutdown_called_, - Poco::Logger * log_) + LoggerPtr log_) : AsynchronousMessageProducer(log_) , connection(configuration_, log_) , routing_keys(routing_keys_) diff --git a/src/Storages/RabbitMQ/RabbitMQProducer.h b/src/Storages/RabbitMQ/RabbitMQProducer.h index 70afbbb9b90..a790eda0d08 100644 --- a/src/Storages/RabbitMQ/RabbitMQProducer.h +++ b/src/Storages/RabbitMQ/RabbitMQProducer.h @@ -24,7 +24,7 @@ public: const size_t channel_id_base_, const bool persistent_, std::atomic & shutdown_called_, - Poco::Logger * log_); + LoggerPtr log_); void produce(const String & message, size_t rows_in_message, const Columns & columns, size_t last_row) override; diff --git a/src/Storages/RabbitMQ/RabbitMQSource.cpp b/src/Storages/RabbitMQ/RabbitMQSource.cpp index 793064c10f8..3cec448fc11 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSource.cpp @@ -70,7 +70,7 @@ RabbitMQSource::RabbitMQSource( , ack_in_suffix(ack_in_suffix_) , non_virtual_header(std::move(headers.first)) , virtual_header(std::move(headers.second)) - , log(&Poco::Logger::get("RabbitMQSource")) + , log(getLogger("RabbitMQSource")) , max_execution_time_ms(max_execution_time_) { storage.incrementReader(); diff --git a/src/Storages/RabbitMQ/RabbitMQSource.h b/src/Storages/RabbitMQ/RabbitMQSource.h index a25b3d50222..21d059bfae2 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.h +++ b/src/Storages/RabbitMQ/RabbitMQSource.h @@ -47,7 +47,7 @@ private: const Block non_virtual_header; const Block virtual_header; - Poco::Logger * log; + LoggerPtr log; RabbitMQConsumerPtr consumer; uint64_t max_execution_time_ms = 0; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index fce2d775b15..025f421db59 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -86,7 +86,7 @@ StorageRabbitMQ::StorageRabbitMQ( , persistent(rabbitmq_settings->rabbitmq_persistent.value) , use_user_setup(rabbitmq_settings->rabbitmq_queue_consume.value) , hash_exchange(num_consumers > 1 || num_queues > 1) - , log(&Poco::Logger::get("StorageRabbitMQ (" + table_id_.table_name + ")")) + , log(getLogger("StorageRabbitMQ (" + table_id_.table_name + ")")) , semaphore(0, static_cast(num_consumers)) , unique_strbase(getRandomName()) , queue_size(std::max(QUEUE_SIZE, static_cast(getMaxBlockSize()))) diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index 120930cf01d..be46caf9798 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -102,7 +102,7 @@ private: bool use_user_setup; bool hash_exchange; - Poco::Logger * log; + LoggerPtr log; RabbitMQConnectionPtr connection; /// Connection for all consumers RabbitMQConfiguration configuration; diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index f49e1d6f25c..a2b41eb4685 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -133,7 +133,7 @@ S3QueueFilesMetadata::S3QueueFilesMetadata(const fs::path & zookeeper_path_, con , zookeeper_processed_path(zookeeper_path_ / "processed") , zookeeper_failed_path(zookeeper_path_ / "failed") , zookeeper_cleanup_lock_path(zookeeper_path_ / "cleanup_lock") - , log(&Poco::Logger::get("S3QueueFilesMetadata")) + , log(getLogger("S3QueueFilesMetadata")) { if (mode == S3QueueMode::UNORDERED && (max_set_size || max_set_age_sec)) { @@ -689,7 +689,7 @@ S3QueueFilesMetadata::ProcessingNodeHolder::ProcessingNodeHolder( , path(path_) , zk_node_path(zk_node_path_) , processing_id(processing_id_) - , log(&Poco::Logger::get("ProcessingNodeHolder")) + , log(getLogger("ProcessingNodeHolder")) { } diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.h b/src/Storages/S3Queue/S3QueueFilesMetadata.h index f3be7c5c3a0..390cb5a64ab 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.h +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.h @@ -93,7 +93,7 @@ private: const fs::path zookeeper_failed_path; const fs::path zookeeper_cleanup_lock_path; - Poco::Logger * log; + LoggerPtr log; std::atomic_bool shutdown = false; BackgroundSchedulePool::TaskHolder task; @@ -169,7 +169,7 @@ private: std::string zk_node_path; std::string processing_id; bool removed = false; - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 27bec039f96..54155ad3ea7 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -60,7 +60,7 @@ StorageS3QueueSource::KeyWithInfoPtr StorageS3QueueSource::FileIterator::next() if (shutdown_called) { - LOG_TEST(&Poco::Logger::get("StorageS3QueueSource"), "Shutdown was called, stopping file iterator"); + LOG_TEST(getLogger("StorageS3QueueSource"), "Shutdown was called, stopping file iterator"); return {}; } @@ -91,7 +91,7 @@ StorageS3QueueSource::StorageS3QueueSource( const std::atomic & table_is_being_dropped_, std::shared_ptr s3_queue_log_, const StorageID & storage_id_, - Poco::Logger * log_) + LoggerPtr log_) : ISource(header_) , WithContext(context_) , name(std::move(name_)) diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index 542f8e8fd8c..82e75020efb 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -67,7 +67,7 @@ public: const std::atomic & table_is_being_dropped_, std::shared_ptr s3_queue_log_, const StorageID & storage_id_, - Poco::Logger * log_); + LoggerPtr log_); ~StorageS3QueueSource() override; @@ -89,7 +89,7 @@ private: const StorageID storage_id; RemoveFileFunc remove_file_func; - Poco::Logger * log; + LoggerPtr log; using ReaderHolder = StorageS3Source::ReaderHolder; ReaderHolder reader; diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index bc33e8cf2a9..48c284c022a 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -75,7 +75,7 @@ namespace return zkutil::extractZooKeeperPath(result_zk_path, true); } - void checkAndAdjustSettings(S3QueueSettings & s3queue_settings, const Settings & settings, Poco::Logger * log) + void checkAndAdjustSettings(S3QueueSettings & s3queue_settings, const Settings & settings, LoggerPtr log) { if (s3queue_settings.mode == S3QueueMode::ORDERED && s3queue_settings.s3queue_processing_threads_num > 1) { @@ -119,7 +119,7 @@ StorageS3Queue::StorageS3Queue( , configuration{configuration_} , format_settings(format_settings_) , reschedule_processing_interval_ms(s3queue_settings->s3queue_polling_min_timeout_ms) - , log(&Poco::Logger::get("StorageS3Queue (" + table_id_.table_name + ")")) + , log(getLogger("StorageS3Queue (" + table_id_.table_name + ")")) { if (configuration.url.key.empty()) { @@ -600,7 +600,7 @@ void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) if (user_format_settings.has(change.name)) user_format_settings.set(change.name, change.value); else - LOG_TRACE(&Poco::Logger::get("StorageS3"), "Remove: {}", change.name); + LOG_TRACE(getLogger("StorageS3"), "Remove: {}", change.name); args.storage_def->settings->changes.removeSetting(change.name); } diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 3d3594dc2ab..5d2be610d58 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -79,7 +79,7 @@ private: std::atomic shutdown_called = false; std::atomic table_is_being_dropped = false; - Poco::Logger * log; + LoggerPtr log; void startup() override; void shutdown(bool is_drop) override; diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h index 16e5b9edfb6..6fc3c5ce592 100644 --- a/src/Storages/StorageAzureBlob.h +++ b/src/Storages/StorageAzureBlob.h @@ -319,7 +319,7 @@ private: ReaderHolder reader; - Poco::Logger * log = &Poco::Logger::get("StorageAzureBlobSource"); + LoggerPtr log = getLogger("StorageAzureBlobSource"); ThreadPool create_reader_pool; ThreadPoolCallbackRunner create_reader_scheduler; diff --git a/src/Storages/StorageAzureBlobCluster.cpp b/src/Storages/StorageAzureBlobCluster.cpp index a6372577fb0..1d587512f38 100644 --- a/src/Storages/StorageAzureBlobCluster.cpp +++ b/src/Storages/StorageAzureBlobCluster.cpp @@ -38,7 +38,7 @@ StorageAzureBlobCluster::StorageAzureBlobCluster( const ConstraintsDescription & constraints_, ContextPtr context_, bool structure_argument_was_provided_) - : IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageAzureBlobCluster (" + table_id_.table_name + ")"), structure_argument_was_provided_) + : IStorageCluster(cluster_name_, table_id_, getLogger("StorageAzureBlobCluster (" + table_id_.table_name + ")"), structure_argument_was_provided_) , configuration{configuration_} , object_storage(std::move(object_storage_)) { diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 6f4b1563a46..d5c135bb81d 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -137,7 +137,7 @@ StorageBuffer::StorageBuffer( , flush_thresholds(flush_thresholds_) , destination_id(destination_id_) , allow_materialized(allow_materialized_) - , log(&Poco::Logger::get("StorageBuffer (" + table_id_.getFullTableName() + ")")) + , log(getLogger("StorageBuffer (" + table_id_.getFullTableName() + ")")) , bg_pool(getContext()->getBufferFlushSchedulePool()) { StorageInMemoryMetadata storage_metadata; @@ -433,7 +433,7 @@ void StorageBuffer::read( } -static void appendBlock(Poco::Logger * log, const Block & from, Block & to) +static void appendBlock(LoggerPtr log, const Block & from, Block & to) { size_t rows = from.rows(); size_t old_rows = to.rows(); diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index ef646a12548..47f6239b173 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -166,7 +166,7 @@ private: Writes lifetime_writes; Writes total_writes; - Poco::Logger * log; + LoggerPtr log; void flushAllBuffers(bool check_thresholds = true); bool flushBuffer(Buffer & buffer, bool check_thresholds, bool locked = false); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 987ea4a4957..2d05efdd74f 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -329,7 +329,7 @@ StorageDistributed::StorageDistributed( , remote_database(remote_database_) , remote_table(remote_table_) , remote_table_function_ptr(remote_table_function_ptr_) - , log(&Poco::Logger::get("StorageDistributed (" + id_.table_name + ")")) + , log(getLogger("StorageDistributed (" + id_.table_name + ")")) , owned_cluster(std::move(owned_cluster_)) , cluster_name(getContext()->getMacros()->expand(cluster_name_)) , has_sharding_key(sharding_key_) diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index b7ed85e87df..161a5983f94 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -238,7 +238,7 @@ private: String remote_table; ASTPtr remote_table_function_ptr; - Poco::Logger * log; + LoggerPtr log; /// Used to implement TableFunctionRemote. std::shared_ptr owned_cluster; diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index 2acbf3f4610..e475211deb3 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -80,7 +80,7 @@ StorageExecutable::StorageExecutable( : IStorage(table_id_) , settings(settings_) , input_queries(input_queries_) - , log(settings.is_executable_pool ? &Poco::Logger::get("StorageExecutablePool") : &Poco::Logger::get("StorageExecutable")) + , log(settings.is_executable_pool ? getLogger("StorageExecutablePool") : getLogger("StorageExecutable")) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns); diff --git a/src/Storages/StorageExecutable.h b/src/Storages/StorageExecutable.h index 37455385675..2be2a84ab49 100644 --- a/src/Storages/StorageExecutable.h +++ b/src/Storages/StorageExecutable.h @@ -45,7 +45,7 @@ public: private: ExecutableSettings settings; std::vector input_queries; - Poco::Logger * log; + LoggerPtr log; std::unique_ptr coordinator; }; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 9f864813de9..8979e068fb5 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1040,7 +1040,7 @@ void StorageFileSource::beforeDestroy() catch (const std::exception & e) { // Cannot throw exception from destructor, will write only error - LOG_ERROR(&Poco::Logger::get("~StorageFileSource"), "Failed to rename file {}: {}", file_path_ref, e.what()); + LOG_ERROR(getLogger("~StorageFileSource"), "Failed to rename file {}: {}", file_path_ref, e.what()); continue; } } diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index b74868597a6..2955eb0f1aa 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -163,7 +163,7 @@ private: mutable std::shared_timed_mutex rwlock; - Poco::Logger * log = &Poco::Logger::get("StorageFile"); + LoggerPtr log = getLogger("StorageFile"); /// Total number of bytes to read (sums for multiple files in case of globs). Needed for progress bar. size_t total_bytes_to_read = 0; diff --git a/src/Storages/StorageFileCluster.cpp b/src/Storages/StorageFileCluster.cpp index c12124f1e07..0cc961bb464 100644 --- a/src/Storages/StorageFileCluster.cpp +++ b/src/Storages/StorageFileCluster.cpp @@ -34,7 +34,7 @@ StorageFileCluster::StorageFileCluster( const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, bool structure_argument_was_provided_) - : IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageFileCluster (" + table_id_.table_name + ")"), structure_argument_was_provided_) + : IStorageCluster(cluster_name_, table_id_, getLogger("StorageFileCluster (" + table_id_.table_name + ")"), structure_argument_was_provided_) , filename(filename_) , format_name(format_name_) , compression_method(compression_method_) diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index efe446a8ccd..b9e082c0b22 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -104,7 +104,7 @@ void StorageJoin::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPt if (disk->exists(path)) disk->removeRecursive(path); else - LOG_INFO(&Poco::Logger::get("StorageJoin"), "Path {} is already removed from disk {}", path, disk->getName()); + LOG_INFO(getLogger("StorageJoin"), "Path {} is already removed from disk {}", path, disk->getName()); disk->createDirectories(path); disk->createDirectories(fs::path(path) / "tmp/"); diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 8914838afed..80abaa3ea2d 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -321,7 +321,7 @@ StorageKeeperMap::StorageKeeperMap( , primary_key(primary_key_) , zookeeper_name(zkutil::extractZooKeeperName(zk_root_path_)) , keys_limit(keys_limit_) - , log(&Poco::Logger::get(fmt::format("StorageKeeperMap ({})", table_id.getNameForLogs()))) + , log(getLogger(fmt::format("StorageKeeperMap ({})", table_id.getNameForLogs()))) { std::string path_prefix = context_->getConfigRef().getString("keeper_map_path_prefix", ""); if (path_prefix.empty()) @@ -776,7 +776,7 @@ void StorageKeeperMap::backupData(BackupEntriesCollector & backup_entries_collec auto with_retries = std::make_shared ( - &Poco::Logger::get(fmt::format("StorageKeeperMapBackup ({})", getStorageID().getNameForLogs())), + getLogger(fmt::format("StorageKeeperMapBackup ({})", getStorageID().getNameForLogs())), [&] { return getClient(); }, WithRetries::KeeperSettings::fromContext(backup_entries_collector.getContext()), backup_entries_collector.getContext()->getProcessListElement(), @@ -808,7 +808,7 @@ void StorageKeeperMap::restoreDataFromBackup(RestorerFromBackup & restorer, cons auto with_retries = std::make_shared ( - &Poco::Logger::get(fmt::format("StorageKeeperMapRestore ({})", getStorageID().getNameForLogs())), + getLogger(fmt::format("StorageKeeperMapRestore ({})", getStorageID().getNameForLogs())), [&] { return getClient(); }, WithRetries::KeeperSettings::fromContext(restorer.getContext()), restorer.getContext()->getProcessListElement(), diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index aa9687243d8..9dca96a24a3 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -146,7 +146,7 @@ private: mutable std::mutex init_mutex; mutable std::optional table_is_valid; - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 050e76c9205..bfe75e61bcd 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -523,7 +523,7 @@ void StorageMaterializedView::backupData(BackupEntriesCollector & backup_entries if (auto table = tryGetTargetTable()) table->backupData(backup_entries_collector, data_path_in_backup, partitions); else - LOG_WARNING(&Poco::Logger::get("StorageMaterializedView"), + LOG_WARNING(getLogger("StorageMaterializedView"), "Inner table does not exist, will not backup any data"); } } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index ab2f7ea2989..7e6c5ca3924 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -74,7 +74,7 @@ namespace ActionLocks extern const StorageActionBlockType PartsMove; } -static MergeTreeTransactionPtr tryGetTransactionForMutation(const MergeTreeMutationEntry & mutation, Poco::Logger * log = nullptr) +static MergeTreeTransactionPtr tryGetTransactionForMutation(const MergeTreeMutationEntry & mutation, LoggerPtr log = nullptr) { assert(!mutation.tid.isEmpty()); if (mutation.tid.isPrehistoric()) @@ -683,7 +683,7 @@ std::optional StorageMergeTree::getIncompleteMutationsS const auto & mutation_entry = current_mutation_it->second; - auto txn = tryGetTransactionForMutation(mutation_entry, log.get()); + auto txn = tryGetTransactionForMutation(mutation_entry, log); /// There's no way a transaction may finish before a mutation that was started by the transaction. /// But sometimes we need to check status of an unrelated mutation, in this case we don't care about transactions. assert(txn || mutation_entry.tid.isPrehistoric() || from_another_mutation); @@ -829,7 +829,7 @@ CancellationCode StorageMergeTree::killMutation(const String & mutation_id) if (!to_kill) return CancellationCode::NotFound; - if (auto txn = tryGetTransactionForMutation(*to_kill, log.get())) + if (auto txn = tryGetTransactionForMutation(*to_kill, log)) { LOG_TRACE(log, "Cancelling transaction {} which had started mutation {}", to_kill->tid, mutation_id); TransactionLog::instance().rollbackTransaction(txn); @@ -1222,7 +1222,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate( if (!part->version.isVisible(first_mutation_tid.start_csn, first_mutation_tid)) continue; - txn = tryGetTransactionForMutation(mutations_begin_it->second, log.get()); + txn = tryGetTransactionForMutation(mutations_begin_it->second, log); if (!txn) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find transaction {} that has started mutation {} " "that is going to be applied to part {}", diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 76a439eabaf..da391909dff 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -55,7 +55,7 @@ StorageMySQL::StorageMySQL( , on_duplicate_clause{on_duplicate_clause_} , mysql_settings(mysql_settings_) , pool(std::make_shared(pool_)) - , log(&Poco::Logger::get("StorageMySQL (" + table_id_.table_name + ")")) + , log(getLogger("StorageMySQL (" + table_id_.table_name + ")")) { StorageInMemoryMetadata storage_metadata; diff --git a/src/Storages/StorageMySQL.h b/src/Storages/StorageMySQL.h index 5303117cf5c..daabd66a530 100644 --- a/src/Storages/StorageMySQL.h +++ b/src/Storages/StorageMySQL.h @@ -92,7 +92,7 @@ private: mysqlxx::PoolWithFailoverPtr pool; - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 8fe2a161dba..9379cb5a1c6 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -72,7 +72,7 @@ StoragePostgreSQL::StoragePostgreSQL( , remote_table_schema(remote_table_schema_) , on_conflict(on_conflict_) , pool(std::move(pool_)) - , log(&Poco::Logger::get("StoragePostgreSQL (" + table_id_.table_name + ")")) + , log(getLogger("StoragePostgreSQL (" + table_id_.table_name + ")")) { StorageInMemoryMetadata storage_metadata; diff --git a/src/Storages/StoragePostgreSQL.h b/src/Storages/StoragePostgreSQL.h index 725a935aa46..1ed4f7a7611 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -79,7 +79,7 @@ private: String on_conflict; postgres::PoolWithFailoverPtr pool; - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/src/Storages/StorageRedis.cpp b/src/Storages/StorageRedis.cpp index ddb1b62c7b0..83bb3c606c9 100644 --- a/src/Storages/StorageRedis.cpp +++ b/src/Storages/StorageRedis.cpp @@ -206,7 +206,7 @@ StorageRedis::StorageRedis( , WithContext(context_->getGlobalContext()) , table_id(table_id_) , configuration(configuration_) - , log(&Poco::Logger::get("StorageRedis")) + , log(getLogger("StorageRedis")) , primary_key(primary_key_) { pool = std::make_shared(configuration.pool_size); diff --git a/src/Storages/StorageRedis.h b/src/Storages/StorageRedis.h index a525a4ed7de..a0eb2bfa580 100644 --- a/src/Storages/StorageRedis.h +++ b/src/Storages/StorageRedis.h @@ -74,7 +74,7 @@ private: StorageID table_id; RedisConfiguration configuration; - Poco::Logger * log; + LoggerPtr log; RedisPoolPtr pool; const String primary_key; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 1e86f3f70f5..c82721d2a18 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -320,7 +320,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( attach, [this] (const std::string & name) { enqueuePartForCheck(name); }) , zookeeper_name(zkutil::extractZooKeeperName(zookeeper_path_)) - , zookeeper_path(zkutil::extractZooKeeperPath(zookeeper_path_, /* check_starts_with_slash */ !attach, log.get())) + , zookeeper_path(zkutil::extractZooKeeperPath(zookeeper_path_, /* check_starts_with_slash */ !attach, log)) , replica_name(replica_name_) , replica_path(fs::path(zookeeper_path) / "replicas" / replica_name_) , reader(*this) @@ -812,7 +812,7 @@ bool StorageReplicatedMergeTree::createTableIfNotExists(const StorageMetadataPtr else { auto metadata_drop_lock = zkutil::EphemeralNodeHolder::existing(drop_lock_path, *zookeeper); - if (!removeTableNodesFromZooKeeper(zookeeper, zookeeper_path, metadata_drop_lock, log.get())) + if (!removeTableNodesFromZooKeeper(zookeeper, zookeeper_path, metadata_drop_lock, log)) { /// Someone is recursively removing table right now, we cannot create new table until old one is removed continue; @@ -1128,12 +1128,12 @@ void StorageReplicatedMergeTree::drop() if (lost_part_count > 0) LOG_INFO(log, "Dropping table with non-zero lost_part_count equal to {}", lost_part_count); } - dropReplica(zookeeper, zookeeper_path, replica_name, log.get(), getSettings(), &has_metadata_in_zookeeper); + dropReplica(zookeeper, zookeeper_path, replica_name, log, getSettings(), &has_metadata_in_zookeeper); } } void StorageReplicatedMergeTree::dropReplica(zkutil::ZooKeeperPtr zookeeper, const String & zookeeper_path, const String & replica, - Poco::Logger * logger, MergeTreeSettingsPtr table_settings, std::optional * has_metadata_out) + LoggerPtr logger, MergeTreeSettingsPtr table_settings, std::optional * has_metadata_out) { if (zookeeper->expired()) throw Exception(ErrorCodes::TABLE_WAS_NOT_DROPPED, "Table was not dropped because ZooKeeper session has expired."); @@ -1251,7 +1251,7 @@ void StorageReplicatedMergeTree::dropReplica(zkutil::ZooKeeperPtr zookeeper, con } } -void StorageReplicatedMergeTree::dropReplica(const String & drop_zookeeper_path, const String & drop_replica, Poco::Logger * logger) +void StorageReplicatedMergeTree::dropReplica(const String & drop_zookeeper_path, const String & drop_replica, LoggerPtr logger) { zkutil::ZooKeeperPtr zookeeper = getZooKeeperIfTableShutDown(); @@ -1266,7 +1266,7 @@ void StorageReplicatedMergeTree::dropReplica(const String & drop_zookeeper_path, bool StorageReplicatedMergeTree::removeTableNodesFromZooKeeper(zkutil::ZooKeeperPtr zookeeper, - const String & zookeeper_path, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock, Poco::Logger * logger) + const String & zookeeper_path, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock, LoggerPtr logger) { bool completely_removed = false; @@ -4181,7 +4181,7 @@ void StorageReplicatedMergeTree::startBeingLeader() return; } - zkutil::checkNoOldLeaders(log.get(), *zookeeper, fs::path(zookeeper_path) / "leader_election"); + zkutil::checkNoOldLeaders(log, *zookeeper, fs::path(zookeeper_path) / "leader_election"); LOG_INFO(log, "Became leader"); is_leader = true; @@ -4275,7 +4275,7 @@ void StorageReplicatedMergeTree::waitForUniquePartsToBeFetchedByOtherReplicas(St auto zookeeper = getZooKeeperIfTableShutDown(); - auto unique_parts_set = findReplicaUniqueParts(replica_name, zookeeper_path, format_version, zookeeper, log.get()); + auto unique_parts_set = findReplicaUniqueParts(replica_name, zookeeper_path, format_version, zookeeper, log); if (unique_parts_set.empty()) { LOG_INFO(log, "Will not wait for unique parts to be fetched because we don't have any unique parts"); @@ -4316,7 +4316,7 @@ void StorageReplicatedMergeTree::waitForUniquePartsToBeFetchedByOtherReplicas(St LOG_INFO(log, "Successfully waited all the parts"); } -std::set StorageReplicatedMergeTree::findReplicaUniqueParts(const String & replica_name_, const String & zookeeper_path_, MergeTreeDataFormatVersion format_version_, zkutil::ZooKeeper::Ptr zookeeper_, Poco::Logger * log_) +std::set StorageReplicatedMergeTree::findReplicaUniqueParts(const String & replica_name_, const String & zookeeper_path_, MergeTreeDataFormatVersion format_version_, zkutil::ZooKeeper::Ptr zookeeper_, LoggerPtr log_) { if (!zookeeper_->exists(fs::path(zookeeper_path_) / "replicas" / replica_name_ / "is_active")) { @@ -9348,7 +9348,7 @@ StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & part, co return unlockSharedDataByID( part.getUniqueId(), shared_id, part.info, replica_name, - part.getDataPartStorage().getDiskType(), zookeeper, *getSettings(), log.get(), zookeeper_path, format_version); + part.getDataPartStorage().getDiskType(), zookeeper, *getSettings(), log, zookeeper_path, format_version); } namespace @@ -9364,7 +9364,7 @@ namespace /// But sometimes we need an opposite. When we deleting all_0_0_0_1 it can be non replicated to other replicas, so we are the only owner of this part. /// In this case when we will drop all_0_0_0_1 we will drop blobs for all_0_0_0. But it will lead to dataloss. For such case we need to check that other replicas /// still need parent part. -std::pair getParentLockedBlobs(const ZooKeeperWithFaultInjectionPtr & zookeeper_ptr, const std::string & zero_copy_part_path_prefix, const MergeTreePartInfo & part_info, MergeTreeDataFormatVersion format_version, Poco::Logger * log) +std::pair getParentLockedBlobs(const ZooKeeperWithFaultInjectionPtr & zookeeper_ptr, const std::string & zero_copy_part_path_prefix, const MergeTreePartInfo & part_info, MergeTreeDataFormatVersion format_version, LoggerPtr log) { NameSet files_not_to_remove; @@ -9455,7 +9455,7 @@ std::pair getParentLockedBlobs(const ZooKeeperWithFaultInjectionP std::pair StorageReplicatedMergeTree::unlockSharedDataByID( String part_id, const String & table_uuid, const MergeTreePartInfo & part_info, const String & replica_name_, const std::string & disk_type, const ZooKeeperWithFaultInjectionPtr & zookeeper_ptr, const MergeTreeSettings & settings, - Poco::Logger * logger, const String & zookeeper_path_old, MergeTreeDataFormatVersion data_format_version) + LoggerPtr logger, const String & zookeeper_path_old, MergeTreeDataFormatVersion data_format_version) { boost::replace_all(part_id, "/", "_"); @@ -10143,7 +10143,7 @@ void StorageReplicatedMergeTree::createZeroCopyLockNode( size_t failed_op = zkutil::getFailedOpIndex(error, responses); if (ops[failed_op]->getPath() == zookeeper_node) { - LOG_WARNING(&Poco::Logger::get("ZeroCopyLocks"), "Replacing persistent lock with ephemeral for path {}. It can happen only in case of local part loss", zookeeper_node); + LOG_WARNING(getLogger("ZeroCopyLocks"), "Replacing persistent lock with ephemeral for path {}. It can happen only in case of local part loss", zookeeper_node); replace_existing_lock = true; continue; } @@ -10201,7 +10201,7 @@ bool StorageReplicatedMergeTree::removeSharedDetachedPart(DiskPtr disk, const St detached_replica_name, disk->getDataSourceDescription().toString(), std::make_shared(zookeeper), local_context->getReplicatedMergeTreeSettings(), - &Poco::Logger::get("StorageReplicatedMergeTree"), + getLogger("StorageReplicatedMergeTree"), detached_zookeeper_path, MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING); @@ -10301,7 +10301,7 @@ void StorageReplicatedMergeTree::backupData( bool exists = false; Strings mutation_ids; { - ZooKeeperRetriesControl retries_ctl("getMutations", log.get(), zookeeper_retries_info, nullptr); + ZooKeeperRetriesControl retries_ctl("getMutations", log, zookeeper_retries_info, nullptr); retries_ctl.retryLoop([&]() { if (!zookeeper || zookeeper->expired()) @@ -10320,7 +10320,7 @@ void StorageReplicatedMergeTree::backupData( bool mutation_id_exists = false; String mutation; - ZooKeeperRetriesControl retries_ctl("getMutation", log.get(), zookeeper_retries_info, nullptr); + ZooKeeperRetriesControl retries_ctl("getMutation", log, zookeeper_retries_info, nullptr); retries_ctl.retryLoop([&]() { if (!zookeeper || zookeeper->expired()) diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 3c3c2f56fe2..c682b1ec88d 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -255,13 +255,13 @@ public: /** Remove a specific replica from zookeeper. */ static void dropReplica(zkutil::ZooKeeperPtr zookeeper, const String & zookeeper_path, const String & replica, - Poco::Logger * logger, MergeTreeSettingsPtr table_settings = nullptr, std::optional * has_metadata_out = nullptr); + LoggerPtr logger, MergeTreeSettingsPtr table_settings = nullptr, std::optional * has_metadata_out = nullptr); - void dropReplica(const String & drop_zookeeper_path, const String & drop_replica, Poco::Logger * logger); + void dropReplica(const String & drop_zookeeper_path, const String & drop_replica, LoggerPtr logger); /// Removes table from ZooKeeper after the last replica was dropped static bool removeTableNodesFromZooKeeper(zkutil::ZooKeeperPtr zookeeper, const String & zookeeper_path, - const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock, Poco::Logger * logger); + const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock, LoggerPtr logger); /// Schedules job to execute in background pool (merge, mutate, drop range and so on) bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) override; @@ -308,7 +308,7 @@ public: const std::string & disk_type, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const MergeTreeSettings & settings, - Poco::Logger * logger, + LoggerPtr logger, const String & zookeeper_path_old, MergeTreeDataFormatVersion data_format_version); @@ -773,7 +773,7 @@ private: String findReplicaHavingCoveringPart(LogEntry & entry, bool active); bool findReplicaHavingCoveringPart(const String & part_name, bool active); String findReplicaHavingCoveringPartImplLowLevel(LogEntry * entry, const String & part_name, String & found_part_name, bool active); - static std::set findReplicaUniqueParts(const String & replica_name_, const String & zookeeper_path_, MergeTreeDataFormatVersion format_version_, zkutil::ZooKeeper::Ptr zookeeper_, Poco::Logger * log_); + static std::set findReplicaUniqueParts(const String & replica_name_, const String & zookeeper_path_, MergeTreeDataFormatVersion format_version_, zkutil::ZooKeeper::Ptr zookeeper_, LoggerPtr log_); /** Download the specified part from the specified replica. * If `to_detached`, the part is placed in the `detached` directory. diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index c376af5a3d7..1a7704b4d67 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1194,7 +1194,7 @@ void ReadFromStorageS3Step::initializePipeline(QueryPipelineBuilder & pipeline, const size_t max_threads = local_context->getSettingsRef().max_threads; const size_t max_parsing_threads = num_streams >= max_threads ? 1 : (max_threads / std::max(num_streams, 1ul)); - LOG_DEBUG(&Poco::Logger::get("StorageS3"), "Reading in {} streams, {} threads per stream", num_streams, max_parsing_threads); + LOG_DEBUG(getLogger("StorageS3"), "Reading in {} streams, {} threads per stream", num_streams, max_parsing_threads); Pipes pipes; pipes.reserve(num_streams); @@ -1347,7 +1347,7 @@ void StorageS3::truncate(const ASTPtr & /* query */, const StorageMetadataPtr &, } for (const auto & error : response.GetResult().GetErrors()) - LOG_WARNING(&Poco::Logger::get("StorageS3"), "Failed to delete {}, error: {}", error.GetKey(), error.GetMessage()); + LOG_WARNING(getLogger("StorageS3"), "Failed to delete {}, error: {}", error.GetKey(), error.GetMessage()); } StorageS3::Configuration StorageS3::updateConfigurationAndGetCopy(ContextPtr local_context) diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index b90a0d394cb..8d020c5e9a2 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -242,7 +242,7 @@ private: size_t max_parsing_threads = 1; bool need_only_count; - Poco::Logger * log = &Poco::Logger::get("StorageS3Source"); + LoggerPtr log = getLogger("StorageS3Source"); ThreadPool create_reader_pool; ThreadPoolCallbackRunner create_reader_scheduler; diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index e1738056e9d..25c2b42b766 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -40,7 +40,7 @@ StorageS3Cluster::StorageS3Cluster( const ConstraintsDescription & constraints_, ContextPtr context_, bool structure_argument_was_provided_) - : IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageS3Cluster (" + table_id_.table_name + ")"), structure_argument_was_provided_) + : IStorageCluster(cluster_name_, table_id_, getLogger("StorageS3Cluster (" + table_id_.table_name + ")"), structure_argument_was_provided_) , s3_configuration{configuration_} { context_->getGlobalContext()->getRemoteHostFilter().checkURL(configuration_.url.uri); diff --git a/src/Storages/StorageSQLite.cpp b/src/Storages/StorageSQLite.cpp index d5db5763da9..85c5e16a1bf 100644 --- a/src/Storages/StorageSQLite.cpp +++ b/src/Storages/StorageSQLite.cpp @@ -42,7 +42,7 @@ StorageSQLite::StorageSQLite( , remote_table_name(remote_table_name_) , database_path(database_path_) , sqlite_db(sqlite_db_) - , log(&Poco::Logger::get("StorageSQLite (" + table_id_.table_name + ")")) + , log(getLogger("StorageSQLite (" + table_id_.table_name + ")")) { StorageInMemoryMetadata storage_metadata; diff --git a/src/Storages/StorageSQLite.h b/src/Storages/StorageSQLite.h index 9da040cbd5c..baacdfb4899 100644 --- a/src/Storages/StorageSQLite.h +++ b/src/Storages/StorageSQLite.h @@ -50,7 +50,7 @@ private: String remote_table_name; String database_path; SQLitePtr sqlite_db; - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 1b0db1da800..7d7f3113cdb 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -218,7 +218,7 @@ void StorageSet::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_sn if (disk->exists(path)) disk->removeRecursive(path); else - LOG_INFO(&Poco::Logger::get("StorageSet"), "Path {} is already removed from disk {}", path, disk->getName()); + LOG_INFO(getLogger("StorageSet"), "Path {} is already removed from disk {}", path, disk->getName()); disk->createDirectories(path); disk->createDirectories(fs::path(path) / "tmp/"); @@ -284,7 +284,7 @@ void StorageSetOrJoinBase::restoreFromFile(const String & file_path) finishInsert(); /// TODO Add speed, compressed bytes, data volume in memory, compression ratio ... Generalize all statistics logging in project. - LOG_INFO(&Poco::Logger::get("StorageSetOrJoinBase"), "Loaded from backup file {}. {} rows, {}. State has {} unique rows.", + LOG_INFO(getLogger("StorageSetOrJoinBase"), "Loaded from backup file {}. {} rows, {}. State has {} unique rows.", file_path, info.rows, ReadableSize(info.bytes), getSize(ctx)); } diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 91f6246d101..359f142949f 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -277,7 +277,7 @@ StorageStripeLog::StorageStripeLog( , index_file_path(table_path + "index.mrk") , file_checker(disk, table_path + "sizes.json") , max_compress_block_size(context_->getSettings().max_compress_block_size) - , log(&Poco::Logger::get("StorageStripeLog")) + , log(getLogger("StorageStripeLog")) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index a05117a9ad5..c7f3e7e21e6 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -123,7 +123,7 @@ private: mutable std::shared_timed_mutex rwlock; - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 0ba72af6fc0..631c06bd87b 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -1284,7 +1284,7 @@ StorageURLWithFailover::StorageURLWithFailover( { Poco::URI poco_uri(uri_option); context_->getRemoteHostFilter().checkURL(poco_uri); - LOG_DEBUG(&Poco::Logger::get("StorageURLDistributed"), "Adding URL option: {}", uri_option); + LOG_DEBUG(getLogger("StorageURLDistributed"), "Adding URL option: {}", uri_option); uri_options.emplace_back(uri_option); } } diff --git a/src/Storages/StorageURLCluster.cpp b/src/Storages/StorageURLCluster.cpp index a0b5fcd6f28..2365887983d 100644 --- a/src/Storages/StorageURLCluster.cpp +++ b/src/Storages/StorageURLCluster.cpp @@ -45,7 +45,7 @@ StorageURLCluster::StorageURLCluster( const ConstraintsDescription & constraints_, const StorageURL::Configuration & configuration_, bool structure_argument_was_provided_) - : IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageURLCluster (" + table_id_.table_name + ")"), structure_argument_was_provided_) + : IStorageCluster(cluster_name_, table_id_, getLogger("StorageURLCluster (" + table_id_.table_name + ")"), structure_argument_was_provided_) , uri(uri_) { context_->getRemoteHostFilter().checkURL(Poco::URI(uri)); diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index a274b1ba4db..259abefb00f 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -45,7 +45,7 @@ StorageXDBC::StorageXDBC( , bridge_helper(bridge_helper_) , remote_database_name(remote_database_name_) , remote_table_name(remote_table_name_) - , log(&Poco::Logger::get("Storage" + bridge_helper->getName())) + , log(getLogger("Storage" + bridge_helper->getName())) { uri = bridge_helper->getMainURI().toString(); } diff --git a/src/Storages/StorageXDBC.h b/src/Storages/StorageXDBC.h index fe678785dc2..cba15a83226 100644 --- a/src/Storages/StorageXDBC.h +++ b/src/Storages/StorageXDBC.h @@ -47,7 +47,7 @@ private: std::string remote_database_name; std::string remote_table_name; - Poco::Logger * log; + LoggerPtr log; std::string getReadMethod() const override; diff --git a/src/Storages/System/StorageSystemDatabases.cpp b/src/Storages/System/StorageSystemDatabases.cpp index 0ffed6c9771..51ecb8f17ca 100644 --- a/src/Storages/System/StorageSystemDatabases.cpp +++ b/src/Storages/System/StorageSystemDatabases.cpp @@ -54,7 +54,7 @@ static String getEngineFull(const ContextPtr & ctx, const DatabasePtr & database return {}; guard.reset(); - LOG_TRACE(&Poco::Logger::get("StorageSystemDatabases"), "Failed to lock database {} ({}), will retry", name, database->getUUID()); + LOG_TRACE(getLogger("StorageSystemDatabases"), "Failed to lock database {} ({}), will retry", name, database->getUUID()); } ASTPtr ast = database->getCreateDatabaseQuery(); diff --git a/src/Storages/System/StorageSystemJemalloc.cpp b/src/Storages/System/StorageSystemJemalloc.cpp index 9c3a075b2c1..15543208dd9 100644 --- a/src/Storages/System/StorageSystemJemalloc.cpp +++ b/src/Storages/System/StorageSystemJemalloc.cpp @@ -77,7 +77,7 @@ void fillJemallocBins(MutableColumns & res_columns) void fillJemallocBins(MutableColumns &) { - LOG_INFO(&Poco::Logger::get("StorageSystemJemallocBins"), "jemalloc is not enabled"); + LOG_INFO(getLogger("StorageSystemJemallocBins"), "jemalloc is not enabled"); } #endif // USE_JEMALLOC diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index d9a12095443..eeb3db342b4 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -56,12 +56,12 @@ private: /// Used to assign unique incremental ids to requests. UInt64 request_id TSA_GUARDED_BY(mutex) = 0; - Poco::Logger * log; + LoggerPtr log; public: explicit StatusRequestsPool(size_t max_threads) : thread_pool(CurrentMetrics::SystemReplicasThreads, CurrentMetrics::SystemReplicasThreadsActive, CurrentMetrics::SystemReplicasThreadsScheduled, max_threads) - , log(&Poco::Logger::get("StatusRequestsPool")) + , log(getLogger("StatusRequestsPool")) {} ~StatusRequestsPool() diff --git a/src/Storages/System/StorageSystemStackTrace.cpp b/src/Storages/System/StorageSystemStackTrace.cpp index e02d4bf1733..b17d04e9895 100644 --- a/src/Storages/System/StorageSystemStackTrace.cpp +++ b/src/Storages/System/StorageSystemStackTrace.cpp @@ -173,7 +173,7 @@ bool wait(int timeout_ms) } using ThreadIdToName = std::unordered_map>; -ThreadIdToName getFilteredThreadNames(const ActionsDAG::Node * predicate, ContextPtr context, const PaddedPODArray & thread_ids, Poco::Logger * log) +ThreadIdToName getFilteredThreadNames(const ActionsDAG::Node * predicate, ContextPtr context, const PaddedPODArray & thread_ids, LoggerPtr log) { ThreadIdToName tid_to_name; MutableColumnPtr all_thread_names = ColumnString::create(); @@ -274,7 +274,7 @@ bool isSignalBlocked(UInt64 tid, int signal) class StackTraceSource : public ISource { public: - StackTraceSource(const Names & column_names, Block header_, ASTPtr && query_, ActionsDAGPtr && filter_dag_, ContextPtr context_, UInt64 max_block_size_, Poco::Logger * log_) + StackTraceSource(const Names & column_names, Block header_, ASTPtr && query_, ActionsDAGPtr && filter_dag_, ContextPtr context_, UInt64 max_block_size_, LoggerPtr log_) : ISource(header_) , context(context_) , header(std::move(header_)) @@ -426,7 +426,7 @@ private: bool send_signal = false; bool read_thread_names = false; - Poco::Logger * log; + LoggerPtr log; std::filesystem::directory_iterator proc_it; std::filesystem::directory_iterator end; @@ -481,7 +481,7 @@ public: ASTPtr && query_, ContextPtr context_, size_t max_block_size_, - Poco::Logger * log_) + LoggerPtr log_) : SourceStepWithFilter(DataStream{.header = std::move(sample_block)}) , column_names(column_names_) , query(query_) @@ -496,7 +496,7 @@ private: ASTPtr query; ContextPtr context; size_t max_block_size; - Poco::Logger * log; + LoggerPtr log; }; } @@ -504,7 +504,7 @@ private: StorageSystemStackTrace::StorageSystemStackTrace(const StorageID & table_id_) : IStorage(table_id_) - , log(&Poco::Logger::get("StorageSystemStackTrace")) + , log(getLogger("StorageSystemStackTrace")) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(ColumnsDescription({ diff --git a/src/Storages/System/StorageSystemStackTrace.h b/src/Storages/System/StorageSystemStackTrace.h index 18216cea1bd..ce1b7f8ccd2 100644 --- a/src/Storages/System/StorageSystemStackTrace.h +++ b/src/Storages/System/StorageSystemStackTrace.h @@ -38,7 +38,7 @@ public: bool isSystemStorage() const override { return true; } protected: - Poco::Logger * log; + LoggerPtr log; }; } diff --git a/src/Storages/UVLoop.h b/src/Storages/UVLoop.h index 4945e1b56fa..dd1d64973d1 100644 --- a/src/Storages/UVLoop.h +++ b/src/Storages/UVLoop.h @@ -63,7 +63,7 @@ public: private: std::unique_ptr loop_ptr; - Poco::Logger * log = &Poco::Logger::get("UVLoop"); + LoggerPtr log = getLogger("UVLoop"); static void onUVWalkClosingCallback(uv_handle_t * handle, void *) { diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index f9ba8e9717f..0764685cb07 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1160,7 +1160,7 @@ StorageWindowView::StorageWindowView( bool attach_) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) - , log(&Poco::Logger::get(fmt::format("StorageWindowView({}.{})", table_id_.database_name, table_id_.table_name))) + , log(getLogger(fmt::format("StorageWindowView({}.{})", table_id_.database_name, table_id_.table_name))) , fire_signal_timeout_s(context_->getSettingsRef().wait_for_window_view_fire_signal_timeout.totalSeconds()) , clean_interval_usec(context_->getSettingsRef().window_view_clean_interval.totalMicroseconds()) { diff --git a/src/Storages/WindowView/StorageWindowView.h b/src/Storages/WindowView/StorageWindowView.h index de8f880c602..969fda8f78e 100644 --- a/src/Storages/WindowView/StorageWindowView.h +++ b/src/Storages/WindowView/StorageWindowView.h @@ -177,7 +177,7 @@ public: const Block & getOutputHeader() const; private: - Poco::Logger * log; + LoggerPtr log; /// Stored query, e.g. SELECT * FROM * GROUP BY tumble(now(), *) ASTPtr select_query; diff --git a/src/TableFunctions/Hive/TableFunctionHive.cpp b/src/TableFunctions/Hive/TableFunctionHive.cpp index d8885087532..e840d5fc8be 100644 --- a/src/TableFunctions/Hive/TableFunctionHive.cpp +++ b/src/TableFunctions/Hive/TableFunctionHive.cpp @@ -46,7 +46,7 @@ public: void parseArguments(const ASTPtr & ast_function_, ContextPtr context_) override; private: - Poco::Logger * logger = &Poco::Logger::get("TableFunctionHive"); + LoggerPtr logger = getLogger("TableFunctionHive"); String cluster_name; String hive_metastore_url; From 109337e91ea4f13a72710d5da263988760c14f83 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 24 Jan 2024 16:00:13 +0300 Subject: [PATCH 187/264] Updated implementation --- base/poco/Foundation/src/Logger.cpp | 60 +++++++++------------------ src/Common/Config/ConfigProcessor.cpp | 2 +- src/Common/Logger.cpp | 2 +- src/Common/Logger.h | 2 +- src/Common/tests/gtest_log.cpp | 6 +-- 5 files changed, 26 insertions(+), 46 deletions(-) diff --git a/base/poco/Foundation/src/Logger.cpp b/base/poco/Foundation/src/Logger.cpp index 7b562217268..939c88cb3c0 100644 --- a/base/poco/Foundation/src/Logger.cpp +++ b/base/poco/Foundation/src/Logger.cpp @@ -25,31 +25,17 @@ namespace { -std::atomic is_logging_infrastructure_destroyed{false}; - -class LoggerMutex +std::mutex & getLoggerMutex() { -public: - void lock() + auto get_logger_mutex_placeholder_memory = []() { - mutex.lock(); - } + static char buffer[sizeof(std::mutex)]{}; + return buffer; + }; - void unlock() - { - mutex.unlock(); - } - - ~LoggerMutex() - { - is_logging_infrastructure_destroyed.store(true, std::memory_order_release); - } - -private: - std::mutex mutex; -}; - -LoggerMutex logger_mutex; + static std::mutex * logger_mutex = new (get_logger_mutex_placeholder_memory()) std::mutex(); + return *logger_mutex; +} } @@ -143,7 +129,7 @@ void Logger::dump(const std::string& msg, const void* buffer, std::size_t length void Logger::setLevel(const std::string& name, int level) { - std::lock_guard lock(logger_mutex); + std::lock_guard lock(getLoggerMutex()); if (_pLoggerMap) { @@ -162,7 +148,7 @@ void Logger::setLevel(const std::string& name, int level) void Logger::setChannel(const std::string& name, Channel* pChannel) { - std::lock_guard lock(logger_mutex); + std::lock_guard lock(getLoggerMutex()); if (_pLoggerMap) { @@ -181,7 +167,7 @@ void Logger::setChannel(const std::string& name, Channel* pChannel) void Logger::setProperty(const std::string& loggerName, const std::string& propertyName, const std::string& value) { - std::lock_guard lock(logger_mutex); + std::lock_guard lock(getLoggerMutex()); if (_pLoggerMap) { @@ -318,12 +304,6 @@ struct LoggerDeleter { void operator()(Poco::Logger * logger) { - if (is_logging_infrastructure_destroyed.load(std::memory_order_acquire)) - { - logger->release(); - return; - } - Logger::destroy(logger->name()); } }; @@ -338,14 +318,14 @@ inline LoggerPtr makeLoggerPtr(Logger & logger) Logger& Logger::get(const std::string& name) { - std::lock_guard lock(logger_mutex); + std::lock_guard lock(getLoggerMutex()); return unsafeGet(name); } LoggerPtr Logger::getShared(const std::string & name) { - std::lock_guard lock(logger_mutex); + std::lock_guard lock(getLoggerMutex()); return makeLoggerPtr(unsafeGet(name)); } @@ -372,21 +352,21 @@ Logger& Logger::unsafeGet(const std::string& name) Logger& Logger::create(const std::string& name, Channel* pChannel, int level) { - std::lock_guard lock(logger_mutex); + std::lock_guard lock(getLoggerMutex()); return unsafeCreate(name, pChannel, level); } LoggerPtr Logger::createShared(const std::string & name, Channel * pChannel, int level) { - std::lock_guard lock(logger_mutex); + std::lock_guard lock(getLoggerMutex()); return makeLoggerPtr(unsafeCreate(name, pChannel, level)); } Logger& Logger::root() { - std::lock_guard lock(logger_mutex); + std::lock_guard lock(getLoggerMutex()); return unsafeGet(ROOT); } @@ -394,7 +374,7 @@ Logger& Logger::root() Logger* Logger::has(const std::string& name) { - std::lock_guard lock(logger_mutex); + std::lock_guard lock(getLoggerMutex()); return find(name); } @@ -402,7 +382,7 @@ Logger* Logger::has(const std::string& name) void Logger::shutdown() { - std::lock_guard lock(logger_mutex); + std::lock_guard lock(getLoggerMutex()); if (_pLoggerMap) { @@ -430,7 +410,7 @@ Logger* Logger::find(const std::string& name) void Logger::destroy(const std::string& name) { - std::lock_guard lock(logger_mutex); + std::lock_guard lock(getLoggerMutex()); if (_pLoggerMap) { @@ -446,7 +426,7 @@ void Logger::destroy(const std::string& name) void Logger::names(std::vector& names) { - std::lock_guard lock(logger_mutex); + std::lock_guard lock(getLoggerMutex()); names.clear(); if (_pLoggerMap) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 502ce9f0376..641e7ddcdaa 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -80,7 +80,7 @@ ConfigProcessor::ConfigProcessor( if (log_to_console && !hasLogger("ConfigProcessor")) { channel_ptr = new Poco::ConsoleChannel; - log = getLogger("ConfigProcessor", channel_ptr.get(), Poco::Message::PRIO_TRACE); + log = createLogger("ConfigProcessor", channel_ptr.get(), Poco::Message::PRIO_TRACE); } else { diff --git a/src/Common/Logger.cpp b/src/Common/Logger.cpp index fe1bb68df5c..c8d557bc3a3 100644 --- a/src/Common/Logger.cpp +++ b/src/Common/Logger.cpp @@ -6,7 +6,7 @@ LoggerPtr getLogger(const std::string & name) return Poco::Logger::getShared(name); } -LoggerPtr getLogger(const std::string & name, Poco::Channel * channel, Poco::Message::Priority level) +LoggerPtr createLogger(const std::string & name, Poco::Channel * channel, Poco::Message::Priority level) { return Poco::Logger::createShared(name, channel, level); } diff --git a/src/Common/Logger.h b/src/Common/Logger.h index b8137c13e86..13e1c6bf8f5 100644 --- a/src/Common/Logger.h +++ b/src/Common/Logger.h @@ -28,7 +28,7 @@ LoggerPtr getLogger(const std::string & name); * If Logger already exists, throws exception. * Logger is destroyed, when last shared ptr that refers to Logger with specified name is destroyed. */ -LoggerPtr getLogger(const std::string & name, Poco::Channel * channel, Poco::Message::Priority level = Poco::Message::PRIO_INFORMATION); +LoggerPtr createLogger(const std::string & name, Poco::Channel * channel, Poco::Message::Priority level = Poco::Message::PRIO_INFORMATION); /** Create raw Poco::Logger that will not be destroyed before program termination. * This can be used in cases when specific Logger instance can be singletone. diff --git a/src/Common/tests/gtest_log.cpp b/src/Common/tests/gtest_log.cpp index 10e85430321..419aac370d6 100644 --- a/src/Common/tests/gtest_log.cpp +++ b/src/Common/tests/gtest_log.cpp @@ -27,7 +27,7 @@ TEST(Logger, TestLog) std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM auto my_channel = Poco::AutoPtr(new Poco::StreamChannel(oss)); - auto log = getLogger("TestLogger", my_channel.get()); + auto log = createLogger("TestLogger", my_channel.get()); log->setLevel("test"); LOG_TEST(log, "Hello World"); @@ -40,7 +40,7 @@ TEST(Logger, TestLog) { std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM auto my_channel = Poco::AutoPtr(new Poco::StreamChannel(oss)); - auto log = getLogger(std::string{level} + "_Logger", my_channel.get()); + auto log = createLogger(std::string{level} + "_Logger", my_channel.get()); log->setLevel(level); LOG_TEST(log, "Hello World"); @@ -84,7 +84,7 @@ TEST(Logger, SideEffects) { std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM auto my_channel = Poco::AutoPtr(new Poco::StreamChannel(oss)); - auto log = getLogger("Logger", my_channel.get()); + auto log = createLogger("Logger", my_channel.get()); log->setLevel("trace"); /// Ensure that parameters are evaluated only once From 485a46f6102043afc7ac3c8ff474ada214caff5d Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 24 Jan 2024 18:53:16 +0300 Subject: [PATCH 188/264] Fixed tests --- base/poco/Foundation/include/Poco/Logger.h | 2 +- base/poco/Foundation/src/Logger.cpp | 11 +++++++++-- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/base/poco/Foundation/include/Poco/Logger.h b/base/poco/Foundation/include/Poco/Logger.h index 79d71c506d5..f91d836f190 100644 --- a/base/poco/Foundation/include/Poco/Logger.h +++ b/base/poco/Foundation/include/Poco/Logger.h @@ -904,7 +904,7 @@ public: /// Returns a pointer to the Logger with the given name if it /// exists, or a null pointer otherwise. - static void destroy(const std::string & name); + static bool destroy(const std::string & name); /// Destroys the logger with the specified name. Does nothing /// if the logger is not found. /// diff --git a/base/poco/Foundation/src/Logger.cpp b/base/poco/Foundation/src/Logger.cpp index 939c88cb3c0..7c54116aaa4 100644 --- a/base/poco/Foundation/src/Logger.cpp +++ b/base/poco/Foundation/src/Logger.cpp @@ -304,7 +304,10 @@ struct LoggerDeleter { void operator()(Poco::Logger * logger) { - Logger::destroy(logger->name()); + if (Logger::destroy(logger->name())) + return; + + logger->release(); } }; @@ -408,7 +411,7 @@ Logger* Logger::find(const std::string& name) } -void Logger::destroy(const std::string& name) +bool Logger::destroy(const std::string& name) { std::lock_guard lock(getLoggerMutex()); @@ -419,8 +422,12 @@ void Logger::destroy(const std::string& name) { if (it->second->release() == 1) _pLoggerMap->erase(it); + + return true; } } + + return false; } From c277148ee15edf4a0c6b402fb2ca205293148eb9 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 25 Jan 2024 12:01:51 +0000 Subject: [PATCH 189/264] enhanced tests, fix --- src/Functions/FunctionsConversion.h | 14 ++++++-------- .../02972_to_string_nullable_timezone.reference | 1 + .../02972_to_string_nullable_timezone.sql | 1 + 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index cd13a6f5cd0..87fa94bd75f 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -1076,7 +1076,6 @@ struct ConvertImpl) { auto datetime_arg = arguments[0]; - auto timezone_arg = arguments[1]; const DateLUTImpl * time_zone = nullptr; const ColumnConst * time_zone_column = nullptr; @@ -1089,14 +1088,13 @@ struct ConvertImplconvertToFullColumnIfConst(); - timezone_arg.column = timezone_arg.column->convertToFullColumnIfConst(); if constexpr (std::is_same_v || std::is_same_v) time_zone = &DateLUT::instance(); /// For argument of Date or DateTime type, second argument with time zone could be specified. if constexpr (std::is_same_v || std::is_same_v) { - if ((time_zone_column = checkAndGetColumnConst(timezone_arg.column.get()))) + if ((time_zone_column = checkAndGetColumnConst(arguments[1].column.get()))) { auto non_null_args = createBlockWithNestedColumns(arguments); time_zone = &extractTimeZoneFromFunctionArguments(non_null_args, 1, 0); @@ -1133,7 +1131,7 @@ struct ConvertImpl 1) - null_map = copyNullMap(timezone_arg.column); + null_map = copyNullMap(arguments[1].column); if (null_map) { @@ -1141,8 +1139,8 @@ struct ConvertImpl 1) { - if (!timezone_arg.column.get()->getDataAt(i).toString().empty()) - time_zone = &DateLUT::instance(timezone_arg.column.get()->getDataAt(i).toString()); + if (!arguments[1].column.get()->getDataAt(i).toString().empty()) + time_zone = &DateLUT::instance(arguments[1].column.get()->getDataAt(i).toString()); else throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Provided time zone must be non-empty"); } @@ -1158,8 +1156,8 @@ struct ConvertImpl 1) { - if (!timezone_arg.column.get()->getDataAt(i).toString().empty()) - time_zone = &DateLUT::instance(timezone_arg.column.get()->getDataAt(i).toString()); + if (!arguments[1].column.get()->getDataAt(i).toString().empty()) + time_zone = &DateLUT::instance(arguments[1].column.get()->getDataAt(i).toString()); else throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Provided time zone must be non-empty"); } diff --git a/tests/queries/0_stateless/02972_to_string_nullable_timezone.reference b/tests/queries/0_stateless/02972_to_string_nullable_timezone.reference index d1e3a643a5f..6c362c0207e 100644 --- a/tests/queries/0_stateless/02972_to_string_nullable_timezone.reference +++ b/tests/queries/0_stateless/02972_to_string_nullable_timezone.reference @@ -1,2 +1,3 @@ 2022-01-01 11:13:14 2022-01-01 11:13:14 +2022-01-01 11:13:14 diff --git a/tests/queries/0_stateless/02972_to_string_nullable_timezone.sql b/tests/queries/0_stateless/02972_to_string_nullable_timezone.sql index d4bdb835336..d8cff4f3c00 100644 --- a/tests/queries/0_stateless/02972_to_string_nullable_timezone.sql +++ b/tests/queries/0_stateless/02972_to_string_nullable_timezone.sql @@ -1,3 +1,4 @@ SET session_timezone='Europe/Amsterdam'; SELECT toString(toDateTime('2022-01-01 12:13:14'), CAST('UTC', 'Nullable(String)')); SELECT toString(toDateTime('2022-01-01 12:13:14'), materialize(CAST('UTC', 'Nullable(String)'))); +SELECT toString(CAST(toDateTime('2022-01-01 12:13:14'), 'Nullable(DateTime)'), materialize(CAST('UTC', 'Nullable(String)'))); From 4b26de9660a317f827bcba4e84cde4176db195d2 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 25 Jan 2024 13:26:13 +0000 Subject: [PATCH 190/264] Rename setting to use it not only in if/multiIf in future --- docs/en/operations/settings/settings.md | 6 +++--- docs/en/sql-reference/data-types/variant.md | 6 +++--- src/Core/Settings.h | 4 ++-- src/Core/SettingsChangesHistory.h | 2 +- src/Functions/if.cpp | 2 +- src/Functions/multiIf.cpp | 2 +- tests/queries/0_stateless/02941_variant_type_alters.sh | 2 +- tests/queries/0_stateless/02943_variant_element.sql | 2 +- tests/queries/0_stateless/02943_variant_read_subcolumns.sh | 2 +- ...43_variant_type_with_different_local_and_global_order.sh | 2 +- .../0_stateless/02944_variant_as_if_multi_if_result.sql | 2 +- 11 files changed, 16 insertions(+), 16 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index dcc46a8b4ba..fb2e04f1e35 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5182,14 +5182,14 @@ Allows creation of experimental [Variant](../../sql-reference/data-types/variant Default value: `false`. -## use_variant_when_no_common_type_in_if {#use_variant_when_no_common_type_in_if} +## use_variant_as_common_type {#use_variant_as_common_type} Allows to use `Variant` type as a result type for [if](../../sql-reference/functions/conditional-functions.md/#if)/[multiIf](../../sql-reference/functions/conditional-functions.md/#multiif) functions when there is no common type for argument types. Example: ```sql -SET use_variant_when_no_common_type_in_if = 1; +SET use_variant_as_common_type = 1; SELECT toTypeName(if(number % 2, number, range(number))) as variant_type FROM numbers(1); SELECT if(number % 2, number, range(number)) as variant FROM numbers(5); ``` @@ -5208,7 +5208,7 @@ SELECT if(number % 2, number, range(number)) as variant FROM numbers(5); ``` ```sql -SET use_variant_when_no_common_type_in_if = 1; +SET use_variant_as_common_type = 1; SELECT toTypeName(multiIf((number % 4) = 0, 42, (number % 4) = 1, [1, 2, 3], (number % 4) = 2, 'Hello, World!', NULL)) AS variant_type FROM numbers(1); SELECT multiIf((number % 4) = 0, 42, (number % 4) = 1, [1, 2, 3], (number % 4) = 2, 'Hello, World!', NULL) AS variant FROM numbers(4); ``` diff --git a/docs/en/sql-reference/data-types/variant.md b/docs/en/sql-reference/data-types/variant.md index 34966d79079..afbd1b798b2 100644 --- a/docs/en/sql-reference/data-types/variant.md +++ b/docs/en/sql-reference/data-types/variant.md @@ -47,10 +47,10 @@ SELECT toTypeName(variant) as type_name, 'Hello, World!'::Variant(UInt64, String └────────────────────────────────────────┴───────────────┘ ``` -Using functions `if/multiIf` when arguments doesn't have common type (setting `use_variant_when_no_common_type_in_if` should be enabled for it): +Using functions `if/multiIf` when arguments doesn't have common type (setting `use_variant_as_common_type` should be enabled for it): ```sql -SET use_variant_when_no_common_type_in_if = 1; +SET use_variant_as_common_type = 1; SELECT if(number % 2, number, range(number)) as variant FROM numbers(5); ``` @@ -65,7 +65,7 @@ SELECT if(number % 2, number, range(number)) as variant FROM numbers(5); ``` ```sql -SET use_variant_when_no_common_type_in_if = 1; +SET use_variant_as_common_type = 1; SELECT multiIf((number % 4) = 0, 42, (number % 4) = 1, [1, 2, 3], (number % 4) = 2, 'Hello, World!', NULL) AS variant FROM numbers(4); ``` diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 11d33594257..da26a621c0e 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -229,7 +229,7 @@ class IColumn; \ M(Bool, force_index_by_date, false, "Throw an exception if there is a partition key in a table, and it is not used.", 0) \ M(Bool, force_primary_key, false, "Throw an exception if there is primary key in a table, and it is not used.", 0) \ - M(Bool, use_skip_indexes, true, "Use data skipping indexes during query execution.", 0) \ + M(Bool, use_skip_indexes, true, "Use data skinipping indexes during query execution.", 0) \ M(Bool, use_skip_indexes_if_final, false, "If query has FINAL, then skipping data based on indexes may produce incorrect result, hence disabled by default.", 0) \ M(String, ignore_data_skipping_indices, "", "Comma separated list of strings or literals with the name of the data skipping indices that should be excluded during query execution.", 0) \ \ @@ -824,7 +824,7 @@ class IColumn; M(Bool, function_json_value_return_type_allow_complex, false, "Allow function JSON_VALUE to return complex type, such as: struct, array, map.", 0) \ M(Bool, use_with_fill_by_sorting_prefix, true, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with different values in sorting prefix are filled independently", 0) \ M(Bool, optimize_uniq_to_count, true, "Rewrite uniq and its variants(except uniqUpTo) to count if subquery has distinct or group by clause.", 0) \ - M(Bool, use_variant_when_no_common_type_in_if, false, "Use Variant as a result type for if/multiIf in case when there is no common type for arguments", 0) \ + M(Bool, use_variant_as_common_type, false, "Use Variant as a result type for if/multiIf in case when there is no common type for arguments", 0) \ \ /** Experimental functions */ \ M(Bool, allow_experimental_materialized_postgresql_table, false, "Allows to use the MaterializedPostgreSQL table engine. Disabled by default, because this feature is experimental", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index b0306bd951b..d00ccf5aa66 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -88,7 +88,7 @@ static std::map sett {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}, {"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"}, {"allow_experimental_variant_type", false, false, "Add new experimental Variant type"}, - {"use_variant_when_no_common_type_in_if", false, false, "Allow to use Variant in if/multiIf if there is no common type"}, + {"use_variant_as_common_type", false, false, "Allow to use Variant in if/multiIf if there is no common type"}, {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, {"parallel_replicas_mark_segment_size", 128, 128, "Add new setting to control segment size in new parallel replicas coordinator implementation"}, {"ignore_materialized_views_with_dropped_target_table", false, false, "Add new setting to allow to ignore materialized views with dropped target table"}, diff --git a/src/Functions/if.cpp b/src/Functions/if.cpp index c247938f885..c784b2fca59 100644 --- a/src/Functions/if.cpp +++ b/src/Functions/if.cpp @@ -264,7 +264,7 @@ public: static constexpr auto name = "if"; static FunctionPtr create(ContextPtr context) { - return std::make_shared(context->getSettingsRef().allow_experimental_variant_type && context->getSettingsRef().use_variant_when_no_common_type_in_if); + return std::make_shared(context->getSettingsRef().allow_experimental_variant_type && context->getSettingsRef().use_variant_as_common_type); } explicit FunctionIf(bool use_variant_when_no_common_type_ = false) : FunctionIfBase(), use_variant_when_no_common_type(use_variant_when_no_common_type_) {} diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index 31662bb353e..cb946b55c73 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -118,7 +118,7 @@ public: types_of_branches.emplace_back(arg); }); - if (context->getSettingsRef().allow_experimental_variant_type && context->getSettingsRef().use_variant_when_no_common_type_in_if) + if (context->getSettingsRef().allow_experimental_variant_type && context->getSettingsRef().use_variant_as_common_type) return getLeastSupertypeOrVariant(types_of_branches); return getLeastSupertype(types_of_branches); diff --git a/tests/queries/0_stateless/02941_variant_type_alters.sh b/tests/queries/0_stateless/02941_variant_type_alters.sh index 9b0d4febd65..7c151d1fe9e 100755 --- a/tests/queries/0_stateless/02941_variant_type_alters.sh +++ b/tests/queries/0_stateless/02941_variant_type_alters.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_when_no_common_type_in_if=1 " +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 " function run() { diff --git a/tests/queries/0_stateless/02943_variant_element.sql b/tests/queries/0_stateless/02943_variant_element.sql index c8eff9775ad..556c0147e56 100644 --- a/tests/queries/0_stateless/02943_variant_element.sql +++ b/tests/queries/0_stateless/02943_variant_element.sql @@ -1,5 +1,5 @@ set allow_experimental_variant_type=1; -set use_variant_when_no_common_type_in_if=1; +set use_variant_as_common_type=1; select variantElement(NULL::Variant(String, UInt64), 'UInt64') from numbers(4); select variantElement(number::Variant(String, UInt64), 'UInt64') from numbers(4); diff --git a/tests/queries/0_stateless/02943_variant_read_subcolumns.sh b/tests/queries/0_stateless/02943_variant_read_subcolumns.sh index 9ccad55191f..88be09c2036 100755 --- a/tests/queries/0_stateless/02943_variant_read_subcolumns.sh +++ b/tests/queries/0_stateless/02943_variant_read_subcolumns.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_when_no_common_type_in_if=1 " +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 " function test() diff --git a/tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.sh b/tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.sh index 9f4df8d7466..d089ed3cb2f 100755 --- a/tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.sh +++ b/tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_when_no_common_type_in_if=1 " +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 " function test1_insert() diff --git a/tests/queries/0_stateless/02944_variant_as_if_multi_if_result.sql b/tests/queries/0_stateless/02944_variant_as_if_multi_if_result.sql index 1121b21e383..d88c0e8b7ae 100644 --- a/tests/queries/0_stateless/02944_variant_as_if_multi_if_result.sql +++ b/tests/queries/0_stateless/02944_variant_as_if_multi_if_result.sql @@ -1,6 +1,6 @@ set allow_experimental_analyzer=0; -- The result type for if function with constant is different with analyzer. set allow_experimental_variant_type=1; -set use_variant_when_no_common_type_in_if=1; +set use_variant_as_common_type=1; select toTypeName(res), if(1, [1,2,3], 'str_1') as res; select toTypeName(res), if(1, [1,2,3], 'str_1'::Nullable(String)) as res; From da1cf46b1495f8fc031285f664e4c9dba225ea34 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 25 Jan 2024 15:47:32 +0100 Subject: [PATCH 191/264] Fix fuzzer --- src/Functions/FunctionsConversion.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 87fa94bd75f..11ce68eba21 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -1131,7 +1131,7 @@ struct ConvertImpl 1) - null_map = copyNullMap(arguments[1].column); + null_map = copyNullMap(arguments[1].column->convertToFullColumnIfConst()); if (null_map) { From 5bb734a4bbcd9b9aa3c00d676127767d5288fa6c Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 25 Jan 2024 18:22:49 +0300 Subject: [PATCH 192/264] ActionsDAG buildFilterActionsDAG refactoring --- src/Interpreters/ActionsDAG.cpp | 6 ++---- src/Interpreters/ActionsDAG.h | 3 +-- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Interpreters/MutationsInterpreter.cpp | 2 +- src/Planner/Planner.cpp | 2 +- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 2 +- src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp | 2 +- src/Processors/SourceWithKeyCondition.h | 2 +- src/Storages/HDFS/StorageHDFS.cpp | 2 +- src/Storages/Hive/StorageHive.cpp | 2 +- src/Storages/IStorageCluster.cpp | 2 +- src/Storages/KVStorageUtils.cpp | 2 +- src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp | 8 ++++---- src/Storages/MergeTree/MergeTreeWhereOptimizer.h | 2 +- src/Storages/S3Queue/StorageS3Queue.cpp | 2 +- src/Storages/StorageAzureBlob.cpp | 2 +- src/Storages/StorageDistributed.cpp | 4 ++-- src/Storages/StorageFile.cpp | 2 +- src/Storages/StorageMerge.cpp | 2 +- src/Storages/StorageS3.cpp | 2 +- src/Storages/StorageURL.cpp | 2 +- src/Storages/System/StorageSystemStackTrace.cpp | 2 +- src/Storages/System/StorageSystemTables.cpp | 2 +- src/Storages/VirtualColumnUtils.cpp | 2 +- 24 files changed, 29 insertions(+), 32 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 7e86d87db14..1124ba94bc1 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -2440,7 +2440,6 @@ bool ActionsDAG::isSortingPreserved( ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( const NodeRawConstPtrs & filter_nodes, const std::unordered_map & node_name_to_input_node_column, - const ContextPtr & context, bool single_output_condition_node) { if (filter_nodes.empty()) @@ -2550,7 +2549,6 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( else index_hint_filter_dag = buildFilterActionsDAG(index_hint_args, node_name_to_input_node_column, - context, false /*single_output_condition_node*/); auto index_hint_function_clone = std::make_shared(); @@ -2589,8 +2587,8 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( if (result_dag_outputs.size() > 1 && single_output_condition_node) { - auto function_builder = FunctionFactory::instance().get("and", context); - result_dag_outputs = { &result_dag->addFunction(function_builder, result_dag_outputs, {}) }; + FunctionOverloadResolverPtr func_builder_and = std::make_unique(std::make_shared()); + result_dag_outputs = { &result_dag->addFunction(func_builder_and, result_dag_outputs, {}) }; } return result_dag; diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index f18ae5d5c75..45f6e5cc717 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -389,8 +389,7 @@ public: */ static ActionsDAGPtr buildFilterActionsDAG( const NodeRawConstPtrs & filter_nodes, - const std::unordered_map & node_name_to_input_node_column, - const ContextPtr & context, + const std::unordered_map & node_name_to_input_node_column = {}, bool single_output_condition_node = true); /// Check if `predicate` is a combination of AND functions. diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 2d994483ba8..946a62c39c1 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2329,7 +2329,7 @@ std::optional InterpreterSelectQuery::getTrivialCount(UInt64 max_paralle filter_nodes.push_back(&analysis_result.before_where->findInOutputs(analysis_result.where_column_name)); } - auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes, {}, context); + auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes); if (!filter_actions_dag) return {}; diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index cc447dfef24..a3d1b84fdc1 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -1274,7 +1274,7 @@ void MutationsInterpreter::Source::read( for (size_t i = 0; i < num_filters; ++i) nodes[i] = &steps[i]->actions()->findInOutputs(names[i]); - filter = ActionsDAG::buildFilterActionsDAG(nodes, {}, context_); + filter = ActionsDAG::buildFilterActionsDAG(nodes); } VirtualColumns virtual_columns(std::move(required_columns), part); diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index b55f0e44038..efccadcbe1a 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -215,7 +215,7 @@ void collectFiltersForAnalysis(const QueryTreeNodePtr & query_tree, const Planne if (!read_from_dummy) continue; - auto filter_actions = ActionsDAG::buildFilterActionsDAG(read_from_dummy->getFilterNodes().nodes, {}, query_context); + auto filter_actions = ActionsDAG::buildFilterActionsDAG(read_from_dummy->getFilterNodes().nodes); auto & table_expression_data = dummy_storage_to_table_expression_data.at(&read_from_dummy->getStorage()); table_expression_data->setFilterActions(std::move(filter_actions)); } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 0465ff54f5a..5ed56f59fc1 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1323,7 +1323,7 @@ static ActionsDAGPtr buildFilterDAG( } } - return ActionsDAG::buildFilterActionsDAG(nodes, node_name_to_input_node_column, context); + return ActionsDAG::buildFilterActionsDAG(nodes, node_name_to_input_node_column); } static void buildIndexes( diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index 5173b18c6bf..d8b3f4fbb8e 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -507,7 +507,7 @@ Pipe ReadFromSystemNumbersStep::makePipe() ActionsDAGPtr ReadFromSystemNumbersStep::buildFilterDAG() { std::unordered_map node_name_to_input_node_column; - return ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, node_name_to_input_node_column, context); + return ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, node_name_to_input_node_column); } void ReadFromSystemNumbersStep::checkLimits(size_t rows) diff --git a/src/Processors/SourceWithKeyCondition.h b/src/Processors/SourceWithKeyCondition.h index 82d46eb74a4..3538adf1d75 100644 --- a/src/Processors/SourceWithKeyCondition.h +++ b/src/Processors/SourceWithKeyCondition.h @@ -22,7 +22,7 @@ protected: for (const auto & column : keys.getColumnsWithTypeAndName()) node_name_to_input_column.insert({column.name, column}); - auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(nodes, node_name_to_input_column, context); + auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(nodes, node_name_to_input_column); key_condition = std::make_shared( filter_actions_dag, context, diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 1e26f1be72c..ab21c4946e4 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -877,7 +877,7 @@ private: void ReadFromHDFS::applyFilters() { - auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context); + auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes); const ActionsDAG::Node * predicate = nullptr; if (filter_actions_dag) predicate = filter_actions_dag->getOutputs().at(0); diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index a7ee8ff9891..6766ecd6b4f 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -818,7 +818,7 @@ private: void ReadFromHive::applyFilters() { - auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context); + auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes); createFiles(filter_actions_dag); } diff --git a/src/Storages/IStorageCluster.cpp b/src/Storages/IStorageCluster.cpp index c9eb07bd9d1..812b213cf33 100644 --- a/src/Storages/IStorageCluster.cpp +++ b/src/Storages/IStorageCluster.cpp @@ -82,7 +82,7 @@ private: void ReadFromCluster::applyFilters() { - auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context); + auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes); const ActionsDAG::Node * predicate = nullptr; if (filter_actions_dag) predicate = filter_actions_dag->getOutputs().at(0); diff --git a/src/Storages/KVStorageUtils.cpp b/src/Storages/KVStorageUtils.cpp index 3031fc6bf9d..5175c93041b 100644 --- a/src/Storages/KVStorageUtils.cpp +++ b/src/Storages/KVStorageUtils.cpp @@ -236,7 +236,7 @@ std::pair getFilterKeys( if (filter_nodes.nodes.empty()) return {{}, true}; - auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context); + auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes); const auto * predicate = filter_actions_dag->getOutputs().at(0); FieldVectorPtr res = std::make_shared(); diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 3f3dadb3cc5..4aecf85ac2a 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -132,8 +132,8 @@ std::optional MergeTreeWhe if (!optimize_result) return {}; - auto filter_actions = reconstructDAG(optimize_result->where_conditions, context); - auto prewhere_filter_actions = reconstructDAG(optimize_result->prewhere_conditions, context); + auto filter_actions = reconstructDAG(optimize_result->where_conditions); + auto prewhere_filter_actions = reconstructDAG(optimize_result->prewhere_conditions); FilterActionsOptimizeResult result = { std::move(filter_actions), std::move(prewhere_filter_actions) }; return result; @@ -343,7 +343,7 @@ ASTPtr MergeTreeWhereOptimizer::reconstructAST(const Conditions & conditions) return function; } -ActionsDAGPtr MergeTreeWhereOptimizer::reconstructDAG(const Conditions & conditions, const ContextPtr & context) +ActionsDAGPtr MergeTreeWhereOptimizer::reconstructDAG(const Conditions & conditions) { if (conditions.empty()) return {}; @@ -354,7 +354,7 @@ ActionsDAGPtr MergeTreeWhereOptimizer::reconstructDAG(const Conditions & conditi for (const auto & condition : conditions) filter_nodes.push_back(condition.node.getDAGNode()); - return ActionsDAG::buildFilterActionsDAG(filter_nodes, {} /*node_name_to_input_node_column*/, context); + return ActionsDAG::buildFilterActionsDAG(filter_nodes); } std::optional MergeTreeWhereOptimizer::optimizeImpl(const RPNBuilderTreeNode & node, diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h index 7a6651210d0..b56219e3c59 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h @@ -123,7 +123,7 @@ private: static ASTPtr reconstructAST(const Conditions & conditions); /// Reconstruct DAG from conditions - static ActionsDAGPtr reconstructDAG(const Conditions & conditions, const ContextPtr & context); + static ActionsDAGPtr reconstructDAG(const Conditions & conditions); void optimizeArbitrary(ASTSelectQuery & select) const; diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 48c284c022a..a8741aed3c5 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -254,7 +254,7 @@ void ReadFromS3Queue::createIterator(const ActionsDAG::Node * predicate) void ReadFromS3Queue::applyFilters() { - auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context); + auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes); const ActionsDAG::Node * predicate = nullptr; if (filter_actions_dag) predicate = filter_actions_dag->getOutputs().at(0); diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index cd841a1a673..01c31eab2b1 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -707,7 +707,7 @@ private: void ReadFromAzureBlob::applyFilters() { - auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context); + auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes); const ActionsDAG::Node * predicate = nullptr; if (filter_actions_dag) predicate = filter_actions_dag->getOutputs().at(0); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 2d05efdd74f..afd9e4aad76 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1102,7 +1102,7 @@ static ActionsDAGPtr getFilterFromQuery(const ASTPtr & ast, ContextPtr context) if (!source) return nullptr; - return ActionsDAG::buildFilterActionsDAG(source->getFilterNodes().nodes, {}, context); + return ActionsDAG::buildFilterActionsDAG(source->getFilterNodes().nodes); } @@ -1597,7 +1597,7 @@ ClusterPtr StorageDistributed::skipUnusedShardsWithAnalyzer( if (nodes.empty()) return nullptr; - auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(nodes, {}, local_context); + auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(nodes); size_t limit = local_context->getSettingsRef().optimize_skip_unused_shards_limit; if (!limit || limit > SSIZE_MAX) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 8979e068fb5..8b8a151fb1d 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1352,7 +1352,7 @@ private: void ReadFromFile::applyFilters() { - auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context); + auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes); const ActionsDAG::Node * predicate = nullptr; if (filter_actions_dag) predicate = filter_actions_dag->getOutputs().at(0); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 0d67403fa2f..8d75382c91c 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -982,7 +982,7 @@ StorageMerge::StorageListWithLocks ReadFromMerge::getSelectedTables( if (!filter_by_database_virtual_column && !filter_by_table_virtual_column) return res; - auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context); + auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes); if (!filter_actions_dag) return res; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 1a7704b4d67..aec967cc95c 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1160,7 +1160,7 @@ void StorageS3::read( void ReadFromStorageS3Step::applyFilters() { - auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, local_context); + auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes); const ActionsDAG::Node * predicate = nullptr; if (filter_actions_dag) predicate = filter_actions_dag->getOutputs().at(0); diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 631c06bd87b..433f4ed7700 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -930,7 +930,7 @@ private: void ReadFromURL::applyFilters() { - auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context); + auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes); const ActionsDAG::Node * predicate = nullptr; if (filter_actions_dag) predicate = filter_actions_dag->getOutputs().at(0); diff --git a/src/Storages/System/StorageSystemStackTrace.cpp b/src/Storages/System/StorageSystemStackTrace.cpp index b17d04e9895..82a5fd4e33f 100644 --- a/src/Storages/System/StorageSystemStackTrace.cpp +++ b/src/Storages/System/StorageSystemStackTrace.cpp @@ -463,7 +463,7 @@ public: void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override { - auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context); + auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes); Pipe pipe(std::make_shared( column_names, getOutputStream().header, diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 92ae643db55..a0f6b03cf89 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -725,7 +725,7 @@ void StorageSystemTables::read( void ReadFromSystemTables::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context); + auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes); const ActionsDAG::Node * predicate = nullptr; if (filter_actions_dag) predicate = filter_actions_dag->getOutputs().at(0); diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index e845e03d122..430ed012fa8 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -211,7 +211,7 @@ bool prepareFilterBlockWithQuery(const ASTPtr & query, ContextPtr context, Block const auto * expr_const_node = actions->tryFindInOutputs(expr_column_name); if (!expr_const_node) return false; - auto filter_actions = ActionsDAG::buildFilterActionsDAG({expr_const_node}, {}, context); + auto filter_actions = ActionsDAG::buildFilterActionsDAG({expr_const_node}); const auto & nodes = filter_actions->getNodes(); bool has_dependent_columns = std::any_of(nodes.begin(), nodes.end(), [&](const auto & node) { From 16a9e95605791b1ea73b4973e115a74ddd019e02 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 Jan 2024 15:40:11 +0000 Subject: [PATCH 193/264] Minor cleanup of msan usage --- src/Common/SymbolIndex.cpp | 16 +--------------- src/Compression/CompressionCodecDeflateQpl.cpp | 13 +++++-------- src/Functions/FunctionsStringSimilarity.cpp | 3 +-- 3 files changed, 7 insertions(+), 25 deletions(-) diff --git a/src/Common/SymbolIndex.cpp b/src/Common/SymbolIndex.cpp index 8e8ffa4d8b7..8dde617fc74 100644 --- a/src/Common/SymbolIndex.cpp +++ b/src/Common/SymbolIndex.cpp @@ -1,6 +1,7 @@ #if defined(__ELF__) && !defined(OS_FREEBSD) #include +#include #include #include @@ -55,21 +56,6 @@ Otherwise you will get only exported symbols from program headers. */ -#if defined(__clang__) -# pragma clang diagnostic ignored "-Wreserved-id-macro" -# pragma clang diagnostic ignored "-Wunused-macros" -#endif - -#define __msan_unpoison_string(X) // NOLINT -#define __msan_unpoison(X, Y) // NOLINT -#if defined(ch_has_feature) -# if ch_has_feature(memory_sanitizer) -# undef __msan_unpoison_string -# undef __msan_unpoison -# include -# endif -#endif - namespace DB { diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index ee0356adde5..292f729a38d 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -6,14 +6,15 @@ #include #include #include -#include -#include -#include "libaccel_config.h" #include +#include +#include #include #include -#include +#include "libaccel_config.h" + +#include namespace DB { @@ -416,9 +417,7 @@ UInt32 CompressionCodecDeflateQpl::doCompressData(const char * source, UInt32 so { /// QPL library is using AVX-512 with some shuffle operations. /// Memory sanitizer don't understand if there was uninitialized memory in SIMD register but it was not used in the result of shuffle. -#if defined(MEMORY_SANITIZER) __msan_unpoison(dest, getMaxCompressedDataSize(source_size)); -#endif Int32 res = HardwareCodecDeflateQpl::RET_ERROR; if (DeflateQplJobHWPool::instance().isJobPoolReady()) res = hw_codec->doCompressData(source, source_size, dest, getMaxCompressedDataSize(source_size)); @@ -439,9 +438,7 @@ void CompressionCodecDeflateQpl::doDecompressData(const char * source, UInt32 so { /// QPL library is using AVX-512 with some shuffle operations. /// Memory sanitizer don't understand if there was uninitialized memory in SIMD register but it was not used in the result of shuffle. -#if defined(MEMORY_SANITIZER) __msan_unpoison(dest, uncompressed_size); -#endif /// Device IOTLB miss has big perf. impact for IAA accelerators. /// To avoid page fault, we need touch buffers related to accelerator in advance. touchBufferWithZeroFilling(dest, uncompressed_size); diff --git a/src/Functions/FunctionsStringSimilarity.cpp b/src/Functions/FunctionsStringSimilarity.cpp index df068531655..aadf5c246fc 100644 --- a/src/Functions/FunctionsStringSimilarity.cpp +++ b/src/Functions/FunctionsStringSimilarity.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -108,10 +109,8 @@ struct NgramDistanceImpl if constexpr (case_insensitive) { -#if defined(MEMORY_SANITIZER) /// Due to PODArray padding accessing more elements should be OK __msan_unpoison(code_points + (N - 1), padding_offset * sizeof(CodePoint)); -#endif /// We really need template lambdas with C++20 to do it inline unrollLowering(code_points, std::make_index_sequence()); } From e431f89339aeefef1fd4cdd5a0781d7a3d254e82 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 25 Jan 2024 18:19:46 +0100 Subject: [PATCH 194/264] Restart CI --- src/Columns/tests/gtest_column_variant.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Columns/tests/gtest_column_variant.cpp b/src/Columns/tests/gtest_column_variant.cpp index 0a6512c46b7..25f276b9600 100644 --- a/src/Columns/tests/gtest_column_variant.cpp +++ b/src/Columns/tests/gtest_column_variant.cpp @@ -677,6 +677,7 @@ TEST(ColumnVariant, ScatterGeneral) selector.push_back(1); selector.push_back(2); selector.push_back(1); + auto columns = column->scatter(3, selector); ASSERT_EQ(columns[0]->size(), 3); ASSERT_EQ((*columns[0])[0].get(), 42); From ff15ab12b8b48db5871dde0106176dc4c7193df2 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 25 Jan 2024 21:06:50 +0000 Subject: [PATCH 195/264] Remove bad implementation of DataTypeVariant::createColumnConst, support use_variant_as_common_type setting in functions array/map --- docs/en/operations/settings/settings.md | 39 ++++++++++- docs/en/sql-reference/data-types/variant.md | 30 ++++++++- src/Columns/ColumnMap.cpp | 5 ++ src/Columns/ColumnMap.h | 1 + src/DataTypes/DataTypeVariant.cpp | 41 ------------ src/DataTypes/DataTypeVariant.h | 1 - src/DataTypes/getLeastSupertype.cpp | 66 +++++++------------ src/DataTypes/getLeastSupertype.h | 5 +- src/Functions/array/array.cpp | 13 +++- src/Functions/map.cpp | 22 +++++-- ...=> 02944_variant_as_common_type.reference} | 7 ++ ...t.sql => 02944_variant_as_common_type.sql} | 13 +++- 12 files changed, 147 insertions(+), 96 deletions(-) rename tests/queries/0_stateless/{02944_variant_as_if_multi_if_result.reference => 02944_variant_as_common_type.reference} (83%) rename tests/queries/0_stateless/{02944_variant_as_if_multi_if_result.sql => 02944_variant_as_common_type.sql} (87%) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index fb2e04f1e35..e8fc095438b 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5184,7 +5184,7 @@ Default value: `false`. ## use_variant_as_common_type {#use_variant_as_common_type} -Allows to use `Variant` type as a result type for [if](../../sql-reference/functions/conditional-functions.md/#if)/[multiIf](../../sql-reference/functions/conditional-functions.md/#multiif) functions when there is no common type for argument types. +Allows to use `Variant` type as a result type for [if](../../sql-reference/functions/conditional-functions.md/#if)/[multiIf](../../sql-reference/functions/conditional-functions.md/#multiif)/[array](../../sql-reference/functions/array-functions.md)/[map](../../sql-reference/functions/tuple-map-functions.md) functions when there is no common type for argument types. Example: @@ -5226,6 +5226,43 @@ SELECT multiIf((number % 4) = 0, 42, (number % 4) = 1, [1, 2, 3], (number % 4) = └───────────────┘ ``` +```sql +SET use_variant_as_common_type = 1; +SELECT toTypeName(array(range(number), number, 'str_' || toString(number))) as array_of_variants_type from numbers(1); +SELECT array(range(number), number, 'str_' || toString(number)) as array_of_variants FROM numbers(3); +``` + +```text +┌─array_of_variants_type────────────────────────┐ +│ Array(Variant(Array(UInt64), String, UInt64)) │ +└───────────────────────────────────────────────┘ + +┌─array_of_variants─┐ +│ [[],0,'str_0'] │ +│ [[0],1,'str_1'] │ +│ [[0,1],2,'str_2'] │ +└───────────────────┘ +``` + +```sql +SET use_variant_as_common_type = 1; +SELECT toTypeName(map('a', range(number), 'b', number, 'c', 'str_' || toString(number))) as map_of_variants_type from numbers(1); +SELECT map('a', range(number), 'b', number, 'c', 'str_' || toString(number)) as map_of_variants FROM numbers(3); +``` + +```text +┌─map_of_variants_type────────────────────────────────┐ +│ Map(String, Variant(Array(UInt64), String, UInt64)) │ +└─────────────────────────────────────────────────────┘ + +┌─map_of_variants───────────────┐ +│ {'a':[],'b':0,'c':'str_0'} │ +│ {'a':[0],'b':1,'c':'str_1'} │ +│ {'a':[0,1],'b':2,'c':'str_2'} │ +└───────────────────────────────┘ +``` + + Default value: `false`. ## max_partition_size_to_drop diff --git a/docs/en/sql-reference/data-types/variant.md b/docs/en/sql-reference/data-types/variant.md index afbd1b798b2..0058e13b4ca 100644 --- a/docs/en/sql-reference/data-types/variant.md +++ b/docs/en/sql-reference/data-types/variant.md @@ -47,7 +47,7 @@ SELECT toTypeName(variant) as type_name, 'Hello, World!'::Variant(UInt64, String └────────────────────────────────────────┴───────────────┘ ``` -Using functions `if/multiIf` when arguments doesn't have common type (setting `use_variant_as_common_type` should be enabled for it): +Using functions `if/multiIf` when arguments don't have common type (setting `use_variant_as_common_type` should be enabled for it): ```sql SET use_variant_as_common_type = 1; @@ -78,6 +78,34 @@ SELECT multiIf((number % 4) = 0, 42, (number % 4) = 1, [1, 2, 3], (number % 4) = └───────────────┘ ``` +Using functions 'array/map' if array elements/map values don't have common type (setting `use_variant_as_common_type` should be enabled for it): + +```sql +SET use_variant_as_common_type = 1; +SELECT array(range(number), number, 'str_' || toString(number)) as array_of_variants FROM numbers(3); +``` + +```text +┌─array_of_variants─┐ +│ [[],0,'str_0'] │ +│ [[0],1,'str_1'] │ +│ [[0,1],2,'str_2'] │ +└───────────────────┘ +``` + +```sql +SET use_variant_as_common_type = 1; +SELECT map('a', range(number), 'b', number, 'c', 'str_' || toString(number)) as map_of_variants FROM numbers(3); +``` + +```text +┌─map_of_variants───────────────┐ +│ {'a':[],'b':0,'c':'str_0'} │ +│ {'a':[0],'b':1,'c':'str_1'} │ +│ {'a':[0,1],'b':2,'c':'str_2'} │ +└───────────────────────────────┘ +``` + ## Reading Variant nested types as subcolumns Variant type supports reading a single nested type from a Variant column using the type name as a subcolumn. diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index ddcde43ca23..307a1ef0f62 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -141,6 +141,11 @@ void ColumnMap::updateHashFast(SipHash & hash) const nested->updateHashFast(hash); } +void ColumnMap::insertFrom(const IColumn & src, size_t start) +{ + nested->insertFrom(assert_cast(src).getNestedColumn(), start); +} + void ColumnMap::insertRangeFrom(const IColumn & src, size_t start, size_t length) { nested->insertRangeFrom( diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index fde8a7e0e67..0bae86231f9 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -64,6 +64,7 @@ public: void updateHashWithValue(size_t n, SipHash & hash) const override; void updateWeakHash32(WeakHash32 & hash) const override; void updateHashFast(SipHash & hash) const override; + void insertFrom(const IColumn & src_, size_t n) override; void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; void expand(const Filter & mask, bool inverted) override; diff --git a/src/DataTypes/DataTypeVariant.cpp b/src/DataTypes/DataTypeVariant.cpp index 3a39fdf9ea8..456b4ea03b6 100644 --- a/src/DataTypes/DataTypeVariant.cpp +++ b/src/DataTypes/DataTypeVariant.cpp @@ -96,47 +96,6 @@ MutableColumnPtr DataTypeVariant::createColumn() const return ColumnVariant::create(std::move(nested_columns)); } -ColumnPtr DataTypeVariant::createColumnConst(size_t size, const DB::Field & field) const -{ - auto column = createColumn(); - if (field.isNull()) - { - column->insertDefault(); - } - else - { - /// We don't have exact mapping Field type -> Data type, so we cannot - /// always know in which variant we need to insert the field by it's type. - /// Examples: - /// Field(42) and Variant(UInt16, String). Type of the Field - UInt64, but we can insert it in UInt16 - /// Field(42) and Variant(Date, String). Type of the Field - UInt64, but we can insert it in Date - - /// Let's first apply FieldToDataType visitor to find best Data type for this field. - /// If we have variant with such type, we will insert this field into it. - /// Otherwise we will try to find the first variant that has default Field value with the same type. - auto field_type = applyVisitor(FieldToDataType(), field); - auto discr = tryGetVariantDiscriminator(field_type); - if (!discr) - { - for (size_t i = 0; i != variants.size(); ++i) - { - if (field.getType() == variants[i]->getDefault().getType()) - { - discr = i; - break; - } - } - } - - if (!discr) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot insert field \"{}\" with type {} into column with type {}", toString(field), field.getTypeName(), getName()); - - assert_cast(*column).insertIntoVariant(field, *discr); - } - - return ColumnConst::create(std::move(column), size); -} - Field DataTypeVariant::getDefault() const { return Null(); diff --git a/src/DataTypes/DataTypeVariant.h b/src/DataTypes/DataTypeVariant.h index 1a1cb6c12f2..d26ce4ea90f 100644 --- a/src/DataTypes/DataTypeVariant.h +++ b/src/DataTypes/DataTypeVariant.h @@ -37,7 +37,6 @@ public: MutableColumnPtr createColumn() const override; - ColumnPtr createColumnConst(size_t size, const Field & field) const override; Field getDefault() const override; bool equals(const IDataType & rhs) const override; diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index 5d67f888c4b..3c64cb755a7 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -59,25 +59,6 @@ DataTypePtr throwOrReturn(const DataTypes & types, std::string_view message_suff if constexpr (on_error == LeastSupertypeOnError::String) return std::make_shared(); - if constexpr (on_error == LeastSupertypeOnError::Variant && std::is_same_v>) - { - DataTypes variants; - for (const auto & type : types) - { - if (isVariant(type)) - { - const DataTypes & nested_variants = assert_cast(*type).getVariants(); - variants.insert(variants.end(), nested_variants.begin(), nested_variants.end()); - } - else - { - variants.push_back(removeNullableOrLowCardinalityNullable(type)); - } - } - - return std::make_shared(variants); - } - if constexpr (on_error == LeastSupertypeOnError::Null) return nullptr; @@ -402,17 +383,7 @@ DataTypePtr getLeastSupertype(const DataTypes & types) if (!all_maps) return throwOrReturn(types, "because some of them are Maps and some of them are not", ErrorCodes::NO_COMMON_TYPE); - DataTypePtr keys_common_type; - if constexpr (on_error == LeastSupertypeOnError::Variant) - { - keys_common_type = getLeastSupertype(key_types); - if (!keys_common_type) - return throwOrReturn(types, "", ErrorCodes::NO_COMMON_TYPE); - } - else - { - keys_common_type = getLeastSupertype(key_types); - } + DataTypePtr keys_common_type = getLeastSupertype(key_types); auto values_common_type = getLeastSupertype(value_types); /// When on_error == LeastSupertypeOnError::Null and we cannot get least supertype for keys or values, @@ -454,17 +425,7 @@ DataTypePtr getLeastSupertype(const DataTypes & types) return getLeastSupertype(nested_types); else { - DataTypePtr nested_type; - if constexpr (on_error == LeastSupertypeOnError::Variant) - { - nested_type = getLeastSupertype(nested_types); - if (!nested_type) - return throwOrReturn(types, "", ErrorCodes::NO_COMMON_TYPE); - } - else - { - nested_type = getLeastSupertype(nested_types); - } + DataTypePtr nested_type = getLeastSupertype(nested_types); /// When on_error == LeastSupertypeOnError::Null and we cannot get least supertype, /// nested_type will be nullptr, we should return nullptr in this case. @@ -684,7 +645,28 @@ DataTypePtr getLeastSupertypeOrString(const DataTypes & types) DataTypePtr getLeastSupertypeOrVariant(const DataTypes & types) { - return getLeastSupertype(types); + auto common_type = getLeastSupertype(types); + if (common_type) + return common_type; + + /// Create Variant with provided arguments as variants. + DataTypes variants; + for (const auto & type : types) + { + /// Nested Variant types are not supported. If we have Variant type + /// we use all its variants in the result Variant. + if (isVariant(type)) + { + const DataTypes & nested_variants = assert_cast(*type).getVariants(); + variants.insert(variants.end(), nested_variants.begin(), nested_variants.end()); + } + else + { + variants.push_back(removeNullableOrLowCardinalityNullable(type)); + } + } + + return std::make_shared(variants); } DataTypePtr tryGetLeastSupertype(const DataTypes & types) diff --git a/src/DataTypes/getLeastSupertype.h b/src/DataTypes/getLeastSupertype.h index d949fad69c5..ceaffbdab7a 100644 --- a/src/DataTypes/getLeastSupertype.h +++ b/src/DataTypes/getLeastSupertype.h @@ -8,7 +8,6 @@ enum class LeastSupertypeOnError { Throw, String, - Variant, Null, }; @@ -25,14 +24,14 @@ DataTypePtr getLeastSupertype(const DataTypes & types); /// All types can be casted to String, because they can be serialized to String. DataTypePtr getLeastSupertypeOrString(const DataTypes & types); -/// Same as getLeastSupertype but in case when there is no supertype for some types +/// Same as getLeastSupertype but in case when there is no supertype for provided types /// it uses Variant of these types as a supertype. Any type can be casted to a Variant /// that contains this type. /// As nested Variants are not allowed, if one of the types is Variant, it's variants /// are used in the resulting Variant. /// Examples: /// (UInt64, String) -> Variant(UInt64, String) -/// (Array(UInt64), Array(String)) -> Array(Variant(UInt64, String)) +/// (Array(UInt64), Array(String)) -> Variant(Array(UInt64), Array(String)) /// (Variant(UInt64, String), Array(UInt32)) -> Variant(UInt64, String, Array(UInt32)) DataTypePtr getLeastSupertypeOrVariant(const DataTypes & types); diff --git a/src/Functions/array/array.cpp b/src/Functions/array/array.cpp index 551f0a6625b..4379283ec86 100644 --- a/src/Functions/array/array.cpp +++ b/src/Functions/array/array.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -14,9 +15,12 @@ class FunctionArray : public IFunction { public: static constexpr auto name = "array"; - static FunctionPtr create(ContextPtr) + + FunctionArray(bool use_variant_as_common_type_ = false) : use_variant_as_common_type(use_variant_as_common_type_) {} + + static FunctionPtr create(ContextPtr context) { - return std::make_shared(); + return std::make_shared(context->getSettingsRef().allow_experimental_variant_type && context->getSettingsRef().use_variant_as_common_type); } bool useDefaultImplementationForNulls() const override { return false; } @@ -31,6 +35,9 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { + if (use_variant_as_common_type) + return std::make_shared(getLeastSupertypeOrVariant(arguments)); + return std::make_shared(getLeastSupertype(arguments)); } @@ -97,6 +104,8 @@ private: } bool addField(DataTypePtr type_res, const Field & f, Array & arr) const; + + bool use_variant_as_common_type = false; }; diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index c950a0491a5..79a18dce99b 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include @@ -30,9 +31,11 @@ class FunctionMap : public IFunction public: static constexpr auto name = "map"; - static FunctionPtr create(ContextPtr) + FunctionMap(bool use_variant_as_common_type_) : use_variant_as_common_type(use_variant_as_common_type_) {} + + static FunctionPtr create(ContextPtr context) { - return std::make_shared(); + return std::make_shared(context->getSettingsRef().allow_experimental_variant_type && context->getSettingsRef().use_variant_as_common_type); } String getName() const override @@ -77,8 +80,16 @@ public: } DataTypes tmp; - tmp.emplace_back(getLeastSupertype(keys)); - tmp.emplace_back(getLeastSupertype(values)); + if (use_variant_as_common_type) + { + tmp.emplace_back(getLeastSupertypeOrVariant(keys)); + tmp.emplace_back(getLeastSupertypeOrVariant(values)); + } + else + { + tmp.emplace_back(getLeastSupertype(keys)); + tmp.emplace_back(getLeastSupertype(values)); + } return std::make_shared(tmp); } @@ -138,6 +149,9 @@ public: return ColumnMap::create(nested_column); } + +private: + bool use_variant_as_common_type = false; }; /// mapFromArrays(keys, values) is a function that allows you to make key-value pair from a pair of arrays diff --git a/tests/queries/0_stateless/02944_variant_as_if_multi_if_result.reference b/tests/queries/0_stateless/02944_variant_as_common_type.reference similarity index 83% rename from tests/queries/0_stateless/02944_variant_as_if_multi_if_result.reference rename to tests/queries/0_stateless/02944_variant_as_common_type.reference index 3803f39253c..0425a8cfa30 100644 --- a/tests/queries/0_stateless/02944_variant_as_if_multi_if_result.reference +++ b/tests/queries/0_stateless/02944_variant_as_common_type.reference @@ -94,3 +94,10 @@ Variant(Array(UInt64), String, UInt64) str_2 Variant(Array(UInt64), String, UInt64) [0,1,2,3] Variant(Array(UInt64), String, UInt64) 4 Variant(Array(UInt64), String, UInt64) str_5 +Array(Variant(String, UInt8)) [1,'str_1',2,'str_2'] +Array(Variant(Array(String), Array(UInt8))) [[1,2,3],['str_1','str_2','str_3']] +Array(Variant(Array(UInt8), Array(Variant(Array(String), Array(UInt8))))) [[[1,2,3],['str_1','str_2','str_3']],[1,2,3]] +Array(Variant(Array(Array(UInt8)), Array(UInt8))) [[1,2,3],[[1,2,3]]] +Map(String, Variant(String, UInt8)) {'a':1,'b':'str_1'} +Map(String, Variant(Map(String, Variant(String, UInt8)), UInt8)) {'a':1,'b':{'c':2,'d':'str_1'}} +Map(String, Variant(Array(Array(UInt8)), Array(UInt8), UInt8)) {'a':1,'b':[1,2,3],'c':[[4,5,6]]} diff --git a/tests/queries/0_stateless/02944_variant_as_if_multi_if_result.sql b/tests/queries/0_stateless/02944_variant_as_common_type.sql similarity index 87% rename from tests/queries/0_stateless/02944_variant_as_if_multi_if_result.sql rename to tests/queries/0_stateless/02944_variant_as_common_type.sql index d88c0e8b7ae..e985cf365dd 100644 --- a/tests/queries/0_stateless/02944_variant_as_if_multi_if_result.sql +++ b/tests/queries/0_stateless/02944_variant_as_common_type.sql @@ -1,4 +1,5 @@ -set allow_experimental_analyzer=0; -- The result type for if function with constant is different with analyzer. +set allow_experimental_analyzer=0; -- The result type for if function with constant is different with analyzer. It wil be fixed after refactoring around constants in analyzer. + set allow_experimental_variant_type=1; set use_variant_as_common_type=1; @@ -63,3 +64,13 @@ select toTypeName(res), multiIf(number % 3 == 0, range(number + 1), number % 3 = select toTypeName(res), multiIf(number % 3 == 0, range(number + 1), number % 3 == 1, number, ('str_' || toString(number))::LowCardinality(String)) as res from numbers(6); select toTypeName(res), multiIf(number % 3 == 0, range(number + 1), number % 3 == 1, number, ('str_' || toString(number))::LowCardinality(Nullable(String))) as res from numbers(6); + +select toTypeName(res), array(1, 'str_1', 2, 'str_2') as res; +select toTypeName(res), array([1, 2, 3], ['str_1', 'str_2', 'str_3']) as res; +select toTypeName(res), array(array([1, 2, 3], ['str_1', 'str_2', 'str_3']), [1, 2, 3]) as res; +select toTypeName(res), array([1, 2, 3], [[1, 2, 3]]) as res; + +select toTypeName(res), map('a', 1, 'b', 'str_1') as res; +select toTypeName(res), map('a', 1, 'b', map('c', 2, 'd', 'str_1')) as res; +select toTypeName(res), map('a', 1, 'b', [1, 2, 3], 'c', [[4, 5, 6]]) as res; + From caeb31940ce9fce6d98ab9629881a099a349703c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 Jan 2024 20:20:58 +0000 Subject: [PATCH 196/264] Allow to build without libssl --- contrib/libssh-cmake/CMakeLists.txt | 2 +- src/Access/Authentication.cpp | 4 ++-- src/Access/AuthenticationData.cpp | 4 ++-- src/Access/UsersConfigAccessStorage.cpp | 2 +- src/Client/Connection.cpp | 4 ++-- src/Client/ConnectionParameters.cpp | 2 +- src/Common/SSH/Wrappers.cpp | 2 +- src/Common/SSH/Wrappers.h | 2 +- src/Common/config.h.in | 1 + src/configure_config.cmake | 3 +++ 10 files changed, 15 insertions(+), 11 deletions(-) diff --git a/contrib/libssh-cmake/CMakeLists.txt b/contrib/libssh-cmake/CMakeLists.txt index 7a3816d4dce..eee3df832fa 100644 --- a/contrib/libssh-cmake/CMakeLists.txt +++ b/contrib/libssh-cmake/CMakeLists.txt @@ -1,4 +1,4 @@ -option (ENABLE_SSH "Enable support for SSH keys and protocol" ON) +option (ENABLE_SSH "Enable support for SSH keys and protocol" ${ENABLE_LIBRARIES}) if (NOT ENABLE_SSH) message(STATUS "Not using SSH") diff --git a/src/Access/Authentication.cpp b/src/Access/Authentication.cpp index 372334bb152..47187d83154 100644 --- a/src/Access/Authentication.cpp +++ b/src/Access/Authentication.cpp @@ -73,7 +73,7 @@ namespace return checkPasswordDoubleSHA1MySQL(scramble, scrambled_password, Util::encodeDoubleSHA1(password_plaintext)); } -#if USE_SSL +#if USE_SSH bool checkSshSignature(const std::vector & keys, std::string_view signature, std::string_view original) { for (const auto & key: keys) @@ -243,7 +243,7 @@ bool Authentication::areCredentialsValid( throw Authentication::Require("ClickHouse X.509 Authentication"); case AuthenticationType::SSH_KEY: -#if USE_SSL +#if USE_SSH return checkSshSignature(auth_data.getSSHKeys(), ssh_credentials->getSignature(), ssh_credentials->getOriginal()); #else throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without OpenSSL"); diff --git a/src/Access/AuthenticationData.cpp b/src/Access/AuthenticationData.cpp index 1036e1e027e..da90a0f5842 100644 --- a/src/Access/AuthenticationData.cpp +++ b/src/Access/AuthenticationData.cpp @@ -320,7 +320,7 @@ std::shared_ptr AuthenticationData::toAST() const } case AuthenticationType::SSH_KEY: { -#if USE_SSL +#if USE_SSH for (const auto & key : getSSHKeys()) node->children.push_back(std::make_shared(key.getBase64(), key.getKeyType())); @@ -353,7 +353,7 @@ AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & que /// For this type of authentication we have ASTPublicSSHKey as children for ASTAuthenticationData if (query.type && query.type == AuthenticationType::SSH_KEY) { -#if USE_SSL +#if USE_SSH AuthenticationData auth_data(*query.type); std::vector keys; diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index 2b0fb3f9b2e..e9b2e1397ab 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -209,7 +209,7 @@ namespace } else if (has_ssh_keys) { -#if USE_SSL +#if USE_SSH user->auth_data = AuthenticationData{AuthenticationType::SSH_KEY}; Poco::Util::AbstractConfiguration::Keys entries; diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 352d2a53892..4848f4c7ee5 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -316,7 +316,7 @@ void Connection::sendHello() "Inter-server secret support is disabled, because ClickHouse was built without SSL library"); #endif } -#if USE_SSL +#if USE_SSH /// Just inform server that we will authenticate using SSH keys. else if (!ssh_private_key.isEmpty()) { @@ -346,7 +346,7 @@ void Connection::sendAddendum() void Connection::performHandshakeForSSHAuth() { -#if USE_SSL +#if USE_SSH String challenge; { writeVarUInt(Protocol::Client::SSHChallengeRequest, *out); diff --git a/src/Client/ConnectionParameters.cpp b/src/Client/ConnectionParameters.cpp index cadabc98447..a0ae47f0620 100644 --- a/src/Client/ConnectionParameters.cpp +++ b/src/Client/ConnectionParameters.cpp @@ -79,7 +79,7 @@ ConnectionParameters::ConnectionParameters(const Poco::Util::AbstractConfigurati } else { -#if USE_SSL +#if USE_SSH std::string filename = config.getString("ssh-key-file"); std::string passphrase; if (config.has("ssh-key-passphrase")) diff --git a/src/Common/SSH/Wrappers.cpp b/src/Common/SSH/Wrappers.cpp index b38dd1f4036..463338dbe3f 100644 --- a/src/Common/SSH/Wrappers.cpp +++ b/src/Common/SSH/Wrappers.cpp @@ -1,5 +1,5 @@ #include -# if USE_SSL +# if USE_SSH # include # pragma GCC diagnostic push diff --git a/src/Common/SSH/Wrappers.h b/src/Common/SSH/Wrappers.h index 9036fac3163..699bba2b042 100644 --- a/src/Common/SSH/Wrappers.h +++ b/src/Common/SSH/Wrappers.h @@ -1,7 +1,7 @@ #pragma once #include #include "config.h" -#if USE_SSL +#if USE_SSH # include # include diff --git a/src/Common/config.h.in b/src/Common/config.h.in index 5b3388a3b7d..8e513dca145 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -5,6 +5,7 @@ #cmakedefine01 USE_CPUID #cmakedefine01 USE_BASE64 #cmakedefine01 USE_SSL +#cmakedefine01 USE_SSH #cmakedefine01 USE_HDFS #cmakedefine01 USE_AWS_S3 #cmakedefine01 USE_AZURE_BLOB_STORAGE diff --git a/src/configure_config.cmake b/src/configure_config.cmake index 7de2d5a9fdd..141e51badbb 100644 --- a/src/configure_config.cmake +++ b/src/configure_config.cmake @@ -164,6 +164,9 @@ endif () if (ENABLE_OPENSSL) set(USE_OPENSSL_INTREE 1) endif () +if (TARGET ch_contrib::ssh) + set(USE_SSH 1) +endif() if (TARGET ch_contrib::fiu) set(FIU_ENABLE 1) endif() From a7b1fe9fa21d6e7ab81f448b1109170b328c78ab Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 25 Jan 2024 23:53:54 +0100 Subject: [PATCH 197/264] Disable copy constructor for MultiVersion. --- src/Common/MultiVersion.h | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/Common/MultiVersion.h b/src/Common/MultiVersion.h index 4796d6c9ec2..6ea337e6a7c 100644 --- a/src/Common/MultiVersion.h +++ b/src/Common/MultiVersion.h @@ -36,6 +36,16 @@ public: { } + /// There is no copy constructor because only one MultiVersion should own the same object. + MultiVersion(MultiVersion && src) { *this = std::move(src); } + + MultiVersion & operator=(MultiVersion && src) + { + if (this != &src) + std::atomic_store(¤t_version, std::atomic_exchange(&src.current_version, Version{})); + return *this; + } + /// Obtain current version for read-only usage. Returns shared_ptr, that manages lifetime of version. Version get() const { From 7f6784937d91c628286a8d630d451460df897b64 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 25 Jan 2024 23:59:24 +0000 Subject: [PATCH 198/264] Fix --- src/DataTypes/getLeastSupertype.cpp | 26 +++++++++++--------------- 1 file changed, 11 insertions(+), 15 deletions(-) diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index 3c64cb755a7..09d44eeb160 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -68,8 +68,8 @@ DataTypePtr throwOrReturn(const DataTypes & types, std::string_view message_suff throw Exception(error_code, "There is no supertype for types {} {}", getExceptionMessagePrefix(types), message_suffix); } -template -DataTypePtr getNumericType(const TypeIndexSet & types, ThrowOrReturnFunc throwOrReturnFunc) +template +DataTypePtr getNumericType(const TypeIndexSet & types) { bool all_numbers = true; @@ -120,7 +120,7 @@ DataTypePtr getNumericType(const TypeIndexSet & types, ThrowOrReturnFunc throwOr if (max_bits_of_signed_integer || max_bits_of_unsigned_integer || max_mantissa_bits_of_floating) { if (!all_numbers) - return throwOrReturnFunc(types, "because some of them are numbers and some of them are not", ErrorCodes::NO_COMMON_TYPE); + return throwOrReturn(types, "because some of them are numbers and some of them are not", ErrorCodes::NO_COMMON_TYPE); /// If there are signed and unsigned types of same bit-width, the result must be signed number with at least one more bit. /// Example, common of Int32, UInt32 = Int64. @@ -135,7 +135,7 @@ DataTypePtr getNumericType(const TypeIndexSet & types, ThrowOrReturnFunc throwOr if (min_bit_width_of_integer != 64) ++min_bit_width_of_integer; else - return throwOrReturnFunc(types, + return throwOrReturn(types, "because some of them are signed integers and some are unsigned integers," " but there is no signed integer type, that can exactly represent all required unsigned integer values", ErrorCodes::NO_COMMON_TYPE); @@ -150,7 +150,7 @@ DataTypePtr getNumericType(const TypeIndexSet & types, ThrowOrReturnFunc throwOr else if (min_mantissa_bits <= 53) return std::make_shared(); else - return throwOrReturnFunc(types, + return throwOrReturn(types, " because some of them are integers and some are floating point," " but there is no floating point type, that can exactly represent all required integers", ErrorCodes::NO_COMMON_TYPE); } @@ -171,7 +171,7 @@ DataTypePtr getNumericType(const TypeIndexSet & types, ThrowOrReturnFunc throwOr else if (min_bit_width_of_integer <= 256) return std::make_shared(); else - return throwOrReturnFunc(types, + return throwOrReturn(types, " because some of them are signed integers and some are unsigned integers," " but there is no signed integer type, that can exactly represent all required unsigned integer values", ErrorCodes::NO_COMMON_TYPE); } @@ -191,7 +191,7 @@ DataTypePtr getNumericType(const TypeIndexSet & types, ThrowOrReturnFunc throwOr else if (min_bit_width_of_integer <= 256) return std::make_shared(); else - return throwOrReturnFunc(types, + return throwOrReturn(types, " but as all data types are unsigned integers, we must have found maximum unsigned integer type", ErrorCodes::NO_COMMON_TYPE); } } @@ -383,7 +383,7 @@ DataTypePtr getLeastSupertype(const DataTypes & types) if (!all_maps) return throwOrReturn(types, "because some of them are Maps and some of them are not", ErrorCodes::NO_COMMON_TYPE); - DataTypePtr keys_common_type = getLeastSupertype(key_types); + auto keys_common_type = getLeastSupertype(key_types); auto values_common_type = getLeastSupertype(value_types); /// When on_error == LeastSupertypeOnError::Null and we cannot get least supertype for keys or values, @@ -425,7 +425,7 @@ DataTypePtr getLeastSupertype(const DataTypes & types) return getLeastSupertype(nested_types); else { - DataTypePtr nested_type = getLeastSupertype(nested_types); + auto nested_type = getLeastSupertype(nested_types); /// When on_error == LeastSupertypeOnError::Null and we cannot get least supertype, /// nested_type will be nullptr, we should return nullptr in this case. @@ -459,8 +459,6 @@ DataTypePtr getLeastSupertype(const DataTypes & types) if (have_nullable) { auto nested_type = getLeastSupertype(nested_types); - if (isVariant(nested_type)) - return nested_type; /// When on_error == LeastSupertypeOnError::Null and we cannot get least supertype, /// nested_type will be nullptr, we should return nullptr in this case. if (!nested_type) @@ -628,8 +626,7 @@ DataTypePtr getLeastSupertype(const DataTypes & types) { /// First, if we have signed integers, try to convert all UInt64 to Int64 if possible. convertUInt64toInt64IfPossible(types, type_ids); - auto throw_or_return = [&](const TypeIndexSet &, std::string_view message_suffix, int error_code){ return throwOrReturn(types, message_suffix, error_code); }; - auto numeric_type = getNumericType(type_ids, throw_or_return); + auto numeric_type = getNumericType(type_ids); if (numeric_type) return numeric_type; } @@ -708,8 +705,7 @@ DataTypePtr getLeastSupertype(const TypeIndexSet & types) return std::make_shared(); } - auto throw_or_return = [](const TypeIndexSet & type_ids, std::string_view message_suffix, int error_code){ return throwOrReturn(type_ids, message_suffix, error_code); }; - auto numeric_type = getNumericType(types, throw_or_return); + auto numeric_type = getNumericType(types); if (numeric_type) return numeric_type; From 92e63bf45c3cd36163131e42d4bd003e543031f4 Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 25 Jan 2024 23:13:31 -0500 Subject: [PATCH 199/264] Load server configs in clickhouse local (#59231) Load server configs in clickhouse local --- src/Interpreters/Context.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 2472fbbe596..75c20b0a520 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4277,11 +4277,11 @@ void Context::setApplicationType(ApplicationType type) /// Lock isn't required, you should set it at start shared->application_type = type; - if (type == ApplicationType::SERVER) - { + if (type == ApplicationType::LOCAL || type == ApplicationType::SERVER) shared->server_settings.loadSettingsFromConfig(Poco::Util::Application::instance().config()); + + if (type == ApplicationType::SERVER) shared->configureServerWideThrottling(); - } } void Context::setDefaultProfiles(const Poco::Util::AbstractConfiguration & config) From 2b75836ad224a663ad62c0cf2aeb9a14eb966b3e Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Fri, 26 Jan 2024 09:33:04 +0000 Subject: [PATCH 200/264] CI: fix ci configuration for nightly job --- tests/ci/ci.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 067aa6173fc..f52f28c3a16 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -919,11 +919,15 @@ def main() -> int: result["ci_flags"] = ci_flags result["jobs_data"] = jobs_data result["docker_data"] = docker_data - if pr_info.number != 0 and not args.docker_digest_or_latest: + if ( + not args.skip_jobs + and pr_info.number != 0 + and not args.docker_digest_or_latest + ): # FIXME: it runs style check before docker build if possible (style-check images is not changed) # find a way to do style check always before docker build and others _check_and_update_for_early_style_check(result) - if pr_info.has_changes_in_documentation_only(): + if not args.skip_jobs and pr_info.has_changes_in_documentation_only(): _update_config_for_docs_only(result) ### CONFIGURE action: end From 7908f69d84dc5a12156dee6ec7c137e1bf60c568 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 26 Jan 2024 14:01:06 +0000 Subject: [PATCH 201/264] Fix special builds and tests --- src/Functions/array/array.cpp | 2 +- src/Functions/map.cpp | 2 +- tests/queries/0_stateless/02942_variant_cast.sql | 1 + 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Functions/array/array.cpp b/src/Functions/array/array.cpp index 4379283ec86..03b51808799 100644 --- a/src/Functions/array/array.cpp +++ b/src/Functions/array/array.cpp @@ -16,7 +16,7 @@ class FunctionArray : public IFunction public: static constexpr auto name = "array"; - FunctionArray(bool use_variant_as_common_type_ = false) : use_variant_as_common_type(use_variant_as_common_type_) {} + explicit FunctionArray(bool use_variant_as_common_type_ = false) : use_variant_as_common_type(use_variant_as_common_type_) {} static FunctionPtr create(ContextPtr context) { diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 79a18dce99b..66cd10a3f0b 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -31,7 +31,7 @@ class FunctionMap : public IFunction public: static constexpr auto name = "map"; - FunctionMap(bool use_variant_as_common_type_) : use_variant_as_common_type(use_variant_as_common_type_) {} + explicit FunctionMap(bool use_variant_as_common_type_) : use_variant_as_common_type(use_variant_as_common_type_) {} static FunctionPtr create(ContextPtr context) { diff --git a/tests/queries/0_stateless/02942_variant_cast.sql b/tests/queries/0_stateless/02942_variant_cast.sql index 33587e3e438..fc2d1d63657 100644 --- a/tests/queries/0_stateless/02942_variant_cast.sql +++ b/tests/queries/0_stateless/02942_variant_cast.sql @@ -1,4 +1,5 @@ set allow_experimental_variant_type=1; +set allow_experimental_analyzer=0; -- It's currently doesn't work with analyzer because of the way it works with constants, but it will be refactored and fixed in future select NULL::Variant(String, UInt64); select 42::UInt64::Variant(String, UInt64); From 92d96c4d0288487fd46a2c6267d4d416f3371608 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 26 Jan 2024 14:40:51 +0100 Subject: [PATCH 202/264] Fix fasttest by pinning pip dependencies After image update it prints the following warning: /ClickHouse/tests/queries/0_stateless/01558_ttest_scipy.python:5: DeprecationWarning: Pyarrow will become a required dependency of pandas in the next major release of pandas (pandas 3.0), (to allow more performant data types, such as the Arrow string type, and better interoperability with other libraries) but was not found to be installed on your system. If this would cause problems for you, please provide us feedback at https://github.com/pandas-dev/pandas/issues/54466 Pyarrow can be added as well, but according to the comment in the issue it bloats the image size 170%, so let's simply downgrade it for now and see how that issue will be resolved (or the warning could be suppressed of course). And pin other dependencies as well to avoid further failures. Signed-off-by: Azat Khuzhin --- docker/test/fasttest/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index a38f59dacac..56ec0199849 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -22,7 +22,7 @@ RUN apt-get update \ zstd \ --yes --no-install-recommends -RUN pip3 install numpy scipy pandas Jinja2 +RUN pip3 install numpy==1.26.3 scipy==1.12.0 pandas==1.5.3 Jinja2==3.1.3 ARG odbc_driver_url="https://github.com/ClickHouse/clickhouse-odbc/releases/download/v1.1.4.20200302/clickhouse-odbc-1.1.4-Linux.tar.gz" From 1624ecd7e6a21d399c42ea0c2e7106e53a2c4334 Mon Sep 17 00:00:00 2001 From: Artur Beglaryan Date: Fri, 26 Jan 2024 15:02:28 +0000 Subject: [PATCH 203/264] Fixed typo in documentation Added missed parameter in xml dictionary config --- docs/en/sql-reference/dictionaries/index.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index 9f86aaf2502..080de94f8b7 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -1805,6 +1805,7 @@ Example of settings: ``` xml + postgresql-hostname 5432 clickhouse qwerty From 94eb33ef27a9ab7c4a99af40772ea287e67efcbf Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 26 Jan 2024 14:29:50 +0100 Subject: [PATCH 204/264] Improve pytest --pdb experience by preserving dockerd on SIGINT (v2) Previous fix (#43392) was not correct, since terminal sends SIGINT to all processes in a group, so simply ignoring it in parent won't work. So instead dockerd will be run in a new session. Signed-off-by: Azat Khuzhin --- docker/test/integration/runner/dockerd-entrypoint.sh | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index b05aef76faf..8882daa38ea 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -23,13 +23,15 @@ if [ -f /sys/fs/cgroup/cgroup.controllers ]; then > /sys/fs/cgroup/cgroup.subtree_control fi -# In case of test hung it is convenient to use pytest --pdb to debug it, -# and on hung you can simply press Ctrl-C and it will spawn a python pdb, -# but on SIGINT dockerd will exit, so ignore it to preserve the daemon. -trap '' INT # Binding to an IP address without --tlsverify is deprecated. Startup is intentionally being slowed # unless --tls=false or --tlsverify=false is set -dockerd --host=unix:///var/run/docker.sock --tls=false --host=tcp://0.0.0.0:2375 --default-address-pool base=172.17.0.0/12,size=24 &>/ClickHouse/tests/integration/dockerd.log & +# +# In case of test hung it is convenient to use pytest --pdb to debug it, +# and on hung you can simply press Ctrl-C and it will spawn a python pdb, +# but on SIGINT dockerd will exit, so we spawn new session to ignore SIGINT by +# docker. +# Note, that if you will run it via runner, it will send SIGINT to docker anyway. +setsid dockerd --host=unix:///var/run/docker.sock --tls=false --host=tcp://0.0.0.0:2375 --default-address-pool base=172.17.0.0/12,size=24 &>/ClickHouse/tests/integration/dockerd.log & set +e reties=0 From e56728ae6c00c6abc92a2d186eb24cb2b2fe57b7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 24 Jan 2024 11:32:09 +0100 Subject: [PATCH 205/264] tests: add s3_plain disk Signed-off-by: Azat Khuzhin --- tests/config/config.d/storage_conf.xml | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index 18652826d83..1429dfff724 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -4,11 +4,17 @@ s3 s3_disk/ - http://localhost:11111/test/common/ + http://localhost:11111/test/s3/ clickhouse clickhouse 20000 + + s3_plain + http://localhost:11111/test/s3_plain/ + clickhouse + clickhouse + cache s3_disk From 6751e01cc3d848132cafb10f0e2195281b6c2575 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 24 Jan 2024 11:59:16 +0100 Subject: [PATCH 206/264] Do not drop parts for static (ro/write-once) storages for Replicated engine Signed-off-by: Azat Khuzhin --- src/Storages/MergeTree/MergeTreeData.cpp | 9 +++++++++ src/Storages/StorageMergeTree.cpp | 3 --- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 39c113c240e..b70e1471b3c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2740,6 +2740,15 @@ void MergeTreeData::renameInMemory(const StorageID & new_table_id) void MergeTreeData::dropAllData() { + /// In case there is read-only/write-once disk we cannot allow to call dropAllData(), but dropping tables is allowed. + /// + /// Note, that one may think that drop on write-once disk should be + /// supported, since it is pretty trivial to implement + /// MetadataStorageFromPlainObjectStorageTransaction::removeDirectory(), + /// however removing part requires moveDirectory() as well. + if (isStaticStorage()) + return; + LOG_TRACE(log, "dropAllData: waiting for locks."); auto settings_ptr = getSettings(); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 7e6c5ca3924..b0473a4bbb0 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -319,9 +319,6 @@ void StorageMergeTree::checkTableCanBeDropped(ContextPtr query_context) const void StorageMergeTree::drop() { shutdown(true); - /// In case there is read-only disk we cannot allow to call dropAllData(), but dropping tables is allowed. - if (isStaticStorage()) - return; dropAllData(); } From 41ebaefeda10c8177dbb92e0651fd833238880b9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 24 Jan 2024 16:27:17 +0100 Subject: [PATCH 207/264] Add a note for MetadataStorageFromPlainObjectStorage to support transactions Signed-off-by: Azat Khuzhin --- .../ObjectStorages/MetadataStorageFromPlainObjectStorage.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h index 1bb008c0c19..58089079a44 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h @@ -106,7 +106,7 @@ public: void commit() override { - /// Nothing to commit. + /// TODO: rewrite with transactions } bool supportsChmod() const override { return false; } From f3b4f28f0d6f3064416653877b1b081f004f0d8f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 24 Jan 2024 16:51:44 +0100 Subject: [PATCH 208/264] Disable mutable operations into {Replicated,}MergeTree with static storage Mutable operations includes: - INSERT - OPTIMIZE - parts manipulation - ... But does not include DROP, since you need an ability to DROP it. Signed-off-by: Azat Khuzhin --- src/Storages/MergeTree/MergeTreeData.cpp | 3 +++ src/Storages/StorageMergeTree.cpp | 21 +++++++++++++++++++++ src/Storages/StorageMergeTree.h | 2 ++ src/Storages/StorageReplicatedMergeTree.cpp | 7 +++++++ 4 files changed, 33 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b70e1471b3c..61bfc2bf9cb 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -90,6 +90,7 @@ #include #include +#include #include #include @@ -7080,6 +7081,8 @@ std::pair MergeTreeData::cloneAn const ReadSettings & read_settings, const WriteSettings & write_settings) { + chassert(!isStaticStorage()); + /// Check that the storage policy contains the disk where the src_part is located. bool does_storage_policy_allow_same_disk = false; for (const DiskPtr & disk : getStoragePolicy()->getDisks()) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index b0473a4bbb0..f9d360f02e7 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -65,6 +65,7 @@ namespace ErrorCodes extern const int NO_SUCH_DATA_PART; extern const int ABORTED; extern const int SUPPORT_IS_DISABLED; + extern const int TABLE_IS_READ_ONLY; } namespace ActionLocks @@ -294,6 +295,8 @@ std::optional StorageMergeTree::totalBytesUncompressed(const Settings &) SinkToStoragePtr StorageMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { + assertNotReadonly(); + const auto & settings = local_context->getSettingsRef(); return std::make_shared( *this, metadata_snapshot, settings.max_partitions_per_insert_block, local_context); @@ -327,6 +330,8 @@ void StorageMergeTree::alter( ContextPtr local_context, AlterLockHolder & table_lock_holder) { + assertNotReadonly(); + if (local_context->getCurrentTransaction() && local_context->getSettingsRef().throw_on_unsupported_query_inside_transaction) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "ALTER METADATA is not supported inside transactions"); @@ -617,6 +622,8 @@ void StorageMergeTree::setMutationCSN(const String & mutation_id, CSN csn) void StorageMergeTree::mutate(const MutationCommands & commands, ContextPtr query_context) { + assertNotReadonly(); + delayMutationOrThrowIfNeeded(nullptr, query_context); /// Validate partition IDs (if any) before starting mutation @@ -807,6 +814,8 @@ std::vector StorageMergeTree::getMutationsStatus() cons CancellationCode StorageMergeTree::killMutation(const String & mutation_id) { + assertNotReadonly(); + LOG_TRACE(log, "Killing mutation {}", mutation_id); UInt64 mutation_version = MergeTreeMutationEntry::tryParseFileName(mutation_id); if (!mutation_version) @@ -1517,6 +1526,8 @@ bool StorageMergeTree::optimize( bool cleanup, ContextPtr local_context) { + assertNotReadonly(); + if (deduplicate) { if (deduplicate_by_columns.empty()) @@ -1762,6 +1773,8 @@ void StorageMergeTree::renameAndCommitEmptyParts(MutableDataPartsVector & new_pa void StorageMergeTree::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr query_context, TableExclusiveLockHolder &) { + assertNotReadonly(); + { /// Asks to complete merges and does not allow them to start. /// This protects against "revival" of data for a removed partition after completion of merge. @@ -2036,6 +2049,8 @@ PartitionCommandsResultInfo StorageMergeTree::attachPartition( void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, ContextPtr local_context) { + assertNotReadonly(); + auto lock1 = lockForShare(local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout); auto lock2 = source_table->lockForShare(local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout); auto merges_blocker = stopMergesAndWait(); @@ -2434,6 +2449,12 @@ PreparedSetsCachePtr StorageMergeTree::getPreparedSetsCache(Int64 mutation_id) return cache; } +void StorageMergeTree::assertNotReadonly() const +{ + if (isStaticStorage()) + throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode due to static storage"); +} + void StorageMergeTree::fillNewPartName(MutableDataPartPtr & part, DataPartsLock &) { part->info.min_block = part->info.max_block = increment.get(); diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 51bf6aa42e7..359fa1d262d 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -273,6 +273,8 @@ private: PreparedSetsCachePtr getPreparedSetsCache(Int64 mutation_id); + void assertNotReadonly() const; + friend class MergeTreeSink; friend class MergeTreeData; friend class MergePlainMergeTreeTask; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c82721d2a18..579cc5f4c30 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4659,6 +4659,9 @@ bool StorageReplicatedMergeTree::fetchPart( zkutil::ZooKeeper::Ptr zookeeper_, bool try_fetch_shared) { + if (isStaticStorage()) + throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode due to static storage"); + auto zookeeper = zookeeper_ ? zookeeper_ : getZooKeeper(); const auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version); @@ -5496,6 +5499,8 @@ void StorageReplicatedMergeTree::assertNotReadonly() const { if (is_readonly) throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode (replica path: {})", replica_path); + if (isStaticStorage()) + throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode due to static storage"); } @@ -5504,6 +5509,8 @@ SinkToStoragePtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, con if (!initialization_done) throw Exception(ErrorCodes::NOT_INITIALIZED, "Table is not initialized yet"); + if (isStaticStorage()) + throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode due to static storage"); /// If table is read-only because it doesn't have metadata in zk yet, then it's not possible to insert into it /// Without this check, we'll write data parts on disk, and afterwards will remove them since we'll fail to commit them into zk /// In case of remote storage like s3, it'll generate unnecessary PUT requests From 9ce18bbf4332a53c7838694365c31ee3861e0a98 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 24 Jan 2024 16:06:53 +0100 Subject: [PATCH 209/264] Implement MetadataStorageFromPlainObjectStorageTransaction::removeDirectory (for DROP TABLE) This is required to remove format_version.txt for MergeTree after DROP (lldb) bt * thread 192, name = 'BgSchPool', stop reason = breakpoint 1.1 frame 0: 0x0000000016c7f452 clickhouse`__cxa_throw(thrown_object=0x00007ffd7a53c480, tinfo=0x00000000011602a0, dest=(clickhouse`DB::Exception::~Exception() at Exception.h:40)) + 18 at cxa_exception.cpp:258 frame 1: 0x00000000109bf709 clickhouse`DB::IMetadataTransaction::throwNotImplemented() + 137 at IMetadataStorage.h:151 frame 2: 0x00000000109bf126 clickhouse`DB::IMetadataTransaction::removeDirectory(this=, (null)=) + 6 at IMetadataStorage.h:96 frame 3: 0x0000000010968978 clickhouse`DB::(anonymous namespace)::RemoveRecursiveObjectStorageOperation::removeMetadataRecursive(this=0x00007ffd7a7327a0, tx=std::__1::shared_ptr::element_type @ 0x00007ffd7a6f6018 strong=4 weak=1, path_to_remove="tmp_insert_all_2_2_0") + 376 at DiskObjectStorageTransaction.cpp:364 frame 4: 0x000000001096893d clickhouse`DB::(anonymous namespace)::RemoveRecursiveObjectStorageOperation::removeMetadataRecursive(this=0x00007ffd7a7327a0, tx=std::__1::shared_ptr::element_type @ 0x00007ffd7a6f6018 strong=4 weak=1, path_to_remove="store/27f/27f6f9d2-ab2e-4817-b771-0e706e485b15/") + 317 at DiskObjectStorageTransaction.cpp:362 frame 5: 0x0000000010965a7f clickhouse`DB::(anonymous namespace)::RemoveRecursiveObjectStorageOperation::execute(this=0x00007ffd7a7327a0, tx=std::__1::shared_ptr::element_type @ 0x00007ffd7a6f6018 strong=4 weak=1) + 95 at DiskObjectStorageTransaction.cpp:372 frame 6: 0x0000000010963a2f clickhouse`DB::DiskObjectStorageTransaction::commit(this=0x00007ffea22b6168) + 79 at DiskObjectStorageTransaction.cpp:903 frame 7: 0x00000000109577e5 clickhouse`DB::DiskObjectStorage::removeSharedRecursive(this=, path="store/27f/27f6f9d2-ab2e-4817-b771-0e706e485b15/", keep_all_batch_data=false, file_names_remove_metadata_only=size=0) + 133 at DiskObjectStorage.cpp:426 frame 8: 0x000000001095af79 clickhouse`DB::DiskObjectStorage::removeRecursive(this=, path=) + 57 at DiskObjectStorage.h:79 * frame 9: 0x0000000010b5c3d5 clickhouse`DB::DatabaseCatalog::dropTableFinally(this=0x00007ffff7096800, table=0x00007ffe853e1bd0) + 1365 at DatabaseCatalog.cpp:1317 frame 10: 0x0000000010b5b7e7 clickhouse`DB::DatabaseCatalog::dropTableDataTask(this=0x00007ffff7096800) + 2375 at DatabaseCatalog.cpp:1270 Signed-off-by: Azat Khuzhin --- .../MetadataStorageFromPlainObjectStorage.cpp | 6 ++++++ .../ObjectStorages/MetadataStorageFromPlainObjectStorage.h | 1 + 2 files changed, 7 insertions(+) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index f20cd67a39f..72d512c6f6b 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -121,6 +121,12 @@ void MetadataStorageFromPlainObjectStorageTransaction::unlinkFile(const std::str metadata_storage.object_storage->removeObject(object); } +void MetadataStorageFromPlainObjectStorageTransaction::removeDirectory(const std::string & path) +{ + for (auto it = metadata_storage.iterateDirectory(path); it->isValid(); it->next()) + metadata_storage.object_storage->removeObject(StoredObject(it->path())); +} + void MetadataStorageFromPlainObjectStorageTransaction::createDirectory(const std::string &) { /// Noop. It is an Object Storage not a filesystem. diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h index 58089079a44..8a8466c3fbe 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h @@ -101,6 +101,7 @@ public: void createDirectoryRecursive(const std::string & path) override; void unlinkFile(const std::string & path) override; + void removeDirectory(const std::string & path) override; UnlinkMetadataFileOperationOutcomePtr unlinkMetadata(const std::string & path) override; From d79c1f7e1c4a98a333c89aa77306ad2c22b2d183 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 Jan 2024 15:38:36 +0100 Subject: [PATCH 210/264] Fix MetadataStorageFromPlainObjectStorage::iterateDirectory() (add prefix) Signed-off-by: Azat Khuzhin --- .../ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 72d512c6f6b..b03809f5b39 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -98,6 +98,8 @@ DirectoryIteratorPtr MetadataStorageFromPlainObjectStorage::iterateDirectory(con { /// Required for MergeTree auto paths = listDirectory(path); + // Prepend path, since iterateDirectory() includes path, unlike listDirectory() + std::for_each(paths.begin(), paths.end(), [&](auto & child) { child = fs::path(path) / child; }); std::vector fs_paths(paths.begin(), paths.end()); return std::make_unique(std::move(fs_paths)); } From 046a7a5df53451e224da8214d783ebbd7e7db0dc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 24 Jan 2024 11:56:09 +0100 Subject: [PATCH 211/264] tests: cover DROP TABLE on s3_plain disk Signed-off-by: Azat Khuzhin --- ...80_s3_plain_DROP_TABLE_MergeTree.reference | 7 +++ .../02980_s3_plain_DROP_TABLE_MergeTree.sh | 35 +++++++++++++++ .../02980_s3_plain_DROP_TABLE_MergeTree.yml | 7 +++ ...n_DROP_TABLE_ReplicatedMergeTree.reference | 7 +++ ...s3_plain_DROP_TABLE_ReplicatedMergeTree.sh | 45 +++++++++++++++++++ ...3_plain_DROP_TABLE_ReplicatedMergeTree.yml | 7 +++ 6 files changed, 108 insertions(+) create mode 100644 tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.reference create mode 100755 tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.sh create mode 100644 tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.yml create mode 100644 tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.reference create mode 100755 tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh create mode 100644 tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.yml diff --git a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.reference b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.reference new file mode 100644 index 00000000000..2bec7a1d508 --- /dev/null +++ b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.reference @@ -0,0 +1,7 @@ +Files before DROP TABLE +format_version.txt + +Files after DETACH TABLE +format_version.txt + +Files after DROP TABLE diff --git a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.sh b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.sh new file mode 100755 index 00000000000..406e53a2175 --- /dev/null +++ b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.sh @@ -0,0 +1,35 @@ +#!/usr/bin/env bash +# Tags: no-fasttest +# Tag no-fasttest: requires S3 + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +config="${BASH_SOURCE[0]/.sh/.yml}" + +$CLICKHOUSE_CLIENT -nm -q " + drop table if exists data; + create table data (key Int) engine=MergeTree() order by key settings disk='s3_plain_disk'; + insert into data values (1); -- { serverError TABLE_IS_READ_ONLY } + optimize table data final; -- { serverError TABLE_IS_READ_ONLY } +" + +path=$($CLICKHOUSE_CLIENT -q "SELECT replace(data_paths[1], 's3_plain', '') FROM system.tables WHERE database = '$CLICKHOUSE_DATABASE' AND table = 'data'") +# trim / to fix "Unable to parse ExceptionName: XMinioInvalidObjectName Message: Object name contains unsupported characters." +path=${path%/} + +echo "Files before DROP TABLE" +clickhouse-disks -C "$config" --disk s3_plain_disk list --recursive "${path:?}" | tail -n+2 + +$CLICKHOUSE_CLIENT -q "detach table data" +echo "Files after DETACH TABLE" +clickhouse-disks -C "$config" --disk s3_plain_disk list --recursive "$path" | tail -n+2 + +$CLICKHOUSE_CLIENT -nm -q " + attach table data; + drop table data; +" +# Check that there is no leftovers: +echo "Files after DROP TABLE" +clickhouse-disks -C "$config" --disk s3_plain_disk list --recursive "$path" | tail -n+2 diff --git a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.yml b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.yml new file mode 100644 index 00000000000..ca5036736d8 --- /dev/null +++ b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.yml @@ -0,0 +1,7 @@ +storage_configuration: + disks: + s3_plain_disk: + type: s3_plain + endpoint: http://localhost:11111/test/s3_plain/ + access_key_id: clickhouse + secret_access_key: clickhouse diff --git a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.reference b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.reference new file mode 100644 index 00000000000..166512b4b85 --- /dev/null +++ b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.reference @@ -0,0 +1,7 @@ +Files before DROP TABLE +format_version.txt + +Files after DETACH TABLE +format_version.txt + +Files after DETACH TABLE diff --git a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh new file mode 100755 index 00000000000..6b8f191edc0 --- /dev/null +++ b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh @@ -0,0 +1,45 @@ +#!/usr/bin/env bash +# Tags: no-fasttest +# Tag no-fasttest: requires S3 + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +config="${BASH_SOURCE[0]/.sh/.yml}" + +$CLICKHOUSE_CLIENT -nm -q " + drop table if exists data_read; + drop table if exists data_write; + + create table data_read (key Int) engine=ReplicatedMergeTree('/tables/{database}/data', 'read') order by key settings disk='s3_plain_disk'; + create table data_write (key Int) engine=ReplicatedMergeTree('/tables/{database}/data', 'write') order by key; + + insert into data_write values (1); + system sync replica data_read; -- { serverError TABLE_IS_READ_ONLY } +" + +path=$($CLICKHOUSE_CLIENT -q "SELECT replace(data_paths[1], 's3_plain', '') FROM system.tables WHERE database = '$CLICKHOUSE_DATABASE' AND table = 'data_read'") +# trim / to fix "Unable to parse ExceptionName: XMinioInvalidObjectName Message: Object name contains unsupported characters." +path=${path%/} + +echo "Files before DROP TABLE" +clickhouse-disks -C "$config" --disk s3_plain_disk list --recursive "${path:?}" | tail -n+2 + +$CLICKHOUSE_CLIENT -nm -q " + detach table data_read; + detach table data_write; +" +echo "Files after DETACH TABLE" +clickhouse-disks -C "$config" --disk s3_plain_disk list --recursive "$path" | tail -n+2 + +$CLICKHOUSE_CLIENT -nm -q " + attach table data_read; + attach table data_write; + + drop table data_read; + drop table data_write; +" +# Check that there is no leftovers: +echo "Files after DETACH TABLE" +clickhouse-disks -C "$config" --disk s3_plain_disk list --recursive "$path" | tail -n+2 diff --git a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.yml b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.yml new file mode 100644 index 00000000000..ca5036736d8 --- /dev/null +++ b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.yml @@ -0,0 +1,7 @@ +storage_configuration: + disks: + s3_plain_disk: + type: s3_plain + endpoint: http://localhost:11111/test/s3_plain/ + access_key_id: clickhouse + secret_access_key: clickhouse From 1ee723d50d1226eff86b0fd6c0297322a5bda682 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 Jan 2024 17:56:25 +0100 Subject: [PATCH 212/264] Prohibit DROP from static storages Signed-off-by: Azat Khuzhin --- src/Storages/MergeTree/MergeTreeData.cpp | 9 --------- src/Storages/StorageMergeTree.cpp | 4 ++++ src/Storages/StorageReplicatedMergeTree.cpp | 4 ++++ .../integration/test_attach_backup_from_s3_plain/test.py | 3 +-- 4 files changed, 9 insertions(+), 11 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 61bfc2bf9cb..62ec058b124 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2741,15 +2741,6 @@ void MergeTreeData::renameInMemory(const StorageID & new_table_id) void MergeTreeData::dropAllData() { - /// In case there is read-only/write-once disk we cannot allow to call dropAllData(), but dropping tables is allowed. - /// - /// Note, that one may think that drop on write-once disk should be - /// supported, since it is pretty trivial to implement - /// MetadataStorageFromPlainObjectStorageTransaction::removeDirectory(), - /// however removing part requires moveDirectory() as well. - if (isStaticStorage()) - return; - LOG_TRACE(log, "dropAllData: waiting for locks."); auto settings_ptr = getSettings(); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index f9d360f02e7..9e1420d811b 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -321,6 +321,10 @@ void StorageMergeTree::checkTableCanBeDropped(ContextPtr query_context) const void StorageMergeTree::drop() { + /// In case there is read-only/write-once disk we do not allow DROP, use DETACH instead. + if (isStaticStorage()) + throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode due to static storage"); + shutdown(true); dropAllData(); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 579cc5f4c30..39c6641a885 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1085,6 +1085,10 @@ zkutil::ZooKeeperPtr StorageReplicatedMergeTree::getZooKeeperIfTableShutDown() c void StorageReplicatedMergeTree::drop() { + /// In case there is read-only/write-once disk we do not allow DROP, use DETACH instead. + if (isStaticStorage()) + throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode due to static storage"); + /// There is also the case when user has configured ClickHouse to wrong ZooKeeper cluster /// or metadata of staled replica were removed manually, /// in this case, has_metadata_in_zookeeper = false, and we also permit to drop the table. diff --git a/tests/integration/test_attach_backup_from_s3_plain/test.py b/tests/integration/test_attach_backup_from_s3_plain/test.py index e575c487b7a..4d48a4438ea 100644 --- a/tests/integration/test_attach_backup_from_s3_plain/test.py +++ b/tests/integration/test_attach_backup_from_s3_plain/test.py @@ -64,8 +64,7 @@ def test_attach_part(table_name, backup_name, storage_policy, min_bytes_for_wide node.query( f""" - -- NOTE: DROP DATABASE cannot be done w/o this due to metadata leftovers - set force_remove_data_recursively_on_drop=1; + detach table ordinary_db.{table_name}; drop database ordinary_db sync; """ ) From aafc149c1eff50c0df95459240065f2eafd73a1b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 Jan 2024 19:24:53 +0100 Subject: [PATCH 213/264] Prohibit CREATE on static storages (ATTACH should be used instead) Signed-off-by: Azat Khuzhin --- src/Storages/StorageMergeTree.cpp | 6 ++++-- src/Storages/StorageReplicatedMergeTree.cpp | 3 +++ 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 9e1420d811b..9896e04c57f 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -115,12 +115,14 @@ StorageMergeTree::StorageMergeTree( , writer(*this) , merger_mutator(*this) { - initializeDirectoriesAndFormatVersion(relative_data_path_, attach, date_column_name); + if (!attach && isStaticStorage()) + throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Creating data on static storage is prohibited, use ATTACH instead"); + initializeDirectoriesAndFormatVersion(relative_data_path_, attach, date_column_name); loadDataParts(has_force_restore_data_flag, std::nullopt); - if (!attach && !getDataPartsForInternalUsage().empty() && !isStaticStorage()) + if (!attach && !getDataPartsForInternalUsage().empty()) throw Exception(ErrorCodes::INCORRECT_DATA, "Data directory for table already containing data parts - probably " "it was unclean DROP table or manual intervention. " diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 39c6641a885..2f5daaff36c 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -338,6 +338,9 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( , replicated_fetches_throttler(std::make_shared(getSettings()->max_replicated_fetches_network_bandwidth, getContext()->getReplicatedFetchesThrottler())) , replicated_sends_throttler(std::make_shared(getSettings()->max_replicated_sends_network_bandwidth, getContext()->getReplicatedSendsThrottler())) { + if (!attach && isStaticStorage()) + throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Creating data on static storage is prohibited, use ATTACH instead"); + initializeDirectoriesAndFormatVersion(relative_data_path_, attach, date_column_name); /// We create and deactivate all tasks for consistency. /// They all will be scheduled and activated by the restarting thread. From d99fb4ce5be1086c0b90d20c85bd7f98414b34a9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 Jan 2024 19:24:55 +0100 Subject: [PATCH 214/264] tests: rewrite test for MergeTree engines in s3_plain using ATTACH Signed-off-by: Azat Khuzhin --- ...80_s3_plain_DROP_TABLE_MergeTree.reference | 9 ++-- .../02980_s3_plain_DROP_TABLE_MergeTree.sh | 40 +++++++++++++---- ...n_DROP_TABLE_ReplicatedMergeTree.reference | 9 ++-- ...s3_plain_DROP_TABLE_ReplicatedMergeTree.sh | 43 +++++++++++++------ 4 files changed, 68 insertions(+), 33 deletions(-) diff --git a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.reference b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.reference index 2bec7a1d508..d510f225161 100644 --- a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.reference +++ b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.reference @@ -1,7 +1,4 @@ -Files before DROP TABLE -format_version.txt - +data after INSERT 1 +data after ATTACH 1 +Files before DETACH TABLE Files after DETACH TABLE -format_version.txt - -Files after DROP TABLE diff --git a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.sh b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.sh index 406e53a2175..2059d33ccfd 100755 --- a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.sh +++ b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.sh @@ -6,11 +6,38 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh +# config for clickhouse-disks (to check leftovers) config="${BASH_SOURCE[0]/.sh/.yml}" +# only in Atomic ATTACH from s3_plain works +new_database="ordinary_$CLICKHOUSE_DATABASE" +$CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 -q "create database $new_database engine=Ordinary" +CLICKHOUSE_CLIENT=${CLICKHOUSE_CLIENT/--database=$CLICKHOUSE_DATABASE/--database=$new_database} +CLICKHOUSE_DATABASE="$new_database" + $CLICKHOUSE_CLIENT -nm -q " drop table if exists data; - create table data (key Int) engine=MergeTree() order by key settings disk='s3_plain_disk'; + create table data (key Int) engine=MergeTree() order by key settings disk='s3_plain_disk'; -- { serverError TABLE_IS_READ_ONLY } + + create table data (key Int) engine=MergeTree() order by key; + insert into data values (1); + select 'data after INSERT', count() from data; +" + +# suppress output +$CLICKHOUSE_CLIENT -q "backup table data to S3('http://localhost:11111/test/backups/$CLICKHOUSE_DATABASE', 'test', 'testtest')" > /dev/null + +$CLICKHOUSE_CLIENT -nm -q " + drop table data; + attach table data (key Int) engine=MergeTree() order by key + settings + max_suspicious_broken_parts=0, + disk=disk(type=s3_plain, + endpoint='http://localhost:11111/test/backups/$CLICKHOUSE_DATABASE', + access_key_id='test', + secret_access_key='testtest'); + select 'data after ATTACH', count() from data; + insert into data values (1); -- { serverError TABLE_IS_READ_ONLY } optimize table data final; -- { serverError TABLE_IS_READ_ONLY } " @@ -19,17 +46,12 @@ path=$($CLICKHOUSE_CLIENT -q "SELECT replace(data_paths[1], 's3_plain', '') FROM # trim / to fix "Unable to parse ExceptionName: XMinioInvalidObjectName Message: Object name contains unsupported characters." path=${path%/} -echo "Files before DROP TABLE" +echo "Files before DETACH TABLE" clickhouse-disks -C "$config" --disk s3_plain_disk list --recursive "${path:?}" | tail -n+2 $CLICKHOUSE_CLIENT -q "detach table data" echo "Files after DETACH TABLE" clickhouse-disks -C "$config" --disk s3_plain_disk list --recursive "$path" | tail -n+2 -$CLICKHOUSE_CLIENT -nm -q " - attach table data; - drop table data; -" -# Check that there is no leftovers: -echo "Files after DROP TABLE" -clickhouse-disks -C "$config" --disk s3_plain_disk list --recursive "$path" | tail -n+2 +# metadata file is left +$CLICKHOUSE_CLIENT --force_remove_data_recursively_on_drop=1 -q "drop database if exists $CLICKHOUSE_DATABASE" diff --git a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.reference b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.reference index 166512b4b85..d510f225161 100644 --- a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.reference +++ b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.reference @@ -1,7 +1,4 @@ -Files before DROP TABLE -format_version.txt - -Files after DETACH TABLE -format_version.txt - +data after INSERT 1 +data after ATTACH 1 +Files before DETACH TABLE Files after DETACH TABLE diff --git a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh index 6b8f191edc0..d8c92c5d6fa 100755 --- a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh +++ b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh @@ -8,14 +8,41 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) config="${BASH_SOURCE[0]/.sh/.yml}" +# only in Atomic ATTACH from s3_plain works +new_database="ordinary_$CLICKHOUSE_DATABASE" +$CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 -q "create database $new_database engine=Ordinary" +CLICKHOUSE_CLIENT=${CLICKHOUSE_CLIENT/--database=$CLICKHOUSE_DATABASE/--database=$new_database} +CLICKHOUSE_DATABASE="$new_database" + $CLICKHOUSE_CLIENT -nm -q " drop table if exists data_read; drop table if exists data_write; - create table data_read (key Int) engine=ReplicatedMergeTree('/tables/{database}/data', 'read') order by key settings disk='s3_plain_disk'; create table data_write (key Int) engine=ReplicatedMergeTree('/tables/{database}/data', 'write') order by key; + create table data_read (key Int) engine=ReplicatedMergeTree('/tables/{database}/data', 'read') order by key settings disk='s3_plain_disk'; -- { serverError TABLE_IS_READ_ONLY } + create table data_read (key Int) engine=ReplicatedMergeTree('/tables/{database}/data', 'read') order by key; insert into data_write values (1); + system sync replica data_read; + select 'data after INSERT', count() from data_read; +" + +# suppress output +$CLICKHOUSE_CLIENT -q "backup table data_read to S3('http://localhost:11111/test/backups/$CLICKHOUSE_DATABASE', 'test', 'testtest')" > /dev/null + +$CLICKHOUSE_CLIENT -nm -q " + drop table data_read; + attach table data_read (key Int) engine=ReplicatedMergeTree('/tables/{database}/data', 'read') order by key + settings + max_suspicious_broken_parts=0, + disk=disk(type=s3_plain, + endpoint='http://localhost:11111/test/backups/$CLICKHOUSE_DATABASE', + access_key_id='test', + secret_access_key='testtest'); + select 'data after ATTACH', count() from data_read; + + insert into data_read values (1); -- { serverError TABLE_IS_READ_ONLY } + optimize table data_read final; -- { serverError TABLE_IS_READ_ONLY } system sync replica data_read; -- { serverError TABLE_IS_READ_ONLY } " @@ -23,7 +50,7 @@ path=$($CLICKHOUSE_CLIENT -q "SELECT replace(data_paths[1], 's3_plain', '') FROM # trim / to fix "Unable to parse ExceptionName: XMinioInvalidObjectName Message: Object name contains unsupported characters." path=${path%/} -echo "Files before DROP TABLE" +echo "Files before DETACH TABLE" clickhouse-disks -C "$config" --disk s3_plain_disk list --recursive "${path:?}" | tail -n+2 $CLICKHOUSE_CLIENT -nm -q " @@ -33,13 +60,5 @@ $CLICKHOUSE_CLIENT -nm -q " echo "Files after DETACH TABLE" clickhouse-disks -C "$config" --disk s3_plain_disk list --recursive "$path" | tail -n+2 -$CLICKHOUSE_CLIENT -nm -q " - attach table data_read; - attach table data_write; - - drop table data_read; - drop table data_write; -" -# Check that there is no leftovers: -echo "Files after DETACH TABLE" -clickhouse-disks -C "$config" --disk s3_plain_disk list --recursive "$path" | tail -n+2 +# metadata file is left +$CLICKHOUSE_CLIENT --force_remove_data_recursively_on_drop=1 -q "drop database if exists $CLICKHOUSE_DATABASE" From 66a40bda0acce258debe6158ce39dacf3300c502 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 26 Jan 2024 11:36:07 +0100 Subject: [PATCH 215/264] Revert "Prohibit DROP from static storages" It is actually OK to DROP at least from read-only disk, as for the write-once it should be OK as well, since CREATE is prohibited anyway, so there should be no leftovers CI: https://s3.amazonaws.com/clickhouse-test-reports/59170/c82050d1925439f0ede2b32acb5b1b8df4acae5d/stress_test__asan_.html CI: https://s3.amazonaws.com/clickhouse-test-reports/59170/c82050d1925439f0ede2b32acb5b1b8df4acae5d/integration_tests__asan__[3_4].html This reverts commit 082e62712c5e2f4c48d1abc6141ccfd1249d26b4. Signed-off-by: Azat Khuzhin --- src/Storages/MergeTree/MergeTreeData.cpp | 9 +++++++++ src/Storages/StorageMergeTree.cpp | 4 ---- src/Storages/StorageReplicatedMergeTree.cpp | 4 ---- .../integration/test_attach_backup_from_s3_plain/test.py | 3 ++- 4 files changed, 11 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 62ec058b124..61bfc2bf9cb 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2741,6 +2741,15 @@ void MergeTreeData::renameInMemory(const StorageID & new_table_id) void MergeTreeData::dropAllData() { + /// In case there is read-only/write-once disk we cannot allow to call dropAllData(), but dropping tables is allowed. + /// + /// Note, that one may think that drop on write-once disk should be + /// supported, since it is pretty trivial to implement + /// MetadataStorageFromPlainObjectStorageTransaction::removeDirectory(), + /// however removing part requires moveDirectory() as well. + if (isStaticStorage()) + return; + LOG_TRACE(log, "dropAllData: waiting for locks."); auto settings_ptr = getSettings(); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 9896e04c57f..ead75113c6d 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -323,10 +323,6 @@ void StorageMergeTree::checkTableCanBeDropped(ContextPtr query_context) const void StorageMergeTree::drop() { - /// In case there is read-only/write-once disk we do not allow DROP, use DETACH instead. - if (isStaticStorage()) - throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode due to static storage"); - shutdown(true); dropAllData(); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 2f5daaff36c..6aa6cc60d7b 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1088,10 +1088,6 @@ zkutil::ZooKeeperPtr StorageReplicatedMergeTree::getZooKeeperIfTableShutDown() c void StorageReplicatedMergeTree::drop() { - /// In case there is read-only/write-once disk we do not allow DROP, use DETACH instead. - if (isStaticStorage()) - throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode due to static storage"); - /// There is also the case when user has configured ClickHouse to wrong ZooKeeper cluster /// or metadata of staled replica were removed manually, /// in this case, has_metadata_in_zookeeper = false, and we also permit to drop the table. diff --git a/tests/integration/test_attach_backup_from_s3_plain/test.py b/tests/integration/test_attach_backup_from_s3_plain/test.py index 4d48a4438ea..e575c487b7a 100644 --- a/tests/integration/test_attach_backup_from_s3_plain/test.py +++ b/tests/integration/test_attach_backup_from_s3_plain/test.py @@ -64,7 +64,8 @@ def test_attach_part(table_name, backup_name, storage_policy, min_bytes_for_wide node.query( f""" - detach table ordinary_db.{table_name}; + -- NOTE: DROP DATABASE cannot be done w/o this due to metadata leftovers + set force_remove_data_recursively_on_drop=1; drop database ordinary_db sync; """ ) From 7cc7fb892c4f2ed4317ba0b86a082624d2b2c1a2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 26 Jan 2024 11:42:37 +0100 Subject: [PATCH 216/264] tests/test_disk_over_web_server: update expected assertion for static storage Signed-off-by: Azat Khuzhin --- tests/integration/test_disk_over_web_server/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_disk_over_web_server/test.py b/tests/integration/test_disk_over_web_server/test.py index a71fdeff302..4b175d188ef 100644 --- a/tests/integration/test_disk_over_web_server/test.py +++ b/tests/integration/test_disk_over_web_server/test.py @@ -172,7 +172,7 @@ def test_incorrect_usage(cluster): assert "Table is read-only" in result result = node2.query_and_get_error("OPTIMIZE TABLE test0 FINAL") - assert "Only read-only operations are supported" in result + assert "Table is in readonly mode due to static storage" in result node2.query("DROP TABLE test0 SYNC") From 7ba31bf55c42132dd35ee2ed0caf284587e5dd47 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 26 Jan 2024 16:28:36 +0100 Subject: [PATCH 217/264] Do not create format_version.txt on write-once storages Signed-off-by: Azat Khuzhin --- src/Storages/MergeTree/MergeTreeData.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 61bfc2bf9cb..6ed07da5c3f 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -301,7 +301,11 @@ void MergeTreeData::initializeDirectoriesAndFormatVersion(const std::string & re if (disk->isBroken()) continue; - if (!disk->isReadOnly()) + /// Write once disk is almost the same as read-only for MergeTree, + /// since it does not support move, that is required for any + /// operation over MergeTree, so avoid writing format_version.txt + /// into it as well, to avoid leaving it after DROP. + if (!disk->isReadOnly() && !disk->isWriteOnce()) { auto buf = disk->writeFile(format_version_path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, getContext()->getWriteSettings()); writeIntText(format_version.toUnderType(), *buf); From 90ab986be6a80227ca281a86ce12db1499d830fe Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 26 Jan 2024 16:37:54 +0100 Subject: [PATCH 218/264] Revert "Prohibit CREATE on static storages (ATTACH should be used instead)" CREATE is used by Replicated database even for ATTACH, and anyway ATTACH creates format_version.txt as well (without previuos patch). This reverts commit 323f1f320ebe7e588d443abf78efa2c60193e7a9. Signed-off-by: Azat Khuzhin --- src/Storages/StorageMergeTree.cpp | 6 ++---- src/Storages/StorageReplicatedMergeTree.cpp | 3 --- 2 files changed, 2 insertions(+), 7 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index ead75113c6d..f9d360f02e7 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -115,14 +115,12 @@ StorageMergeTree::StorageMergeTree( , writer(*this) , merger_mutator(*this) { - if (!attach && isStaticStorage()) - throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Creating data on static storage is prohibited, use ATTACH instead"); - initializeDirectoriesAndFormatVersion(relative_data_path_, attach, date_column_name); + loadDataParts(has_force_restore_data_flag, std::nullopt); - if (!attach && !getDataPartsForInternalUsage().empty()) + if (!attach && !getDataPartsForInternalUsage().empty() && !isStaticStorage()) throw Exception(ErrorCodes::INCORRECT_DATA, "Data directory for table already containing data parts - probably " "it was unclean DROP table or manual intervention. " diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 6aa6cc60d7b..579cc5f4c30 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -338,9 +338,6 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( , replicated_fetches_throttler(std::make_shared(getSettings()->max_replicated_fetches_network_bandwidth, getContext()->getReplicatedFetchesThrottler())) , replicated_sends_throttler(std::make_shared(getSettings()->max_replicated_sends_network_bandwidth, getContext()->getReplicatedSendsThrottler())) { - if (!attach && isStaticStorage()) - throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Creating data on static storage is prohibited, use ATTACH instead"); - initializeDirectoriesAndFormatVersion(relative_data_path_, attach, date_column_name); /// We create and deactivate all tasks for consistency. /// They all will be scheduled and activated by the restarting thread. From 332924a2b48910b9d9fb2fcfab7c10c78391426f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 26 Jan 2024 15:29:42 +0100 Subject: [PATCH 219/264] tests: adjust 02980_s3_plain_DROP_TABLE tests Signed-off-by: Azat Khuzhin --- ...80_s3_plain_DROP_TABLE_MergeTree.reference | 26 +++++++++++++++++++ .../02980_s3_plain_DROP_TABLE_MergeTree.sh | 9 +++---- ...n_DROP_TABLE_ReplicatedMergeTree.reference | 26 +++++++++++++++++++ ...s3_plain_DROP_TABLE_ReplicatedMergeTree.sh | 8 +++--- 4 files changed, 60 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.reference b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.reference index d510f225161..531163e1d84 100644 --- a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.reference +++ b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.reference @@ -1,4 +1,30 @@ data after INSERT 1 data after ATTACH 1 Files before DETACH TABLE +all_1_1_0 + +backups/ordinary_default/data/ordinary_default/data/all_1_1_0: +primary.cidx +serialization.json +metadata_version.txt +default_compression_codec.txt +data.bin +data.cmrk3 +count.txt +columns.txt +checksums.txt + Files after DETACH TABLE +all_1_1_0 + +backups/ordinary_default/data/ordinary_default/data/all_1_1_0: +primary.cidx +serialization.json +metadata_version.txt +default_compression_codec.txt +data.bin +data.cmrk3 +count.txt +columns.txt +checksums.txt + diff --git a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.sh b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.sh index 2059d33ccfd..386c29704b6 100755 --- a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.sh +++ b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.sh @@ -1,6 +1,7 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-random-settings, no-random-merge-tree-settings # Tag no-fasttest: requires S3 +# Tag no-random-settings, no-random-merge-tree-settings: to avoid creating extra files like serialization.json, this test too exocit anyway CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -17,15 +18,13 @@ CLICKHOUSE_DATABASE="$new_database" $CLICKHOUSE_CLIENT -nm -q " drop table if exists data; - create table data (key Int) engine=MergeTree() order by key settings disk='s3_plain_disk'; -- { serverError TABLE_IS_READ_ONLY } - create table data (key Int) engine=MergeTree() order by key; insert into data values (1); select 'data after INSERT', count() from data; " # suppress output -$CLICKHOUSE_CLIENT -q "backup table data to S3('http://localhost:11111/test/backups/$CLICKHOUSE_DATABASE', 'test', 'testtest')" > /dev/null +$CLICKHOUSE_CLIENT -q "backup table data to S3('http://localhost:11111/test/s3_plain/backups/$CLICKHOUSE_DATABASE', 'test', 'testtest')" > /dev/null $CLICKHOUSE_CLIENT -nm -q " drop table data; @@ -33,7 +32,7 @@ $CLICKHOUSE_CLIENT -nm -q " settings max_suspicious_broken_parts=0, disk=disk(type=s3_plain, - endpoint='http://localhost:11111/test/backups/$CLICKHOUSE_DATABASE', + endpoint='http://localhost:11111/test/s3_plain/backups/$CLICKHOUSE_DATABASE', access_key_id='test', secret_access_key='testtest'); select 'data after ATTACH', count() from data; diff --git a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.reference b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.reference index d510f225161..ad9897b3381 100644 --- a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.reference +++ b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.reference @@ -1,4 +1,30 @@ data after INSERT 1 data after ATTACH 1 Files before DETACH TABLE +all_0_0_0 + +backups/ordinary_default/data/ordinary_default/data_read/all_0_0_0: +primary.cidx +serialization.json +metadata_version.txt +default_compression_codec.txt +data.bin +data.cmrk3 +count.txt +columns.txt +checksums.txt + Files after DETACH TABLE +all_0_0_0 + +backups/ordinary_default/data/ordinary_default/data_read/all_0_0_0: +primary.cidx +serialization.json +metadata_version.txt +default_compression_codec.txt +data.bin +data.cmrk3 +count.txt +columns.txt +checksums.txt + diff --git a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh index d8c92c5d6fa..f31ef518c62 100755 --- a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh +++ b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh @@ -1,6 +1,7 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-random-settings, no-random-merge-tree-settings # Tag no-fasttest: requires S3 +# Tag no-random-settings, no-random-merge-tree-settings: to avoid creating extra files like serialization.json, this test too exocit anyway CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -19,7 +20,6 @@ $CLICKHOUSE_CLIENT -nm -q " drop table if exists data_write; create table data_write (key Int) engine=ReplicatedMergeTree('/tables/{database}/data', 'write') order by key; - create table data_read (key Int) engine=ReplicatedMergeTree('/tables/{database}/data', 'read') order by key settings disk='s3_plain_disk'; -- { serverError TABLE_IS_READ_ONLY } create table data_read (key Int) engine=ReplicatedMergeTree('/tables/{database}/data', 'read') order by key; insert into data_write values (1); @@ -28,7 +28,7 @@ $CLICKHOUSE_CLIENT -nm -q " " # suppress output -$CLICKHOUSE_CLIENT -q "backup table data_read to S3('http://localhost:11111/test/backups/$CLICKHOUSE_DATABASE', 'test', 'testtest')" > /dev/null +$CLICKHOUSE_CLIENT -q "backup table data_read to S3('http://localhost:11111/test/s3_plain/backups/$CLICKHOUSE_DATABASE', 'test', 'testtest')" > /dev/null $CLICKHOUSE_CLIENT -nm -q " drop table data_read; @@ -36,7 +36,7 @@ $CLICKHOUSE_CLIENT -nm -q " settings max_suspicious_broken_parts=0, disk=disk(type=s3_plain, - endpoint='http://localhost:11111/test/backups/$CLICKHOUSE_DATABASE', + endpoint='http://localhost:11111/test/s3_plain/backups/$CLICKHOUSE_DATABASE', access_key_id='test', secret_access_key='testtest'); select 'data after ATTACH', count() from data_read; From 5227d7c9bce3d1b9cac2b8d3b8ef0c7b61a2a516 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 26 Jan 2024 11:31:44 +0100 Subject: [PATCH 220/264] Fix 02475_bson_each_row_format flakiness (due to small parsing block) CI: https://s3.amazonaws.com/clickhouse-test-reports/59170/c82050d1925439f0ede2b32acb5b1b8df4acae5d/stateless_tests__release_/run.log Signed-off-by: Azat Khuzhin --- tests/queries/0_stateless/02475_bson_each_row_format.sh | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tests/queries/0_stateless/02475_bson_each_row_format.sh b/tests/queries/0_stateless/02475_bson_each_row_format.sh index aa58d27fa50..f5c48608639 100755 --- a/tests/queries/0_stateless/02475_bson_each_row_format.sh +++ b/tests/queries/0_stateless/02475_bson_each_row_format.sh @@ -5,6 +5,12 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +# In case of parallel parsing and small block +# (--min_chunk_bytes_for_parallel_parsing) we may have multiple blocks, and +# this will break sorting order, so let's limit number of threads to avoid +# reordering. +CLICKHOUSE_CLIENT+="--allow_repeated_settings --max_threads 1" + echo "Integers" $CLICKHOUSE_CLIENT -q "insert into function file(02475_data.bsonEachRow) select number::Bool as bool, number::Int8 as int8, number::UInt8 as uint8, number::Int16 as int16, number::UInt16 as uint16, number::Int32 as int32, number::UInt32 as uint32, number::Int64 as int64, number::UInt64 as uint64 from numbers(5) settings engine_file_truncate_on_insert=1" $CLICKHOUSE_CLIENT -q "select * from file(02475_data.bsonEachRow, auto, 'bool Bool, int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64')" From 50f0a34ecb8d686d2b185351df444445ce4b8030 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 26 Jan 2024 16:59:40 +0100 Subject: [PATCH 221/264] Enable Rust on OSX Signed-off-by: Azat Khuzhin --- contrib/corrosion-cmake/CMakeLists.txt | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/contrib/corrosion-cmake/CMakeLists.txt b/contrib/corrosion-cmake/CMakeLists.txt index 8adc2c0b23a..04871c761ab 100644 --- a/contrib/corrosion-cmake/CMakeLists.txt +++ b/contrib/corrosion-cmake/CMakeLists.txt @@ -1,8 +1,5 @@ if (NOT ENABLE_LIBRARIES) set(DEFAULT_ENABLE_RUST FALSE) -elseif((CMAKE_TOOLCHAIN_FILE MATCHES "darwin") AND (CMAKE_TOOLCHAIN_FILE MATCHES "aarch64")) - message(STATUS "Rust is not available on aarch64-apple-darwin") - set(DEFAULT_ENABLE_RUST FALSE) else() list (APPEND CMAKE_MODULE_PATH "${ClickHouse_SOURCE_DIR}/contrib/corrosion/cmake") find_package(Rust) @@ -19,7 +16,9 @@ message(STATUS "Checking Rust toolchain for current target") # See https://doc.rust-lang.org/nightly/rustc/platform-support.html -if((CMAKE_TOOLCHAIN_FILE MATCHES "linux/toolchain-x86_64") AND (CMAKE_TOOLCHAIN_FILE MATCHES "musl")) +if(CMAKE_TOOLCHAIN_FILE MATCHES "ppc64le") + set(Rust_CARGO_TARGET "powerpc64le-unknown-linux-gnu") +elseif((CMAKE_TOOLCHAIN_FILE MATCHES "linux/toolchain-x86_64") AND (CMAKE_TOOLCHAIN_FILE MATCHES "musl")) set(Rust_CARGO_TARGET "x86_64-unknown-linux-musl") elseif(CMAKE_TOOLCHAIN_FILE MATCHES "linux/toolchain-x86_64") set(Rust_CARGO_TARGET "x86_64-unknown-linux-gnu") @@ -29,14 +28,14 @@ elseif(CMAKE_TOOLCHAIN_FILE MATCHES "linux/toolchain-aarch64") set(Rust_CARGO_TARGET "aarch64-unknown-linux-gnu") elseif((CMAKE_TOOLCHAIN_FILE MATCHES "darwin") AND (CMAKE_TOOLCHAIN_FILE MATCHES "x86_64")) set(Rust_CARGO_TARGET "x86_64-apple-darwin") +elseif((CMAKE_TOOLCHAIN_FILE MATCHES "darwin") AND (CMAKE_TOOLCHAIN_FILE MATCHES "darwin")) + set(Rust_CARGO_TARGET "aarch64-apple-darwin") elseif((CMAKE_TOOLCHAIN_FILE MATCHES "freebsd") AND (CMAKE_TOOLCHAIN_FILE MATCHES "x86_64")) set(Rust_CARGO_TARGET "x86_64-unknown-freebsd") elseif(CMAKE_TOOLCHAIN_FILE MATCHES "linux/toolchain-riscv64") set(Rust_CARGO_TARGET "riscv64gc-unknown-linux-gnu") -endif() - -if(CMAKE_TOOLCHAIN_FILE MATCHES "ppc64le") - set(Rust_CARGO_TARGET "powerpc64le-unknown-linux-gnu") +else() + message(FATAL_ERROR "Unsupported rust target") endif() message(STATUS "Switched Rust target to ${Rust_CARGO_TARGET}") From 6795d0207f98657e499c66871678c23412ac04ba Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 26 Jan 2024 16:37:54 +0000 Subject: [PATCH 222/264] Update test_reload_after_fail_in_cache_dictionary for analyzer --- tests/analyzer_integration_broken_tests.txt | 1 - tests/integration/test_dictionaries_update_and_reload/test.py | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/analyzer_integration_broken_tests.txt b/tests/analyzer_integration_broken_tests.txt index 23f22209451..e1d4de59a23 100644 --- a/tests/analyzer_integration_broken_tests.txt +++ b/tests/analyzer_integration_broken_tests.txt @@ -1,7 +1,6 @@ test_access_for_functions/test.py::test_access_rights_for_function test_build_sets_from_multiple_threads/test.py::test_set test_concurrent_backups_s3/test.py::test_concurrent_backups -test_dictionaries_update_and_reload/test.py::test_reload_after_fail_in_cache_dictionary test_distributed_backward_compatability/test.py::test_distributed_in_tuple test_distributed_type_object/test.py::test_distributed_type_object test_executable_table_function/test.py::test_executable_function_input_python diff --git a/tests/integration/test_dictionaries_update_and_reload/test.py b/tests/integration/test_dictionaries_update_and_reload/test.py index 3d96d0b8dd4..648ea847afb 100644 --- a/tests/integration/test_dictionaries_update_and_reload/test.py +++ b/tests/integration/test_dictionaries_update_and_reload/test.py @@ -281,7 +281,7 @@ def test_reload_after_fail_in_cache_dictionary(started_cluster): query_and_get_error = instance.query_and_get_error # Can't get a value from the cache dictionary because the source (table `test.xypairs`) doesn't respond. - expected_error = "Table test.xypairs does not exist" + expected_error = "UNKNOWN_TABLE" update_error = "Could not update cache dictionary cache_xypairs now" assert expected_error in query_and_get_error( "SELECT dictGetUInt64('cache_xypairs', 'y', toUInt64(1))" From 8fcedddcda46f59b8ddf701105b8f8cabff60ba7 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 26 Jan 2024 17:46:49 +0100 Subject: [PATCH 223/264] Fix abort in iceberg metadata on bad file paths --- src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp b/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp index e01a9a831c0..df1536f53fc 100644 --- a/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp +++ b/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp @@ -596,10 +596,11 @@ Strings IcebergMetadata::getDataFiles() const auto status = status_int_column->getInt(i); const auto data_path = std::string(file_path_string_column->getDataAt(i).toView()); const auto pos = data_path.find(configuration.url.key); - const auto file_path = data_path.substr(pos); if (pos == std::string::npos) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected to find {} in data path: {}", configuration.url.key, data_path); + const auto file_path = data_path.substr(pos); + if (ManifestEntryStatus(status) == ManifestEntryStatus::DELETED) { LOG_TEST(log, "Processing delete file for path: {}", file_path); From 163a3e44d38b249155f4c056e6388b0cffe26153 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 26 Jan 2024 21:22:39 +0100 Subject: [PATCH 224/264] Update run.sh --- docker/test/stateless/run.sh | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index b7c04a6fabd..8eeb01449b4 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -246,16 +246,16 @@ clickhouse-client -q "system flush logs" ||: stop_logs_replication # Try to get logs while server is running -successfuly_saved=0 +failed_to_save_logs=0 for table in query_log zookeeper_log trace_log transactions_info_log metric_log do - clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst - successfuly_saved=$? + error=$( { clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst } 2>&1 ) + [[ "0" != "${#err}" ]] && failed_to_save_logs=1 if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then - clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst - successfuly_saved=$((successfuly_saved | $?)) - clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst - successfuly_saved=$((successfuly_saved | $?)) + error=$( { clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst } 2>&1 ) + [[ "0" != "${#err}" ]] && failed_to_save_logs=1 + error=$( { clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst } 2>&1 ) + [[ "0" != "${#err}" ]] && failed_to_save_logs=1 fi done @@ -280,7 +280,7 @@ fi # If server crashed dump system logs with clickhouse-local -if [ $successfuly_saved -ne 0 ]; then +if [ $failed_to_save_logs -ne 0 ]; then # Compress tables. # # NOTE: From 4273ae4f4ff401691acf3712d862336e1bf6cfb6 Mon Sep 17 00:00:00 2001 From: Peignon Melvyn Date: Fri, 26 Jan 2024 21:27:20 +0100 Subject: [PATCH 225/264] Update view.md --- docs/en/sql-reference/statements/alter/view.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/view.md b/docs/en/sql-reference/statements/alter/view.md index 517e64e3e5b..59045afdeb6 100644 --- a/docs/en/sql-reference/statements/alter/view.md +++ b/docs/en/sql-reference/statements/alter/view.md @@ -8,8 +8,6 @@ sidebar_label: VIEW You can modify `SELECT` query that was specified when a [materialized view](../create/view.md#materialized) was created with the `ALTER TABLE … MODIFY QUERY` statement without interrupting ingestion process. -The `allow_experimental_alter_materialized_view_structure` setting must be enabled. - This command is created to change materialized view created with `TO [db.]name` clause. It does not change the structure of the underling storage table and it does not change the columns' definition of the materialized view, because of this the application of this command is very limited for materialized views are created without `TO [db.]name` clause. **Example with TO table** From 1e61521dccc055890e5e329b2d512d6d21e016b4 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 26 Jan 2024 23:33:04 +0100 Subject: [PATCH 226/264] Add missing setting optimize_injective_functions_in_group_by to SettingsChangesHistory --- src/Core/SettingsChangesHistory.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 859ba99b5f7..dff0ebb759c 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -99,7 +99,8 @@ static std::map sett {"output_format_pretty_color", true, "auto", "Setting is changed to allow also for auto value, disabling ANSI escapes if output is not a tty"}, {"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"}, {"max_estimated_execution_time", 0, 0, "Separate max_execution_time and max_estimated_execution_time"}, - {"iceberg_engine_ignore_schema_evolution", false, false, "Allow to ignore schema evolution in Iceberg table engine"}}}, + {"iceberg_engine_ignore_schema_evolution", false, false, "Allow to ignore schema evolution in Iceberg table engine"}, + {"optimize_injective_functions_in_group_by", false, true, "Replace injective functions by it's arguments in GROUP BY section in analyzer"}}}, {"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}, {"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, From 44e42052b184834466970a7d95a3943eba572dec Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 27 Jan 2024 13:12:21 +0100 Subject: [PATCH 227/264] Fix perf tests after sumMap starts to filter out -0. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Before perf tests was relying on the following: SELECT sumMap(['foo', 'bar'], [-0., -0.]) ┌─sumMap(['foo', 'bar'], [-0., -0.])─┐ │ (['bar','foo'],[-0,-0]) │ └────────────────────────────────────┘ While it got changed, and now: ┌─sumMap(['foo', 'bar'], [-0., -0.])─┐ │ ([],[]) │ └────────────────────────────────────┘ But it works for nan: SELECT sumMap(['foo', 'bar'], [nan, nan]) ┌─sumMap(['foo', 'bar'], [nan, nan])─┐ │ (['bar','foo'],[nan,nan]) │ └────────────────────────────────────┘ Signed-off-by: Azat Khuzhin --- tests/performance/scripts/compare.sh | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/performance/scripts/compare.sh b/tests/performance/scripts/compare.sh index 7dc522dca7a..92ba383f965 100755 --- a/tests/performance/scripts/compare.sh +++ b/tests/performance/scripts/compare.sh @@ -444,10 +444,10 @@ create view query_logs as create table query_run_metric_arrays engine File(TSV, 'analyze/query-run-metric-arrays.tsv') as with ( - -- sumMapState with the list of all keys with '-0.' values. Negative zero is because - -- sumMap removes keys with positive zeros. + -- sumMapState with the list of all keys with 'nan' values. 'nan' is because + -- sumMap removes keys with positive/negative zeros. with (select groupUniqArrayArray(mapKeys(ProfileEvents)) from query_logs) as all_names - select arrayReduce('sumMapState', [(all_names, arrayMap(x->-0., all_names))]) + select arrayReduce('sumMapState', [(all_names, arrayMap(x->nan, all_names))]) ) as all_metrics select test, query_index, version, query_id, (finalizeAggregation( @@ -460,13 +460,13 @@ create table query_run_metric_arrays engine File(TSV, 'analyze/query-run-metric- ), arrayReduce('sumMapState', [( ['client_time', 'server_time', 'memory_usage'], - arrayMap(x->if(x != 0., x, -0.), [ + arrayMap(x->if(x != 0., x, nan), [ toFloat64(query_runs.time), toFloat64(query_duration_ms / 1000.), toFloat64(memory_usage)]))]) ] )) as metrics_tuple).1 metric_names, - metrics_tuple.2 metric_values + arrayMap(x->if(isNaN(x),0,x), metrics_tuple.2) metric_values from query_logs right join query_runs on query_logs.query_id = query_runs.query_id From 5d62cdd58cb6f4988742bf40c1a27c318aeb6172 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 25 Jan 2024 21:05:02 +0300 Subject: [PATCH 228/264] Added setting update_insert_deduplication_token_in_dependent_materialized_views --- src/Core/Settings.h | 1 + .../Transforms/buildPushingToViewsChain.cpp | 25 +++++ ...ation_token_hierarchical_inserts.reference | 9 ++ ...duplication_token_hierarchical_inserts.sql | 103 ++++++++++++++++++ 4 files changed, 138 insertions(+) create mode 100644 tests/queries/0_stateless/02972_insert_deduplication_token_hierarchical_inserts.reference create mode 100644 tests/queries/0_stateless/02972_insert_deduplication_token_hierarchical_inserts.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e0b3ca39899..a09f2d2331d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -588,6 +588,7 @@ class IColumn; M(Bool, normalize_function_names, true, "Normalize function names to their canonical names", 0) \ M(Bool, enable_early_constant_folding, true, "Enable query optimization where we analyze function and subqueries results and rewrite query if there're constants there", 0) \ M(Bool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views if the block is not a duplicate for the table. Use true to always deduplicate in dependent tables.", 0) \ + M(Bool, update_insert_deduplication_token_in_dependent_materialized_views, false, "Should update insert deduplication token with table identifier during insert in dependent materialized views.", 0) \ M(Bool, materialized_views_ignore_errors, false, "Allows to ignore errors for MATERIALIZED VIEW, and deliver original block to the table regardless of MVs", 0) \ M(Bool, ignore_materialized_views_with_dropped_target_table, false, "Ignore MVs with dropped taraget table during pushing to views", 0) \ M(Bool, allow_experimental_refreshable_materialized_view, false, "Allow refreshable materialized views (CREATE MATERIALIZED VIEW REFRESH ...).", 0) \ diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 960cc019001..0cbbf4ef482 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -244,7 +244,32 @@ Chain buildPushingToViewsChain( // Do not deduplicate insertions into MV if the main insertion is Ok if (disable_deduplication_for_children) + { insert_context->setSetting("insert_deduplicate", Field{false}); + } + else if (insert_settings.update_insert_deduplication_token_in_dependent_materialized_views) + { + /** Update deduplication token passed to dependent MV with current table id. So it is possible to properly handle + * deduplication in complex INSERT flows. + * + * Example: + * + * landing -┬--> mv_1_1 ---> ds_1_1 ---> mv_2_1 --┬-> ds_2_1 ---> mv_3_1 ---> ds_3_1 + * | | + * └--> mv_1_2 ---> ds_1_2 ---> mv_2_2 --┘ + * + * Here we want to avoid deduplication for two different blocks generated from `mv_2_1` and `mv_2_2` that will + * be inserted into `ds_2_1`. + */ + auto insert_deduplication_token = insert_settings.insert_deduplication_token.toString(); + + if (table_id.hasUUID()) + insert_deduplication_token += "_" + toString(table_id.uuid); + else + insert_deduplication_token += "_" + table_id.getFullNameNotQuoted(); + + insert_context->setSetting("insert_deduplication_token", insert_deduplication_token); + } // Processing of blocks for MVs is done block by block, and there will // be no parallel reading after (plus it is not a costless operation) diff --git a/tests/queries/0_stateless/02972_insert_deduplication_token_hierarchical_inserts.reference b/tests/queries/0_stateless/02972_insert_deduplication_token_hierarchical_inserts.reference new file mode 100644 index 00000000000..a5a9a5539ba --- /dev/null +++ b/tests/queries/0_stateless/02972_insert_deduplication_token_hierarchical_inserts.reference @@ -0,0 +1,9 @@ +0 +landing all_1_1_0 0 +ds_1_2 all_1_1_0 0 +ds_1_1 all_1_1_0 0 +ds_2_1 all_1_1_0 0 +ds_2_1 all_2_2_0 0 +ds_3_1 all_1_1_0 0 +ds_3_1 all_2_2_0 0 +10 diff --git a/tests/queries/0_stateless/02972_insert_deduplication_token_hierarchical_inserts.sql b/tests/queries/0_stateless/02972_insert_deduplication_token_hierarchical_inserts.sql new file mode 100644 index 00000000000..4208543c284 --- /dev/null +++ b/tests/queries/0_stateless/02972_insert_deduplication_token_hierarchical_inserts.sql @@ -0,0 +1,103 @@ +SET insert_deduplicate = 1; +SET deduplicate_blocks_in_dependent_materialized_views = 1; +SET update_insert_deduplication_token_in_dependent_materialized_views = 1; +SET insert_deduplication_token = 'test'; + +DROP TABLE IF EXISTS landing; +CREATE TABLE landing +( + timestamp UInt64, + value UInt64 +) +ENGINE = MergeTree ORDER BY tuple() SETTINGS non_replicated_deduplication_window = 1000; + +DROP TABLE IF EXISTS ds_1_1; +CREATE TABLE ds_1_1 +( + t UInt64, + v UInt64 +) +ENGINE = MergeTree ORDER BY tuple() SETTINGS non_replicated_deduplication_window = 1000; + +DROP VIEW IF EXISTS mv_1_1; +CREATE MATERIALIZED VIEW mv_1_1 TO ds_1_1 as +SELECT + timestamp t, sum(value) v +FROM landing +GROUP BY t; + +DROP TABLE IF EXISTS ds_1_2; +CREATE TABLE ds_1_2 +( + t UInt64, + v UInt64 +) +ENGINE = MergeTree ORDER BY tuple() SETTINGS non_replicated_deduplication_window = 1000; + +DROP VIEW IF EXISTS mv_1_2; +CREATE MATERIALIZED VIEW mv_1_2 TO ds_1_2 as +SELECT + timestamp t, sum(value) v +FROM landing +GROUP BY t; + +DROP TABLE IF EXISTS ds_2_1; +CREATE TABLE ds_2_1 +( + l String, + t DateTime, + v UInt64 +) +ENGINE = MergeTree ORDER BY tuple() SETTINGS non_replicated_deduplication_window = 1000; + +DROP VIEW IF EXISTS mv_2_1; +CREATE MATERIALIZED VIEW mv_2_1 TO ds_2_1 as +SELECT '2_1' l, t, v +FROM ds_1_1; + +DROP VIEW IF EXISTS mv_2_2; +CREATE MATERIALIZED VIEW mv_2_2 TO ds_2_1 as +SELECT '2_2' l, t, v +FROM ds_1_2; + +DROP TABLE IF EXISTS ds_3_1; +CREATE TABLE ds_3_1 +( + l String, + t DateTime, + v UInt64 +) +ENGINE = MergeTree ORDER BY tuple() SETTINGS non_replicated_deduplication_window = 1000; + +DROP VIEW IF EXISTS mv_3_1; +CREATE MATERIALIZED VIEW mv_3_1 TO ds_3_1 as +SELECT '3_1' l, t, v +FROM ds_2_1; + +INSERT INTO landing SELECT 1 as timestamp, 1 AS value FROM numbers(10); + +SELECT sleep(3); + +INSERT INTO landing SELECT 1 as timestamp, 1 AS value FROM numbers(10); + +SYSTEM FLUSH LOGS; +SELECT table, name, error FROM system.part_log +WHERE database = currentDatabase() +ORDER BY event_time; + +SELECT count() FROM landing; + +DROP TABLE landing; + +DROP TABLE ds_1_1; +DROP VIEW mv_1_1; + +DROP TABLE ds_1_2; +DROP VIEW mv_1_2; + +DROP TABLE ds_2_1; +DROP VIEW mv_2_1; +DROP VIEW mv_2_2; + +DROP TABLE ds_3_1; +DROP VIEW mv_3_1; From 4ae425461fa7c3fb6b9a32d4f209ce558322497f Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 25 Jan 2024 21:22:48 +0300 Subject: [PATCH 229/264] Added documentation --- docs/en/operations/settings/settings.md | 28 ++++++++++++++++++++++++- 1 file changed, 27 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 75d05d55366..3fd409e2d68 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1991,6 +1991,32 @@ If an INSERTed block is skipped due to deduplication in the source table, there At the same time, this behaviour “breaks” `INSERT` idempotency. If an `INSERT` into the main table was successful and `INSERT` into a materialized view failed (e.g. because of communication failure with ClickHouse Keeper) a client will get an error and can retry the operation. However, the materialized view won’t receive the second insert because it will be discarded by deduplication in the main (source) table. The setting `deduplicate_blocks_in_dependent_materialized_views` allows for changing this behaviour. On retry, a materialized view will receive the repeat insert and will perform a deduplication check by itself, ignoring check result for the source table, and will insert rows lost because of the first failure. +## update_insert_deduplication_token_in_dependent_materialized_views {#update-insert-deduplication-token-in-dependent-materialized-views} + +Allows to update insert deduplication token with table identifier during insert in dependent materialized views. + +Possible values: + + 0 — Disabled. + 1 — Enabled. + +Default value: 0. + +Usage: + +If setting `update_insert_deduplication_token_in_dependent_materialized_views` is enabled, `insert_deduplication_token` is passed to dependent materialized views. But in complex INSERT flows it is possible that we want to avoid deduplication for dependent materialized views. + +Example: +``` +landing -┬--> mv_1_1 ---> ds_1_1 ---> mv_2_1 --┬-> ds_2_1 ---> mv_3_1 ---> ds_3_1 + | | + └--> mv_1_2 ---> ds_1_2 ---> mv_2_2 --┘ +``` + +In this example we want to avoid deduplication for two different blocks generated from `mv_2_1` and `mv_2_2` that will be inserted into `ds_2_1`. Without `update_insert_deduplication_token_in_dependent_materialized_views` setting, those two different blocks will be deduplicated, because different blocks from `mv_2_1` and `mv_2_2` will have the same `insert_deduplication_token`. + +If setting `update_insert_deduplication_token_in_dependent_materialized_views` is enabled, during each insert into dependent materialized views `insert_deduplication_token` is updated with table identifier, so block from `mv_2_1` and block from `mv_2_2` will have different `insert_deduplication_token` and will not be deduplicated. + ## insert_deduplication_token {#insert_deduplication_token} The setting allows a user to provide own deduplication semantic in MergeTree/ReplicatedMergeTree @@ -5165,7 +5191,7 @@ SETTINGS(dictionary_use_async_executor=1, max_threads=8); ## storage_metadata_write_full_object_key {#storage_metadata_write_full_object_key} When set to `true` the metadata files are written with `VERSION_FULL_OBJECT_KEY` format version. With that format full object storage key names are written to the metadata files. -When set to `false` the metadata files are written with the previous format version, `VERSION_INLINE_DATA`. With that format only suffixes of object storage key names are are written to the metadata files. The prefix for all of object storage key names is set in configurations files at `storage_configuration.disks` section. +When set to `false` the metadata files are written with the previous format version, `VERSION_INLINE_DATA`. With that format only suffixes of object storage key names are are written to the metadata files. The prefix for all of object storage key names is set in configurations files at `storage_configuration.disks` section. Default value: `false`. From 4a10b7125298b9133ec66b2654138ed459010d33 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 26 Jan 2024 13:49:32 +0300 Subject: [PATCH 230/264] Updated documentation --- docs/en/operations/settings/settings.md | 52 ++++++++++++------------- 1 file changed, 26 insertions(+), 26 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 3fd409e2d68..5f197990f4d 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1991,32 +1991,6 @@ If an INSERTed block is skipped due to deduplication in the source table, there At the same time, this behaviour “breaks” `INSERT` idempotency. If an `INSERT` into the main table was successful and `INSERT` into a materialized view failed (e.g. because of communication failure with ClickHouse Keeper) a client will get an error and can retry the operation. However, the materialized view won’t receive the second insert because it will be discarded by deduplication in the main (source) table. The setting `deduplicate_blocks_in_dependent_materialized_views` allows for changing this behaviour. On retry, a materialized view will receive the repeat insert and will perform a deduplication check by itself, ignoring check result for the source table, and will insert rows lost because of the first failure. -## update_insert_deduplication_token_in_dependent_materialized_views {#update-insert-deduplication-token-in-dependent-materialized-views} - -Allows to update insert deduplication token with table identifier during insert in dependent materialized views. - -Possible values: - - 0 — Disabled. - 1 — Enabled. - -Default value: 0. - -Usage: - -If setting `update_insert_deduplication_token_in_dependent_materialized_views` is enabled, `insert_deduplication_token` is passed to dependent materialized views. But in complex INSERT flows it is possible that we want to avoid deduplication for dependent materialized views. - -Example: -``` -landing -┬--> mv_1_1 ---> ds_1_1 ---> mv_2_1 --┬-> ds_2_1 ---> mv_3_1 ---> ds_3_1 - | | - └--> mv_1_2 ---> ds_1_2 ---> mv_2_2 --┘ -``` - -In this example we want to avoid deduplication for two different blocks generated from `mv_2_1` and `mv_2_2` that will be inserted into `ds_2_1`. Without `update_insert_deduplication_token_in_dependent_materialized_views` setting, those two different blocks will be deduplicated, because different blocks from `mv_2_1` and `mv_2_2` will have the same `insert_deduplication_token`. - -If setting `update_insert_deduplication_token_in_dependent_materialized_views` is enabled, during each insert into dependent materialized views `insert_deduplication_token` is updated with table identifier, so block from `mv_2_1` and block from `mv_2_2` will have different `insert_deduplication_token` and will not be deduplicated. - ## insert_deduplication_token {#insert_deduplication_token} The setting allows a user to provide own deduplication semantic in MergeTree/ReplicatedMergeTree @@ -2066,6 +2040,32 @@ SELECT * FROM test_table └───┘ ``` +## update_insert_deduplication_token_in_dependent_materialized_views {#update-insert-deduplication-token-in-dependent-materialized-views} + +Allows to update `insert_deduplication_token` with table identifier during insert in dependent materialized views, if setting `deduplicate_blocks_in_dependent_materialized_views` is enabled and `insert_deduplication_token` is set. + +Possible values: + + 0 — Disabled. + 1 — Enabled. + +Default value: 0. + +Usage: + +If setting `deduplicate_blocks_in_dependent_materialized_views` is enabled, `insert_deduplication_token` is passed to dependent materialized views. But in complex INSERT flows it is possible that we want to avoid deduplication for dependent materialized views. + +Example: +``` +landing -┬--> mv_1_1 ---> ds_1_1 ---> mv_2_1 --┬-> ds_2_1 ---> mv_3_1 ---> ds_3_1 + | | + └--> mv_1_2 ---> ds_1_2 ---> mv_2_2 --┘ +``` + +In this example we want to avoid deduplication for two different blocks generated from `mv_2_1` and `mv_2_2` that will be inserted into `ds_2_1`. Without `update_insert_deduplication_token_in_dependent_materialized_views` setting enabled, those two different blocks will be deduplicated, because different blocks from `mv_2_1` and `mv_2_2` will have the same `insert_deduplication_token`. + +If setting `update_insert_deduplication_token_in_dependent_materialized_views` is enabled, during each insert into dependent materialized views `insert_deduplication_token` is updated with table identifier, so block from `mv_2_1` and block from `mv_2_2` will have different `insert_deduplication_token` and will not be deduplicated. + ## insert_keeper_max_retries The setting sets the maximum number of retries for ClickHouse Keeper (or ZooKeeper) requests during insert into replicated MergeTree. Only Keeper requests which failed due to network error, Keeper session timeout, or request timeout are considered for retries. From 0f46200d5aaba14cbfbb2e923826b1394ebc4d2c Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 26 Jan 2024 20:09:22 +0300 Subject: [PATCH 231/264] Fixed tests --- src/Processors/Transforms/buildPushingToViewsChain.cpp | 5 +++-- ...insert_deduplication_token_hierarchical_inserts.reference | 4 ++-- ...02972_insert_deduplication_token_hierarchical_inserts.sql | 2 +- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 0cbbf4ef482..8ddc3ab0c61 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -247,7 +247,8 @@ Chain buildPushingToViewsChain( { insert_context->setSetting("insert_deduplicate", Field{false}); } - else if (insert_settings.update_insert_deduplication_token_in_dependent_materialized_views) + else if (insert_settings.update_insert_deduplication_token_in_dependent_materialized_views && + !insert_settings.insert_deduplication_token.value.empty()) { /** Update deduplication token passed to dependent MV with current table id. So it is possible to properly handle * deduplication in complex INSERT flows. @@ -261,7 +262,7 @@ Chain buildPushingToViewsChain( * Here we want to avoid deduplication for two different blocks generated from `mv_2_1` and `mv_2_2` that will * be inserted into `ds_2_1`. */ - auto insert_deduplication_token = insert_settings.insert_deduplication_token.toString(); + auto insert_deduplication_token = insert_settings.insert_deduplication_token.value; if (table_id.hasUUID()) insert_deduplication_token += "_" + toString(table_id.uuid); diff --git a/tests/queries/0_stateless/02972_insert_deduplication_token_hierarchical_inserts.reference b/tests/queries/0_stateless/02972_insert_deduplication_token_hierarchical_inserts.reference index a5a9a5539ba..71c9053d644 100644 --- a/tests/queries/0_stateless/02972_insert_deduplication_token_hierarchical_inserts.reference +++ b/tests/queries/0_stateless/02972_insert_deduplication_token_hierarchical_inserts.reference @@ -1,9 +1,9 @@ 0 -landing all_1_1_0 0 -ds_1_2 all_1_1_0 0 ds_1_1 all_1_1_0 0 +ds_1_2 all_1_1_0 0 ds_2_1 all_1_1_0 0 ds_2_1 all_2_2_0 0 ds_3_1 all_1_1_0 0 ds_3_1 all_2_2_0 0 +landing all_1_1_0 0 10 diff --git a/tests/queries/0_stateless/02972_insert_deduplication_token_hierarchical_inserts.sql b/tests/queries/0_stateless/02972_insert_deduplication_token_hierarchical_inserts.sql index 4208543c284..2e982cde738 100644 --- a/tests/queries/0_stateless/02972_insert_deduplication_token_hierarchical_inserts.sql +++ b/tests/queries/0_stateless/02972_insert_deduplication_token_hierarchical_inserts.sql @@ -83,7 +83,7 @@ INSERT INTO landing SELECT 1 as timestamp, 1 AS value FROM numbers(10); SYSTEM FLUSH LOGS; SELECT table, name, error FROM system.part_log WHERE database = currentDatabase() -ORDER BY event_time; +ORDER BY event_time, table, name; SELECT count() FROM landing; From 7d48d0da72af35a2f63d5b6fcf53303e15c5c330 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 27 Jan 2024 15:57:33 +0300 Subject: [PATCH 232/264] Fixed tests --- .../02972_insert_deduplication_token_hierarchical_inserts.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02972_insert_deduplication_token_hierarchical_inserts.sql b/tests/queries/0_stateless/02972_insert_deduplication_token_hierarchical_inserts.sql index 2e982cde738..242133e9122 100644 --- a/tests/queries/0_stateless/02972_insert_deduplication_token_hierarchical_inserts.sql +++ b/tests/queries/0_stateless/02972_insert_deduplication_token_hierarchical_inserts.sql @@ -83,7 +83,7 @@ INSERT INTO landing SELECT 1 as timestamp, 1 AS value FROM numbers(10); SYSTEM FLUSH LOGS; SELECT table, name, error FROM system.part_log WHERE database = currentDatabase() -ORDER BY event_time, table, name; +ORDER BY table, name; SELECT count() FROM landing; From 71cd1efd8bdf0d1265f578575217fb8b5404123f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 27 Jan 2024 15:17:40 +0100 Subject: [PATCH 233/264] tests: fix 02980_s3_plain_DROP_TABLE_ReplicatedMergeTree flakiness In case of commit failed (likely due to fault injection in tests) part name will have different name, not all_0_0_0, so replace numbers with X. Signed-off-by: Azat Khuzhin --- ...2980_s3_plain_DROP_TABLE_ReplicatedMergeTree.reference | 8 ++++---- .../02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh | 5 +++-- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.reference b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.reference index ad9897b3381..1e191b719a5 100644 --- a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.reference +++ b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.reference @@ -1,9 +1,9 @@ data after INSERT 1 data after ATTACH 1 Files before DETACH TABLE -all_0_0_0 +all_X_X_X -backups/ordinary_default/data/ordinary_default/data_read/all_0_0_0: +backups/ordinary_default/data/ordinary_default/data_read/all_X_X_X: primary.cidx serialization.json metadata_version.txt @@ -15,9 +15,9 @@ columns.txt checksums.txt Files after DETACH TABLE -all_0_0_0 +all_X_X_X -backups/ordinary_default/data/ordinary_default/data_read/all_0_0_0: +backups/ordinary_default/data/ordinary_default/data_read/all_X_X_X: primary.cidx serialization.json metadata_version.txt diff --git a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh index f31ef518c62..bf20247c7aa 100755 --- a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh +++ b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh @@ -51,14 +51,15 @@ path=$($CLICKHOUSE_CLIENT -q "SELECT replace(data_paths[1], 's3_plain', '') FROM path=${path%/} echo "Files before DETACH TABLE" -clickhouse-disks -C "$config" --disk s3_plain_disk list --recursive "${path:?}" | tail -n+2 +# sed to match any part, since in case of fault injection part name may not be all_0_0_0 but all_1_1_0 +clickhouse-disks -C "$config" --disk s3_plain_disk list --recursive "${path:?}" | tail -n+2 | sed 's/all_[^_]*_[^_]*_0/all_X_X_X/g' $CLICKHOUSE_CLIENT -nm -q " detach table data_read; detach table data_write; " echo "Files after DETACH TABLE" -clickhouse-disks -C "$config" --disk s3_plain_disk list --recursive "$path" | tail -n+2 +clickhouse-disks -C "$config" --disk s3_plain_disk list --recursive "$path" | tail -n+2 | sed 's/all_[^_]*_[^_]*_0/all_X_X_X/g' # metadata file is left $CLICKHOUSE_CLIENT --force_remove_data_recursively_on_drop=1 -q "drop database if exists $CLICKHOUSE_DATABASE" From d94547d338897ef6a2bb53a65946b53fab7358d0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 27 Jan 2024 16:00:49 +0100 Subject: [PATCH 234/264] Use fresh ZooKeeper client on DROP (to have higher chances on success) In case of DROP the client can be expired, and even though StorageReplicatedMergeTree::drop() uses getZooKeeperIfTableShutDown(), which creates new client if current is expired, it is not enough, since current_zookeeper (cached client) will be used implicitly from dropAllData(). This had been found by stress tests, that leads to DROP query hang [1]. [1]: https://s3.amazonaws.com/clickhouse-test-reports/59255/94eb33ef27a9ab7c4a99af40772ea287e67efcbf/stress_test__tsan_.html
stacktrace 2024.01.26 17:41:41.138577 [ 3319 ] {} DatabaseCatalog: Cannot drop table test_vzfk1xoc.alter_table1 (455a674c-161d-44d8-abc2-fd205bad1116). Will retry later.: Code: 999. Coordination::Exception: Session expired. (KEEPER_EXCEPTION), Stack trace (when copying this message, always include the lines below): 0. ./contrib/llvm-project/libcxx/include/exception:134: Poco::Exception::Exception(String const&, int) @ 0x000000001e06a5e3 in /usr/bin/clickhouse 1. ./build_docker/./src/Common/Exception.cpp:96: DB::Exception::Exception(DB::Exception::MessageMasked&&, int, bool) @ 0x000000000f697c74 in /usr/bin/clickhouse 2. ./src/Common/Exception.h:0: Coordination::Exception::Exception(char const (&) [16], Coordination::Error) @ 0x000000001adc298b in /usr/bin/clickhouse 3. ./src/Common/ZooKeeper/IKeeper.h:0: Coordination::ZooKeeper::pushRequest(Coordination::ZooKeeper::RequestInfo&&) @ 0x000000001adfb418 in /usr/bin/clickhouse 4. ./build_docker/./src/Common/ZooKeeper/ZooKeeperImpl.cpp:1343: Coordination::ZooKeeper::get(String const&, std::function, std::shared_ptr>) @ 0x000000001adfd8e5 in /usr/bin/clickhouse 5. ./contrib/llvm-project/libcxx/include/__functional/function.h:818: ? @ 0x000000001ad890bd in /usr/bin/clickhouse 6. ./contrib/llvm-project/libcxx/include/__functional/function.h:818: ? @ 0x000000001ad88a36 in /usr/bin/clickhouse 7. ./build_docker/./src/Common/ZooKeeper/ZooKeeper.cpp:580: zkutil::ZooKeeper::tryGetWatch(String const&, String&, Coordination::Stat*, std::function, Coordination::Error*) @ 0x000000001ad898bf in /usr/bin/clickhouse 8. ./build_docker/./src/Common/ZooKeeper/ZooKeeper.cpp:570: zkutil::ZooKeeper::tryGet(String const&, String&, Coordination::Stat*, std::shared_ptr const&, Coordination::Error*) @ 0x000000001ad89554 in /usr/bin/clickhouse 9. ./build_docker/./src/Common/ZooKeeper/ZooKeeperWithFaultInjection.cpp:0: DB::ZooKeeperWithFaultInjection::tryGet(String const&, String&, Coordination::Stat*, std::shared_ptr const&, Coordination::Error*) @ 0x000000001ae110dc in /usr/bin/clickhouse 10. ./contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:701: DB::StorageReplicatedMergeTree::unlockSharedDataByID(String, String const&, DB::MergeTreePartInfo const&, String const&, String const&, std::shared_ptr const&, DB::MergeTreeSettings const&, std::shared_ptr, String const&, StrongTypedef) @ 0x000000001975527e in /usr/bin/clickhouse 11. ./build_docker/./src/Storages/StorageReplicatedMergeTree.cpp:0: DB::StorageReplicatedMergeTree::removeSharedDetachedPart(std::shared_ptr, String const&, String const&, String const&, String const&, String const&, std::shared_ptr const&, std::shared_ptr const&) @ 0x000000001976c980 in /usr/bin/clickhouse 12. ./build_docker/./src/Storages/StorageReplicatedMergeTree.cpp:10171: DB::StorageReplicatedMergeTree::removeDetachedPart(std::shared_ptr, String const&, String const&) @ 0x000000001976bdc6 in /usr/bin/clickhouse 13. ./build_docker/./src/Storages/MergeTree/MergeTreeData.cpp:2798: DB::MergeTreeData::dropAllData() @ 0x0000000019c643c8 in /usr/bin/clickhouse 14. ./build_docker/./src/Storages/StorageReplicatedMergeTree.cpp:1117: DB::StorageReplicatedMergeTree::drop() @ 0x0000000019664e0a in /usr/bin/clickhouse 15. ./build_docker/./src/Interpreters/DatabaseCatalog.cpp:0: DB::DatabaseCatalog::dropTableFinally(DB::DatabaseCatalog::TableMarkedAsDropped const&) @ 0x0000000017bf1fac in /usr/bin/clickhouse 16. ./build_docker/./src/Interpreters/DatabaseCatalog.cpp:0: DB::DatabaseCatalog::dropTableDataTask() @ 0x0000000017bf175e in /usr/bin/clickhouse 17. ./contrib/llvm-project/libcxx/include/__functional/function.h:717: ? @ 0x0000000017bf94c2 in /usr/bin/clickhouse 18. ./contrib/llvm-project/libcxx/include/__functional/function.h:0: ? @ 0x00000000170bd0b8 in /usr/bin/clickhouse 19. ./contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:701: DB::BackgroundSchedulePool::threadFunction() @ 0x00000000170c06ce in /usr/bin/clickhouse 20. ./build_docker/./src/Core/BackgroundSchedulePool.cpp:0: void std::__function::__policy_invoker::__call_impl::ThreadFromGlobalPoolImpl, StrongTypedef, char const*)::$_0>(DB::BackgroundSchedulePool::BackgroundSchedulePool(unsigned long, StrongTypedef, StrongTypedef, char const*)::$_0&&)::'lambda'(), void ()>>(std::__function::__policy_storage const*) @ 0x00000000170c0feb in /usr/bin/clickhouse 21. ./base/base/../base/wide_integer_impl.h:809: ThreadPoolImpl::worker(std::__list_iterator) @ 0x000000000f7c6a2f in /usr/bin/clickhouse 22. ./contrib/llvm-project/libcxx/include/__memory/unique_ptr.h:302: void* std::__thread_proxy[abi:v15000]>, void ThreadPoolImpl::scheduleImpl(std::function, Priority, std::optional, bool)::'lambda0'()>>(void*) @ 0x000000000f7cc8f2 in /usr/bin/clickhouse 23. ? @ 0x000000000733904f in /usr/bin/clickhouse 24. ? @ 0x00007f28b2319ac3 25. ? @ 0x00007f28b23ab850
Signed-off-by: Azat Khuzhin --- src/Storages/StorageReplicatedMergeTree.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c82721d2a18..695b78a10db 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1096,6 +1096,8 @@ void StorageReplicatedMergeTree::drop() /// Table can be shut down, restarting thread is not active /// and calling StorageReplicatedMergeTree::getZooKeeper()/getAuxiliaryZooKeeper() won't suffice. zookeeper = getZooKeeperIfTableShutDown(); + /// Update zookeeper client, since existing may be expired, while ZooKeeper is required inside dropAllData(). + current_zookeeper = zookeeper; /// If probably there is metadata in ZooKeeper, we don't allow to drop the table. if (!zookeeper) From cf1df96cc1c19f9fec621ba1a407148474edba01 Mon Sep 17 00:00:00 2001 From: Peignon Melvyn Date: Sat, 27 Jan 2024 17:27:00 +0100 Subject: [PATCH 235/264] Update view.md Improved documentation. --- docs/en/sql-reference/statements/create/view.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index f6158acd9a4..028d0b09a1a 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -97,7 +97,7 @@ This feature is deprecated and will be removed in the future. For your convenience, the old documentation is located [here](https://pastila.nl/?00f32652/fdf07272a7b54bda7e13b919264e449f.md) -## Refreshable Materialized View {#refreshable-materialized-view} +## Refreshable Materialized View [Experimental] {#refreshable-materialized-view} ```sql CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name @@ -120,7 +120,8 @@ Differences from regular non-refreshable materialized views: :::note Refreshable materialized views are a work in progress. Setting `allow_experimental_refreshable_materialized_view = 1` is required for creating one. Current limitations: - * not compatible with Replicated database or table engines, + * not compatible with Replicated database or table engines + * It is not supported in ClickHouse Cloud * require [Atomic database engine](../../../engines/database-engines/atomic.md), * no retries for failed refresh - we just skip to the next scheduled refresh time, * no limit on number of concurrent refreshes. From 9d3b4e530810b2d05f62706a22062519ba18b30b Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 27 Jan 2024 16:40:11 +0000 Subject: [PATCH 236/264] lower memory usage --- src/Columns/ColumnAggregateFunction.cpp | 4 ++-- src/Columns/ColumnArray.cpp | 12 +++++++++--- src/Columns/ColumnArray.h | 1 + src/Columns/ColumnConst.cpp | 2 +- src/Columns/ColumnDecimal.cpp | 4 ++-- src/Columns/ColumnDecimal.h | 1 + src/Columns/ColumnFixedString.cpp | 2 +- src/Columns/ColumnFixedString.h | 5 +++++ src/Columns/ColumnLowCardinality.h | 2 ++ src/Columns/ColumnMap.cpp | 5 +++++ src/Columns/ColumnMap.h | 1 + src/Columns/ColumnNullable.cpp | 8 +++++++- src/Columns/ColumnNullable.h | 1 + src/Columns/ColumnString.cpp | 7 ++++++- src/Columns/ColumnString.h | 1 + src/Columns/ColumnTuple.cpp | 7 +++++++ src/Columns/ColumnTuple.h | 1 + src/Columns/ColumnVector.cpp | 8 ++++---- src/Columns/ColumnVector.h | 5 +++++ src/Columns/IColumn.h | 5 +---- src/Common/PODArray.h | 7 +++++++ src/Core/Block.cpp | 2 +- src/DataTypes/Serializations/SerializationString.cpp | 2 +- src/Storages/MergeTree/MergeTreeReadTask.cpp | 4 ++++ src/Storages/MergeTree/MergeTreeSequentialSource.cpp | 5 +++++ 25 files changed, 81 insertions(+), 21 deletions(-) diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index 2018015b46d..43e55494747 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -337,7 +337,7 @@ ColumnPtr ColumnAggregateFunction::indexImpl(const PaddedPODArray & indexe assert(limit <= indexes.size()); auto res = createView(); - res->data.resize(limit); + res->data.resize_exact(limit); for (size_t i = 0; i < limit; ++i) res->data[i] = data[indexes[i]]; @@ -626,7 +626,7 @@ void ColumnAggregateFunction::getPermutation(PermutationSortDirection /*directio size_t /*limit*/, int /*nan_direction_hint*/, IColumn::Permutation & res) const { size_t s = data.size(); - res.resize(s); + res.resize_exact(s); iota(res.data(), s, IColumn::Permutation::value_type(0)); } diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 1cb8188bce6..fddfa2ac6b2 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -109,7 +109,7 @@ MutableColumnPtr ColumnArray::cloneResized(size_t to_size) const offset = getOffsets().back(); } - res->getOffsets().resize(to_size); + res->getOffsets().resize_exact(to_size); for (size_t i = from_size; i < to_size; ++i) res->getOffsets()[i] = offset; } @@ -427,6 +427,12 @@ void ColumnArray::reserve(size_t n) getData().reserve(n); /// The average size of arrays is not taken into account here. Or it is considered to be no more than 1. } +void ColumnArray::shrinkToFit() +{ + getOffsets().shrink_to_fit(); + getData().shrinkToFit(); +} + void ColumnArray::ensureOwnership() { getData().ensureOwnership(); @@ -603,7 +609,7 @@ void ColumnArray::expand(const IColumn::Filter & mask, bool inverted) ssize_t index = mask.size() - 1; ssize_t from = offsets_data.size() - 1; - offsets_data.resize(mask.size()); + offsets_data.resize_exact(mask.size()); UInt64 last_offset = offsets_data[from]; while (index >= 0) { @@ -831,7 +837,7 @@ ColumnPtr ColumnArray::indexImpl(const PaddedPODArray & indexes, size_t limit auto res = ColumnArray::create(data->cloneEmpty()); Offsets & res_offsets = res->getOffsets(); - res_offsets.resize(limit); + res_offsets.resize_exact(limit); size_t current_offset = 0; for (size_t i = 0; i < limit; ++i) diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index 2a9bfa405e5..407f44a6f3c 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -108,6 +108,7 @@ public: void updatePermutationWithCollation(const Collator & collator, PermutationSortDirection direction, PermutationSortStability stability, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges& equal_ranges) const override; void reserve(size_t n) override; + void shrinkToFit() override; void ensureOwnership() override; size_t byteSize() const override; size_t byteSizeAt(size_t n) const override; diff --git a/src/Columns/ColumnConst.cpp b/src/Columns/ColumnConst.cpp index 9aa0f5cfa49..57d02e72820 100644 --- a/src/Columns/ColumnConst.cpp +++ b/src/Columns/ColumnConst.cpp @@ -128,7 +128,7 @@ MutableColumns ColumnConst::scatter(ColumnIndex num_columns, const Selector & se void ColumnConst::getPermutation(PermutationSortDirection /*direction*/, PermutationSortStability /*stability*/, size_t /*limit*/, int /*nan_direction_hint*/, Permutation & res) const { - res.resize(s); + res.resize_exact(s); iota(res.data(), s, IColumn::Permutation::value_type(0)); } diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index 20fc5d8e1fe..aef81727ffd 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -159,7 +159,7 @@ void ColumnDecimal::getPermutation(IColumn::PermutationSortDirection directio }; size_t data_size = data.size(); - res.resize(data_size); + res.resize_exact(data_size); if (limit >= data_size) limit = 0; @@ -318,7 +318,7 @@ MutableColumnPtr ColumnDecimal::cloneResized(size_t size) const if (size > 0) { auto & new_col = static_cast(*res); - new_col.data.resize(size); + new_col.data.resize_exact(size); size_t count = std::min(this->size(), size); diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index fb24ae4554b..840dc23dc36 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -55,6 +55,7 @@ public: size_t allocatedBytes() const override { return data.allocated_bytes(); } void protect() override { data.protect(); } void reserve(size_t n) override { data.reserve(n); } + void shrinkToFit() override { data.shrink_to_fit(); } void insertFrom(const IColumn & src, size_t n) override { data.push_back(static_cast(src).getData()[n]); } void insertData(const char * src, size_t /*length*/) override; diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index a18e5c522a1..1ba59ce4f4b 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -39,7 +39,7 @@ MutableColumnPtr ColumnFixedString::cloneResized(size_t size) const if (size > 0) { auto & new_col = assert_cast(*new_col_holder); - new_col.chars.resize(size * n); + new_col.chars.resize_exact(size * n); size_t count = std::min(this->size(), size); memcpy(new_col.chars.data(), chars.data(), count * n * sizeof(chars[0])); diff --git a/src/Columns/ColumnFixedString.h b/src/Columns/ColumnFixedString.h index 445432b7b28..70c26ba8496 100644 --- a/src/Columns/ColumnFixedString.h +++ b/src/Columns/ColumnFixedString.h @@ -181,6 +181,11 @@ public: chars.reserve(n * size); } + void shrinkToFit() override + { + chars.shrink_to_fit(); + } + void resize(size_t size) { chars.resize(n * size); diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index 91bd5945fd9..c6fcfc0a4b7 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -159,6 +159,7 @@ public: } void reserve(size_t n) override { idx.reserve(n); } + void shrinkToFit() override { idx.shrinkToFit(); } /// Don't count the dictionary size as it can be shared between different blocks. size_t byteSize() const override { return idx.getPositions()->byteSize(); } @@ -295,6 +296,7 @@ public: void popBack(size_t n) { positions->popBack(n); } void reserve(size_t n) { positions->reserve(n); } + void shrinkToFit() { positions->shrinkToFit(); } UInt64 getMaxPositionForCurrentType() const; diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index ddcde43ca23..61abe9d2598 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -229,6 +229,11 @@ void ColumnMap::reserve(size_t n) nested->reserve(n); } +void ColumnMap::shrinkToFit() +{ + nested->shrinkToFit(); +} + void ColumnMap::ensureOwnership() { nested->ensureOwnership(); diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index fde8a7e0e67..118c5b3d3df 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -83,6 +83,7 @@ public: void updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const override; void reserve(size_t n) override; + void shrinkToFit() override; void ensureOwnership() override; size_t byteSize() const override; size_t byteSizeAt(size_t n) const override; diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 4ee6bb3d586..1a2bc378ff1 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -97,7 +97,7 @@ MutableColumnPtr ColumnNullable::cloneResized(size_t new_size) const if (new_size > 0) { - new_null_map->getData().resize(new_size); + new_null_map->getData().resize_exact(new_size); size_t count = std::min(size(), new_size); memcpy(new_null_map->getData().data(), getNullMapData().data(), count * sizeof(getNullMapData()[0])); @@ -678,6 +678,12 @@ void ColumnNullable::reserve(size_t n) getNullMapData().reserve(n); } +void ColumnNullable::shrinkToFit() +{ + getNestedColumn().shrinkToFit(); + getNullMapData().shrink_to_fit(); +} + void ColumnNullable::ensureOwnership() { getNestedColumn().ensureOwnership(); diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index b57fdf3064d..988783abfa3 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -109,6 +109,7 @@ public: void updatePermutationWithCollation(const Collator & collator, IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int null_direction_hint, Permutation & res, EqualRanges& equal_ranges) const override; void reserve(size_t n) override; + void shrinkToFit() override; void ensureOwnership() override; size_t byteSize() const override; size_t byteSizeAt(size_t n) const override; diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 50fe90ad8ef..b1ce50f9a2e 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -71,8 +71,8 @@ MutableColumnPtr ColumnString::cloneResized(size_t to_size) const /// Empty strings are just zero terminating bytes. res->chars.resize_fill(res->chars.size() + to_size - from_size); + res->offsets.resize_exact(to_size); - res->offsets.resize(to_size); for (size_t i = from_size; i < to_size; ++i) { ++offset; @@ -494,6 +494,11 @@ void ColumnString::reserve(size_t n) offsets.reserve(n); } +void ColumnString::shrinkToFit() +{ + chars.shrink_to_fit(); + offsets.shrink_to_fit(); +} void ColumnString::getExtremes(Field & min, Field & max) const { diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index 345513248fd..f2844b40aff 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -259,6 +259,7 @@ public: ColumnPtr compress() const override; void reserve(size_t n) override; + void shrinkToFit() override; void getExtremes(Field & min, Field & max) const override; diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 356bb0493d2..55899bbd50b 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -444,6 +444,13 @@ void ColumnTuple::reserve(size_t n) getColumn(i).reserve(n); } +void ColumnTuple::shrinkToFit() +{ + const size_t tuple_size = columns.size(); + for (size_t i = 0; i < tuple_size; ++i) + getColumn(i).shrinkToFit(); +} + void ColumnTuple::ensureOwnership() { const size_t tuple_size = columns.size(); diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index 79099f4c098..d5d0915de96 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -91,6 +91,7 @@ public: void updatePermutationWithCollation(const Collator & collator, IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges& equal_ranges) const override; void reserve(size_t n) override; + void shrinkToFit() override; void ensureOwnership() override; size_t byteSize() const override; size_t byteSizeAt(size_t n) const override; diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index b1cf449dfde..802d4293b90 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -237,7 +237,7 @@ void ColumnVector::getPermutation(IColumn::PermutationSortDirection direction size_t limit, int nan_direction_hint, IColumn::Permutation & res) const { size_t data_size = data.size(); - res.resize(data_size); + res.resize_exact(data_size); if (data_size == 0) return; @@ -424,7 +424,7 @@ MutableColumnPtr ColumnVector::cloneResized(size_t size) const if (size > 0) { auto & new_col = static_cast(*res); - new_col.data.resize(size); + new_col.data.resize_exact(size); size_t count = std::min(this->size(), size); memcpy(new_col.data.data(), data.data(), count * sizeof(data[0])); @@ -628,8 +628,8 @@ inline void doFilterAligned(const UInt8 *& filt_pos, const UInt8 *& filt_end_ali filt_pos += SIMD_ELEMENTS; data_pos += SIMD_ELEMENTS; } - /// resize to the real size. - res_data.resize(current_offset); + /// Resize to the real size. + res_data.resize_exact(current_offset); } ) diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index 517375f8eb4..cfc2ad6272e 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -182,6 +182,11 @@ public: data.reserve(n); } + void shrinkToFit() override + { + data.shrink_to_fit(); + } + const char * getFamilyName() const override { return TypeName.data(); } TypeIndex getDataType() const override { return TypeToTypeIndex; } diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 1ccdf255bf4..4f992263605 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -399,10 +399,7 @@ public: /// Requests the removal of unused capacity. /// It is a non-binding request to reduce the capacity of the underlying container to its size. - virtual MutablePtr shrinkToFit() const - { - return cloneResized(size()); - } + virtual void shrinkToFit() {} /// If we have another column as a source (owner of data), copy all data to ourself and reset source. virtual void ensureOwnership() {} diff --git a/src/Common/PODArray.h b/src/Common/PODArray.h index 6a048d1c6c0..1a4047a2588 100644 --- a/src/Common/PODArray.h +++ b/src/Common/PODArray.h @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -238,6 +239,12 @@ public: resize_assume_reserved(n); } + template + void shrink_to_fit(TAllocatorParams &&... allocator_params) + { + realloc(PODArrayDetails::minimum_memory_for_elements(size(), ELEMENT_SIZE, pad_left, pad_right), std::forward(allocator_params)...); + } + void resize_assume_reserved(const size_t n) /// NOLINT { c_end = c_start + PODArrayDetails::byte_size(n, ELEMENT_SIZE); diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index a7d5b0a869f..dfd60b994f4 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -601,7 +601,7 @@ Block Block::shrinkToFit() const { Columns new_columns(data.size(), nullptr); for (size_t i = 0; i < data.size(); ++i) - new_columns[i] = data[i].column->shrinkToFit(); + new_columns[i] = data[i].column->cloneResized(data[i].column->size()); return cloneWithColumns(new_columns); } diff --git a/src/DataTypes/Serializations/SerializationString.cpp b/src/DataTypes/Serializations/SerializationString.cpp index b2b083fd466..e2935a78c32 100644 --- a/src/DataTypes/Serializations/SerializationString.cpp +++ b/src/DataTypes/Serializations/SerializationString.cpp @@ -208,7 +208,7 @@ static NO_INLINE void deserializeBinarySSE2(ColumnString::Chars & data, ColumnSt data[offset - 1] = 0; } - data.resize(offset); + data.resize_exact(offset); } diff --git a/src/Storages/MergeTree/MergeTreeReadTask.cpp b/src/Storages/MergeTree/MergeTreeReadTask.cpp index dcfed700fac..41c7531b6a6 100644 --- a/src/Storages/MergeTree/MergeTreeReadTask.cpp +++ b/src/Storages/MergeTree/MergeTreeReadTask.cpp @@ -184,7 +184,11 @@ MergeTreeReadTask::BlockAndProgress MergeTreeReadTask::read(const BlockSizeParam Block block; if (read_result.num_rows != 0) + { + for (const auto & column : read_result.columns) + column->assumeMutableRef().shrinkToFit(); block = sample_block.cloneWithColumns(read_result.columns); + } BlockAndProgress res = { .block = std::move(block), diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 29af7fb4820..d0fbc316024 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -140,6 +140,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( if (storage.supportsSubcolumns()) options.withSubcolumns(); + columns_for_reader = storage_snapshot->getColumnsByNames(options, columns_to_read); } else @@ -156,6 +157,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( read_settings.local_fs_method = LocalFSReadMethod::pread; if (read_with_direct_io) read_settings.direct_io_threshold = 1; + /// Configure throttling switch (type) { @@ -224,7 +226,10 @@ try for (size_t i = 0; i < num_columns; ++i) { if (header.has(it->name)) + { + columns[i]->assumeMutableRef().shrinkToFit(); res_columns.emplace_back(std::move(columns[i])); + } ++it; } From b7a6dbfbb9c31970aaeefd3fc1423c5de9d014b4 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sat, 27 Jan 2024 18:22:37 +0100 Subject: [PATCH 237/264] Check engine to avoid possible seg fault --- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 9ce1c856622..5b04ffb2b17 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -716,7 +716,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti setEngine(create); /// We have to check access rights again (in case engine was changed). - if (create.storage) + if (create.storage && create.storage->engine) { auto source_access_type = StorageFactory::instance().getSourceAccessType(create.storage->engine->name); if (source_access_type != AccessType::NONE) From 24ab61932e438c887f7224092d9524397881e9c3 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sat, 27 Jan 2024 22:42:43 +0100 Subject: [PATCH 238/264] Update run.sh --- docker/test/stateless/run.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 8eeb01449b4..19f9783cd01 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -249,12 +249,12 @@ stop_logs_replication failed_to_save_logs=0 for table in query_log zookeeper_log trace_log transactions_info_log metric_log do - error=$( { clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst } 2>&1 ) + error=$( { clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst; } 2>&1 ) [[ "0" != "${#err}" ]] && failed_to_save_logs=1 if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then - error=$( { clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst } 2>&1 ) + error=$( { clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst; } 2>&1 ) [[ "0" != "${#err}" ]] && failed_to_save_logs=1 - error=$( { clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst } 2>&1 ) + error=$( { clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst; } 2>&1 ) [[ "0" != "${#err}" ]] && failed_to_save_logs=1 fi done From 07750ee9494cdbd8e7cb838eec79f1a00f6b8afb Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 26 Jan 2024 19:38:58 +0300 Subject: [PATCH 239/264] Added AtomicLogger --- base/poco/Foundation/src/Logger.cpp | 13 ++--- src/Common/AtomicLogger.h | 51 +++++++++++++++++++ src/Common/Exception.cpp | 10 ++++ src/Common/Exception.h | 3 ++ src/Common/Logger.h | 1 - src/Common/logger_useful.h | 4 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 8 +-- src/Storages/MergeTree/MergeTreeData.cpp | 26 +++++----- src/Storages/MergeTree/MergeTreeData.h | 7 +-- src/Storages/StorageMergeTree.cpp | 6 +-- src/Storages/StorageReplicatedMergeTree.cpp | 16 +++--- 11 files changed, 103 insertions(+), 42 deletions(-) create mode 100644 src/Common/AtomicLogger.h diff --git a/base/poco/Foundation/src/Logger.cpp b/base/poco/Foundation/src/Logger.cpp index 7c54116aaa4..e67b05a9225 100644 --- a/base/poco/Foundation/src/Logger.cpp +++ b/base/poco/Foundation/src/Logger.cpp @@ -304,16 +304,12 @@ struct LoggerDeleter { void operator()(Poco::Logger * logger) { - if (Logger::destroy(logger->name())) - return; - - logger->release(); + Logger::destroy(logger->name()); } }; inline LoggerPtr makeLoggerPtr(Logger & logger) { - logger.duplicate(); return std::shared_ptr(&logger, LoggerDeleter()); } @@ -329,8 +325,13 @@ Logger& Logger::get(const std::string& name) LoggerPtr Logger::getShared(const std::string & name) { std::lock_guard lock(getLoggerMutex()); + bool logger_exists = _pLoggerMap && _pLoggerMap->contains(name); - return makeLoggerPtr(unsafeGet(name)); + Logger & logger = unsafeGet(name); + if (logger_exists) + logger.duplicate(); + + return makeLoggerPtr(logger); } Logger& Logger::unsafeGet(const std::string& name) diff --git a/src/Common/AtomicLogger.h b/src/Common/AtomicLogger.h new file mode 100644 index 00000000000..bfbe7caad10 --- /dev/null +++ b/src/Common/AtomicLogger.h @@ -0,0 +1,51 @@ +#pragma once + +#include + +#include +#include +#include + + +/** AtomicLogger allows to atomically change logger. + * Standard library does not have atomic_shared_ptr, and we do not use std::atomic* operations, + * because standard library implementation uses fixed table of mutexes, and it is better to avoid contention here. + */ +class AtomicLogger +{ +public: + explicit AtomicLogger(LoggerPtr logger_) + : logger(std::move(logger_)) + {} + + explicit AtomicLogger(const std::string & log_name) + : AtomicLogger(::getLogger(log_name)) + {} + + void store(LoggerPtr new_logger) + { + std::lock_guard lock(log_mutex); + logger = std::move(new_logger); + } + + void store(const std::string & new_log_name) + { + auto new_logger = ::getLogger(new_log_name); + store(std::move(new_logger)); + } + + LoggerPtr load() const + { + DB::SharedLockGuard lock(log_mutex); + return logger; + } + + String loadName() const + { + DB::SharedLockGuard lock(log_mutex); + return logger->name(); + } +private: + mutable DB::SharedMutex log_mutex; + LoggerPtr logger TSA_GUARDED_BY(log_mutex); +}; diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index ff83f6ba807..7e73e2c0783 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -257,6 +257,11 @@ void tryLogCurrentException(LoggerPtr logger, const std::string & start_of_messa tryLogCurrentException(logger.get(), start_of_message); } +void tryLogCurrentException(const AtomicLogger & logger, const std::string & start_of_message) +{ + tryLogCurrentException(logger.load(), start_of_message); +} + static void getNoSpaceLeftInfoMessage(std::filesystem::path path, String & msg) { path = std::filesystem::absolute(path); @@ -529,6 +534,11 @@ void tryLogException(std::exception_ptr e, LoggerPtr logger, const std::string & } } +void tryLogException(std::exception_ptr e, const AtomicLogger & logger, const std::string & start_of_message) +{ + tryLogException(e, logger.load(), start_of_message); +} + std::string getExceptionMessage(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace) { return getExceptionMessageAndPattern(e, with_stacktrace, check_embedded_stacktrace).text; diff --git a/src/Common/Exception.h b/src/Common/Exception.h index 8afed6034cf..817a4400080 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -11,6 +11,7 @@ #include #include #include +#include #include #include @@ -245,6 +246,7 @@ using Exceptions = std::vector; void tryLogCurrentException(const char * log_name, const std::string & start_of_message = ""); void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_message = ""); void tryLogCurrentException(LoggerPtr logger, const std::string & start_of_message = ""); +void tryLogCurrentException(const AtomicLogger & logger, const std::string & start_of_message = ""); /** Prints current exception in canonical format. @@ -290,6 +292,7 @@ struct ExecutionStatus /// TODO: Logger leak constexpr overload void tryLogException(std::exception_ptr e, const char * log_name, const std::string & start_of_message = ""); void tryLogException(std::exception_ptr e, LoggerPtr logger, const std::string & start_of_message = ""); +void tryLogException(std::exception_ptr e, const AtomicLogger & logger, const std::string & start_of_message = ""); std::string getExceptionMessage(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace = false); PreformattedMessage getExceptionMessageAndPattern(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace = false); diff --git a/src/Common/Logger.h b/src/Common/Logger.h index 13e1c6bf8f5..6dcdea9a9d8 100644 --- a/src/Common/Logger.h +++ b/src/Common/Logger.h @@ -43,7 +43,6 @@ LoggerRawPtr getRawLogger(const std::string & name); LoggerRawPtr createRawLogger(const std::string & name, Poco::Channel * channel, Poco::Message::Priority level = Poco::Message::PRIO_INFORMATION); - /** Returns true, if currently Logger with specified name is created. * Otherwise, returns false. */ diff --git a/src/Common/logger_useful.h b/src/Common/logger_useful.h index 3899d060b7c..8e78e93e198 100644 --- a/src/Common/logger_useful.h +++ b/src/Common/logger_useful.h @@ -9,6 +9,7 @@ #include #include #include +#include namespace Poco { class Logger; } @@ -20,7 +21,8 @@ using LogSeriesLimiterPtr = std::shared_ptr; namespace { - [[maybe_unused]] const ::Poco::Logger * getLoggerHelper(const LoggerPtr & logger) { return logger.get(); } + [[maybe_unused]] LoggerPtr getLoggerHelper(const LoggerPtr & logger) { return logger; } + [[maybe_unused]] LoggerPtr getLoggerHelper(const AtomicLogger & logger) { return logger.load(); } [[maybe_unused]] const ::Poco::Logger * getLoggerHelper(const ::Poco::Logger * logger) { return logger; } [[maybe_unused]] std::unique_ptr getLoggerHelper(std::unique_ptr && logger) { return logger; } [[maybe_unused]] std::unique_ptr getLoggerHelper(std::unique_ptr && logger) { return logger; } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 87f23b0da2a..19730dc05c1 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1663,7 +1663,7 @@ try metadata_manager->deleteAll(true); metadata_manager->assertAllDeleted(true); - getDataPartStorage().rename(to.parent_path(), to.filename(), storage.log, remove_new_dir_if_exists, fsync_dir); + getDataPartStorage().rename(to.parent_path(), to.filename(), storage.log.load(), remove_new_dir_if_exists, fsync_dir); metadata_manager->updateAll(true); auto new_projection_root_path = to.string(); @@ -1758,7 +1758,7 @@ void IMergeTreeDataPart::remove() } bool is_temporary_part = is_temp || state == MergeTreeDataPartState::Temporary; - getDataPartStorage().remove(std::move(can_remove_callback), checksums, projection_checksums, is_temporary_part, storage.log); + getDataPartStorage().remove(std::move(can_remove_callback), checksums, projection_checksums, is_temporary_part, storage.log.load()); } std::optional IMergeTreeDataPart::getRelativePathForPrefix(const String & prefix, bool detached, bool broken) const @@ -1775,7 +1775,7 @@ std::optional IMergeTreeDataPart::getRelativePathForPrefix(const String if (detached && parent_part) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot detach projection"); - return getDataPartStorage().getRelativePathForPrefix(storage.log, prefix, detached, broken); + return getDataPartStorage().getRelativePathForPrefix(storage.log.load(), prefix, detached, broken); } std::optional IMergeTreeDataPart::getRelativePathForDetachedPart(const String & prefix, bool broken) const @@ -1841,7 +1841,7 @@ MutableDataPartStoragePtr IMergeTreeDataPart::makeCloneOnDisk( throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not clone data part {} to empty directory.", name); String path_to_clone = fs::path(storage.relative_data_path) / directory_name / ""; - return getDataPartStorage().clonePart(path_to_clone, getDataPartStorage().getPartDirectory(), disk, read_settings, write_settings, storage.log, cancellation_hook); + return getDataPartStorage().clonePart(path_to_clone, getDataPartStorage().getPartDirectory(), disk, read_settings, write_settings, storage.log.load(), cancellation_hook); } UInt64 IMergeTreeDataPart::getIndexSizeFromFile() const diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 39c113c240e..f0eeb3602d5 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -354,8 +354,7 @@ MergeTreeData::MergeTreeData( , merging_params(merging_params_) , require_part_metadata(require_part_metadata_) , broken_part_callback(broken_part_callback_) - , log_name(std::make_shared(table_id_.getNameForLogs())) - , log(getLogger(*log_name)) + , log(table_id_.getNameForLogs()) , storage_settings(std::move(storage_settings_)) , pinned_part_uuids(std::make_shared()) , data_parts_by_info(data_parts_indexes.get()) @@ -1296,7 +1295,7 @@ MergeTreeData::LoadPartResult MergeTreeData::loadDataPart( res.is_broken = true; tryLogCurrentException(log, fmt::format("while loading part {} on path {}", part_name, part_path)); - res.size_of_part = calculatePartSizeSafe(res.part, log); + res.size_of_part = calculatePartSizeSafe(res.part, log.load()); auto part_size_str = res.size_of_part ? formatReadableSizeWithBinarySuffix(*res.size_of_part) : "failed to calculate size"; LOG_ERROR(log, @@ -1327,7 +1326,7 @@ MergeTreeData::LoadPartResult MergeTreeData::loadDataPart( if (part_disk_ptr->exists(marker_path)) { /// NOTE: getBytesOnDisk() cannot be used here, since it may be zero if checksums.txt does not exist. - res.size_of_part = calculatePartSizeSafe(res.part, log); + res.size_of_part = calculatePartSizeSafe(res.part, log.load()); res.is_broken = true; auto part_size_str = res.size_of_part ? formatReadableSizeWithBinarySuffix(*res.size_of_part) : "failed to calculate size"; @@ -2114,7 +2113,7 @@ size_t MergeTreeData::clearOldTemporaryDirectories(const String & root_path, siz { /// Actually we don't rely on temporary_directories_lifetime when removing old temporaries directories, /// it's just an extra level of protection just in case we have a bug. - LOG_INFO(LogFrequencyLimiter(log, 10), "{} is in use (by merge/mutation/INSERT) (consider increasing temporary_directories_lifetime setting)", full_path); + LOG_INFO(LogFrequencyLimiter(log.load(), 10), "{} is in use (by merge/mutation/INSERT) (consider increasing temporary_directories_lifetime setting)", full_path); continue; } else if (!disk->exists(it->path())) @@ -2734,8 +2733,7 @@ void MergeTreeData::rename(const String & new_table_path, const StorageID & new_ void MergeTreeData::renameInMemory(const StorageID & new_table_id) { IStorage::renameInMemory(new_table_id); - std::atomic_store(&log_name, std::make_shared(new_table_id.getNameForLogs())); - log = getLogger(*log_name); + log.store(new_table_id.getNameForLogs()); } void MergeTreeData::dropAllData() @@ -6249,13 +6247,13 @@ ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules( log, "Would like to reserve space on volume '{}' by TTL rule of table '{}' but volume was not found", move_ttl_entry->destination_name, - *std::atomic_load(&log_name)); + log.loadName()); else if (move_ttl_entry->destination_type == DataDestinationType::DISK && !move_ttl_entry->if_exists) LOG_WARNING( log, "Would like to reserve space on disk '{}' by TTL rule of table '{}' but disk was not found", move_ttl_entry->destination_name, - *std::atomic_load(&log_name)); + log.loadName()); } else if (is_insert && !perform_ttl_move_on_insert) { @@ -6264,7 +6262,7 @@ ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules( "TTL move on insert to {} {} for table {} is disabled", (move_ttl_entry->destination_type == DataDestinationType::VOLUME ? "volume" : "disk"), move_ttl_entry->destination_name, - *std::atomic_load(&log_name)); + log.loadName()); } else { @@ -6280,13 +6278,13 @@ ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules( log, "Would like to reserve space on volume '{}' by TTL rule of table '{}' but there is not enough space", move_ttl_entry->destination_name, - *std::atomic_load(&log_name)); + log.loadName()); else if (move_ttl_entry->destination_type == DataDestinationType::DISK) LOG_WARNING( log, "Would like to reserve space on disk '{}' by TTL rule of table '{}' but there is not enough space", move_ttl_entry->destination_name, - *std::atomic_load(&log_name)); + log.loadName()); } } } @@ -7989,7 +7987,7 @@ bool MergeTreeData::insertQueryIdOrThrowNoLock(const String & query_id, size_t m throw Exception( ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES, "Too many simultaneous queries for table {}. Maximum is: {}", - *std::atomic_load(&log_name), + log.loadName(), max_queries); query_id_set.insert(query_id); return true; @@ -8181,7 +8179,7 @@ ReservationPtr MergeTreeData::balancedReservation( } // Record submerging big parts in the tagger to clean them up. - tagger_ptr->emplace(*this, part_name, std::move(covered_parts), log); + tagger_ptr->emplace(*this, part_name, std::move(covered_parts), log.load()); } } } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index caef247500a..29aafcc3a13 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -462,7 +462,7 @@ public: /// Load the set of data parts from disk. Call once - immediately after the object is created. void loadDataParts(bool skip_sanity_checks, std::optional> expected_parts); - String getLogName() const { return *std::atomic_load(&log_name); } + String getLogName() const { return log.loadName(); } Int64 getMaxBlockNumber() const; @@ -1115,10 +1115,7 @@ protected: /// Engine-specific methods BrokenPartCallback broken_part_callback; - /// log_name will change during table RENAME. Use atomic_shared_ptr to allow concurrent RW. - /// NOTE clang-14 doesn't have atomic_shared_ptr yet. Use std::atomic* operations for now. - std::shared_ptr log_name; - LoggerPtr log; + AtomicLogger log; /// Storage settings. /// Use get and set to receive readonly versions. diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 7e6c5ca3924..b402191f895 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -683,7 +683,7 @@ std::optional StorageMergeTree::getIncompleteMutationsS const auto & mutation_entry = current_mutation_it->second; - auto txn = tryGetTransactionForMutation(mutation_entry, log); + auto txn = tryGetTransactionForMutation(mutation_entry, log.load()); /// There's no way a transaction may finish before a mutation that was started by the transaction. /// But sometimes we need to check status of an unrelated mutation, in this case we don't care about transactions. assert(txn || mutation_entry.tid.isPrehistoric() || from_another_mutation); @@ -829,7 +829,7 @@ CancellationCode StorageMergeTree::killMutation(const String & mutation_id) if (!to_kill) return CancellationCode::NotFound; - if (auto txn = tryGetTransactionForMutation(*to_kill, log)) + if (auto txn = tryGetTransactionForMutation(*to_kill, log.load())) { LOG_TRACE(log, "Cancelling transaction {} which had started mutation {}", to_kill->tid, mutation_id); TransactionLog::instance().rollbackTransaction(txn); @@ -1222,7 +1222,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate( if (!part->version.isVisible(first_mutation_tid.start_csn, first_mutation_tid)) continue; - txn = tryGetTransactionForMutation(mutations_begin_it->second, log); + txn = tryGetTransactionForMutation(mutations_begin_it->second, log.load()); if (!txn) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find transaction {} that has started mutation {} " "that is going to be applied to part {}", diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c82721d2a18..9c2f9dba485 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -320,7 +320,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( attach, [this] (const std::string & name) { enqueuePartForCheck(name); }) , zookeeper_name(zkutil::extractZooKeeperName(zookeeper_path_)) - , zookeeper_path(zkutil::extractZooKeeperPath(zookeeper_path_, /* check_starts_with_slash */ !attach, log)) + , zookeeper_path(zkutil::extractZooKeeperPath(zookeeper_path_, /* check_starts_with_slash */ !attach, log.load())) , replica_name(replica_name_) , replica_path(fs::path(zookeeper_path) / "replicas" / replica_name_) , reader(*this) @@ -812,7 +812,7 @@ bool StorageReplicatedMergeTree::createTableIfNotExists(const StorageMetadataPtr else { auto metadata_drop_lock = zkutil::EphemeralNodeHolder::existing(drop_lock_path, *zookeeper); - if (!removeTableNodesFromZooKeeper(zookeeper, zookeeper_path, metadata_drop_lock, log)) + if (!removeTableNodesFromZooKeeper(zookeeper, zookeeper_path, metadata_drop_lock, log.load())) { /// Someone is recursively removing table right now, we cannot create new table until old one is removed continue; @@ -1128,7 +1128,7 @@ void StorageReplicatedMergeTree::drop() if (lost_part_count > 0) LOG_INFO(log, "Dropping table with non-zero lost_part_count equal to {}", lost_part_count); } - dropReplica(zookeeper, zookeeper_path, replica_name, log, getSettings(), &has_metadata_in_zookeeper); + dropReplica(zookeeper, zookeeper_path, replica_name, log.load(), getSettings(), &has_metadata_in_zookeeper); } } @@ -4181,7 +4181,7 @@ void StorageReplicatedMergeTree::startBeingLeader() return; } - zkutil::checkNoOldLeaders(log, *zookeeper, fs::path(zookeeper_path) / "leader_election"); + zkutil::checkNoOldLeaders(log.load(), *zookeeper, fs::path(zookeeper_path) / "leader_election"); LOG_INFO(log, "Became leader"); is_leader = true; @@ -4275,7 +4275,7 @@ void StorageReplicatedMergeTree::waitForUniquePartsToBeFetchedByOtherReplicas(St auto zookeeper = getZooKeeperIfTableShutDown(); - auto unique_parts_set = findReplicaUniqueParts(replica_name, zookeeper_path, format_version, zookeeper, log); + auto unique_parts_set = findReplicaUniqueParts(replica_name, zookeeper_path, format_version, zookeeper, log.load()); if (unique_parts_set.empty()) { LOG_INFO(log, "Will not wait for unique parts to be fetched because we don't have any unique parts"); @@ -9348,7 +9348,7 @@ StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & part, co return unlockSharedDataByID( part.getUniqueId(), shared_id, part.info, replica_name, - part.getDataPartStorage().getDiskType(), zookeeper, *getSettings(), log, zookeeper_path, format_version); + part.getDataPartStorage().getDiskType(), zookeeper, *getSettings(), log.load(), zookeeper_path, format_version); } namespace @@ -10301,7 +10301,7 @@ void StorageReplicatedMergeTree::backupData( bool exists = false; Strings mutation_ids; { - ZooKeeperRetriesControl retries_ctl("getMutations", log, zookeeper_retries_info, nullptr); + ZooKeeperRetriesControl retries_ctl("getMutations", log.load(), zookeeper_retries_info, nullptr); retries_ctl.retryLoop([&]() { if (!zookeeper || zookeeper->expired()) @@ -10320,7 +10320,7 @@ void StorageReplicatedMergeTree::backupData( bool mutation_id_exists = false; String mutation; - ZooKeeperRetriesControl retries_ctl("getMutation", log, zookeeper_retries_info, nullptr); + ZooKeeperRetriesControl retries_ctl("getMutation", log.load(), zookeeper_retries_info, nullptr); retries_ctl.retryLoop([&]() { if (!zookeeper || zookeeper->expired()) From 50309dfd908e5e1da4addb3f53d1b750687c473d Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 27 Jan 2024 16:30:36 +0300 Subject: [PATCH 240/264] Fixed tests --- base/poco/Foundation/include/Poco/Logger.h | 11 -- base/poco/Foundation/src/Logger.cpp | 114 ++++++++++++++------- src/Common/tests/gtest_log.cpp | 3 - 3 files changed, 78 insertions(+), 50 deletions(-) diff --git a/base/poco/Foundation/include/Poco/Logger.h b/base/poco/Foundation/include/Poco/Logger.h index f91d836f190..cf202718662 100644 --- a/base/poco/Foundation/include/Poco/Logger.h +++ b/base/poco/Foundation/include/Poco/Logger.h @@ -904,13 +904,6 @@ public: /// Returns a pointer to the Logger with the given name if it /// exists, or a null pointer otherwise. - static bool destroy(const std::string & name); - /// Destroys the logger with the specified name. Does nothing - /// if the logger is not found. - /// - /// After a logger has been destroyed, all references to it - /// become invalid. - static void shutdown(); /// Shuts down the logging framework and releases all /// Loggers. @@ -940,8 +933,6 @@ public: static const std::string ROOT; /// The name of the root logger (""). protected: - typedef std::map LoggerMap; - Logger(const std::string & name, Channel * pChannel, int level); ~Logger(); @@ -962,8 +953,6 @@ private: std::string _name; Channel * _pChannel; std::atomic_int _level; - - static LoggerMap * _pLoggerMap; }; diff --git a/base/poco/Foundation/src/Logger.cpp b/base/poco/Foundation/src/Logger.cpp index e67b05a9225..d8c90e7ccb9 100644 --- a/base/poco/Foundation/src/Logger.cpp +++ b/base/poco/Foundation/src/Logger.cpp @@ -20,6 +20,7 @@ #include "Poco/NumberParser.h" #include "Poco/String.h" +#include #include namespace @@ -37,12 +38,20 @@ std::mutex & getLoggerMutex() return *logger_mutex; } +struct LoggerEntry +{ + Poco::Logger * logger; + bool owned_by_shared_ptr = false; +}; + +using LoggerMap = std::unordered_map; +LoggerMap * _pLoggerMap = nullptr; + } namespace Poco { -Logger::LoggerMap* Logger::_pLoggerMap = 0; const std::string Logger::ROOT; @@ -134,12 +143,12 @@ void Logger::setLevel(const std::string& name, int level) if (_pLoggerMap) { std::string::size_type len = name.length(); - for (LoggerMap::iterator it = _pLoggerMap->begin(); it != _pLoggerMap->end(); ++it) + for (auto & it : *_pLoggerMap) { if (len == 0 || - (it->first.compare(0, len, name) == 0 && (it->first.length() == len || it->first[len] == '.'))) + (it.first.compare(0, len, name) == 0 && (it.first.length() == len || it.first[len] == '.'))) { - it->second->setLevel(level); + it.second.logger->setLevel(level); } } } @@ -153,12 +162,12 @@ void Logger::setChannel(const std::string& name, Channel* pChannel) if (_pLoggerMap) { std::string::size_type len = name.length(); - for (LoggerMap::iterator it = _pLoggerMap->begin(); it != _pLoggerMap->end(); ++it) + for (auto & it : *_pLoggerMap) { if (len == 0 || - (it->first.compare(0, len, name) == 0 && (it->first.length() == len || it->first[len] == '.'))) + (it.first.compare(0, len, name) == 0 && (it.first.length() == len || it.first[len] == '.'))) { - it->second->setChannel(pChannel); + it.second.logger->setChannel(pChannel); } } } @@ -172,12 +181,12 @@ void Logger::setProperty(const std::string& loggerName, const std::string& prope if (_pLoggerMap) { std::string::size_type len = loggerName.length(); - for (LoggerMap::iterator it = _pLoggerMap->begin(); it != _pLoggerMap->end(); ++it) + for (auto & it : *_pLoggerMap) { if (len == 0 || - (it->first.compare(0, len, loggerName) == 0 && (it->first.length() == len || it->first[len] == '.'))) + (it.first.compare(0, len, loggerName) == 0 && (it.first.length() == len || it.first[len] == '.'))) { - it->second->setProperty(propertyName, value); + it.second.logger->setProperty(propertyName, value); } } } @@ -304,10 +313,31 @@ struct LoggerDeleter { void operator()(Poco::Logger * logger) { - Logger::destroy(logger->name()); + std::lock_guard lock(getLoggerMutex()); + + /// If logger infrastructure is destroyed just decrement logger reference count + if (!_pLoggerMap) + { + logger->release(); + return; + } + + auto it = _pLoggerMap->find(logger->name()); + assert(it != _pLoggerMap->end()); + + /** If reference count is 1, this means this shared pointer owns logger + * and need destroy it. + */ + size_t reference_count_before_release = logger->release(); + if (reference_count_before_release == 1) + { + assert(it->second.owned_by_shared_ptr); + _pLoggerMap->erase(it); + } } }; + inline LoggerPtr makeLoggerPtr(Logger & logger) { return std::shared_ptr(&logger, LoggerDeleter()); @@ -315,25 +345,52 @@ inline LoggerPtr makeLoggerPtr(Logger & logger) } + Logger& Logger::get(const std::string& name) { std::lock_guard lock(getLoggerMutex()); - return unsafeGet(name); + Logger & logger = unsafeGet(name); + + /** If there are already shared pointer created for this logger + * we need to increment Logger reference count and now logger + * is owned by logger infrastructure. + */ + auto it = _pLoggerMap->find(name); + if (it->second.owned_by_shared_ptr) + { + it->second.logger->duplicate(); + it->second.owned_by_shared_ptr = false; + } + + return logger; } + LoggerPtr Logger::getShared(const std::string & name) { std::lock_guard lock(getLoggerMutex()); bool logger_exists = _pLoggerMap && _pLoggerMap->contains(name); Logger & logger = unsafeGet(name); + + /** If logger already exists, then this shared pointer does not own it. + * If logger does not exists, logger infrastructure could be already destroyed + * or logger was created. + */ if (logger_exists) + { logger.duplicate(); + } + else if (_pLoggerMap) + { + _pLoggerMap->find(name)->second.owned_by_shared_ptr = true; + } return makeLoggerPtr(logger); } + Logger& Logger::unsafeGet(const std::string& name) { Logger* pLogger = find(name); @@ -390,10 +447,14 @@ void Logger::shutdown() if (_pLoggerMap) { - for (LoggerMap::iterator it = _pLoggerMap->begin(); it != _pLoggerMap->end(); ++it) + for (auto & it : *_pLoggerMap) { - it->second->release(); + if (it.second.owned_by_shared_ptr) + continue; + + it.second.logger->release(); } + delete _pLoggerMap; _pLoggerMap = 0; } @@ -406,32 +467,12 @@ Logger* Logger::find(const std::string& name) { LoggerMap::iterator it = _pLoggerMap->find(name); if (it != _pLoggerMap->end()) - return it->second; + return it->second.logger; } return 0; } -bool Logger::destroy(const std::string& name) -{ - std::lock_guard lock(getLoggerMutex()); - - if (_pLoggerMap) - { - LoggerMap::iterator it = _pLoggerMap->find(name); - if (it != _pLoggerMap->end()) - { - if (it->second->release() == 1) - _pLoggerMap->erase(it); - - return true; - } - } - - return false; -} - - void Logger::names(std::vector& names) { std::lock_guard lock(getLoggerMutex()); @@ -539,7 +580,8 @@ void Logger::add(Logger* pLogger) { if (!_pLoggerMap) _pLoggerMap = new LoggerMap; - _pLoggerMap->insert(LoggerMap::value_type(pLogger->name(), pLogger)); + + _pLoggerMap->emplace(pLogger->name(), LoggerEntry{pLogger, false /*owned_by_shared_ptr*/}); } diff --git a/src/Common/tests/gtest_log.cpp b/src/Common/tests/gtest_log.cpp index 419aac370d6..622497fe2f5 100644 --- a/src/Common/tests/gtest_log.cpp +++ b/src/Common/tests/gtest_log.cpp @@ -32,7 +32,6 @@ TEST(Logger, TestLog) LOG_TEST(log, "Hello World"); EXPECT_EQ(oss.str(), "Hello World\n"); - Poco::Logger::destroy("TestLogger"); } { /// Test logs invisible for other levels @@ -45,8 +44,6 @@ TEST(Logger, TestLog) LOG_TEST(log, "Hello World"); EXPECT_EQ(oss.str(), ""); - - Poco::Logger::destroy(std::string{level} + "_Logger"); } } From bccd8b5e270caccf1011971c424e2acee64b9e95 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 27 Jan 2024 19:17:17 +0300 Subject: [PATCH 241/264] Fixed tests --- base/poco/Foundation/src/Logger.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/base/poco/Foundation/src/Logger.cpp b/base/poco/Foundation/src/Logger.cpp index d8c90e7ccb9..cfc063c8979 100644 --- a/base/poco/Foundation/src/Logger.cpp +++ b/base/poco/Foundation/src/Logger.cpp @@ -422,7 +422,10 @@ LoggerPtr Logger::createShared(const std::string & name, Channel * pChannel, int { std::lock_guard lock(getLoggerMutex()); - return makeLoggerPtr(unsafeCreate(name, pChannel, level)); + Logger & logger = unsafeCreate(name, pChannel, level); + _pLoggerMap->find(name)->second.owned_by_shared_ptr = true; + + return makeLoggerPtr(logger); } Logger& Logger::root() From 5165fb7286a145091e9cf89a92284a34d22c31c0 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 28 Jan 2024 13:55:23 +0300 Subject: [PATCH 242/264] Fixed build --- src/Common/AtomicLogger.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/AtomicLogger.h b/src/Common/AtomicLogger.h index bfbe7caad10..4bda55e070b 100644 --- a/src/Common/AtomicLogger.h +++ b/src/Common/AtomicLogger.h @@ -47,5 +47,5 @@ public: } private: mutable DB::SharedMutex log_mutex; - LoggerPtr logger TSA_GUARDED_BY(log_mutex); + LoggerPtr logger; }; From febd4ce8d794749da1346428709f72ce0e34495f Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sun, 28 Jan 2024 13:41:33 +0100 Subject: [PATCH 243/264] Update run.sh --- docker/test/stateless/run.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 19f9783cd01..5c1398bf308 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -249,12 +249,12 @@ stop_logs_replication failed_to_save_logs=0 for table in query_log zookeeper_log trace_log transactions_info_log metric_log do - error=$( { clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst; } 2>&1 ) + err=$( { clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst; } 2>&1 ) [[ "0" != "${#err}" ]] && failed_to_save_logs=1 if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then - error=$( { clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst; } 2>&1 ) + err=$( { clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst; } 2>&1 ) [[ "0" != "${#err}" ]] && failed_to_save_logs=1 - error=$( { clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst; } 2>&1 ) + err=$( { clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst; } 2>&1 ) [[ "0" != "${#err}" ]] && failed_to_save_logs=1 fi done From 9dd6362ae7488169b4dc8be9be92e3a982131cdb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jan 2024 13:44:56 +0100 Subject: [PATCH 244/264] Play UI: improve rendering of errors from JSON formats --- programs/server/play.html | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/programs/server/play.html b/programs/server/play.html index e6662693bd7..507a96382a7 100644 --- a/programs/server/play.html +++ b/programs/server/play.html @@ -993,7 +993,16 @@ function renderError(response) { clear(); - document.getElementById('error').innerText = response ? response : "No response."; + + let message = response; + try { + let json = JSON.parse(response); + if (json.exception) { + message = json.exception; + } + } catch (e) {} + + document.getElementById('error').innerText = message ? message : "No response."; document.getElementById('error').style.display = 'block'; document.getElementById('logo-container').style.display = 'none'; } From e0f0b21e907eb0d20655f83641f26261793910cf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jan 2024 14:24:13 +0100 Subject: [PATCH 245/264] Remove KQL --- src/Interpreters/executeQuery.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 533d58aaa8f..3ff3db6b8ff 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -102,6 +102,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int QUERY_WAS_CANCELLED; extern const int INCORRECT_DATA; + extern const int SUPPORT_IS_DISABLED; } @@ -709,10 +710,7 @@ static std::tuple executeQueryImpl( { if (settings.dialect == Dialect::kusto && !internal) { - ParserKQLStatement parser(end, settings.allow_settings_after_format_in_insert); - - /// TODO: parser should fail early when max_query_size limit is reached. - ast = parseKQLQuery(parser, begin, end, "", max_query_size, settings.max_parser_depth); + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Kusto dialect is disabled until these two bugs will be fixed: https://github.com/ClickHouse/ClickHouse/issues/59037 and https://github.com/ClickHouse/ClickHouse/issues/59036"); } else if (settings.dialect == Dialect::prql && !internal) { From 27cd21848360815c5db49647b5d8efc6e55f2b99 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jan 2024 15:10:57 +0100 Subject: [PATCH 246/264] No debug symbols in Rust --- rust/prql/Cargo.toml | 2 +- rust/skim/Cargo.toml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/rust/prql/Cargo.toml b/rust/prql/Cargo.toml index f29aa4aaef9..ca8a19b41ac 100644 --- a/rust/prql/Cargo.toml +++ b/rust/prql/Cargo.toml @@ -13,7 +13,7 @@ serde_json = "1.0" crate-type = ["staticlib"] [profile.release] -debug = true +debug = false [profile.release-thinlto] inherits = "release" diff --git a/rust/skim/Cargo.toml b/rust/skim/Cargo.toml index 22af40c3e33..0839b888d8b 100644 --- a/rust/skim/Cargo.toml +++ b/rust/skim/Cargo.toml @@ -17,7 +17,7 @@ cxx-build = "1.0.83" crate-type = ["staticlib"] [profile.release] -debug = true +debug = false [profile.release-thinlto] inherits = "release" From b3b2b7a1e6352379d90d72bc773737f386ec1c36 Mon Sep 17 00:00:00 2001 From: Maksim Alekseev Date: Sun, 28 Jan 2024 17:29:08 +0300 Subject: [PATCH 247/264] Fix architecture name in select of Rust target --- contrib/corrosion-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/corrosion-cmake/CMakeLists.txt b/contrib/corrosion-cmake/CMakeLists.txt index 04871c761ab..9b98ed6efb3 100644 --- a/contrib/corrosion-cmake/CMakeLists.txt +++ b/contrib/corrosion-cmake/CMakeLists.txt @@ -28,7 +28,7 @@ elseif(CMAKE_TOOLCHAIN_FILE MATCHES "linux/toolchain-aarch64") set(Rust_CARGO_TARGET "aarch64-unknown-linux-gnu") elseif((CMAKE_TOOLCHAIN_FILE MATCHES "darwin") AND (CMAKE_TOOLCHAIN_FILE MATCHES "x86_64")) set(Rust_CARGO_TARGET "x86_64-apple-darwin") -elseif((CMAKE_TOOLCHAIN_FILE MATCHES "darwin") AND (CMAKE_TOOLCHAIN_FILE MATCHES "darwin")) +elseif((CMAKE_TOOLCHAIN_FILE MATCHES "darwin") AND (CMAKE_TOOLCHAIN_FILE MATCHES "aarch64")) set(Rust_CARGO_TARGET "aarch64-apple-darwin") elseif((CMAKE_TOOLCHAIN_FILE MATCHES "freebsd") AND (CMAKE_TOOLCHAIN_FILE MATCHES "x86_64")) set(Rust_CARGO_TARGET "x86_64-unknown-freebsd") From 29700fbad154328fcaeda059a6b4644f95edee11 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Sun, 28 Jan 2024 15:33:27 +0100 Subject: [PATCH 248/264] Fix special build --- src/Columns/ColumnMap.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index 307a1ef0f62..4a758ca248a 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -141,9 +141,9 @@ void ColumnMap::updateHashFast(SipHash & hash) const nested->updateHashFast(hash); } -void ColumnMap::insertFrom(const IColumn & src, size_t start) +void ColumnMap::insertFrom(const IColumn & src, size_t n) { - nested->insertFrom(assert_cast(src).getNestedColumn(), start); + nested->insertFrom(assert_cast(src).getNestedColumn(), n); } void ColumnMap::insertRangeFrom(const IColumn & src, size_t start, size_t length) From d36c92a21900c7f47680ee67cb93864d02105bdf Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Sun, 28 Jan 2024 15:09:39 +0000 Subject: [PATCH 249/264] CI: Add rust dir to build digest --- tests/ci/ci_config.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index ece7f2f7bae..df72c02cb7d 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -213,6 +213,7 @@ class BuildConfig: "./programs", "./packages", "./docker/packager/packager", + "./rust", ], exclude_files=[".md"], docker=["clickhouse/binary-builder"], From aa6c7e78beb843b1c830a947615836a89291519b Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 28 Jan 2024 15:47:17 +0000 Subject: [PATCH 250/264] fix deadlock in AsyncLoader::stop() --- src/Common/AsyncLoader.cpp | 44 ++++++++++++++++++------- src/Common/AsyncLoader.h | 5 --- src/Common/tests/gtest_async_loader.cpp | 12 +++++-- 3 files changed, 43 insertions(+), 18 deletions(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index e9de95363bc..140194e10b4 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -195,13 +196,6 @@ void LoadTask::remove() } } -void LoadTask::detach() -{ - jobs.clear(); - goal_jobs.clear(); -} - - AsyncLoader::AsyncLoader(std::vector pool_initializers, bool log_failures_, bool log_progress_) : log_failures(log_failures_) , log_progress(log_progress_) @@ -214,7 +208,22 @@ AsyncLoader::AsyncLoader(std::vector pool_initializers, bool lo AsyncLoader::~AsyncLoader() { - stop(); + // All `LoadTask` objects should be destructed before AsyncLoader destruction because they hold a reference. + // To make sure we check for all pending jobs to be finished. + std::unique_lock lock{mutex}; + if (scheduled_jobs.empty() && finished_jobs.empty()) + return; + + std::vector scheduled; + std::vector finished; + scheduled.reserve(scheduled_jobs.size()); + finished.reserve(finished_jobs.size()); + for (const auto & [job, _] : scheduled_jobs) + scheduled.push_back(job->name); + for (const auto & job : finished_jobs) + finished.push_back(job->name); + LOG_ERROR(log, "Bug. Destruction with pending ({}) and finished ({}) load jobs.", fmt::join(scheduled, ", "), fmt::join(finished, ", ")); + abort(); } void AsyncLoader::start() @@ -236,6 +245,17 @@ void AsyncLoader::wait() for (auto & p : pools) p.thread_pool->wait(); lock.lock(); + + // If there is no way for all jobs to finish, throw LOGICAL_ERROR instead of deadlock + if (!scheduled_jobs.empty() && !hasWorker(lock)) + { + std::vector names; + names.reserve(scheduled_jobs.size()); + for (const auto & [job, _] : scheduled_jobs) + names.push_back(job->name); + LOG_ERROR(log, "Waiting for load jobs to finish while being stopped: {}.", fmt::join(names, ", ")); + abort(); + } } } @@ -243,10 +263,12 @@ void AsyncLoader::stop() { { std::unique_lock lock{mutex}; - is_running = false; - // NOTE: there is no need to notify because workers never wait + is_running = false; // NOTE: there is no need to notify because workers never wait } - wait(); + + // Wait for all currently running jobs to finish (and do NOT wait all pending jobs) + for (auto & p : pools) + p.thread_pool->wait(); } void AsyncLoader::schedule(LoadTask & task) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index c2a9c901f1c..b1b336d24dc 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -198,10 +198,6 @@ public: // Remove all jobs of this task from AsyncLoader. void remove(); - // Do not track jobs in this task. - // WARNING: Jobs will never be removed() and are going to be stored as finished jobs until ~AsyncLoader(). - void detach(); - // Return the final jobs in this tasks. This job subset should be used as `dependencies` for dependent jobs or tasks: // auto load_task = loadSomethingAsync(async_loader, load_after_task.goals(), something); const LoadJobSet & goals() const { return goal_jobs.empty() ? jobs : goal_jobs; } @@ -333,7 +329,6 @@ private: public: AsyncLoader(std::vector pool_initializers, bool log_failures_, bool log_progress_); - // Stops AsyncLoader before destruction // WARNING: all tasks instances should be destructed before associated AsyncLoader. ~AsyncLoader(); diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index d978d23750c..950c7bbab76 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -622,7 +622,13 @@ TEST(AsyncLoader, CustomDependencyFailure) auto dependent_job1 = makeLoadJob({ collect_job }, "dependent_job1", dependent_job_func); auto dependent_job2 = makeLoadJob({ collect_job }, "dependent_job2", dependent_job_func); auto dependent_job3 = makeLoadJob({ collect_job }, "dependent_job3", dependent_job_func); - auto task = t.schedule({ dependent_job1, dependent_job2, dependent_job3 }); // Other jobs should be discovery automatically + auto task = t.schedule({ + dependent_job1, dependent_job2, dependent_job3, + collect_job, + late_dep1, late_dep2, late_dep3, + good_dep1, good_dep2, good_dep3, + evil_dep1, evil_dep2, evil_dep3, + }); t.loader.wait(collect_job, true); canceled_sync.arrive_and_wait(); // (A) @@ -1022,8 +1028,10 @@ TEST(AsyncLoader, SetMaxThreads) }; // Generate enough independent jobs + std::vector tasks; + tasks.reserve(1000); for (int i = 0; i < 1000; i++) - t.schedule({makeLoadJob({}, "job", job_func)})->detach(); + tasks.push_back(t.schedule({makeLoadJob({}, "job", job_func)})); t.loader.start(); while (sync_index < syncs.size()) From 41883ddb34d39dba8aaf5e06099c7623575239b6 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sun, 28 Jan 2024 18:15:11 +0100 Subject: [PATCH 251/264] Update run.sh --- docker/test/stateless/run.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 5c1398bf308..89e8eb13a6d 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -250,11 +250,14 @@ failed_to_save_logs=0 for table in query_log zookeeper_log trace_log transactions_info_log metric_log do err=$( { clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst; } 2>&1 ) + echo $err [[ "0" != "${#err}" ]] && failed_to_save_logs=1 if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then err=$( { clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst; } 2>&1 ) + echo $err [[ "0" != "${#err}" ]] && failed_to_save_logs=1 err=$( { clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst; } 2>&1 ) + echo $err [[ "0" != "${#err}" ]] && failed_to_save_logs=1 fi done From 4dcf27e5b20f3c00e3cec066eb0c462b6dda3f38 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jan 2024 18:27:15 +0100 Subject: [PATCH 252/264] /binary: allow specifying user/password/host --- programs/server/binary.html | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/programs/server/binary.html b/programs/server/binary.html index 988dd33a72a..74095dff537 100644 --- a/programs/server/binary.html +++ b/programs/server/binary.html @@ -60,10 +60,16 @@ /// If it is hosted on server, assume that it is the address of ClickHouse. if (location.protocol != 'file:') { host = location.origin; - user = 'default'; add_http_cors_header = false; } + if (window.location.search) { + const params = new URLSearchParams(window.location.search); + if (params.has('host')) { host = params.get('host'); } + if (params.has('user')) { user = params.get('user'); } + if (params.has('password')) { password = params.get('password'); } + } + let map = L.map('space', { crs: L.CRS.Simple, center: [-512, 512], From c7663c6d92b7841c118f23cc5c1f66589dff1c8b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jan 2024 18:31:10 +0100 Subject: [PATCH 253/264] Remove tests --- .../02366_kql_create_table.reference | 4 - .../0_stateless/02366_kql_create_table.sql | 29 -- .../0_stateless/02366_kql_datatype.reference | 105 ----- .../0_stateless/02366_kql_datatype.sql | 117 ------ .../0_stateless/02366_kql_distinct.reference | 27 -- .../0_stateless/02366_kql_distinct.sql | 28 -- .../0_stateless/02366_kql_extend.reference | 32 -- .../queries/0_stateless/02366_kql_extend.sql | 61 --- .../02366_kql_func_binary.reference | 7 - .../0_stateless/02366_kql_func_binary.sql | 8 - .../02366_kql_func_datetime.reference | 76 ---- .../0_stateless/02366_kql_func_datetime.sql | 86 ----- .../02366_kql_func_dynamic.reference | 152 -------- .../0_stateless/02366_kql_func_dynamic.sql | 161 -------- .../0_stateless/02366_kql_func_ip.reference | 123 ------ .../queries/0_stateless/02366_kql_func_ip.sql | 131 ------- .../0_stateless/02366_kql_func_math.reference | 4 - .../0_stateless/02366_kql_func_math.sql | 7 - .../02366_kql_func_scalar.reference | 16 - .../0_stateless/02366_kql_func_scalar.sql | 26 -- .../02366_kql_func_string.reference | 360 ------------------ .../0_stateless/02366_kql_func_string.sql | 313 --------------- .../02366_kql_makeseries.reference | 60 --- .../0_stateless/02366_kql_makeseries.sql | 77 ---- .../0_stateless/02366_kql_mvexpand.reference | 65 ---- .../0_stateless/02366_kql_mvexpand.sql | 35 -- ...02366_kql_native_interval_format.reference | 23 -- .../02366_kql_native_interval_format.sql.j2 | 16 - .../02366_kql_operator_in_sql.reference | 60 --- .../0_stateless/02366_kql_operator_in_sql.sql | 42 -- .../0_stateless/02366_kql_summarize.reference | 92 ----- .../0_stateless/02366_kql_summarize.sql | 102 ----- .../0_stateless/02366_kql_tabular.reference | 139 ------- .../queries/0_stateless/02366_kql_tabular.sql | 88 ----- 34 files changed, 2672 deletions(-) delete mode 100644 tests/queries/0_stateless/02366_kql_create_table.reference delete mode 100644 tests/queries/0_stateless/02366_kql_create_table.sql delete mode 100644 tests/queries/0_stateless/02366_kql_datatype.reference delete mode 100644 tests/queries/0_stateless/02366_kql_datatype.sql delete mode 100644 tests/queries/0_stateless/02366_kql_distinct.reference delete mode 100644 tests/queries/0_stateless/02366_kql_distinct.sql delete mode 100644 tests/queries/0_stateless/02366_kql_extend.reference delete mode 100644 tests/queries/0_stateless/02366_kql_extend.sql delete mode 100644 tests/queries/0_stateless/02366_kql_func_binary.reference delete mode 100644 tests/queries/0_stateless/02366_kql_func_binary.sql delete mode 100644 tests/queries/0_stateless/02366_kql_func_datetime.reference delete mode 100644 tests/queries/0_stateless/02366_kql_func_datetime.sql delete mode 100644 tests/queries/0_stateless/02366_kql_func_dynamic.reference delete mode 100644 tests/queries/0_stateless/02366_kql_func_dynamic.sql delete mode 100644 tests/queries/0_stateless/02366_kql_func_ip.reference delete mode 100644 tests/queries/0_stateless/02366_kql_func_ip.sql delete mode 100644 tests/queries/0_stateless/02366_kql_func_math.reference delete mode 100644 tests/queries/0_stateless/02366_kql_func_math.sql delete mode 100644 tests/queries/0_stateless/02366_kql_func_scalar.reference delete mode 100644 tests/queries/0_stateless/02366_kql_func_scalar.sql delete mode 100644 tests/queries/0_stateless/02366_kql_func_string.reference delete mode 100644 tests/queries/0_stateless/02366_kql_func_string.sql delete mode 100644 tests/queries/0_stateless/02366_kql_makeseries.reference delete mode 100644 tests/queries/0_stateless/02366_kql_makeseries.sql delete mode 100644 tests/queries/0_stateless/02366_kql_mvexpand.reference delete mode 100644 tests/queries/0_stateless/02366_kql_mvexpand.sql delete mode 100644 tests/queries/0_stateless/02366_kql_native_interval_format.reference delete mode 100644 tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 delete mode 100644 tests/queries/0_stateless/02366_kql_operator_in_sql.reference delete mode 100644 tests/queries/0_stateless/02366_kql_operator_in_sql.sql delete mode 100644 tests/queries/0_stateless/02366_kql_summarize.reference delete mode 100644 tests/queries/0_stateless/02366_kql_summarize.sql delete mode 100644 tests/queries/0_stateless/02366_kql_tabular.reference delete mode 100644 tests/queries/0_stateless/02366_kql_tabular.sql diff --git a/tests/queries/0_stateless/02366_kql_create_table.reference b/tests/queries/0_stateless/02366_kql_create_table.reference deleted file mode 100644 index 35136b5ff42..00000000000 --- a/tests/queries/0_stateless/02366_kql_create_table.reference +++ /dev/null @@ -1,4 +0,0 @@ --- test create table -- -Theodore -Diaz -Theodore Diaz 28 diff --git a/tests/queries/0_stateless/02366_kql_create_table.sql b/tests/queries/0_stateless/02366_kql_create_table.sql deleted file mode 100644 index b266679b06a..00000000000 --- a/tests/queries/0_stateless/02366_kql_create_table.sql +++ /dev/null @@ -1,29 +0,0 @@ -DROP TABLE IF EXISTS Customers; -CREATE TABLE Customers -( - FirstName Nullable(String), - LastName String, - Occupation String, - Education String, - Age Nullable(UInt8) -) ENGINE = Memory; - -INSERT INTO Customers VALUES ('Theodore','Diaz','Skilled Manual','Bachelors',28),('Stephanie','Cox','Management abcd defg','Bachelors',33),('Peter','Nara','Skilled Manual','Graduate Degree',26),('Latoya','Shen','Professional','Graduate Degree',25),('Apple','','Skilled Manual','Bachelors',28),(NULL,'why','Professional','Partial College',38); -Select '-- test create table --' ; -Select * from kql(Customers|project FirstName) limit 1;; -DROP TABLE IF EXISTS kql_table1; -CREATE TABLE kql_table1 ENGINE = Memory AS select *, now() as new_column From kql(Customers | project LastName | filter LastName=='Diaz'); -select LastName from kql_table1 limit 1; -DROP TABLE IF EXISTS kql_table2; -CREATE TABLE kql_table2 -( - FirstName Nullable(String), - LastName String, - Age Nullable(UInt8) -) ENGINE = Memory; -INSERT INTO kql_table2 select * from kql(Customers|project FirstName,LastName,Age | filter FirstName=='Theodore'); -select * from kql_table2 limit 1; --- select * from kql(Customers | where FirstName !in ("test", "test2")); -DROP TABLE IF EXISTS Customers; -DROP TABLE IF EXISTS kql_table1; -DROP TABLE IF EXISTS kql_table2; \ No newline at end of file diff --git a/tests/queries/0_stateless/02366_kql_datatype.reference b/tests/queries/0_stateless/02366_kql_datatype.reference deleted file mode 100644 index fe666f3734c..00000000000 --- a/tests/queries/0_stateless/02366_kql_datatype.reference +++ /dev/null @@ -1,105 +0,0 @@ --- bool -true -\N --- int -123 -\N --- long -123 -255 --1 -\N -456 --- real -0.01 -\N -nan -inf --inf --- datetime -2015-12-31 23:59:59.900000000 -2015-12-31 00:00:00.000000000 -2014-05-25 08:20:03.123456000 -2014-11-08 15:55:55.000000000 -2014-11-08 15:55:00.000000000 -2014-11-08 00:00:00.000000000 -\N -2014-05-25 08:20:03.123456000 -2014-11-08 15:55:55.123456000 --- time -1216984.12345 -45055.123 -86400 --86400 -6.000000000000001e-9 -6e-7 -172800 -259200 --- guid -\N --- timespan (time) -172800 -1800 -10 -0.1 -0.00001 -1e-7 -1120343 --- null -1 -\N \N \N \N \N --- decimal -\N -123.345 -100000 --- dynamic -\N -1 -86400 -[1,2,3] -[[1],[2],[3]] -['a','b','c'] --- cast functions -true -1 --- tobool("false") -false -1 --- tobool(1) -true -1 --- tobool(123) -true -1 --- tobool("abc") -\N -\N --- todouble() -123.4 -\N --- toreal() -123.4 -\N --- toint() -1 -\N --- tostring() -123 -1 --- todatetime() -1 -\N --- make_timespan() -01:12:00 01:12:30 1.12:30:55 --- totimespan() -1e-7 -60 -\N -1120343 --- tolong() -123 -\N --- todecimal() -123.345 -\N -\N diff --git a/tests/queries/0_stateless/02366_kql_datatype.sql b/tests/queries/0_stateless/02366_kql_datatype.sql deleted file mode 100644 index ecd29504298..00000000000 --- a/tests/queries/0_stateless/02366_kql_datatype.sql +++ /dev/null @@ -1,117 +0,0 @@ -set dialect = 'kusto'; - -print '-- bool' -print bool(true); -print bool(true); -print bool(null); -print '-- int'; -print int(123); -print int(null); -print int('4'); -- { clientError BAD_ARGUMENTS } -print '-- long'; -print long(123); -print long(0xff); -print long(-1); -print long(null); -print 456; -print '-- real'; -print real(0.01); -print real(null); -print real(nan); -print real(+inf); -print real(-inf); -print double('4.2'); -- { clientError BAD_ARGUMENTS } -print '-- datetime'; -print datetime(2015-12-31 23:59:59.9); -print datetime(2015-12-31); -print datetime('2014-05-25T08:20:03.123456'); -print datetime('2014-11-08 15:55:55'); -print datetime('2014-11-08 15:55'); -print datetime('2014-11-08'); -print datetime(null); -print datetime('2014-05-25T08:20:03.123456Z'); -print datetime('2014-11-08 15:55:55.123456Z'); -print '-- time'; -print time('14.02:03:04.12345'); -print time('12:30:55.123'); -print time(1d); -print time(-1d); -print time(6nanoseconds); -print time(6tick); -print time(2); -print time(2) + 1d; -print '-- guid' -print guid(74be27de-1e4e-49d9-b579-fe0b331d3642); -print guid(null); -print '-- timespan (time)'; -print timespan(2d); -- 2 days ---print timespan(1.5h); -- 1.5 hour -print timespan(30m); -- 30 minutes -print timespan(10s); -- 10 seconds ---print timespan(0.1s); -- 0.1 second -print timespan(100ms); -- 100 millisecond -print timespan(10microsecond); -- 10 microseconds -print timespan(1tick); -- 100 nanoseconds ---print timespan(1.5h) / timespan(30m); -print timespan('12.23:12:23') / timespan(1s); -print '-- null'; -print isnull(null); -print bool(null), int(null), long(null), real(null), double(null); -print '-- decimal'; -print decimal(null); -print decimal(123.345); -print decimal(1e5); -print '-- dynamic'; -- no support for mixed types and bags for now -print dynamic(null); -print dynamic(1); -print dynamic(timespan(1d)); -print dynamic([1,2,3]); -print dynamic([[1], [2], [3]]); -print dynamic(['a', "b", 'c']); -print '-- cast functions' -print '--tobool("true")'; -- == true -print tobool('true'); -- == true -print tobool('true') == toboolean('true'); -- == true -print '-- tobool("false")'; -- == false -print tobool('false'); -- == false -print tobool('false') == toboolean('false'); -- == false -print '-- tobool(1)'; -- == true -print tobool(1); -- == true -print tobool(1) == toboolean(1); -- == true -print '-- tobool(123)'; -- == true -print tobool(123); -- == true -print tobool(123) == toboolean(123); -- == true -print '-- tobool("abc")'; -- == null -print tobool('abc'); -- == null -print tobool('abc') == toboolean('abc'); -- == null -print '-- todouble()'; -print todouble('123.4'); -print todouble('abc') == null; -print '-- toreal()'; -print toreal("123.4"); -print toreal('abc') == null; -print '-- toint()'; -print toint("123") == int(123); -print toint('abc'); -print '-- tostring()'; -print tostring(123); -print tostring(null) == ''; -print '-- todatetime()'; -print todatetime("2015-12-24") == datetime(2015-12-24); -print todatetime('abc') == null; -print '-- make_timespan()'; -print v1=make_timespan(1,12), v2=make_timespan(1,12,30), v3=make_timespan(1,12,30,55.123); -print '-- totimespan()'; -print totimespan(1tick); -print totimespan('0.00:01:00'); -print totimespan('abc'); -print totimespan('12.23:12:23') / totimespan(1s); --- print totimespan(strcat('12.', '23', ':12:', '23')) / timespan(1s); -> 1120343 -print '-- tolong()'; -print tolong('123'); -print tolong('abc'); -print '-- todecimal()'; -print todecimal(123.345); -print todecimal(null); -print todecimal('abc'); --- print todecimal(4 * 2 + 3); -> 11 diff --git a/tests/queries/0_stateless/02366_kql_distinct.reference b/tests/queries/0_stateless/02366_kql_distinct.reference deleted file mode 100644 index 2100f44f18c..00000000000 --- a/tests/queries/0_stateless/02366_kql_distinct.reference +++ /dev/null @@ -1,27 +0,0 @@ --- distinct * -- -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Peter Nara Skilled Manual Graduate Degree 26 -Latoya Shen Professional Graduate Degree 25 -Apple Skilled Manual Bachelors 28 -\N why Professional Partial College 38 --- distinct one column -- -Skilled Manual -Management abcd defg -Professional --- distinct two column -- -Skilled Manual Bachelors -Management abcd defg Bachelors -Skilled Manual Graduate Degree -Professional Graduate Degree -Professional Partial College --- distinct with where -- -Skilled Manual Bachelors -Management abcd defg Bachelors -Skilled Manual Graduate Degree -Professional Graduate Degree -Professional Partial College --- distinct with where, order -- -Skilled Manual Bachelors -Skilled Manual Graduate Degree -Professional Graduate Degree diff --git a/tests/queries/0_stateless/02366_kql_distinct.sql b/tests/queries/0_stateless/02366_kql_distinct.sql deleted file mode 100644 index 3c997eb4865..00000000000 --- a/tests/queries/0_stateless/02366_kql_distinct.sql +++ /dev/null @@ -1,28 +0,0 @@ -DROP TABLE IF EXISTS Customers; -CREATE TABLE Customers -( - FirstName Nullable(String), - LastName String, - Occupation String, - Education String, - Age Nullable(UInt8) -) ENGINE = Memory; - -INSERT INTO Customers VALUES ('Theodore','Diaz','Skilled Manual','Bachelors',28), ('Stephanie','Cox','Management abcd defg','Bachelors',33),('Peter','Nara','Skilled Manual','Graduate Degree',26),('Latoya','Shen','Professional','Graduate Degree',25),('Apple','','Skilled Manual','Bachelors',28),(NULL,'why','Professional','Partial College',38); - -set dialect = 'kusto'; - -print '-- distinct * --'; -Customers | distinct *; - -print '-- distinct one column --'; -Customers | distinct Occupation; - -print '-- distinct two column --'; -Customers | distinct Occupation, Education; - -print '-- distinct with where --'; -Customers where Age <30 | distinct Occupation, Education; - -print '-- distinct with where, order --'; -Customers |where Age <30 | order by Age| distinct Occupation, Education; diff --git a/tests/queries/0_stateless/02366_kql_extend.reference b/tests/queries/0_stateless/02366_kql_extend.reference deleted file mode 100644 index 2936c9ea19c..00000000000 --- a/tests/queries/0_stateless/02366_kql_extend.reference +++ /dev/null @@ -1,32 +0,0 @@ --- extend #1 -- -Aldi Apple 4 2016-09-10 400 -Costco Apple 2 2016-09-11 200 --- extend #2 -- -Apple 200 -Apple 400 --- extend #3 -- -Apple cost 480 on average based on 5 samples. -Snargaluff cost 28080 on average based on 5 samples. --- extend #4 -- -1 --- extend #5 -- -Aldi Apple 4 2016-09-10 Apple was purchased from Aldi for $4 on 2016-09-10 400 -Costco Apple 2 2016-09-11 Apple was purchased from Costco for $2 on 2016-09-11 200 --- extend #6 -- -Aldi Apple 2016-09-10 400 -Costco Apple 2016-09-11 200 -Aldi Apple 2016-09-10 600 -Costco Snargaluff 2016-09-12 10000 -Aldi Apple 2016-09-12 700 -Aldi Snargaluff 2016-09-11 40000 -Costco Snargaluff 2016-09-12 10400 -Aldi Apple 2016-09-12 500 -Aldi Snargaluff 2016-09-11 60000 -Costco Snargaluff 2016-09-10 20000 --- extend #7 -- -5 --- extend #8 -- --- extend #9 -- --- extend #10 -- --- extend #11 -- -5 [2,1] diff --git a/tests/queries/0_stateless/02366_kql_extend.sql b/tests/queries/0_stateless/02366_kql_extend.sql deleted file mode 100644 index 0a3c1f3dcd4..00000000000 --- a/tests/queries/0_stateless/02366_kql_extend.sql +++ /dev/null @@ -1,61 +0,0 @@ --- datatable(Supplier:string, Fruit:string, Price: real, Purchase:datetime) --- [ --- 'Aldi','Apple',4,'2016-09-10', --- 'Costco','Apple',2,'2016-09-11', --- 'Aldi','Apple',6,'2016-09-10', --- 'Costco','Snargaluff',100,'2016-09-12', --- 'Aldi','Apple',7,'2016-09-12', --- 'Aldi','Snargaluff',400,'2016-09-11', --- 'Costco','Snargaluff',104,'2016-09-12', --- 'Aldi','Apple',5,'2016-09-12', --- 'Aldi','Snargaluff',600,'2016-09-11', --- 'Costco','Snargaluff',200,'2016-09-10', --- ] - - -DROP TABLE IF EXISTS Ledger; -CREATE TABLE Ledger -( - Supplier Nullable(String), - Fruit String , - Price Float64, - Purchase Date -) ENGINE = Memory; -INSERT INTO Ledger VALUES ('Aldi','Apple',4,'2016-09-10'), ('Costco','Apple',2,'2016-09-11'), ('Aldi','Apple',6,'2016-09-10'), ('Costco','Snargaluff',100,'2016-09-12'), ('Aldi','Apple',7,'2016-09-12'), ('Aldi','Snargaluff',400,'2016-09-11'),('Costco','Snargaluff',104,'2016-09-12'),('Aldi','Apple',5,'2016-09-12'),('Aldi','Snargaluff',600,'2016-09-11'),('Costco','Snargaluff',200,'2016-09-10'); - --- This test requies sorting after some of aggregations but I don't know KQL, sorry -set max_bytes_before_external_group_by = 0; -set dialect = 'kusto'; - -print '-- extend #1 --'; -Ledger | extend PriceInCents = 100 * Price | take 2; - -print '-- extend #2 --'; -Ledger | extend PriceInCents = 100 * Price | sort by PriceInCents asc | project Fruit, PriceInCents | take 2; - -print '-- extend #3 --'; -Ledger | extend PriceInCents = 100 * Price | sort by PriceInCents asc | project Fruit, PriceInCents | summarize AveragePrice = avg(PriceInCents), Purchases = count() by Fruit | extend Sentence = strcat(Fruit, ' cost ', tostring(AveragePrice), ' on average based on ', tostring(Purchases), ' samples.') | project Sentence; - -print '-- extend #4 --'; -Ledger | extend a = Price | extend b = a | extend c = a, d = b + 500 | extend Pass = bool(b == a and c == a and d == b + 500) | summarize binary_all_and(Pass); - -print '-- extend #5 --'; -Ledger | take 2 | extend strcat(Fruit, ' was purchased from ', Supplier, ' for $', tostring(Price), ' on ', tostring(Purchase)) | extend PriceInCents = 100 * Price; - -print '-- extend #6 --'; -Ledger | extend Price = 100 * Price; - -print '-- extend #7 --'; -print a = 4 | extend a = 5; - -print '-- extend #8 --'; --- print x = 5 | extend array_sort_desc(range(0, x), range(1, x + 1)) - -print '-- extend #9 --'; -print x = 19 | extend = 4 + ; -- { clientError SYNTAX_ERROR } - -print '-- extend #10 --'; -Ledger | extend PriceInCents = * Price | sort by PriceInCents asc | project Fruit, PriceInCents | summarize AveragePrice = avg(PriceInCents), Purchases = count() by Fruit | extend Sentence = strcat(Fruit, ' cost ', tostring(AveragePrice), ' on average based on ', tostring(Purchases), ' samples.') | project Sentence; -- { clientError SYNTAX_ERROR } - -print '-- extend #11 --'; -- should ideally return this in the future: 5 [2,1] because of the alias ex -print x = 5 | extend ex = array_sort_desc(dynamic([1, 2]), dynamic([3, 4])); diff --git a/tests/queries/0_stateless/02366_kql_func_binary.reference b/tests/queries/0_stateless/02366_kql_func_binary.reference deleted file mode 100644 index 6276cd6d867..00000000000 --- a/tests/queries/0_stateless/02366_kql_func_binary.reference +++ /dev/null @@ -1,7 +0,0 @@ - -- binary functions -4 7 -1 -1 -1 -7 3 -1 diff --git a/tests/queries/0_stateless/02366_kql_func_binary.sql b/tests/queries/0_stateless/02366_kql_func_binary.sql deleted file mode 100644 index 824022b564c..00000000000 --- a/tests/queries/0_stateless/02366_kql_func_binary.sql +++ /dev/null @@ -1,8 +0,0 @@ -set dialect='kusto'; -print ' -- binary functions'; -print binary_and(4,7), binary_or(4,7); -print binary_shift_left(1, 1) == binary_shift_left(1, 65); -print binary_shift_right(2, 1) == binary_shift_right(2, 65); -print binary_shift_right(binary_shift_left(1, 65), 65) == 1; -print binary_xor(2, 5), bitset_count_ones(42); -print bitset_count_ones(binary_shift_left(binary_and(4,7), 1)); diff --git a/tests/queries/0_stateless/02366_kql_func_datetime.reference b/tests/queries/0_stateless/02366_kql_func_datetime.reference deleted file mode 100644 index 40d8d7e19ac..00000000000 --- a/tests/queries/0_stateless/02366_kql_func_datetime.reference +++ /dev/null @@ -1,76 +0,0 @@ --- dayofmonth() -31 --- dayofweek() -4.00:00:00 --- dayofyear() -365 --- getmonth() -10 --- getyear() -2015 --- hoursofday() -23 --- startofday() -2017-01-01 00:00:00.000000000 -2016-12-31 00:00:00.000000000 -2017-01-02 00:00:00.000000000 --- endofday() -2017-01-01 23:59:59.999999000 -2016-12-31 23:59:59.999999000 -2017-01-02 23:59:59.999999000 --- endofmonth() -2017-01-31 23:59:59.999999000 -2016-12-31 23:59:59.999999000 -2017-02-28 23:59:59.999999000 -2022-09-30 23:59:59.999999000 --- startofweek() -2017-01-01 00:00:00.000000000 -2016-12-25 00:00:00.000000000 -2017-01-08 00:00:00.000000000 --- endofweek() -2017-01-07 23:59:59.999999000 -2016-12-31 23:59:59.999999000 -2017-01-14 23:59:59.999999000 --- startofyear() -2017-01-01 00:00:00.000000000 -2016-01-01 00:00:00.000000000 -2018-01-01 00:00:00.000000000 --- endofyear() -2017-12-31 23:59:59.999999000 -2016-12-31 23:59:59.999999000 -2018-12-31 23:59:59.999999000 --- unixtime_seconds_todatetime() -2019-01-01 00:00:00.000000000 -1970-01-02 00:00:00.000000000 -1969-12-31 00:00:00.000000000 --- unixtime_microseconds_todatetime -2019-01-01 00:00:00.000000 --- unixtime_milliseconds_todatetime() -2019-01-01 00:00:00.000 --- unixtime_nanoseconds_todatetime() -2019-01-01 00:00:00.000000000 --- weekofyear() -52 --- monthofyear() -12 --- weekofyear() -52 --- now() -1 --- make_datetime() -1 -2017-10-01 12:10:00.0000000 -2017-10-01 12:11:00.0000000 --- format_datetime -15-12-14 02:03:04.1234500 -17-01-29 [09:00:05] 2017-01-29 [09:00:05] 17-01-29 [09:00:05 AM] --- format_timespan() -02:03:04.1234500 -29.09:00:05:12 --- ago() --- datetime_diff() -17 2 13 4 29 2 5 10 --- datetime_part() -2017 4 10 44 30 303 01 02 03 --- datetime_add() -2018-01-01 00:00:00.0000000 2017-04-01 00:00:00.0000000 2017-02-01 00:00:00.0000000 2017-01-08 00:00:00.0000000 2017-01-02 00:00:00.0000000 2017-01-01 01:00:00.0000000 2017-01-01 00:01:00.0000000 2017-01-01 00:00:01.0000000 diff --git a/tests/queries/0_stateless/02366_kql_func_datetime.sql b/tests/queries/0_stateless/02366_kql_func_datetime.sql deleted file mode 100644 index b1fba4166a9..00000000000 --- a/tests/queries/0_stateless/02366_kql_func_datetime.sql +++ /dev/null @@ -1,86 +0,0 @@ -set dialect = 'kusto'; - -print '-- dayofmonth()'; -print dayofmonth(datetime(2015-12-31)); -print '-- dayofweek()'; -print dayofweek(datetime(2015-12-31)); -print '-- dayofyear()'; -print dayofyear(datetime(2015-12-31)); -print '-- getmonth()'; -print getmonth(datetime(2015-10-12)); -print '-- getyear()'; -print getyear(datetime(2015-10-12)); -print '-- hoursofday()'; -print hourofday(datetime(2015-12-31 23:59:59.9)); -print '-- startofday()'; -print startofday(datetime(2017-01-01 10:10:17)); -print startofday(datetime(2017-01-01 10:10:17), -1); -print startofday(datetime(2017-01-01 10:10:17), 1); -print '-- endofday()'; -print endofday(datetime(2017-01-01 10:10:17)); -print endofday(datetime(2017-01-01 10:10:17), -1); -print endofday(datetime(2017-01-01 10:10:17), 1); -print '-- endofmonth()'; -print endofmonth(datetime(2017-01-01 10:10:17)); -print endofmonth(datetime(2017-01-01 10:10:17), -1); -print endofmonth(datetime(2017-01-01 10:10:17), 1); -print endofmonth(datetime(2022-09-23)); -print '-- startofweek()'; -print startofweek(datetime(2017-01-01 10:10:17)); -print startofweek(datetime(2017-01-01 10:10:17), -1); -print startofweek(datetime(2017-01-01 10:10:17), 1); -print '-- endofweek()'; -print endofweek(datetime(2017-01-01 10:10:17)); -print endofweek(datetime(2017-01-01 10:10:17), -1); -print endofweek(datetime(2017-01-01 10:10:17), 1); -print '-- startofyear()'; -print startofyear(datetime(2017-01-01 10:10:17)); -print startofyear(datetime(2017-01-01 10:10:17), -1); -print startofyear(datetime(2017-01-01 10:10:17), 1); -print '-- endofyear()'; -print endofyear(datetime(2017-01-01 10:10:17)); -print endofyear(datetime(2017-01-01 10:10:17), -1); -print endofyear(datetime(2017-01-01 10:10:17), 1); -print '-- unixtime_seconds_todatetime()'; -print unixtime_seconds_todatetime(1546300800); -print unixtime_seconds_todatetime(1d); -print unixtime_seconds_todatetime(-1d); -print '-- unixtime_microseconds_todatetime'; -print unixtime_microseconds_todatetime(1546300800000000); -print '-- unixtime_milliseconds_todatetime()'; -print unixtime_milliseconds_todatetime(1546300800000); -print '-- unixtime_nanoseconds_todatetime()'; -print unixtime_nanoseconds_todatetime(1546300800000000000); -print '-- weekofyear()'; -print week_of_year(datetime(2000-01-01)); -print '-- monthofyear()'; -print monthofyear(datetime(2015-12-31)); -print '-- weekofyear()'; -print week_of_year(datetime(2000-01-01)); -print '-- now()'; -print getyear(now(-2d))>1900; -print '-- make_datetime()'; -print make_datetime(2017,10,01,12,10) == datetime(2017-10-01 12:10:00); -print year_month_day_hour_minute = make_datetime(2017,10,01,12,10); -print year_month_day_hour_minute_second = make_datetime(2017,10,01,12,11,0.1234567); -print '-- format_datetime'; -print format_datetime(datetime(2015-12-14 02:03:04.12345), 'y-M-d h:m:s.fffffff'); -print v1=format_datetime(datetime(2017-01-29 09:00:05),'yy-MM-dd [HH:mm:ss]'), v2=format_datetime(datetime(2017-01-29 09:00:05), 'yyyy-M-dd [H:mm:ss]'), v3=format_datetime(datetime(2017-01-29 09:00:05), 'yy-MM-dd [hh:mm:ss tt]'); -print '-- format_timespan()'; -print format_timespan(time('14.02:03:04.12345'), 'h:m:s.fffffff'); -print v1=format_timespan(time('29.09:00:05.12345'), 'dd.hh:mm:ss:FF'); --- print v2=format_timespan(time('29.09:00:05.12345'), 'ddd.h:mm:ss [fffffff]'); == '029.9:00:05 [1234500]' -print '-- ago()'; --- print ago(1d) - now(); -print '-- datetime_diff()'; -print year = datetime_diff('year',datetime(2017-01-01),datetime(2000-12-31)), quarter = datetime_diff('quarter',datetime(2017-07-01),datetime(2017-03-30)), month = datetime_diff('month',datetime(2017-01-01),datetime(2015-12-30)), week = datetime_diff('week',datetime(2017-10-29 00:00),datetime(2017-09-30 23:59)), day = datetime_diff('day',datetime(2017-10-29 00:00),datetime(2017-09-30 23:59)), hour = datetime_diff('hour',datetime(2017-10-31 01:00),datetime(2017-10-30 23:59)), minute = datetime_diff('minute',datetime(2017-10-30 23:05:01),datetime(2017-10-30 23:00:59)), second = datetime_diff('second',datetime(2017-10-30 23:00:10.100),datetime(2017-10-30 23:00:00.900)); --- millisecond = datetime_diff('millisecond',datetime(2017-10-30 23:00:00.200100),datetime(2017-10-30 23:00:00.100900)), --- microsecond = datetime_diff('microsecond',datetime(2017-10-30 23:00:00.1009001),datetime(2017-10-30 23:00:00.1008009)), --- nanosecond = datetime_diff('nanosecond',datetime(2017-10-30 23:00:00.0000000),datetime(2017-10-30 23:00:00.0000007)) -print '-- datetime_part()'; -print year = datetime_part("year", datetime(2017-10-30 01:02:03.7654321)),quarter = datetime_part("quarter", datetime(2017-10-30 01:02:03.7654321)),month = datetime_part("month", datetime(2017-10-30 01:02:03.7654321)),weekOfYear = datetime_part("week_of_year", datetime(2017-10-30 01:02:03.7654321)),day = datetime_part("day", datetime(2017-10-30 01:02:03.7654321)),dayOfYear = datetime_part("dayOfYear", datetime(2017-10-30 01:02:03.7654321)),hour = datetime_part("hour", datetime(2017-10-30 01:02:03.7654321)),minute = datetime_part("minute", datetime(2017-10-30 01:02:03.7654321)),second = datetime_part("second", datetime(2017-10-30 01:02:03.7654321)); --- millisecond = datetime_part("millisecond", dt), --- microsecond = datetime_part("microsecond", dt), --- nanosecond = datetime_part("nanosecond", dt) -print '-- datetime_add()'; -print year = datetime_add('year',1,make_datetime(2017,1,1)),quarter = datetime_add('quarter',1,make_datetime(2017,1,1)),month = datetime_add('month',1,make_datetime(2017,1,1)),week = datetime_add('week',1,make_datetime(2017,1,1)),day = datetime_add('day',1,make_datetime(2017,1,1)),hour = datetime_add('hour',1,make_datetime(2017,1,1)),minute = datetime_add('minute',1,make_datetime(2017,1,1)),second = datetime_add('second',1,make_datetime(2017,1,1)); \ No newline at end of file diff --git a/tests/queries/0_stateless/02366_kql_func_dynamic.reference b/tests/queries/0_stateless/02366_kql_func_dynamic.reference deleted file mode 100644 index 564f1eebc4b..00000000000 --- a/tests/queries/0_stateless/02366_kql_func_dynamic.reference +++ /dev/null @@ -1,152 +0,0 @@ --- constant index value -1 c ['A',NULL,'C'] --- array_length() -1 -1 --- array_sum() -1 -1 --- array_index_of() -3 -1 --- array_iif() -[1,5,3] -[1,5,3] -[1,5,NULL] -[NULL,NULL,NULL] --- array_concat() -[1,2,3,4,5,6] --- array_reverse() -[] -[1] -[4,3,2,1] -['example','an','is','this'] --- array_rotate_left() -[] -[] -[] -[3,4,5,1,2] -[1,2,3,4,5] -[3,4,5,1,2] -[4,5,1,2,3] -[1,2,3,4,5] -[4,5,1,2,3] --- array_rotate_right() -[] -[] -[] -[4,5,1,2,3] -[1,2,3,4,5] -[4,5,1,2,3] -[3,4,5,1,2] -[1,2,3,4,5] -[3,4,5,1,2] --- array_shift_left() -[] -[] -[] -[3,4,5,NULL,NULL] -[NULL,NULL,1,2,3] -[3,4,5,-1,-1] -['c','',''] --- array_shift_right() -[] -[] -[] -[3,4,5,NULL,NULL] -[NULL,NULL,1,2,3] -[3,4,5,-1,-1] -['c','',''] --- array_slice() -[3,4] --- array_split() -[[1],[2,3],[4,5]] -[[1,2],[3,4,5]] -[[1],[2,3],[4,5]] -[[1,2,3,4],[],[4,5]] --- array_sort_asc() -(['a','c','c','d',NULL]) -([1,2,3,4]) -['a','b','c'] -(['p','q','r'],['hello','clickhouse','world']) -([NULL,'a','c','c','d']) -([NULL,'a','c','c','d']) -([NULL,NULL,NULL]) -[1,2,3,NULL,NULL] -['a','e','b','c','d'] -(['George','John','Paul','Ringo']) -(['blue','green','yellow',NULL,NULL]) -([NULL,NULL,'blue','green','yellow']) --- array_sort_desc() -(['d','c','c','a',NULL]) -([4,3,2,1]) -['c','b','a'] -(['r','q','p'],['world','clickhouse','hello']) -([NULL,'d','c','c','a']) -([NULL,'d','c','c','a']) -([NULL,NULL,NULL]) -[3,2,1,NULL,NULL] -['d','c','b','e','a'] -(['Ringo','Paul','John','George']) -(['yellow','green','blue',NULL,NULL]) -([NULL,NULL,'yellow','green','blue']) --- jaccard_index() -0.75 -0 -0 -nan -0 -0.75 -0.25 --- pack_array() -1 2 4 [1,2,4] -['ab','0.0.0.42','4.2'] --- repeat() -[] -[1,1,1] -['asd','asd','asd'] -[86400,86400,86400] -[true,true,true] -[NULL] -[NULL] --- set_difference() -[] -[] -[] -[] -[4,5,6] -[4] -[1,3] -[1,2,3] -['d','s'] -['Chewbacca','Han Solo'] --- set_has_element() -0 -1 -0 -1 -0 --- set_intersect() -[] -[1,2,3] -[1,2,3] -[] -[5] -[] -['a'] -['Darth Vader'] --- set_union() -[] -[1,2,3] -[1,2,3,4,5,6] -[1,2,3,4] -[1,2,3,4,5] -[1,2,3] -['a','d','f','s'] -['Chewbacca','Darth Sidious','Darth Vader','Han Solo'] --- zip() -[] -[[1,2],[3,4],[5,6]] -[['Darth','Vader','has a suit'],['Master','Yoda','doesn\'t have a suit']] -[[1,10],[2,20],[3,NULL]] -[[NULL,1],[NULL,2],[NULL,3]] diff --git a/tests/queries/0_stateless/02366_kql_func_dynamic.sql b/tests/queries/0_stateless/02366_kql_func_dynamic.sql deleted file mode 100644 index b0956f032d0..00000000000 --- a/tests/queries/0_stateless/02366_kql_func_dynamic.sql +++ /dev/null @@ -1,161 +0,0 @@ -DROP TABLE IF EXISTS array_test; -CREATE TABLE array_test (floats Array(Float64), - strings Array(String), - nullable_strings Array(Nullable(String)) - ) ENGINE=Memory; -INSERT INTO array_test VALUES([1.0, 2.5], ['a', 'c'], ['A', NULL, 'C']); -set dialect = 'kusto'; -print '-- constant index value'; -array_test | project floats[0], strings[1], nullable_strings; -print '-- array_length()'; -print array_length(dynamic(['John', 'Denver', 'Bob', 'Marley'])) == 4; -print array_length(dynamic([1, 2, 3])) == 3; -print '-- array_sum()'; -print array_sum(dynamic([2, 5, 3])) == 10; -print array_sum(dynamic([2.5, 5.5, 3])) == 11; -print '-- array_index_of()'; -print array_index_of(dynamic(['John', 'Denver', 'Bob', 'Marley']), 'Marley'); -print array_index_of(dynamic([1, 2, 3]), 2); -print '-- array_iif()'; -print array_iif(dynamic([true,false,true]), dynamic([1,2,3]), dynamic([4,5,6])); -print array_iif(dynamic([1,0,1]), dynamic([1,2,3]), dynamic([4,5,6])); -print array_iif(dynamic([true,false,true]), dynamic([1,2]), dynamic([4,5,6])); -print array_iif(dynamic(['a','b','c']), dynamic([1,2,3]), dynamic([4,5,6])); -print '-- array_concat()'; -print array_concat(dynamic([1,2,3]),dynamic([4,5,6])); -print '-- array_reverse()'; -print array_reverse(dynamic([])); -print array_reverse(dynamic([1])); -print array_reverse(dynamic([1,2,3,4])); -print array_reverse(dynamic(["this", "is", "an", "example"])); -print '-- array_rotate_left()'; -print array_rotate_left(dynamic([]), 0); -print array_rotate_left(dynamic([]), 500); -print array_rotate_left(dynamic([]), -500); -print array_rotate_left(dynamic([1,2,3,4,5]), 2); -print array_rotate_left(dynamic([1,2,3,4,5]), 5); -print array_rotate_left(dynamic([1,2,3,4,5]), 7); -print array_rotate_left(dynamic([1,2,3,4,5]), -2); -print array_rotate_left(dynamic([1,2,3,4,5]), -5); -print array_rotate_left(dynamic([1,2,3,4,5]), -7); -print '-- array_rotate_right()'; -print array_rotate_right(dynamic([]), 0); -print array_rotate_right(dynamic([]), 500); -print array_rotate_right(dynamic([]), -500); -print array_rotate_right(dynamic([1,2,3,4,5]), 2); -print array_rotate_right(dynamic([1,2,3,4,5]), 5); -print array_rotate_right(dynamic([1,2,3,4,5]), 7); -print array_rotate_right(dynamic([1,2,3,4,5]), -2); -print array_rotate_right(dynamic([1,2,3,4,5]), -5); -print array_rotate_right(dynamic([1,2,3,4,5]), -7); -print '-- array_shift_left()'; -print array_shift_left(dynamic([]), 0); -print array_shift_left(dynamic([]), 555); -print array_shift_left(dynamic([]), -555); -print array_shift_left(dynamic([1,2,3,4,5]), 2); -print array_shift_left(dynamic([1,2,3,4,5]), -2); -print array_shift_left(dynamic([1,2,3,4,5]), 2, -1); -print array_shift_left(dynamic(['a', 'b', 'c']), 2); -print '-- array_shift_right()'; -print array_shift_left(dynamic([]), 0); -print array_shift_left(dynamic([]), 555); -print array_shift_left(dynamic([]), -555); -print array_shift_right(dynamic([1,2,3,4,5]), -2); -print array_shift_right(dynamic([1,2,3,4,5]), 2); -print array_shift_right(dynamic([1,2,3,4,5]), -2, -1); -print array_shift_right(dynamic(['a', 'b', 'c']), -2); -print '-- array_slice()'; ---print array_slice(dynamic([1,2,3]), 1, 2); -- will enable whe analyzer dixed -print array_slice(dynamic([1,2,3,4,5]), -3, -2); -print '-- array_split()'; -print array_split(dynamic([1,2,3,4,5]), dynamic([1,-2])); -print array_split(dynamic([1,2,3,4,5]), 2); -print array_split(dynamic([1,2,3,4,5]), dynamic([1,3])); -print array_split(dynamic([1,2,3,4,5]), dynamic([-1,-2])); -print '-- array_sort_asc()'; -print array_sort_asc(dynamic([null, 'd', 'a', 'c', 'c'])); -print array_sort_asc(dynamic([4, 1, 3, 2])); -print array_sort_asc(dynamic(['b', 'a', 'c']), dynamic(['q', 'p', 'r']))[0]; -print array_sort_asc(dynamic(['q', 'p', 'r']), dynamic(['clickhouse','hello', 'world'])); -print array_sort_asc( dynamic(['d', null, 'a', 'c', 'c']) , false); -print array_sort_asc( dynamic(['d', null, 'a', 'c', 'c']) , 1 > 2); -print array_sort_asc( dynamic([null, null, null]) , false); -print array_sort_asc(dynamic([2, 1, null,3, null]), dynamic([20, 10, 40, 30, 50]), 1 < 2)[0]; -print array_sort_asc(dynamic(['1','3','4','5','2']),dynamic(["a","b","c","d","e"]), dynamic(["a","b","c","d","e"]), dynamic(["a","b","c","d","e"]))[3]; -print array_sort_asc(split("John,Paul,George,Ringo", ",")); -print array_sort_asc(dynamic([null,"blue","yellow","green",null])); -print array_sort_asc(dynamic([null,"blue","yellow","green",null]), false); -print '-- array_sort_desc()'; -print array_sort_desc(dynamic([null, 'd', 'a', 'c', 'c'])); -print array_sort_desc(dynamic([4, 1, 3, 2])); -print array_sort_desc(dynamic(['b', 'a', 'c']), dynamic(['q', 'p', 'r']))[0]; -print array_sort_desc(dynamic(['q', 'p', 'r']), dynamic(['clickhouse','hello', 'world'])); -print array_sort_desc( dynamic(['d', null, 'a', 'c', 'c']) , false); -print array_sort_desc( dynamic(['d', null, 'a', 'c', 'c']) , 1 > 2); -print array_sort_desc( dynamic([null, null, null]) , false); -print array_sort_desc(dynamic([2, 1, null,3, null]), dynamic([20, 10, 40, 30, 50]), 1 < 2)[0]; -print array_sort_desc(dynamic(['1','3','4','5','2']),dynamic(["a","b","c","d","e"]), dynamic(["a","b","c","d","e"]), dynamic(["a","b","c","d","e"]))[3]; -print array_sort_desc(split("John,Paul,George,Ringo", ",")); -print array_sort_desc(dynamic([null,"blue","yellow","green",null])); -print array_sort_desc(dynamic([null,"blue","yellow","green",null]), false); -print '-- jaccard_index()'; -print jaccard_index(dynamic([1, 1, 2, 2, 3, 3]), dynamic([1, 2, 3, 4, 4, 4])); -print jaccard_index(dynamic([1, 2, 3]), dynamic([])); -print jaccard_index(dynamic([]), dynamic([1, 2, 3, 4])); -print jaccard_index(dynamic([]), dynamic([])); -print jaccard_index(dynamic([1, 2, 3]), dynamic([4, 5, 6, 7])); -print jaccard_index(dynamic(['a', 's', 'd']), dynamic(['f', 'd', 's', 'a'])); -print jaccard_index(dynamic(['Chewbacca', 'Darth Vader', 'Han Solo']), dynamic(['Darth Sidious', 'Darth Vader'])); -print '-- pack_array()'; -print pack_array(); -- { clientError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -print x = 1 | extend y = x * 2 | extend z = y * 2 | extend pack_array(x,y,z); -print pack_array(strcat('a', 'b'), format_ipv4(42), tostring(4.2)); -print '-- repeat()'; -print repeat(1, 0); -print repeat(1, 3); -print repeat("asd", 3); -print repeat(timespan(1d), 3); -print repeat(true, 3); -print repeat(1, -3); -print repeat(6.7,-4); -print '-- set_difference()'; -print set_difference(dynamic([]), dynamic([])); -print set_difference(dynamic([]), dynamic([9])); -print set_difference(dynamic([]), dynamic(["asd"])); -print set_difference(dynamic([1, 1, 2, 2, 3, 3]), dynamic([1, 2, 3])); -print array_sort_asc(set_difference(dynamic([1, 4, 2, 3, 5, 4, 6]), dynamic([1, 2, 3])))[0]; -print set_difference(dynamic([4]), dynamic([1, 2, 3])); -print array_sort_asc(set_difference(dynamic([1, 2, 3, 4, 5]), dynamic([5]), dynamic([2, 4])))[0]; -print array_sort_asc(set_difference(dynamic([1, 2, 3]), dynamic([])))[0]; -print array_sort_asc(set_difference(dynamic(['a', 's', 'd']), dynamic(['a', 'f'])))[0]; -print array_sort_asc(set_difference(dynamic(['Chewbacca', 'Darth Vader', 'Han Solo']), dynamic(['Darth Sidious', 'Darth Vader'])))[0]; -print '-- set_has_element()'; -print set_has_element(dynamic([]), 9); -print set_has_element(dynamic(["this", "is", "an", "example"]), "example"); -print set_has_element(dynamic(["this", "is", "an", "example"]), "examplee"); -print set_has_element(dynamic([1, 2, 3]), 2); -print set_has_element(dynamic([1, 2, 3, 4.2]), 4); -print '-- set_intersect()'; -print set_intersect(dynamic([]), dynamic([])); -print array_sort_asc(set_intersect(dynamic([1, 1, 2, 2, 3, 3]), dynamic([1, 2, 3])))[0]; -print array_sort_asc(set_intersect(dynamic([1, 4, 2, 3, 5, 4, 6]), dynamic([1, 2, 3])))[0]; -print set_intersect(dynamic([4]), dynamic([1, 2, 3])); -print set_intersect(dynamic([1, 2, 3, 4, 5]), dynamic([1, 3, 5]), dynamic([2, 5])); -print set_intersect(dynamic([1, 2, 3]), dynamic([])); -print set_intersect(dynamic(['a', 's', 'd']), dynamic(['a', 'f'])); -print set_intersect(dynamic(['Chewbacca', 'Darth Vader', 'Han Solo']), dynamic(['Darth Sidious', 'Darth Vader'])); -print '-- set_union()'; -print set_union(dynamic([]), dynamic([])); -print array_sort_asc(set_union(dynamic([1, 1, 2, 2, 3, 3]), dynamic([1, 2, 3])))[0]; -print array_sort_asc(set_union(dynamic([1, 4, 2, 3, 5, 4, 6]), dynamic([1, 2, 3])))[0]; -print array_sort_asc(set_union(dynamic([4]), dynamic([1, 2, 3])))[0]; -print array_sort_asc(set_union(dynamic([1, 3, 4]), dynamic([5]), dynamic([2, 4])))[0]; -print array_sort_asc(set_union(dynamic([1, 2, 3]), dynamic([])))[0]; -print array_sort_asc(set_union(dynamic(['a', 's', 'd']), dynamic(['a', 'f'])))[0]; -print array_sort_asc(set_union(dynamic(['Chewbacca', 'Darth Vader', 'Han Solo']), dynamic(['Darth Sidious', 'Darth Vader'])))[0]; -print '-- zip()'; -print zip(dynamic([]), dynamic([])); -print zip(dynamic([1,3,5]), dynamic([2,4,6])); -print zip(dynamic(['Darth','Master']), dynamic(['Vader','Yoda']), dynamic(['has a suit','doesn\'t have a suit'])); -print zip(dynamic([1,2,3]), dynamic([10,20])); -print zip(dynamic([]), dynamic([1,2,3])); \ No newline at end of file diff --git a/tests/queries/0_stateless/02366_kql_func_ip.reference b/tests/queries/0_stateless/02366_kql_func_ip.reference deleted file mode 100644 index 2a0bbf53fff..00000000000 --- a/tests/queries/0_stateless/02366_kql_func_ip.reference +++ /dev/null @@ -1,123 +0,0 @@ --- ipv4_is_private(\'127.0.0.1\') -0 --- ipv4_is_private(\'10.1.2.3\') -1 --- ipv4_is_private(\'192.168.1.1/24\') -1 -ipv4_is_private(strcat(\'192.\',\'168.\',\'1.\',\'1\',\'/24\')) -1 --- ipv4_is_private(\'abc\') -\N --- ipv4_netmask_suffix(\'192.168.1.1/24\') -24 --- ipv4_netmask_suffix(\'192.168.1.1\') -32 --- ipv4_netmask_suffix(\'127.0.0.1/16\') -16 --- ipv4_netmask_suffix(\'abc\') -\N -ipv4_netmask_suffix(strcat(\'127.\', \'0.\', \'0.1/16\')) -16 --- ipv4_is_in_range(\'127.0.0.1\', \'127.0.0.1\') -1 --- ipv4_is_in_range(\'192.168.1.6\', \'192.168.1.1/24\') -1 --- ipv4_is_in_range(\'192.168.1.1\', \'192.168.2.1/24\') -0 --- ipv4_is_in_range(strcat(\'192.\',\'168.\', \'1.1\'), \'192.168.2.1/24\') -0 --- ipv4_is_in_range(\'abc\', \'127.0.0.1\') -\N --- parse_ipv6(127.0.0.1) -0000:0000:0000:0000:0000:ffff:7f00:0001 --- parse_ipv6(fe80::85d:e82c:9446:7994) -fe80:0000:0000:0000:085d:e82c:9446:7994 --- parse_ipv4(\'127.0.0.1\') -2130706433 --- parse_ipv4(\'192.1.168.1\') < parse_ipv4(\'192.1.168.2\') -1 --- parse_ipv4(arrayStringConcat([\'127\', \'0\', \'0\', \'1\'], \'.\')) --- parse_ipv4_mask(\'127.0.0.1\', 24) == 2130706432 -2130706432 --- parse_ipv4_mask(\'abc\', 31) -\N -\N --- parse_ipv4_mask(\'192.1.168.2\', 31) == parse_ipv4_mask(\'192.1.168.3\', 31) -3221334018 -3221334018 --- ipv4_is_match(\'127.0.0.1\', \'127.0.0.1\') -1 --- ipv4_is_match(\'192.168.1.1\', \'192.168.1.255\') -0 --- ipv4_is_match(\'192.168.1.1/24\', \'192.168.1.255/24\') -1 --- ipv4_is_match(\'192.168.1.1\', \'192.168.1.255\', 24) -1 --- ipv4_is_match(\'abc\', \'def\', 24) -\N --- ipv4_compare() -0 --1 -1 -0 -0 -0 -0 -0 -0 -0 -0 --- format_ipv4() -192.168.1.0 -192.168.1.1 -192.168.1.0 -192.168.1.0 -1 -1 -127.0.0.0 --- format_ipv4_mask() -192.168.1.0/24 -192.168.1.0/24 -192.168.1.0/24 -192.168.1.1/32 -192.168.1.0/24 -1 -1 -127.0.0.0/24 --- parse_ipv6_mask() -0000:0000:0000:0000:0000:0000:0000:0000 -fe80:0000:0000:0000:085d:e82c:9446:7900 -0000:0000:0000:0000:0000:ffff:c0a8:ff00 -0000:0000:0000:0000:0000:ffff:c0a8:ff00 -0000:0000:0000:0000:0000:ffff:ffff:ffff -fe80:0000:0000:0000:085d:e82c:9446:7994 -fe80:0000:0000:0000:085d:e82c:9446:7900 -0000:0000:0000:0000:0000:ffff:c0a8:ffff -0000:0000:0000:0000:0000:ffff:c0a8:ff00 --- ipv6_is_match() -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 diff --git a/tests/queries/0_stateless/02366_kql_func_ip.sql b/tests/queries/0_stateless/02366_kql_func_ip.sql deleted file mode 100644 index c9b335f203a..00000000000 --- a/tests/queries/0_stateless/02366_kql_func_ip.sql +++ /dev/null @@ -1,131 +0,0 @@ -set dialect='kusto'; -print '-- ipv4_is_private(\'127.0.0.1\')'; -print ipv4_is_private('127.0.0.1'); -print '-- ipv4_is_private(\'10.1.2.3\')'; -print ipv4_is_private('10.1.2.3'); -print '-- ipv4_is_private(\'192.168.1.1/24\')'; -print ipv4_is_private('192.168.1.1/24'); -print 'ipv4_is_private(strcat(\'192.\',\'168.\',\'1.\',\'1\',\'/24\'))'; -print ipv4_is_private(strcat('192.','168.','1.','1','/24')); -print '-- ipv4_is_private(\'abc\')'; -print ipv4_is_private('abc'); -- == null - -print '-- ipv4_netmask_suffix(\'192.168.1.1/24\')'; -print ipv4_netmask_suffix('192.168.1.1/24'); -- == 24 -print '-- ipv4_netmask_suffix(\'192.168.1.1\')'; -print ipv4_netmask_suffix('192.168.1.1'); -- == 32 -print '-- ipv4_netmask_suffix(\'127.0.0.1/16\')'; -print ipv4_netmask_suffix('127.0.0.1/16'); -- == 16 -print '-- ipv4_netmask_suffix(\'abc\')'; -print ipv4_netmask_suffix('abc'); -- == null -print 'ipv4_netmask_suffix(strcat(\'127.\', \'0.\', \'0.1/16\'))'; -print ipv4_netmask_suffix(strcat('127.', '0.', '0.1/16')); -- == 16 - -print '-- ipv4_is_in_range(\'127.0.0.1\', \'127.0.0.1\')'; -print ipv4_is_in_range('127.0.0.1', '127.0.0.1'); -- == true -print '-- ipv4_is_in_range(\'192.168.1.6\', \'192.168.1.1/24\')'; -print ipv4_is_in_range('192.168.1.6', '192.168.1.1/24'); -- == true -print '-- ipv4_is_in_range(\'192.168.1.1\', \'192.168.2.1/24\')'; -print ipv4_is_in_range('192.168.1.1', '192.168.2.1/24'); -- == false -print '-- ipv4_is_in_range(strcat(\'192.\',\'168.\', \'1.1\'), \'192.168.2.1/24\')'; -print ipv4_is_in_range(strcat('192.','168.', '1.1'), '192.168.2.1/24'); -- == false -print '-- ipv4_is_in_range(\'abc\', \'127.0.0.1\')'; -- == null -print ipv4_is_in_range('abc', '127.0.0.1'); - -print '-- parse_ipv6(127.0.0.1)'; -print parse_ipv6('127.0.0.1'); -print '-- parse_ipv6(fe80::85d:e82c:9446:7994)'; -print parse_ipv6('fe80::85d:e82c:9446:7994'); -print '-- parse_ipv4(\'127.0.0.1\')'; -print parse_ipv4('127.0.0.1'); -print '-- parse_ipv4(\'192.1.168.1\') < parse_ipv4(\'192.1.168.2\')'; -print parse_ipv4('192.1.168.1') < parse_ipv4('192.1.168.2'); -print '-- parse_ipv4(arrayStringConcat([\'127\', \'0\', \'0\', \'1\'], \'.\'))'; -print parse_ipv4(arrayStringConcat(['127', '0', '0', '1'], '.')); -- { clientError UNKNOWN_FUNCTION } - -print '-- parse_ipv4_mask(\'127.0.0.1\', 24) == 2130706432'; -print parse_ipv4_mask('127.0.0.1', 24); -print '-- parse_ipv4_mask(\'abc\', 31)'; -print parse_ipv4_mask('abc', 31) -print '-- parse_ipv4_mask(\'192.1.168.2\', 1000)'; -print parse_ipv4_mask('192.1.168.2', 1000); -print '-- parse_ipv4_mask(\'192.1.168.2\', 31) == parse_ipv4_mask(\'192.1.168.3\', 31)'; ---print parse_ipv4_mask('192.1.168.2', 31) == parse_ipv4_mask('192.1.168.3', 31); // this qual failed in analyzer 3221334018 -print parse_ipv4_mask('192.1.168.2', 31); -print parse_ipv4_mask('192.1.168.3', 31); -print '-- ipv4_is_match(\'127.0.0.1\', \'127.0.0.1\')'; -print ipv4_is_match('127.0.0.1', '127.0.0.1'); -print '-- ipv4_is_match(\'192.168.1.1\', \'192.168.1.255\')'; -print ipv4_is_match('192.168.1.1', '192.168.1.255'); -print '-- ipv4_is_match(\'192.168.1.1/24\', \'192.168.1.255/24\')'; -print ipv4_is_match('192.168.1.1/24', '192.168.1.255/24'); -print '-- ipv4_is_match(\'192.168.1.1\', \'192.168.1.255\', 24)'; -print ipv4_is_match('192.168.1.1', '192.168.1.255', 24); -print '-- ipv4_is_match(\'abc\', \'def\', 24)'; -print ipv4_is_match('abc', 'dev', 24); -print '-- ipv4_compare()'; -print ipv4_compare('127.0.0.1', '127.0.0.1'); -print ipv4_compare('192.168.1.1', '192.168.1.255'); -print ipv4_compare('192.168.1.255', '192.168.1.1'); -print ipv4_compare('192.168.1.1/24', '192.168.1.255/24'); -print ipv4_compare('192.168.1.1', '192.168.1.255', 24); -print ipv4_compare('192.168.1.1/24', '192.168.1.255'); -print ipv4_compare('192.168.1.1', '192.168.1.255/24'); -print ipv4_compare('192.168.1.1/30', '192.168.1.255/24'); -print ipv4_compare('192.168.1.1', '192.168.1.0', 31); -print ipv4_compare('192.168.1.1/24', '192.168.1.255', 31); -print ipv4_compare('192.168.1.1', '192.168.1.255', 24); -print '-- format_ipv4()'; -print format_ipv4('192.168.1.255', 24); -print format_ipv4('192.168.1.1', 32); -print format_ipv4('192.168.1.1/24', 32); -print format_ipv4(3232236031, 24); -print format_ipv4('192.168.1.1/24', -1) == ''; -print format_ipv4('abc', 24) == ''; -print format_ipv4(strcat('127.0', '.0.', '1', '/32'), 12 + 12); -print '-- format_ipv4_mask()'; -print format_ipv4_mask('192.168.1.255', 24); -print format_ipv4_mask(3232236031, 24); -print format_ipv4_mask('192.168.1.1', 24); -print format_ipv4_mask('192.168.1.1', 32); -print format_ipv4_mask('192.168.1.1/24', 32); -print format_ipv4_mask('192.168.1.1/24', -1) == ''; -print format_ipv4_mask('abc', 24) == ''; -print format_ipv4_mask(strcat('127.0', '.0.', '1', '/32'), 12 + 12); -print '-- parse_ipv6_mask()'; -print parse_ipv6_mask("127.0.0.1", 24); -print parse_ipv6_mask("fe80::85d:e82c:9446:7994", 120); -print parse_ipv6_mask("192.168.255.255", 120); -print parse_ipv6_mask("192.168.255.255/24", 124); -print parse_ipv6_mask("255.255.255.255", 128); -print parse_ipv6_mask("fe80::85d:e82c:9446:7994", 128); -print parse_ipv6_mask("fe80::85d:e82c:9446:7994/120", 124); -print parse_ipv6_mask("::192.168.255.255", 128); -print parse_ipv6_mask("::192.168.255.255/24", 128); -print '-- ipv6_is_match()'; -print ipv6_is_match('::ffff:7f00:1', '127.0.0.1') == true; -print ipv6_is_match('fe80::85d:e82c:9446:7994', 'fe80::85d:e82c:9446:7995') == false; -print ipv6_is_match('192.168.1.1/24', '192.168.1.255/24') == true; -print ipv6_is_match('fe80::85d:e82c:9446:7994/127', 'fe80::85d:e82c:9446:7995/127') == true; -print ipv6_is_match('fe80::85d:e82c:9446:7994', 'fe80::85d:e82c:9446:7995', 127) == true; -print ipv6_is_match('192.168.1.1', '192.168.1.1'); -- // Equal IPs -print ipv6_is_match('192.168.1.1/24', '192.168.1.255'); -- // 24 bit IP4-prefix is used for comparison -print ipv6_is_match('192.168.1.1', '192.168.1.255/24'); -- // 24 bit IP4-prefix is used for comparison -print ipv6_is_match('192.168.1.1/30', '192.168.1.255/24'); -- // 24 bit IP4-prefix is used for comparison -print ipv6_is_match('fe80::85d:e82c:9446:7994', 'fe80::85d:e82c:9446:7994'); -- // Equal IPs -print ipv6_is_match('fe80::85d:e82c:9446:7994/120', 'fe80::85d:e82c:9446:7998'); -- // 120 bit IP6-prefix is used for comparison -print ipv6_is_match('fe80::85d:e82c:9446:7994', 'fe80::85d:e82c:9446:7998/120'); -- // 120 bit IP6-prefix is used for comparison -print ipv6_is_match('fe80::85d:e82c:9446:7994/120', 'fe80::85d:e82c:9446:7998/120'); -- // 120 bit IP6-prefix is used for comparison -print ipv6_is_match('192.168.1.1', '::ffff:c0a8:0101'); -- // Equal IPs -print ipv6_is_match('192.168.1.1/24', '::ffff:c0a8:01ff'); -- // 24 bit IP-prefix is used for comparison -print ipv6_is_match('::ffff:c0a8:0101', '192.168.1.255/24'); -- // 24 bit IP-prefix is used for comparison -print ipv6_is_match('::192.168.1.1/30', '192.168.1.255/24'); -- // 24 bit IP-prefix is used for comparison -print ipv6_is_match('192.168.1.1', '192.168.1.0', 31); -- // 31 bit IP4-prefix is used for comparison -print ipv6_is_match('192.168.1.1/24', '192.168.1.255', 31); -- // 24 bit IP4-prefix is used for comparison -print ipv6_is_match('192.168.1.1', '192.168.1.255', 24); -- // 24 bit IP4-prefix is used for comparison -print ipv6_is_match('fe80::85d:e82c:9446:7994', 'fe80::85d:e82c:9446:7995', 127); -- // 127 bit IP6-prefix is used for comparison -print ipv6_is_match('fe80::85d:e82c:9446:7994/127', 'fe80::85d:e82c:9446:7998', 120); -- // 120 bit IP6-prefix is used for comparison -print ipv6_is_match('fe80::85d:e82c:9446:7994/120', 'fe80::85d:e82c:9446:7998', 127); -- // 120 bit IP6-prefix is used for comparison -print ipv6_is_match('192.168.1.1/24', '::ffff:c0a8:01ff', 127); -- // 127 bit IP6-prefix is used for comparison -print ipv6_is_match('::ffff:c0a8:0101', '192.168.1.255', 120); -- // 120 bit IP6-prefix is used for comparison -print ipv6_is_match('::192.168.1.1/30', '192.168.1.255/24', 127); -- // 120 bit IP6-prefix is used for comparison \ No newline at end of file diff --git a/tests/queries/0_stateless/02366_kql_func_math.reference b/tests/queries/0_stateless/02366_kql_func_math.reference deleted file mode 100644 index 92f283abcb6..00000000000 --- a/tests/queries/0_stateless/02366_kql_func_math.reference +++ /dev/null @@ -1,4 +0,0 @@ --- isnan -- -1 -0 -0 diff --git a/tests/queries/0_stateless/02366_kql_func_math.sql b/tests/queries/0_stateless/02366_kql_func_math.sql deleted file mode 100644 index 4e83622eb6b..00000000000 --- a/tests/queries/0_stateless/02366_kql_func_math.sql +++ /dev/null @@ -1,7 +0,0 @@ -set dialect = 'kusto'; -print '-- isnan --'; -print isnan(double(nan)); -print isnan(4.2); -print isnan(4); -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } -print isnan(real(+inf)); -print isnan(dynamic(null)); -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } diff --git a/tests/queries/0_stateless/02366_kql_func_scalar.reference b/tests/queries/0_stateless/02366_kql_func_scalar.reference deleted file mode 100644 index b7fa62c5d43..00000000000 --- a/tests/queries/0_stateless/02366_kql_func_scalar.reference +++ /dev/null @@ -1,16 +0,0 @@ --- bin_at() -4.5 --12:0:0 -2017-05-14 12:00:00.000000000 -2017-05-14 00:00:00.000000000 -2018-02-25 15:14:00.000000000 5 -2018-02-24 15:14:00.000000000 3 -2018-02-23 15:14:00.000000000 4 --- bin() -4 -1970-05-11 00:00:00.000000000 -336:0:0 -1970-05-11 13:45:07.345000000 -1970-05-11 13:45:07.345623000 -2022-09-26 10:13:23.987232000 -1970-05-11 13:45:07.456336000 diff --git a/tests/queries/0_stateless/02366_kql_func_scalar.sql b/tests/queries/0_stateless/02366_kql_func_scalar.sql deleted file mode 100644 index d7e94cfd9d1..00000000000 --- a/tests/queries/0_stateless/02366_kql_func_scalar.sql +++ /dev/null @@ -1,26 +0,0 @@ -DROP TABLE IF EXISTS Bin_at_test; -CREATE TABLE Bin_at_test -( - `Date` DateTime('UTC'), - Num Nullable(UInt8) -) ENGINE = Memory; -INSERT INTO Bin_at_test VALUES ('2018-02-24T15:14:01',3), ('2018-02-23T16:14:01',4), ('2018-02-26T15:14:01',5); - -set dialect = 'kusto'; -print '-- bin_at()'; -print bin_at(6.5, 2.5, 7); -print bin_at(1h, 1d, 12h); -print bin_at(datetime(2017-05-15 10:20:00.0), 1d, datetime(1970-01-01 12:00:00.0)); -print bin_at(datetime(2017-05-17 10:20:00.0), 7d, datetime(2017-06-04 00:00:00.0)); -Bin_at_test | summarize sum(Num) by d = todatetime(bin_at(Date, 1d, datetime('2018-02-24 15:14:00'))) | order by d; -print '-- bin()'; -print bin(4.5, 1); -print bin(datetime(1970-05-11 13:45:07), 1d); -print bin(16d, 7d); -print bin(datetime(1970-05-11 13:45:07.345623), 1ms); --- print bin(datetime(2022-09-26 10:13:23.987234), 6ms); -> 2022-09-26 10:13:23.982000000 -print bin(datetime(1970-05-11 13:45:07.345623), 1microsecond); -print bin(datetime(2022-09-26 10:13:23.987234), 6microseconds); -print bin(datetime(1970-05-11 13:45:07.456345672), 16microseconds); --- print bin(datetime(2022-09-26 10:13:23.987234128), 1tick); -> 2022-09-26 10:13:23.987234100 --- print bin(datetime(2022-09-26 10:13:23.987234128), 99nanosecond); -> null diff --git a/tests/queries/0_stateless/02366_kql_func_string.reference b/tests/queries/0_stateless/02366_kql_func_string.reference deleted file mode 100644 index 9bdd38ca5db..00000000000 --- a/tests/queries/0_stateless/02366_kql_func_string.reference +++ /dev/null @@ -1,360 +0,0 @@ --- test String Functions -- --- Customers |where Education contains \'degree\' -Latoya Shen Professional Graduate Degree 25 -Peter Nara Skilled Manual Graduate Degree 26 - --- Customers |where Education !contains \'degree\' -\N why Professional Partial College 38 -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Apple Skilled Manual Bachelors 28 - --- Customers |where Education contains \'Degree\' -Latoya Shen Professional Graduate Degree 25 -Peter Nara Skilled Manual Graduate Degree 26 - --- Customers |where Education !contains \'Degree\' -\N why Professional Partial College 38 -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Apple Skilled Manual Bachelors 28 - --- Customers | where FirstName endswith \'RE\' -Theodore Diaz Skilled Manual Bachelors 28 - --- Customers | where ! FirstName endswith \'RE\' -Latoya Shen Professional Graduate Degree 25 -Peter Nara Skilled Manual Graduate Degree 26 -Stephanie Cox Management abcd defg Bachelors 33 -Apple Skilled Manual Bachelors 28 - ---Customers | where FirstName endswith_cs \'re\' -Theodore Diaz Skilled Manual Bachelors 28 - --- Customers | where FirstName !endswith_cs \'re\' -Latoya Shen Professional Graduate Degree 25 -Peter Nara Skilled Manual Graduate Degree 26 -Stephanie Cox Management abcd defg Bachelors 33 -Apple Skilled Manual Bachelors 28 - --- Customers | where Occupation == \'Skilled Manual\' -Peter Nara Skilled Manual Graduate Degree 26 -Theodore Diaz Skilled Manual Bachelors 28 -Apple Skilled Manual Bachelors 28 - --- Customers | where Occupation != \'Skilled Manual\' -\N why Professional Partial College 38 -Latoya Shen Professional Graduate Degree 25 -Stephanie Cox Management abcd defg Bachelors 33 - --- Customers | where Occupation has \'skilled\' -Peter Nara Skilled Manual Graduate Degree 26 -Theodore Diaz Skilled Manual Bachelors 28 -Apple Skilled Manual Bachelors 28 - --- Customers | where Occupation !has \'skilled\' -\N why Professional Partial College 38 -Latoya Shen Professional Graduate Degree 25 -Stephanie Cox Management abcd defg Bachelors 33 - --- Customers | where Occupation has \'Skilled\' -Peter Nara Skilled Manual Graduate Degree 26 -Theodore Diaz Skilled Manual Bachelors 28 -Apple Skilled Manual Bachelors 28 - --- Customers | where Occupation !has \'Skilled\' -\N why Professional Partial College 38 -Latoya Shen Professional Graduate Degree 25 -Stephanie Cox Management abcd defg Bachelors 33 - --- Customers | where Occupation hasprefix_cs \'Ab\' - --- Customers | where Occupation !hasprefix_cs \'Ab\' -\N why Professional Partial College 38 -Latoya Shen Professional Graduate Degree 25 -Peter Nara Skilled Manual Graduate Degree 26 -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Apple Skilled Manual Bachelors 28 - --- Customers | where Occupation hasprefix_cs \'ab\' -Stephanie Cox Management abcd defg Bachelors 33 - --- Customers | where Occupation !hasprefix_cs \'ab\' -\N why Professional Partial College 38 -Latoya Shen Professional Graduate Degree 25 -Peter Nara Skilled Manual Graduate Degree 26 -Theodore Diaz Skilled Manual Bachelors 28 -Apple Skilled Manual Bachelors 28 - --- Customers | where Occupation hassuffix \'Ent\' -Stephanie Cox Management abcd defg Bachelors 33 - --- Customers | where Occupation !hassuffix \'Ent\' -\N why Professional Partial College 38 -Latoya Shen Professional Graduate Degree 25 -Peter Nara Skilled Manual Graduate Degree 26 -Theodore Diaz Skilled Manual Bachelors 28 -Apple Skilled Manual Bachelors 28 - --- Customers | where Occupation hassuffix \'ent\' -Stephanie Cox Management abcd defg Bachelors 33 - --- Customers | where Occupation hassuffix \'ent\' -Stephanie Cox Management abcd defg Bachelors 33 - --- Customers |where Education in (\'Bachelors\',\'High School\') -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Apple Skilled Manual Bachelors 28 - --- Customers | where Education !in (\'Bachelors\',\'High School\') -\N why Professional Partial College 38 -Latoya Shen Professional Graduate Degree 25 -Peter Nara Skilled Manual Graduate Degree 26 - --- Customers | where FirstName matches regex \'P.*r\' -Peter Nara Skilled Manual Graduate Degree 26 - --- Customers | where FirstName startswith \'pet\' -Peter Nara Skilled Manual Graduate Degree 26 - --- Customers | where FirstName !startswith \'pet\' -Latoya Shen Professional Graduate Degree 25 -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Apple Skilled Manual Bachelors 28 - --- Customers | where FirstName startswith_cs \'pet\' - --- Customers | where FirstName !startswith_cs \'pet\' -Latoya Shen Professional Graduate Degree 25 -Peter Nara Skilled Manual Graduate Degree 26 -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Apple Skilled Manual Bachelors 28 - --- Customers | where isempty(LastName) -Apple Skilled Manual Bachelors 28 - --- Customers | where isnotempty(LastName) -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Peter Nara Skilled Manual Graduate Degree 26 -Latoya Shen Professional Graduate Degree 25 -\N why Professional Partial College 38 - --- Customers | where isnotnull(FirstName) -Latoya Shen Professional Graduate Degree 25 -Peter Nara Skilled Manual Graduate Degree 26 -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Apple Skilled Manual Bachelors 28 - --- Customers | where isnull(FirstName) -\N why Professional Partial College 38 - --- Customers | project url_decode(\'https%3A%2F%2Fwww.test.com%2Fhello%20word\') | take 1 -https://www.test.com/hello word - --- Customers | project url_encode(\'https://www.test.com/hello word\') | take 1 -https%3A%2F%2Fwww.test.com%2Fhello%20word - --- Customers | project name_abbr = strcat(substring(FirstName,0,3), \' \', substring(LastName,2)) -\N -Lat en -Pet ra -The az -Ste x -App - --- Customers | project name = strcat(FirstName, \' \', LastName) -\N -Latoya Shen -Peter Nara -Theodore Diaz -Stephanie Cox -Apple - --- Customers | project FirstName, strlen(FirstName) -\N \N -Latoya 6 -Peter 5 -Theodore 8 -Stephanie 9 -Apple 5 - --- Customers | project strrep(FirstName,2,\'_\') -\N -Latoya_Latoya -Peter_Peter -Theodore_Theodore -Stephanie_Stephanie -Apple_Apple - --- Customers | project toupper(FirstName) -\N -LATOYA -PETER -THEODORE -STEPHANIE -APPLE - --- Customers | project tolower(FirstName) -\N -latoya -peter -theodore -stephanie -apple - --- support subquery for in orerator (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/in-cs-operator) (subquery need to be wraped with bracket inside bracket); TODO: case-insensitive not supported yet -Latoya Shen Professional Graduate Degree 25 -Peter Nara Skilled Manual Graduate Degree 26 -Theodore Diaz Skilled Manual Bachelors 28 -Apple Skilled Manual Bachelors 28 - --- has_all (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/has-all-operator); TODO: subquery not supported yet -Peter Nara Skilled Manual Graduate Degree 26 -Theodore Diaz Skilled Manual Bachelors 28 -Apple Skilled Manual Bachelors 28 - --- has_any (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/has-anyoperator); TODO: subquery not supported yet -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Peter Nara Skilled Manual Graduate Degree 26 -Apple Skilled Manual Bachelors 28 - --- countof (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/countoffunction) -3 -3 -1 - --- extract ( https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/extractfunction) -PINEAPPLE ice cream is 20 -PINEAPPLE -20 - -20 -\N -\N -\N -\N -\N -45.6 -45.6 - --- extract_all (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/extractallfunction); TODO: captureGroups not supported yet -[['T','h','e'],['p','ric','e'],['P','INEAPPL','E'],['i','c','e'],['c','rea','m']] - --- extract_json (https://learn.microsoft.com/en-us/azure/data-explorer/kusto/query/extractjsonfunction) - - -John -iPhone -\N -26 -26 -26 -26 -\N - --- split (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/splitfunction) -['aa','bb'] -['bbb'] -[''] -['a','','b'] -['aa','cc'] -['aabbcc'] -['aaa','bbb','ccc'] -[NULL] - --- strcat_delim (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/strcat-delimfunction); TODO: only support string now. -1-2-Ab - --- indexof (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/indexoffunction); TODO: length and occurrence not supported yet -2 -2 --1 --- base64_encode_fromguid() -8jMxriJurkmwahbmqbIS6w== --- base64_decode_toarray() -[] -[75,117,115,116,111] --- base64_decode_toguid() -10e99626-bc2b-4c75-bb3e-fe606de25700 -1 --- base64_encode_tostring - -S3VzdG8x --- base64_decode_tostring - -Kusto1 --- parse_url() -{"Scheme":"scheme","Host":"","Port":"0","Path":"/this/is/a/path","Username":"username","Password":"password","Query Parameters":{"k1":"v1","k2":"v2"},"Fragment":"fragment"} --- parse_urlquery() -{"Query Parameters":{"k1":"v1","k2":"v2","k3":"v3"}} --- strcmp() -0 1 -1 1 --- substring() -CD --- translate() -kusto xxx --- trim() -https://www.ibm.com -Te st1 - asd -asd -sd --- trim_start() -www.ibm.com -Te st1// $ -asdw - -asd --- trim_end() -https -- Te st1 -wasd - -asd --- trim, trim_start, trim_end all at once ---https://bing.com-- -- https://bing.com-- --https://bing.com https://bing.com --- replace_regex -Number was: 1 --- has_any_index() -0 1 -1 -1 --- parse_version() -1000000020000000300000040 -1000000020000000000000000 -1000000020000000000000000 -\N -\N -\N -\N -1000000020000000300000004 -1000000020000000000000000 -1000000020000000300000000 -1000000000000000000000000 --- parse_json() -[1,2,3] -[{"a":123.5,"b":"{\\"c\\":456}"}] --- parse_command_line() -[NULL] -[NULL] --- reverse() -321 -43.321 - -dsa -][ -]3,2,1[ -]\'redaV\',\'htraD\'[ -000000000.00:00:21 51-01-7102 -Peter Nara Skilled Manual Graduate Degree 26 -Latoya Shen Professional Graduate Degree 25 --- parse_csv() -[''] -['aaa'] -['aa','b','cc'] -['record1','a','b','c'] diff --git a/tests/queries/0_stateless/02366_kql_func_string.sql b/tests/queries/0_stateless/02366_kql_func_string.sql deleted file mode 100644 index d251b04e08b..00000000000 --- a/tests/queries/0_stateless/02366_kql_func_string.sql +++ /dev/null @@ -1,313 +0,0 @@ --- Tags: no-fasttest - -DROP TABLE IF EXISTS Customers; -CREATE TABLE Customers -( - FirstName Nullable(String), - LastName String, - Occupation String, - Education String, - Age Nullable(UInt8) -) ENGINE = Memory; - -INSERT INTO Customers VALUES ('Theodore','Diaz','Skilled Manual','Bachelors',28), ('Stephanie','Cox','Management abcd defg','Bachelors',33),('Peter','Nara','Skilled Manual','Graduate Degree',26),('Latoya','Shen','Professional','Graduate Degree',25),('Apple','','Skilled Manual','Bachelors',28),(NULL,'why','Professional','Partial College',38); - --- datatable (Version:string) [ --- '1.2.3.4', --- '1.2', --- '1.2.3', --- '1' --- ] - -DROP TABLE IF EXISTS Versions; -CREATE TABLE Versions -( - Version String -) ENGINE = Memory; -INSERT INTO Versions VALUES ('1.2.3.4'),('1.2'),('1.2.3'),('1'); - - -set dialect='kusto'; -print '-- test String Functions --'; - -print '-- Customers |where Education contains \'degree\''; -Customers |where Education contains 'degree' | order by LastName; -print ''; -print '-- Customers |where Education !contains \'degree\''; -Customers |where Education !contains 'degree' | order by LastName; -print ''; -print '-- Customers |where Education contains \'Degree\''; -Customers |where Education contains 'Degree' | order by LastName; -print ''; -print '-- Customers |where Education !contains \'Degree\''; -Customers |where Education !contains 'Degree' | order by LastName; -print ''; -print '-- Customers | where FirstName endswith \'RE\''; -Customers | where FirstName endswith 'RE' | order by LastName; -print ''; -print '-- Customers | where ! FirstName endswith \'RE\''; -Customers | where FirstName ! endswith 'RE' | order by LastName; -print ''; -print '--Customers | where FirstName endswith_cs \'re\''; -Customers | where FirstName endswith_cs 're' | order by LastName; -print ''; -print '-- Customers | where FirstName !endswith_cs \'re\''; -Customers | where FirstName !endswith_cs 're' | order by LastName; -print ''; -print '-- Customers | where Occupation == \'Skilled Manual\''; -Customers | where Occupation == 'Skilled Manual' | order by LastName; -print ''; -print '-- Customers | where Occupation != \'Skilled Manual\''; -Customers | where Occupation != 'Skilled Manual' | order by LastName; -print ''; -print '-- Customers | where Occupation has \'skilled\''; -Customers | where Occupation has 'skilled' | order by LastName; -print ''; -print '-- Customers | where Occupation !has \'skilled\''; -Customers | where Occupation !has 'skilled' | order by LastName; -print ''; -print '-- Customers | where Occupation has \'Skilled\''; -Customers | where Occupation has 'Skilled'| order by LastName; -print ''; -print '-- Customers | where Occupation !has \'Skilled\''; -Customers | where Occupation !has 'Skilled'| order by LastName; -print ''; -print '-- Customers | where Occupation hasprefix_cs \'Ab\''; -Customers | where Occupation hasprefix_cs 'Ab'| order by LastName; -print ''; -print '-- Customers | where Occupation !hasprefix_cs \'Ab\''; -Customers | where Occupation !hasprefix_cs 'Ab'| order by LastName; -print ''; -print '-- Customers | where Occupation hasprefix_cs \'ab\''; -Customers | where Occupation hasprefix_cs 'ab'| order by LastName; -print ''; -print '-- Customers | where Occupation !hasprefix_cs \'ab\''; -Customers | where Occupation !hasprefix_cs 'ab'| order by LastName; -print ''; -print '-- Customers | where Occupation hassuffix \'Ent\''; -Customers | where Occupation hassuffix 'Ent'| order by LastName; -print ''; -print '-- Customers | where Occupation !hassuffix \'Ent\''; -Customers | where Occupation !hassuffix 'Ent'| order by LastName; -print ''; -print '-- Customers | where Occupation hassuffix \'ent\''; -Customers | where Occupation hassuffix 'ent'| order by LastName; -print ''; -print '-- Customers | where Occupation hassuffix \'ent\''; -Customers | where Occupation hassuffix 'ent'| order by LastName; -print ''; -print '-- Customers |where Education in (\'Bachelors\',\'High School\')'; -Customers |where Education in ('Bachelors','High School')| order by LastName; -print ''; -print '-- Customers | where Education !in (\'Bachelors\',\'High School\')'; -Customers | where Education !in ('Bachelors','High School')| order by LastName; -print ''; -print '-- Customers | where FirstName matches regex \'P.*r\''; -Customers | where FirstName matches regex 'P.*r'| order by LastName; -print ''; -print '-- Customers | where FirstName startswith \'pet\''; -Customers | where FirstName startswith 'pet'| order by LastName; -print ''; -print '-- Customers | where FirstName !startswith \'pet\''; -Customers | where FirstName !startswith 'pet'| order by LastName; -print ''; -print '-- Customers | where FirstName startswith_cs \'pet\''; -Customers | where FirstName startswith_cs 'pet'| order by LastName; -print ''; -print '-- Customers | where FirstName !startswith_cs \'pet\''; -Customers | where FirstName !startswith_cs 'pet'| order by LastName; -print ''; -print '-- Customers | where isempty(LastName)'; -Customers | where isempty(LastName); -print ''; -print '-- Customers | where isnotempty(LastName)'; -Customers | where isnotempty(LastName); -print ''; -print '-- Customers | where isnotnull(FirstName)'; -Customers | where isnotnull(FirstName)| order by LastName; -print ''; -print '-- Customers | where isnull(FirstName)'; -Customers | where isnull(FirstName)| order by LastName; -print ''; -print '-- Customers | project url_decode(\'https%3A%2F%2Fwww.test.com%2Fhello%20word\') | take 1'; -Customers | project url_decode('https%3A%2F%2Fwww.test.com%2Fhello%20word') | take 1; -print ''; -print '-- Customers | project url_encode(\'https://www.test.com/hello word\') | take 1'; -Customers | project url_encode('https://www.test.com/hello word') | take 1; -print ''; -print '-- Customers | project name_abbr = strcat(substring(FirstName,0,3), \' \', substring(LastName,2))'; -Customers | project name_abbr = strcat(substring(FirstName,0,3), ' ', substring(LastName,2))| order by LastName; -print ''; -print '-- Customers | project name = strcat(FirstName, \' \', LastName)'; -Customers | project name = strcat(FirstName, ' ', LastName)| order by LastName; -print ''; -print '-- Customers | project FirstName, strlen(FirstName)'; -Customers | project FirstName, strlen(FirstName)| order by LastName; -print ''; -print '-- Customers | project strrep(FirstName,2,\'_\')'; -Customers | project strrep(FirstName,2,'_')| order by LastName; -print ''; -print '-- Customers | project toupper(FirstName)'; -Customers | project toupper(FirstName)| order by LastName; -print ''; -print '-- Customers | project tolower(FirstName)'; -Customers | project tolower(FirstName)| order by LastName; -print ''; -print '-- support subquery for in orerator (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/in-cs-operator) (subquery need to be wraped with bracket inside bracket); TODO: case-insensitive not supported yet'; -Customers | where Age in ((Customers|project Age|where Age < 30)) | order by LastName; --- Customer | where LastName in~ ("diaz", "cox") -print ''; -print '-- has_all (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/has-all-operator); TODO: subquery not supported yet'; -Customers | where Occupation has_all ('manual', 'skilled') | order by LastName; -print ''; -print '-- has_any (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/has-anyoperator); TODO: subquery not supported yet'; -Customers|where Occupation has_any ('Skilled','abcd'); -print ''; -print '-- countof (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/countoffunction)'; -Customers | project countof('The cat sat on the mat', 'at') | take 1; -Customers | project countof('The cat sat on the mat', 'at', 'normal') | take 1; -Customers | project countof('The cat sat on the mat', '\\s.he', 'regex') | take 1; -print ''; -print '-- extract ( https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/extractfunction)'; -print extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 0, 'The price of PINEAPPLE ice cream is 20'); -print extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 1, 'The price of PINEAPPLE ice cream is 20'); -print extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 2, 'The price of PINEAPPLE ice cream is 20'); -print extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 3, 'The price of PINEAPPLE ice cream is 20'); -print extract('(\\b[A-Z]+\\b).+(\\b\\d+)', 2, 'The price of PINEAPPLE ice cream is 20', typeof(real)); -print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(bool)); -print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(date)); -print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(guid)); -print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(int)); -print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(long)); -print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(real)); -print extract("x=([0-9.]+)", 1, "hello x=45.6|wo" , typeof(decimal)); -print ''; -print '-- extract_all (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/extractallfunction); TODO: captureGroups not supported yet'; -Customers | project extract_all('(\\w)(\\w+)(\\w)','The price of PINEAPPLE ice cream is 20') | take 1; -print ''; -print '-- extract_json (https://learn.microsoft.com/en-us/azure/data-explorer/kusto/query/extractjsonfunction)'; -print extract_json('', ''); -- { serverError BAD_ARGUMENTS } -print extract_json('a', ''); -- { serverError BAD_ARGUMENTS } -print extract_json('$.firstName', ''); -print extract_json('$.phoneNumbers[0].type', ''); -print extractjson('$.firstName', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}'); -print extract_json('$.phoneNumbers[0].type', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(string)); -print extract_json('$.phoneNumbers[0].type', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(int)); -print extract_json('$.age', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}'); -print extract_json('$.age', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(int)); -print extract_json('$.age', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(long)); --- print extract_json('$.age', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(bool)); -> true -print extract_json('$.age', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(double)); -print extract_json('$.age', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(guid)); --- print extract_json('$.phoneNumbers', '{"firstName":"John","lastName":"doe","age":26,"address":{"streetAddress":"naist street","city":"Nara","postalCode":"630-0192"},"phoneNumbers":[{"type":"iPhone","number":"0123-4567-8888"},{"type":"home","number":"0123-4567-8910"}]}', typeof(dynamic)); we won't be able to handle this particular case for a while, because it should return a dictionary -print ''; -print '-- split (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/splitfunction)'; -Customers | project split('aa_bb', '_') | take 1; -Customers | project split('aaa_bbb_ccc', '_', 1) | take 1; -Customers | project split('', '_') | take 1; -Customers | project split('a__b', '_') | take 1; -Customers | project split('aabbcc', 'bb') | take 1; -Customers | project split('aabbcc', '') | take 1; -Customers | project split('aaa_bbb_ccc', '_', -1) | take 1; -Customers | project split('aaa_bbb_ccc', '_', 10) | take 1; -print ''; -print '-- strcat_delim (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/strcat-delimfunction); TODO: only support string now.'; -Customers | project strcat_delim('-', '1', '2', strcat('A','b')) | take 1; --- Customers | project strcat_delim('-', '1', '2', 'A' , 1s); -print ''; -print '-- indexof (https://docs.microsoft.com/en-us/azure/data-explorer/kusto/query/indexoffunction); TODO: length and occurrence not supported yet'; -Customers | project indexof('abcdefg','cde') | take 1; -Customers | project indexof('abcdefg','cde',2) | take 1; -Customers | project indexof('abcdefg','cde',6) | take 1; -print '-- base64_encode_fromguid()'; --- print base64_encode_fromguid(guid(null)); -print base64_encode_fromguid(guid('ae3133f2-6e22-49ae-b06a-16e6a9b212eb')); -print base64_encode_fromguid(dynamic(null)); -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } -print base64_encode_fromguid("abcd1231"); -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } -print '-- base64_decode_toarray()'; -print base64_decode_toarray(''); -print base64_decode_toarray('S3VzdG8='); -print '-- base64_decode_toguid()'; -print base64_decode_toguid("JpbpECu8dUy7Pv5gbeJXAA=="); -print base64_decode_toguid(base64_encode_fromguid(guid('ae3133f2-6e22-49ae-b06a-16e6a9b212eb'))) == guid('ae3133f2-6e22-49ae-b06a-16e6a9b212eb'); -print '-- base64_encode_tostring'; -print base64_encode_tostring(''); -print base64_encode_tostring('Kusto1'); -print '-- base64_decode_tostring'; -print base64_decode_tostring(''); -print base64_decode_tostring('S3VzdG8x'); -print '-- parse_url()'; -print parse_url('scheme://username:password@host:1234/this/is/a/path?k1=v1&k2=v2#fragment'); -print '-- parse_urlquery()'; -print parse_urlquery('k1=v1&k2=v2&k3=v3'); -print '-- strcmp()'; -print strcmp('ABC','ABC'), strcmp('abc','ABC'), strcmp('ABC','abc'), strcmp('abcde','abc'); -print '-- substring()'; -print substring("ABCD", -2, 2); -print '-- translate()'; -print translate('krasp', 'otsku', 'spark'), translate('abc', '', 'ab'), translate('abc', 'x', 'abc'); -print '-- trim()'; -print trim("--", "--https://www.ibm.com--"); -print trim("[^\w]+", strcat("- ","Te st", "1", "// $")); -print trim("", " asd "); -print trim("a$", "asd"); -print trim("^a", "asd"); -print '-- trim_start()'; -print trim_start("https://", "https://www.ibm.com"); -print trim_start("[^\w]+", strcat("- ","Te st", "1", "// $")); -print trim_start("asd$", "asdw"); -print trim_start("asd$", "asd"); -print trim_start("d$", "asd"); -print '-- trim_end()'; -print trim_end("://www.ibm.com", "https://www.ibm.com"); -print trim_end("[^\w]+", strcat("- ","Te st", "1", "// $")); -print trim_end("^asd", "wasd"); -print trim_end("^asd", "asd"); -print trim_end("^a", "asd"); -print '-- trim, trim_start, trim_end all at once'; -print str = "--https://bing.com--", pattern = '--' | extend start = trim_start(pattern, str), end = trim_end(pattern, str), both = trim(pattern, str); -print '-- replace_regex'; -print replace_regex(strcat('Number is ', '1'), 'is (\d+)', 'was: \1'); -print '-- has_any_index()'; -print has_any_index('this is an example', dynamic(['this', 'example'])), has_any_index("this is an example", dynamic(['not', 'example'])), has_any_index("this is an example", dynamic(['not', 'found'])), has_any_index("this is an example", dynamic([])); -print '-- parse_version()'; -print parse_version(42); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } --- print parse_version(''); -> NULL -print parse_version('1.2.3.40'); -print parse_version('1.2'); -print parse_version(strcat('1.', '2')); -print parse_version('1.2.4.5.6'); -print parse_version('moo'); -print parse_version('moo.boo.foo'); -print parse_version(strcat_delim('.', 'moo', 'boo', 'foo')); -Versions | project parse_version(Version); -print '-- parse_json()'; -print parse_json(dynamic([1, 2, 3])); -print parse_json('{"a":123.5, "b":"{\\"c\\":456}"}'); -print '-- parse_command_line()'; -print parse_command_line(55, 'windows'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } --- print parse_command_line((52 + 3) * 4 % 2, 'windows'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -print parse_command_line('', 'windows'); -print parse_command_line(strrep(' ', 6), 'windows'); --- print parse_command_line('echo \"hello world!\" print$?', 'windows'); -> ["echo","hello world!","print$?"] --- print parse_command_line("yolo swag 'asd bcd' \"moo moo \"", 'windows'); -> ["yolo","swag","'asd","bcd'","moo moo "] --- print parse_command_line(strcat_delim(' ', "yolo", "swag", "\'asd bcd\'", "\"moo moo \""), 'windows'); -> ["yolo","swag","'asd","bcd'","moo moo "] -print '-- reverse()'; -print reverse(123); -print reverse(123.34); -print reverse(''); -print reverse("asd"); -print reverse(dynamic([])); -print reverse(dynamic([1, 2, 3])); -print reverse(dynamic(['Darth', "Vader"])); -print reverse(datetime(2017-10-15 12:00)); --- print reverse(timespan(3h)); -> 00:00:30 -Customers | where Education contains 'degree' | order by reverse(FirstName); -print '-- parse_csv()'; -print parse_csv(''); -print parse_csv(65); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -print parse_csv('aaa'); -print result=parse_csv('aa,b,cc'); -print result_multi_record=parse_csv('record1,a,b,c\nrecord2,x,y,z'); --- print result=parse_csv('aa,"b,b,b",cc,"Escaping quotes: ""Title""","line1\nline2"'); -> ["aa","b,b,b","cc","Escaping quotes: \"Title\"","line1\nline2"] --- print parse_csv(strcat(strcat_delim(',', 'aa', '"b,b,b"', 'cc', '"Escaping quotes: ""Title"""', '"line1\nline2"'), '\r\n', strcat_delim(',', 'asd', 'qcf'))); -> ["aa","b,b,b","cc","Escaping quotes: \"Title\"","line1\nline2"] diff --git a/tests/queries/0_stateless/02366_kql_makeseries.reference b/tests/queries/0_stateless/02366_kql_makeseries.reference deleted file mode 100644 index 8e7fde997bf..00000000000 --- a/tests/queries/0_stateless/02366_kql_makeseries.reference +++ /dev/null @@ -1,60 +0,0 @@ --- from to -Costco Snargaluff ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [200,0,102] -Costco Apple ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [0,2,0] -Aldi Snargaluff ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [0,500,0] -Aldi Apple ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [5,0,6] --- from -Costco Snargaluff ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [200,0,102] -Costco Apple ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000'] [0,2] -Aldi Snargaluff ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000'] [0,500] -Aldi Apple ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [5,0,6] --- to -Costco Snargaluff ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [200,0,102] -Costco Apple ['2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [2,0] -Aldi Snargaluff ['2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [500,0] -Aldi Apple ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [5,0,6] --- without from/to -Costco Snargaluff ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [200,0,102] -Costco Apple ['2016-09-11 00:00:00.000000000'] [2] -Aldi Snargaluff ['2016-09-11 00:00:00.000000000'] [500] -Aldi Apple ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [5,0,6] --- without by -['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [70,334,54] --- without aggregation alias -Costco Snargaluff ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [200,0,102] -Aldi Snargaluff ['2016-09-11 00:00:00.000000000'] [500] -Aldi Apple ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [5,0,6] -Costco Apple ['2016-09-11 00:00:00.000000000'] [2] --- assign group alias -Costco Snargaluff ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [200,0,102] -Aldi Snargaluff ['2016-09-11 00:00:00.000000000'] [500] -Aldi Apple ['2016-09-10 00:00:00.000000000','2016-09-11 00:00:00.000000000','2016-09-12 00:00:00.000000000'] [5,0,6] -Costco Apple ['2016-09-11 00:00:00.000000000'] [2] --- 3d step -Costco Snargaluff ['2016-09-10 00:00:00.000000000'] [134.66666666666666] -Costco Apple ['2016-09-10 00:00:00.000000000'] [2] -Aldi Snargaluff ['2016-09-10 00:00:00.000000000'] [500] -Aldi Apple ['2016-09-10 00:00:00.000000000'] [5.5] --- numeric column -Costco Snargaluff [10,11,12,13,14] [200,0,102,0,0] -Aldi Snargaluff [10,11,12,13,14] [0,500,0,0,0] -Aldi Apple [10,11,12,13,14] [5,0,6,0,0] -Costco Apple [10,11,12,13,14] [0,2,0,0,0] --- from -Costco Snargaluff [10,11,12] [200,0,102] -Aldi Snargaluff [10,11] [0,500] -Aldi Apple [10,11,12] [5,0,6] -Costco Apple [10,11] [0,2] --- to -Costco Snargaluff [8,12,16] [200,102,0] -Aldi Snargaluff [8,12,16] [500,0,0] -Aldi Apple [8,12,16] [5,6,0] -Costco Apple [8,12,16] [2,0,0] --- without from/to -Costco Snargaluff [10,12] [200,102] -Aldi Snargaluff [10] [500] -Aldi Apple [10,12] [5,6] -Costco Apple [10] [2] --- without by -[10,12] [202,54] -['2017-01-01 00:00:00.000000000','2017-01-02 00:00:00.000000000','2017-01-03 00:00:00.000000000','2017-01-04 00:00:00.000000000','2017-01-05 00:00:00.000000000','2017-01-06 00:00:00.000000000','2017-01-07 00:00:00.000000000','2017-01-08 00:00:00.000000000','2017-01-09 00:00:00.000000000'] [4,3,5,0,10.5,4,3,8,6.5] diff --git a/tests/queries/0_stateless/02366_kql_makeseries.sql b/tests/queries/0_stateless/02366_kql_makeseries.sql deleted file mode 100644 index c9ca91c0be0..00000000000 --- a/tests/queries/0_stateless/02366_kql_makeseries.sql +++ /dev/null @@ -1,77 +0,0 @@ --- Azure Data Explore Test Data --- let make_series_test_table = datatable (Supplier:string, Fruit:string, Price: real, Purchase:datetime) --- [ --- 'Aldi','Apple',4,'2016-09-10', --- 'Costco','Apple',2,'2016-09-11', --- 'Aldi','Apple',6,'2016-09-10', --- 'Costco','Snargaluff',100,'2016-09-12', --- 'Aldi','Apple',7,'2016-09-12', --- 'Aldi','Snargaluff',400,'2016-09-11', --- 'Costco','Snargaluff',104,'2016-09-12', --- 'Aldi','Apple',5,'2016-09-12', --- 'Aldi','Snargaluff',600,'2016-09-11', --- 'Costco','Snargaluff',200,'2016-09-10', --- ]; -DROP TABLE IF EXISTS make_series_test_table; -CREATE TABLE make_series_test_table -( - Supplier Nullable(String), - Fruit String , - Price Float64, - Purchase Date -) ENGINE = Memory; -INSERT INTO make_series_test_table VALUES ('Aldi','Apple',4,'2016-09-10'), ('Costco','Apple',2,'2016-09-11'), ('Aldi','Apple',6,'2016-09-10'), ('Costco','Snargaluff',100,'2016-09-12'), ('Aldi','Apple',7,'2016-09-12'), ('Aldi','Snargaluff',400,'2016-09-11'),('Costco','Snargaluff',104,'2016-09-12'),('Aldi','Apple',5,'2016-09-12'),('Aldi','Snargaluff',600,'2016-09-11'),('Costco','Snargaluff',200,'2016-09-10'); -DROP TABLE IF EXISTS make_series_test_table2; -CREATE TABLE make_series_test_table2 -( - Supplier Nullable(String), - Fruit String , - Price Int32, - Purchase Int32 -) ENGINE = Memory; -INSERT INTO make_series_test_table2 VALUES ('Aldi','Apple',4,10),('Costco','Apple',2,11),('Aldi','Apple',6,10),('Costco','Snargaluff',100,12),('Aldi','Apple',7,12),('Aldi','Snargaluff',400,11),('Costco','Snargaluff',104,12),('Aldi','Apple',5,12),('Aldi','Snargaluff',600,11),('Costco','Snargaluff',200,10); -DROP TABLE IF EXISTS make_series_test_table3; -CREATE TABLE make_series_test_table3 -( - timestamp datetime, - metric Float64, -) ENGINE = Memory; -INSERT INTO make_series_test_table3 VALUES (parseDateTimeBestEffort('2016-12-31T06:00', 'UTC'), 50), (parseDateTimeBestEffort('2017-01-01', 'UTC'), 4), (parseDateTimeBestEffort('2017-01-02', 'UTC'), 3), (parseDateTimeBestEffort('2017-01-03', 'UTC'), 4), (parseDateTimeBestEffort('2017-01-03T03:00', 'UTC'), 6), (parseDateTimeBestEffort('2017-01-05', 'UTC'), 8), (parseDateTimeBestEffort('2017-01-05T13:40', 'UTC'), 13), (parseDateTimeBestEffort('2017-01-06', 'UTC'), 4), (parseDateTimeBestEffort('2017-01-07', 'UTC'), 3), (parseDateTimeBestEffort('2017-01-08', 'UTC'), 8), (parseDateTimeBestEffort('2017-01-08T21:00', 'UTC'), 8), (parseDateTimeBestEffort('2017-01-09', 'UTC'), 2), (parseDateTimeBestEffort('2017-01-09T12:00', 'UTC'), 11), (parseDateTimeBestEffort('2017-01-10T05:00', 'UTC'), 5); - --- This test requies sorting after some of aggregations but I don't know KQL, sorry -set max_bytes_before_external_group_by = 0; -set dialect = 'kusto'; - -print '-- from to'; -make_series_test_table | make-series PriceAvg = avg(Price) default=0 on Purchase from datetime(2016-09-10) to datetime(2016-09-13) step 1d by Supplier, Fruit | order by Supplier, Fruit; -print '-- from'; -make_series_test_table | make-series PriceAvg = avg(Price) default=0 on Purchase from datetime(2016-09-10) step 1d by Supplier, Fruit | order by Supplier, Fruit; -print '-- to'; -make_series_test_table | make-series PriceAvg = avg(Price) default=0 on Purchase to datetime(2016-09-13) step 1d by Supplier, Fruit | order by Supplier, Fruit; -print '-- without from/to'; -make_series_test_table | make-series PriceAvg = avg(Price) default=0 on Purchase step 1d by Supplier, Fruit | order by Supplier, Fruit; -print '-- without by'; -make_series_test_table | make-series PriceAvg = avg(Price) default=0 on Purchase step 1d; -print '-- without aggregation alias'; -make_series_test_table | make-series avg(Price) default=0 on Purchase step 1d by Supplier, Fruit; -print '-- assign group alias'; -make_series_test_table | make-series avg(Price) default=0 on Purchase step 1d by Supplier_Name = Supplier, Fruit; -print '-- 3d step'; -make_series_test_table | make-series PriceAvg = avg(Price) default=0 on Purchase from datetime(2016-09-10) to datetime(2016-09-13) step 3d by Supplier, Fruit | order by Supplier, Fruit; - -print '-- numeric column' -print '-- from to'; -make_series_test_table2 | make-series PriceAvg=avg(Price) default=0 on Purchase from 10 to 15 step 1.0 by Supplier, Fruit; -print '-- from'; -make_series_test_table2 | make-series PriceAvg=avg(Price) default=0 on Purchase from 10 step 1.0 by Supplier, Fruit; -print '-- to'; -make_series_test_table2 | make-series PriceAvg=avg(Price) default=0 on Purchase to 18 step 4.0 by Supplier, Fruit; -print '-- without from/to'; -make_series_test_table2 | make-series PriceAvg=avg(Price) default=0 on Purchase step 2.0 by Supplier, Fruit; -print '-- without by'; -make_series_test_table2 | make-series PriceAvg=avg(Price) default=0 on Purchase step 2.0; - -make_series_test_table3 | make-series avg(metric) default=0 on timestamp from datetime(2017-01-01) to datetime(2017-01-10) step 1d - --- print '-- summarize --' --- make_series_test_table | summarize count() by format_datetime(bin(Purchase, 1d), 'yy-MM-dd'); diff --git a/tests/queries/0_stateless/02366_kql_mvexpand.reference b/tests/queries/0_stateless/02366_kql_mvexpand.reference deleted file mode 100644 index 25be070eb0b..00000000000 --- a/tests/queries/0_stateless/02366_kql_mvexpand.reference +++ /dev/null @@ -1,65 +0,0 @@ --- mv-expand -- --- mv_expand_test_table | mv-expand c -- -1 ['Salmon','Steak','Chicken'] 1 [5,6,7,8] -1 ['Salmon','Steak','Chicken'] 2 [5,6,7,8] -1 ['Salmon','Steak','Chicken'] 3 [5,6,7,8] -1 ['Salmon','Steak','Chicken'] 4 [5,6,7,8] --- mv_expand_test_table | mv-expand c, d -- -1 ['Salmon','Steak','Chicken'] 1 5 -1 ['Salmon','Steak','Chicken'] 2 6 -1 ['Salmon','Steak','Chicken'] 3 7 -1 ['Salmon','Steak','Chicken'] 4 8 --- mv_expand_test_table | mv-expand b | mv-expand c -- -1 Salmon 1 [5,6,7,8] -1 Salmon 2 [5,6,7,8] -1 Salmon 3 [5,6,7,8] -1 Salmon 4 [5,6,7,8] -1 Steak 1 [5,6,7,8] -1 Steak 2 [5,6,7,8] -1 Steak 3 [5,6,7,8] -1 Steak 4 [5,6,7,8] -1 Chicken 1 [5,6,7,8] -1 Chicken 2 [5,6,7,8] -1 Chicken 3 [5,6,7,8] -1 Chicken 4 [5,6,7,8] --- mv_expand_test_table | mv-expand with_itemindex=index b, c, d -- -0 1 Salmon 1 5 -1 1 Steak 2 6 -2 1 Chicken 3 7 -3 1 4 8 --- mv_expand_test_table | mv-expand array_concat(c,d) -- -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 1 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 2 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 3 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 4 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 5 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 6 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 7 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 8 --- mv_expand_test_table | mv-expand x = c, y = d -- -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 1 5 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 2 6 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 3 7 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 4 8 --- mv_expand_test_table | mv-expand xy = array_concat(c, d) -- -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 1 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 2 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 3 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 4 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 5 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 6 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 7 -1 ['Salmon','Steak','Chicken'] [1,2,3,4] [5,6,7,8] 8 --- mv_expand_test_table | mv-expand xy = array_concat(c, d) limit 2| summarize count() by xy -- -1 1 -2 1 --- mv_expand_test_table | mv-expand with_itemindex=index c,d to typeof(bool) -- -0 1 ['Salmon','Steak','Chicken'] 1 true -1 1 ['Salmon','Steak','Chicken'] 2 true -2 1 ['Salmon','Steak','Chicken'] 3 true -3 1 ['Salmon','Steak','Chicken'] 4 true --- mv_expand_test_table | mv-expand c to typeof(bool) -- -1 ['Salmon','Steak','Chicken'] [5,6,7,8] true -1 ['Salmon','Steak','Chicken'] [5,6,7,8] true -1 ['Salmon','Steak','Chicken'] [5,6,7,8] true -1 ['Salmon','Steak','Chicken'] [5,6,7,8] true diff --git a/tests/queries/0_stateless/02366_kql_mvexpand.sql b/tests/queries/0_stateless/02366_kql_mvexpand.sql deleted file mode 100644 index e7798609646..00000000000 --- a/tests/queries/0_stateless/02366_kql_mvexpand.sql +++ /dev/null @@ -1,35 +0,0 @@ --- datatable(a: int, b: dynamic, c: dynamic, d: dynamic) [ --- 1, dynamic(['Salmon', 'Steak', 'Chicken']), dynamic([1, 2, 3, 4]), dynamic([5, 6, 7, 8]) --- ] - -DROP TABLE IF EXISTS mv_expand_test_table; -CREATE TABLE mv_expand_test_table -( - a UInt8, - b Array(String), - c Array(Int8), - d Array(Int8) -) ENGINE = Memory; -INSERT INTO mv_expand_test_table VALUES (1, ['Salmon', 'Steak','Chicken'],[1,2,3,4],[5,6,7,8]); -set dialect='kusto'; -print '-- mv-expand --'; -print '-- mv_expand_test_table | mv-expand c --'; -mv_expand_test_table | mv-expand c; -print '-- mv_expand_test_table | mv-expand c, d --'; -mv_expand_test_table | mv-expand c, d; -print '-- mv_expand_test_table | mv-expand b | mv-expand c --'; -mv_expand_test_table | mv-expand b | mv-expand c; -print '-- mv_expand_test_table | mv-expand with_itemindex=index b, c, d --'; -mv_expand_test_table | mv-expand with_itemindex=index b, c, d; -print '-- mv_expand_test_table | mv-expand array_concat(c,d) --'; -mv_expand_test_table | mv-expand array_concat(c,d); -print '-- mv_expand_test_table | mv-expand x = c, y = d --'; -mv_expand_test_table | mv-expand x = c, y = d; -print '-- mv_expand_test_table | mv-expand xy = array_concat(c, d) --'; -mv_expand_test_table | mv-expand xy = array_concat(c, d); -print '-- mv_expand_test_table | mv-expand xy = array_concat(c, d) limit 2| summarize count() by xy --'; -mv_expand_test_table | mv-expand xy = array_concat(c, d) limit 2| summarize count() by xy; -print '-- mv_expand_test_table | mv-expand with_itemindex=index c,d to typeof(bool) --'; -mv_expand_test_table | mv-expand with_itemindex=index c,d to typeof(bool); -print '-- mv_expand_test_table | mv-expand c to typeof(bool) --'; -mv_expand_test_table | mv-expand c to typeof(bool); diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.reference b/tests/queries/0_stateless/02366_kql_native_interval_format.reference deleted file mode 100644 index 8a12c6885c4..00000000000 --- a/tests/queries/0_stateless/02366_kql_native_interval_format.reference +++ /dev/null @@ -1,23 +0,0 @@ -numeric -kusto -00:00:00 -00:00:00.0000001 -00:00:00.0010000 -00:00:42 -01:06:00 -2.18:00:00 -5.00:00:00 -7.00:00:00 -14.00:00:00 -('00:01:12','21.00:00:00','00:00:00.0000002') -numeric -99 -100 -1 -42 -66 -66 -5 -1 -2 -(72,3,200) diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 b/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 deleted file mode 100644 index 0731687222d..00000000000 --- a/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 +++ /dev/null @@ -1,16 +0,0 @@ -select value from system.settings where name = 'interval_output_format'; - -{% for format in ['kusto', 'numeric'] -%} -select '{{ format }}'; -set interval_output_format = '{{ format }}'; -select toIntervalNanosecond(99); -select toIntervalNanosecond(100); -select toIntervalMillisecond(1); -select toIntervalSecond(42); -select toIntervalMinute(66); -select toIntervalHour(66); -select toIntervalDay(5); -select toIntervalWeek(1); -select toIntervalWeek(2); -select toIntervalSecond(72) + toIntervalWeek(3) + toIntervalNanosecond(200); -{% endfor -%} diff --git a/tests/queries/0_stateless/02366_kql_operator_in_sql.reference b/tests/queries/0_stateless/02366_kql_operator_in_sql.reference deleted file mode 100644 index 4e0987aa5c3..00000000000 --- a/tests/queries/0_stateless/02366_kql_operator_in_sql.reference +++ /dev/null @@ -1,60 +0,0 @@ --- #1 -- -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Apple Skilled Manual Bachelors 28 --- #2 -- -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Peter Nara Skilled Manual Graduate Degree 26 -Latoya Shen Professional Graduate Degree 25 -Apple Skilled Manual Bachelors 28 --- #3 -- -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Latoya Shen Professional Graduate Degree 25 -Apple Skilled Manual Bachelors 28 --- #4 -- -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Latoya Shen Professional Graduate Degree 25 -Apple Skilled Manual Bachelors 28 --- #5 -- -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Latoya Shen Professional Graduate Degree 25 -Apple Skilled Manual Bachelors 28 --- #6 -- -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Latoya Shen Professional Graduate Degree 25 -Apple Skilled Manual Bachelors 28 --- #7 -- -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Latoya Shen Professional Graduate Degree 25 -Apple Skilled Manual Bachelors 28 --- #8 -- -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Latoya Shen Professional Graduate Degree 25 -Apple Skilled Manual Bachelors 28 --- #9 -- -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Peter Nara Skilled Manual Graduate Degree 26 -Latoya Shen Professional Graduate Degree 25 -Apple Skilled Manual Bachelors 28 --- #10 -- --- #11 -- --- #12 -- --- #13 -- --- #14 -- -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Latoya Shen Professional Graduate Degree 25 -Apple Skilled Manual Bachelors 28 --- #15 -- -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management abcd defg Bachelors 33 -Latoya Shen Professional Graduate Degree 25 -Apple Skilled Manual Bachelors 28 diff --git a/tests/queries/0_stateless/02366_kql_operator_in_sql.sql b/tests/queries/0_stateless/02366_kql_operator_in_sql.sql deleted file mode 100644 index 0b02faa0680..00000000000 --- a/tests/queries/0_stateless/02366_kql_operator_in_sql.sql +++ /dev/null @@ -1,42 +0,0 @@ -DROP TABLE IF EXISTS Customers; -CREATE TABLE Customers -( - FirstName Nullable(String), - LastName String, - Occupation String, - Education String, - Age Nullable(UInt8) -) ENGINE = Memory; - -INSERT INTO Customers VALUES ('Theodore','Diaz','Skilled Manual','Bachelors',28),('Stephanie','Cox','Management abcd defg','Bachelors',33),('Peter','Nara','Skilled Manual','Graduate Degree',26),('Latoya','Shen','Professional','Graduate Degree',25),('Apple','','Skilled Manual','Bachelors',28),(NULL,'why','Professional','Partial College',38); -Select '-- #1 --' ; -select * from kql($$Customers | where FirstName !in ('Peter', 'Latoya')$$); -Select '-- #2 --' ; -select * from kql($$Customers | where FirstName !in ("test", "test2")$$); -Select '-- #3 --' ; -select * from kql($$Customers | where FirstName !contains 'Pet'$$); -Select '-- #4 --' ; -select * from kql($$Customers | where FirstName !contains_cs 'Pet'$$); -Select '-- #5 --' ; -select * from kql($$Customers | where FirstName !endswith 'ter'$$); -Select '-- #6 --' ; -select * from kql($$Customers | where FirstName !endswith_cs 'ter'$$); -Select '-- #7 --' ; -select * from kql($$Customers | where FirstName != 'Peter'$$); -Select '-- #8 --' ; -select * from kql($$Customers | where FirstName !has 'Peter'$$); -Select '-- #9 --' ; -select * from kql($$Customers | where FirstName !has_cs 'peter'$$); -Select '-- #10 --' ; --- select * from kql($$Customers | where FirstName !hasprefix 'Peter'$$); -- will enable when analyzer fixed `and` issue -Select '-- #11 --' ; ---select * from kql($$Customers | where FirstName !hasprefix_cs 'Peter'$$); -Select '-- #12 --' ; ---select * from kql($$Customers | where FirstName !hassuffix 'Peter'$$); -Select '-- #13 --' ; ---select * from kql($$Customers | where FirstName !hassuffix_cs 'Peter'$$); -Select '-- #14 --' ; -select * from kql($$Customers | where FirstName !startswith 'Peter'$$); -Select '-- #15 --' ; -select * from kql($$Customers | where FirstName !startswith_cs 'Peter'$$); -DROP TABLE IF EXISTS Customers; diff --git a/tests/queries/0_stateless/02366_kql_summarize.reference b/tests/queries/0_stateless/02366_kql_summarize.reference deleted file mode 100644 index aeb42feb6be..00000000000 --- a/tests/queries/0_stateless/02366_kql_summarize.reference +++ /dev/null @@ -1,92 +0,0 @@ --- test summarize -- -12 25 46 32.416666666666664 389 -Skilled Manual 5 26 36 30.2 151 -Professional 6 25 46 34.166666666666664 205 -Management abcd defg 1 33 33 33 33 -Skilled Manual 0 -Professional 2 -Management abcd defg 0 -Skilled Manual 36 -Professional 38 -Management abcd defg 33 -Skilled Manual 26 -Professional 25 -Management abcd defg 33 -Skilled Manual 30.2 -Professional 29.25 -Management abcd defg 33 -Skilled Manual 151 -Professional 117 -Management abcd defg 33 -4 -2 -40 2 -30 4 -20 6 -Skilled Manual 5 -Professional 6 -Management abcd defg 1 --- make_list() -- -Skilled Manual ['Bachelors','Graduate Degree','High School','Partial College','Bachelors'] -Professional ['Graduate Degree','Partial College','Partial College','Partial College','Partial College','Partial College'] -Management abcd defg ['Bachelors'] -Skilled Manual ['Bachelors','Graduate Degree'] -Professional ['Graduate Degree','Partial College'] -Management abcd defg ['Bachelors'] --- make_list_if() -- -Skilled Manual ['Edward','Christine'] -Professional ['Dalton','Angel'] -Management abcd defg ['Stephanie'] -Skilled Manual ['Edward'] -Professional ['Dalton'] -Management abcd defg ['Stephanie'] --- make_set() -- -Skilled Manual ['Graduate Degree','High School','Partial College','Bachelors'] -Professional ['Graduate Degree','Partial College'] -Management abcd defg ['Bachelors'] -Skilled Manual ['Graduate Degree','Bachelors'] -Professional ['Graduate Degree','Partial College'] -Management abcd defg ['Bachelors'] --- make_set_if() -- -Skilled Manual ['Partial College','High School'] -Professional ['Partial College'] -Management abcd defg ['Bachelors'] -Skilled Manual ['High School'] -Professional ['Partial College'] -Management abcd defg ['Bachelors'] --- stdev() -- -6.855102059227432 --- stdevif() -- -7.557189365836421 --- binary_all_and -- -42 --- binary_all_or -- -46 --- binary_all_xor -- -4 -43.8 -25.55 30.5 43.8 -30.5 -35 -[25,35,45] --- Summarize following sort -- -Skilled Manual 5 -Professional 6 -Management abcd defg 1 --- summarize with bin -- -0 1 -245000 2 -0 1 -245 2 -0 1 -245 2 -2015-10-12 00:00:00.000000000 -2016-10-12 00:00:00.000000000 --- make_list_with_nulls -- -['Theodore','Stephanie','Peter','Latoya','Joshua','Edward','Dalton','Christine','Cameron','Angel','Apple',NULL] -Skilled Manual ['Theodore','Peter','Edward','Christine','Apple'] -Professional ['Latoya','Joshua','Dalton','Cameron','Angel',NULL] -Management abcd defg ['Stephanie'] -Skilled Manual ['Theodore','Peter','Edward','Christine','Apple'] [28,26,36,33,28] -Professional ['Latoya','Joshua','Dalton','Cameron','Angel',NULL] [25,26,42,28,46,38] -Management abcd defg ['Stephanie'] [33] diff --git a/tests/queries/0_stateless/02366_kql_summarize.sql b/tests/queries/0_stateless/02366_kql_summarize.sql deleted file mode 100644 index bb12d1f251f..00000000000 --- a/tests/queries/0_stateless/02366_kql_summarize.sql +++ /dev/null @@ -1,102 +0,0 @@ --- datatable(FirstName:string, LastName:string, Occupation:string, Education:string, Age:int) [ --- 'Theodore', 'Diaz', 'Skilled Manual', 'Bachelors', 28, --- 'Stephanie', 'Cox', 'Management abcd defg', 'Bachelors', 33, --- 'Peter', 'Nara', 'Skilled Manual', 'Graduate Degree', 26, --- 'Latoya', 'Shen', 'Professional', 'Graduate Degree', 25, --- 'Joshua', 'Lee', 'Professional', 'Partial College', 26, --- 'Edward', 'Hernandez', 'Skilled Manual', 'High School', 36, --- 'Dalton', 'Wood', 'Professional', 'Partial College', 42, --- 'Christine', 'Nara', 'Skilled Manual', 'Partial College', 33, --- 'Cameron', 'Rodriguez', 'Professional', 'Partial College', 28, --- 'Angel', 'Stewart', 'Professional', 'Partial College', 46, --- 'Apple', '', 'Skilled Manual', 'Bachelors', 28, --- dynamic(null), 'why', 'Professional', 'Partial College', 38 --- ] - -DROP TABLE IF EXISTS Customers; -CREATE TABLE Customers -( - FirstName Nullable(String), - LastName String, - Occupation String, - Education String, - Age Nullable(UInt8) -) ENGINE = Memory; - -INSERT INTO Customers VALUES ('Theodore','Diaz','Skilled Manual','Bachelors',28),('Stephanie','Cox','Management abcd defg','Bachelors',33),('Peter','Nara','Skilled Manual','Graduate Degree',26),('Latoya','Shen','Professional','Graduate Degree',25),('Joshua','Lee','Professional','Partial College',26),('Edward','Hernandez','Skilled Manual','High School',36),('Dalton','Wood','Professional','Partial College',42),('Christine','Nara','Skilled Manual','Partial College',33),('Cameron','Rodriguez','Professional','Partial College',28),('Angel','Stewart','Professional','Partial College',46),('Apple','','Skilled Manual','Bachelors',28),(NULL,'why','Professional','Partial College',38); - -drop table if exists EventLog; -create table EventLog -( - LogEntry String, - Created Int64 -) ENGINE = Memory; - -insert into EventLog values ('Darth Vader has entered the room.', 546), ('Rambo is suspciously looking at Darth Vader.', 245234), ('Darth Sidious electrocutes both using Force Lightning.', 245554); - -drop table if exists Dates; -create table Dates -( - EventTime DateTime, -) ENGINE = Memory; - -Insert into Dates VALUES ('2015-10-12') , ('2016-10-12') -Select '-- test summarize --' ; -set dialect='kusto'; -Customers | summarize count(), min(Age), max(Age), avg(Age), sum(Age); -Customers | summarize count(), min(Age), max(Age), avg(Age), sum(Age) by Occupation | order by Occupation; -Customers | summarize countif(Age>40) by Occupation | order by Occupation; -Customers | summarize MyMax = maxif(Age, Age<40) by Occupation | order by Occupation; -Customers | summarize MyMin = minif(Age, Age<40) by Occupation | order by Occupation; -Customers | summarize MyAvg = avgif(Age, Age<40) by Occupation | order by Occupation; -Customers | summarize MySum = sumif(Age, Age<40) by Occupation | order by Occupation; -Customers | summarize dcount(Education); -Customers | summarize dcountif(Education, Occupation=='Professional'); -Customers | summarize count_ = count() by bin(Age, 10) | order by count_ asc; -Customers | summarize job_count = count() by Occupation | where job_count > 0 | order by Occupation; -Customers | summarize 'Edu Count'=count() by Education | sort by 'Edu Count' desc; -- { clientError 62 } - -print '-- make_list() --'; -Customers | summarize f_list = make_list(Education) by Occupation | sort by Occupation; -Customers | summarize f_list = make_list(Education, 2) by Occupation | sort by Occupation; -print '-- make_list_if() --'; -Customers | summarize f_list = make_list_if(FirstName, Age>30) by Occupation | sort by Occupation; -Customers | summarize f_list = make_list_if(FirstName, Age>30, 1) by Occupation | sort by Occupation; -print '-- make_set() --'; -Customers | summarize f_list = make_set(Education) by Occupation | sort by Occupation; -Customers | summarize f_list = make_set(Education, 2) by Occupation | sort by Occupation; -print '-- make_set_if() --'; -Customers | summarize f_list = make_set_if(Education, Age>30) by Occupation | sort by Occupation; -Customers | summarize f_list = make_set_if(Education, Age>30, 1) by Occupation | sort by Occupation; -print '-- stdev() --'; -Customers | project Age | summarize stdev(Age); -print '-- stdevif() --'; -Customers | project Age | summarize stdevif(Age, Age%2==0); -print '-- binary_all_and --'; -Customers | project Age | where Age > 40 | summarize binary_all_and(Age); -print '-- binary_all_or --'; -Customers | project Age | where Age > 40 | summarize binary_all_or(Age); -print '-- binary_all_xor --'; -Customers | project Age | where Age > 40 | summarize binary_all_xor(Age); - -Customers | project Age | summarize percentile(Age, 95); -Customers | project Age | summarize percentiles(Age, 5, 50, 95)|project round(percentiles_Age[0],2),round(percentiles_Age[1],2),round(percentiles_Age[2],2); -Customers | project Age | summarize percentiles(Age, 5, 50, 95)[1]; -Customers | summarize w=count() by AgeBucket=bin(Age, 5) | summarize percentilew(AgeBucket, w, 75); -Customers | summarize w=count() by AgeBucket=bin(Age, 5) | summarize percentilesw(AgeBucket, w, 50, 75, 99.9); - -print '-- Summarize following sort --'; -Customers | sort by FirstName | summarize count() by Occupation | sort by Occupation; - -print '-- summarize with bin --'; -EventLog | summarize count=count() by bin(Created, 1000) | sort by count asc; -EventLog | summarize count=count() by bin(unixtime_seconds_todatetime(Created/1000), 1s) | sort by count asc; -EventLog | summarize count=count() by time_label=bin(Created/1000, 1s) | sort by count asc; -Dates | project bin(datetime(EventTime), 1m); -print '-- make_list_with_nulls --'; -Customers | summarize t = make_list_with_nulls(FirstName); -Customers | summarize f_list = make_list_with_nulls(FirstName) by Occupation | sort by Occupation; -Customers | summarize f_list = make_list_with_nulls(FirstName), a_list = make_list_with_nulls(Age) by Occupation | sort by Occupation; --- TODO: --- arg_max() --- arg_min() diff --git a/tests/queries/0_stateless/02366_kql_tabular.reference b/tests/queries/0_stateless/02366_kql_tabular.reference deleted file mode 100644 index e70c02ce34f..00000000000 --- a/tests/queries/0_stateless/02366_kql_tabular.reference +++ /dev/null @@ -1,139 +0,0 @@ --- test Query only has table name: -- -Theodore Diaz Skilled Manual Bachelors 28 -Stephanie Cox Management Bachelors 33 -Peter Nara Skilled Manual Graduate Degree 26 -Latoya Shen Professional Graduate Degree 25 -Joshua Lee Professional Partial College 26 -Edward Hernandez Skilled Manual High School 36 -Dalton Wood Professional Partial College 42 -Christine Nara Skilled Manual Partial College 33 -Cameron Rodriguez Professional Partial College 28 -Angel Stewart Professional Partial College 46 --- Query has Column Selection -- -Theodore Diaz Skilled Manual -Stephanie Cox Management -Peter Nara Skilled Manual -Latoya Shen Professional -Joshua Lee Professional -Edward Hernandez Skilled Manual -Dalton Wood Professional -Christine Nara Skilled Manual -Cameron Rodriguez Professional -Angel Stewart Professional --- Query has limit -- -Theodore Diaz Skilled Manual -Stephanie Cox Management -Peter Nara Skilled Manual -Latoya Shen Professional -Joshua Lee Professional -Theodore Diaz Skilled Manual -Stephanie Cox Management -Peter Nara Skilled Manual -Latoya Shen Professional -Joshua Lee Professional --- Query has second limit with bigger value -- -Theodore Diaz Skilled Manual -Stephanie Cox Management -Peter Nara Skilled Manual -Latoya Shen Professional -Joshua Lee Professional --- Query has second limit with smaller value -- -Theodore Diaz Skilled Manual -Stephanie Cox Management -Peter Nara Skilled Manual --- Query has second Column selection -- -Theodore Diaz -Stephanie Cox -Peter Nara --- Query has second Column selection with extra column -- --- Query with desc sort -- -Theodore -Stephanie -Peter -Latoya -Joshua -Skilled Manual -Skilled Manual -Professional -Professional -Management --- Query with asc sort -- -Management -Professional -Professional -Skilled Manual -Skilled Manual --- Query with sort (without keyword asc desc) -- -Theodore -Stephanie -Peter -Latoya -Joshua -Skilled Manual -Skilled Manual -Professional -Professional -Management --- Query with sort 2 Columns with different direction -- -Stephanie Cox Management -Latoya Shen Professional -Joshua Lee Professional -Peter Nara Skilled Manual -Theodore Diaz Skilled Manual --- Query with second sort -- -Stephanie Cox Management -Latoya Shen Professional -Joshua Lee Professional -Peter Nara Skilled Manual -Theodore Diaz Skilled Manual --- Test String Equals (==) -- -Theodore Diaz Skilled Manual -Peter Nara Skilled Manual -Edward Hernandez Skilled Manual -Christine Nara Skilled Manual --- Test String Not equals (!=) -- -Stephanie Cox Management -Latoya Shen Professional -Joshua Lee Professional -Dalton Wood Professional -Cameron Rodriguez Professional -Angel Stewart Professional --- Test Filter using a list (in) -- -Theodore Diaz Skilled Manual Bachelors -Stephanie Cox Management Bachelors -Edward Hernandez Skilled Manual High School --- Test Filter using a list (!in) -- -Peter Nara Skilled Manual Graduate Degree -Latoya Shen Professional Graduate Degree -Joshua Lee Professional Partial College -Dalton Wood Professional Partial College -Christine Nara Skilled Manual Partial College -Cameron Rodriguez Professional Partial College -Angel Stewart Professional Partial College --- Test Filter using common string operations (contains_cs) -- -Joshua Lee Professional Partial College -Dalton Wood Professional Partial College -Christine Nara Skilled Manual Partial College -Cameron Rodriguez Professional Partial College -Angel Stewart Professional Partial College --- Test Filter using common string operations (startswith_cs) -- -Latoya Shen Professional Graduate Degree -Joshua Lee Professional Partial College -Dalton Wood Professional Partial College -Cameron Rodriguez Professional Partial College -Angel Stewart Professional Partial College --- Test Filter using common string operations (endswith_cs) -- -Latoya Shen Professional Graduate Degree -Joshua Lee Professional Partial College --- Test Filter using numerical equal (==) -- -Peter Nara Skilled Manual Graduate Degree 26 -Joshua Lee Professional Partial College 26 --- Test Filter using numerical great and less (> , <) -- -Stephanie Cox Management Bachelors 33 -Edward Hernandez Skilled Manual High School 36 -Christine Nara Skilled Manual Partial College 33 --- Test Filter using multi where -- -Dalton Wood Professional Partial College 42 -Angel Stewart Professional Partial College 46 --- Complex query with unknown function -- --- Missing column in front of startsWith -- diff --git a/tests/queries/0_stateless/02366_kql_tabular.sql b/tests/queries/0_stateless/02366_kql_tabular.sql deleted file mode 100644 index f73c4c09cca..00000000000 --- a/tests/queries/0_stateless/02366_kql_tabular.sql +++ /dev/null @@ -1,88 +0,0 @@ -DROP TABLE IF EXISTS Customers; -CREATE TABLE Customers -( - FirstName Nullable(String), - LastName String, - Occupation String, - Education String, - Age Nullable(UInt8) -) ENGINE = Memory; - -INSERT INTO Customers VALUES ('Theodore','Diaz','Skilled Manual','Bachelors',28), ('Stephanie','Cox','Management','Bachelors',33), ('Peter','Nara','Skilled Manual','Graduate Degree',26), ('Latoya','Shen','Professional','Graduate Degree',25), ('Joshua','Lee','Professional','Partial College',26), ('Edward','Hernandez','Skilled Manual','High School',36), ('Dalton','Wood','Professional','Partial College',42), ('Christine','Nara','Skilled Manual','Partial College',33), ('Cameron','Rodriguez','Professional','Partial College',28), ('Angel','Stewart','Professional','Partial College',46); - -set dialect='kusto'; -print '-- test Query only has table name: --'; -Customers; - -print '-- Query has Column Selection --'; -Customers | project FirstName,LastName,Occupation; - -print '-- Query has limit --'; -Customers | project FirstName,LastName,Occupation | take 5; -Customers | project FirstName,LastName,Occupation | limit 5; - -print '-- Query has second limit with bigger value --'; -Customers | project FirstName,LastName,Occupation | take 5 | take 7; - -print '-- Query has second limit with smaller value --'; -Customers | project FirstName,LastName,Occupation | take 5 | take 3; - -print '-- Query has second Column selection --'; -Customers | project FirstName,LastName,Occupation | take 3 | project FirstName,LastName; - -print '-- Query has second Column selection with extra column --'; -Customers| project FirstName,LastName,Occupation | take 3 | project FirstName,LastName,Education;-- { serverError 47 } - -print '-- Query with desc sort --'; -Customers | project FirstName | take 5 | sort by FirstName desc; -Customers | project Occupation | take 5 | order by Occupation desc; - -print '-- Query with asc sort --'; -Customers | project Occupation | take 5 | sort by Occupation asc; - -print '-- Query with sort (without keyword asc desc) --'; -Customers | project FirstName | take 5 | sort by FirstName; -Customers | project Occupation | take 5 | order by Occupation; - -print '-- Query with sort 2 Columns with different direction --'; -Customers | project FirstName,LastName,Occupation | take 5 | sort by Occupation asc, LastName desc; - -print '-- Query with second sort --'; -Customers | project FirstName,LastName,Occupation | take 5 | sort by Occupation desc |sort by Occupation asc, LastName desc; - -print '-- Test String Equals (==) --'; -Customers | project FirstName,LastName,Occupation | where Occupation == 'Skilled Manual'; - -print '-- Test String Not equals (!=) --'; -Customers | project FirstName,LastName,Occupation | where Occupation != 'Skilled Manual'; - -print '-- Test Filter using a list (in) --'; -Customers | project FirstName,LastName,Occupation,Education | where Education in ('Bachelors','High School'); - -print '-- Test Filter using a list (!in) --'; -set dialect='kusto'; -Customers | project FirstName,LastName,Occupation,Education | where Education !in ('Bachelors','High School'); - -print '-- Test Filter using common string operations (contains_cs) --'; -Customers | project FirstName,LastName,Occupation,Education | where Education contains_cs 'Coll'; - -print '-- Test Filter using common string operations (startswith_cs) --'; -Customers | project FirstName,LastName,Occupation,Education | where Occupation startswith_cs 'Prof'; - -print '-- Test Filter using common string operations (endswith_cs) --'; -Customers | project FirstName,LastName,Occupation,Education | where FirstName endswith_cs 'a'; - -print '-- Test Filter using numerical equal (==) --'; -Customers | project FirstName,LastName,Occupation,Education,Age | where Age == 26; - -print '-- Test Filter using numerical great and less (> , <) --'; -Customers | project FirstName,LastName,Occupation,Education,Age | where Age > 30 and Age < 40; - -print '-- Test Filter using multi where --'; -Customers | project FirstName,LastName,Occupation,Education,Age | where Age > 30 | where Occupation == 'Professional'; - -print '-- Complex query with unknown function --'; -hits | where CounterID == 62 and EventDate >= '2013-07-14' and EventDate <= '2013-07-15' and IsRefresh == 0 and DontCountHits == 0 | summarize count() by d=bin(poopoo(EventTime), 1m) | order by d | limit 10; -- { clientError UNKNOWN_FUNCTION } - -print '-- Missing column in front of startsWith --'; -StormEvents | where startswith "W" | summarize Count=count() by State; -- { clientError SYNTAX_ERROR } From f57edeb7dd9b2c7b6b98ee919decf1f7b8cef97c Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sun, 28 Jan 2024 18:32:02 +0100 Subject: [PATCH 254/264] impl --- .../00165_jit_aggregate_functions.reference | 70 ------------ .../00165_jit_aggregate_functions.sql | 102 ------------------ 2 files changed, 172 deletions(-) diff --git a/tests/queries/1_stateful/00165_jit_aggregate_functions.reference b/tests/queries/1_stateful/00165_jit_aggregate_functions.reference index fa084170f53..62baba2af8b 100644 --- a/tests/queries/1_stateful/00165_jit_aggregate_functions.reference +++ b/tests/queries/1_stateful/00165_jit_aggregate_functions.reference @@ -68,73 +68,3 @@ Simple functions with non compilable function without key 4611686725751467379 9223371678237104442 3626326766789368100 61384643584599682996279588 408650940859.2896 104735.01095549858 8873898 9223372036854775807 4611686018427387904 3818489297630359920 Simple functions if combinator without key 4611687533683519016 9223371678237104442 4124667747700004330 930178817930.5122 321189.2280948817 4434274 9223372036854775806 4611686018427387904 2265422677606390266 -Aggregation without JIT compilation -Simple functions -1704509 4611700827100483880 9223360787015464643 10441337359398154812 19954243669348.844 9648741.579254271 523264 9223372036854775807 4611686018427387904 4544239379628300646 -732797 4611701940806302259 9223355550934604746 977192643464016658 2054229034942.3723 51998323.94457991 475698 9223372036854775807 4611686018427387904 4091184823334377716 -598875 4611701407242345792 9223362250391155632 9312163881623734456 27615161624211.875 12261797.824844675 337212 9223372036854775807 4611686018427387904 3725992504798702670 -792887 4611699550286611812 9223290551912005343 6930300520201292824 27479710385933.586 53095331.60360441 252197 9223372036854775807 4611686018427387904 6536441508464694614 -3807842 4611710821592843606 9223326163906184987 16710274896338005145 85240848090850.69 22373416.533275086 196036 9223372036854775807 4611686018427387904 1797862753609257231 -25703952 4611709443519524003 9223353913449113943 9946868158853570839 67568783303242.086 3154349.826950714 147211 9223372036854775807 4611686018427387904 8737124378202300429 -716829 4611852156092872082 9223361623076951140 15381015774917924786 170693446547158.72 201431892.4773785 90109 9223372036854775807 4611686018427387904 8209915323001116338 -59183 4611730685242027332 9223354909338698162 8078812522502896568 94622946187035.42 1425270865.0901496 85379 9223372036854775807 4611686018427387904 8909082036598843562 -33010362 4611704682869732882 9223268545373999677 2064452191838585926 26532987929602.555 3695122.4062526934 77807 9223372036854775807 4611686018427387904 5411365383789552292 -800784 4611752907938305166 9223340418389788041 18082918611792817587 233352070043266.62 36535786.81446395 77492 9223372036854775807 4611686018427387904 2059255810151375435 -20810645 4611712185532639162 9223218900001937412 4996531385439292694 68246505203164.63 6316535.831023813 73213 9223372036854775807 4611686018427387904 8852740550386113674 -25843850 4611690025407720929 9223346023778617822 12755881190906812868 185015319325648.16 9962165.34831339 68945 9223372036854775807 4611686018427387904 7849665866595760148 -23447120 4611796031755620254 9223329309291309758 17231649548755339966 255019232629204.38 7937191.271698021 67570 9223372036854775807 4611686018427387904 3435410911925610424 -14739804 4611692230555590277 9223313509005166531 2458378896777063244 38308020331864.36 14590240.469105456 64174 9223372036854775807 4611686018427387904 511910855240035342 -32077710 4611884228437061959 9223352444952988904 12965822147651192908 214467085941034.7 7257521.096258734 60456 9223372036854775807 4611686018427387904 2256071920672551964 -22446879 4611846229717089436 9223124373140579096 13530160492087688838 231724477077663.4 4737362.521046629 58389 9223372036854775807 4611686018427387904 6236276364886386410 -170282 4611833225706935900 9223371583739401906 8076893424988479310 141657635880324.8 1613795518.1065989 57017 9223372036854775807 4611686018427387904 4755775861151848768 -11482817 4611708000353743073 9223337838355779113 14841435427430843458 283531099960470.8 9938452.835998287 52345 9223372036854775807 4611686018427387904 5371586112642152558 -63469 4611695097019173921 9223353530156141191 6296784708578574520 120762239817777.88 579655378.4603049 52142 9223372036854775807 4611686018427387904 4150567963952988110 -29103473 4611744585914335132 9223333530281362537 5908285283932344933 123712996438970.34 867841.595541967 47758 9223372036854775807 4611686018427387904 3238284030821087319 -Simple functions with non compilable function -1704509 4611700827100483880 9223360787015464643 10441337359398154812 3620921835565807284859452 19954243669348.844 9648741.579254271 523264 9223372036854775807 4611686018427387904 4544239379628300646 -732797 4611701940806302259 9223355550934604746 977192643464016658 3289442827160604417733394 2054229034942.3723 51998323.94457991 475698 9223372036854775807 4611686018427387904 4091184823334377716 -598875 4611701407242345792 9223362250391155632 9312163881623734456 2330921446573746856380600 27615161624211.875 12261797.824844675 337212 9223372036854775807 4611686018427387904 3725992504798702670 -792887 4611699550286611812 9223290551912005343 6930300520201292824 1745179600137886041476120 27479710385933.586 53095331.60360441 252197 9223372036854775807 4611686018427387904 6536441508464694614 -3807842 4611710821592843606 9223326163906184987 16710274896338005145 1356295121550317411019929 85240848090850.69 22373416.533275086 196036 9223372036854775807 4611686018427387904 1797862753609257231 -25703952 4611709443519524003 9223353913449113943 9946868158853570839 1018731388338768841564439 67568783303242.086 3154349.826950714 147211 9223372036854775807 4611686018427387904 8737124378202300429 -716829 4611852156092872082 9223361623076951140 15381015774917924786 623810478612337115371442 170693446547158.72 201431892.4773785 90109 9223372036854775807 4611686018427387904 8209915323001116338 -59183 4611730685242027332 9223354909338698162 8078812522502896568 589916507545680254024632 94622946187035.42 1425270865.0901496 85379 9223372036854775807 4611686018427387904 8909082036598843562 -33010362 4611704682869732882 9223268545373999677 2064452191838585926 538517864195994778911814 26532987929602.555 3695122.4062526934 77807 9223372036854775807 4611686018427387904 5411365383789552292 -800784 4611752907938305166 9223340418389788041 18082918611792817587 535545510122473785781683 233352070043266.62 36535786.81446395 77492 9223372036854775807 4611686018427387904 2059255810151375435 -20810645 4611712185532639162 9223218900001937412 4996531385439292694 506405014842860050255126 68246505203164.63 6316535.831023813 73213 9223372036854775807 4611686018427387904 8852740550386113674 -25843850 4611690025407720929 9223346023778617822 12755881190906812868 476547495537329753708996 185015319325648.16 9962165.34831339 68945 9223372036854775807 4611686018427387904 7849665866595760148 -23447120 4611796031755620254 9223329309291309758 17231649548755339966 467236365548464278670014 255019232629204.38 7937191.271698021 67570 9223372036854775807 4611686018427387904 3435410911925610424 -14739804 4611692230555590277 9223313509005166531 2458378896777063244 444126268697527941770060 38308020331864.36 14590240.469105456 64174 9223372036854775807 4611686018427387904 511910855240035342 -32077710 4611884228437061959 9223352444952988904 12965822147651192908 417407443977973675608140 214467085941034.7 7257521.096258734 60456 9223372036854775807 4611686018427387904 2256071920672551964 -22446879 4611846229717089436 9223124373140579096 13530160492087688838 403462269796593691082374 231724477077663.4 4737362.521046629 58389 9223372036854775807 4611686018427387904 6236276364886386410 -170282 4611833225706935900 9223371583739401906 8076893424988479310 394417911933408911581006 141657635880324.8 1613795518.1065989 57017 9223372036854775807 4611686018427387904 4755775861151848768 -11482817 4611708000353743073 9223337838355779113 14841435427430843458 361995300393829962204226 283531099960470.8 9938452.835998287 52345 9223372036854775807 4611686018427387904 5371586112642152558 -63469 4611695097019173921 9223353530156141191 6296784708578574520 360843057610541117735096 120762239817777.88 579655378.4603049 52142 9223372036854775807 4611686018427387904 4150567963952988110 -29103473 4611744585914335132 9223333530281362537 5908285283932344933 330534668598011678200421 123712996438970.34 867841.595541967 47758 9223372036854775807 4611686018427387904 3238284030821087319 -Simple functions if combinator -1704509 4611700827100483880 9223310246721229500 16398241567152875142 62618822667209.71 2224726.7626273884 261874 9223372036854775806 4611686018427387904 4518874482384062894 -732797 4611721382223060002 9223355550934604746 16281585268876620522 68472164943295.68 5898616.931652982 237784 9223372036854775806 4611686018427387904 3641900047478154650 -598875 4611701407242345792 9223362250391155632 3577699408183553052 21300140553347.42 53771550.26565126 167966 9223372036854775806 4611686018427387904 1688477495230210408 -792887 4611699550286611812 9223164887726235740 7088177025760385824 56461952267903.89 92835869.96920013 125539 9223372036854775806 4611686018427387904 4850868151095058072 -3807842 4611710821592843606 9223283397553859544 5756765290752687660 58835559208469.4 39794091.419183925 97845 9223372036854775806 4611686018427387904 6845214684357194564 -25703952 4611784761593342388 9223241341744449690 4782279928971192568 65182094768443.91 9276773.708181158 73368 9223372036854775806 4611686018427387904 1384302533387727316 -716829 4611852156092872082 9223361623076951140 8613712481895484190 191445613359755.62 291083243.75407773 44993 9223372036854775806 4611686018427387904 6344483471397203854 -59183 4611730685242027332 9223354909338698162 18369075291092794110 429013599530392 5925109959.715378 42817 9223372036854775806 4611686018427387904 5909305558020042898 -33010362 4611704682869732882 9223092117352620518 9991152681891671022 257099731913529.5 12412830.045471078 38861 9223372036854775806 4611686018427387904 4672855013852508626 -800784 4611752907938305166 9223309994342931384 5251877538869750510 135472890315726.03 53535427.52018088 38767 9223372036854775806 4611686018427387904 7801864489649220514 -20810645 4611712185532639162 9223218900001937412 11803718472901310700 323593455407553 10496765.20741332 36477 9223372036854775806 4611686018427387904 5941995311893397960 -25843850 4611744529689964352 9223346023778617822 127137885677350808 3700925266420.715 18966925.191309396 34353 9223372036854775806 4611686018427387904 6700111718676827412 -23447120 4611796031755620254 9223329309291309758 1841522159325376278 54534534450526.42 6271211.193812284 33768 9223372036854775806 4611686018427387904 2325654077031843898 -14739804 4611762063154116632 9223007205463222212 16302703534054321116 506987919332451.8 6885575.861759452 32156 9223372036854775806 4611686018427387904 2114922310535979832 -32077710 4612033458080771112 9223352444952988904 421072759851674408 13955745719596.793 12220152.393889504 30172 9223372036854775806 4611686018427387904 4399934528735249092 -22446879 4611846229717089436 9223124373140579096 6577134317587565298 224866980668999.47 2482202.163802278 29249 9223372036854775806 4611686018427387904 8763910740678180498 -170282 4611833225706935900 9223371583739401906 15764226366913732386 551447384017691 2515144222.953728 28587 9223372036854775806 4611686018427387904 8217388408377809010 -11482817 4611990575414646848 9223302669582414438 9828522700609834800 378121905921203.2 34845264.2080656 25993 9223372036854775806 4611686018427387904 4689180182672571856 -63469 4612175339998036670 9222961628400798084 17239621485933250238 663164390134376.5 7825349797.6059 25996 9223372036854775806 4611686018427387904 2067736879306995526 -29103473 4611744585914335132 9223035551850347954 12590190375872647672 525927999326314.7 26049107.15514301 23939 9223372036854775806 4611686018427387904 8318055464870862444 -Simple functions without key -4611686725751467379 9223371678237104442 3626326766789368100 408650940859.2896 104735.01095549858 8873898 9223372036854775807 4611686018427387904 3818489297630359920 -Simple functions with non compilable function without key -4611686725751467379 9223371678237104442 3626326766789368100 61384643584599682996279588 408650940859.2896 104735.01095549858 8873898 9223372036854775807 4611686018427387904 3818489297630359920 -Simple functions if combinator without key -4611687533683519016 9223371678237104442 4124667747700004330 930178817930.5122 321189.2280948817 4434274 9223372036854775806 4611686018427387904 2265422677606390266 diff --git a/tests/queries/1_stateful/00165_jit_aggregate_functions.sql b/tests/queries/1_stateful/00165_jit_aggregate_functions.sql index 157d5892ad8..7fd0e1822a5 100644 --- a/tests/queries/1_stateful/00165_jit_aggregate_functions.sql +++ b/tests/queries/1_stateful/00165_jit_aggregate_functions.sql @@ -1,4 +1,3 @@ -SET compile_aggregate_expressions = 1; SET min_count_to_compile_aggregate_expression = 0; SET max_bytes_before_external_group_by='200M'; -- might be randomized to 1 leading to timeout @@ -101,104 +100,3 @@ SELECT FROM test.hits ORDER BY min_watch_id DESC LIMIT 20; - -SET compile_aggregate_expressions = 0; - -SELECT 'Aggregation without JIT compilation'; - -SELECT 'Simple functions'; - -SELECT - CounterID, - min(WatchID), - max(WatchID), - sum(WatchID), - avg(WatchID), - avgWeighted(WatchID, CounterID), - count(WatchID), - groupBitOr(WatchID), - groupBitAnd(WatchID), - groupBitXor(WatchID) -FROM test.hits -GROUP BY CounterID ORDER BY count() DESC LIMIT 20; - -SELECT 'Simple functions with non compilable function'; -SELECT - CounterID, - min(WatchID), - max(WatchID), - sum(WatchID), - sum(toUInt128(WatchID)), - avg(WatchID), - avgWeighted(WatchID, CounterID), - count(WatchID), - groupBitOr(WatchID), - groupBitAnd(WatchID), - groupBitXor(WatchID) -FROM test.hits -GROUP BY CounterID ORDER BY count() DESC LIMIT 20; - -SELECT 'Simple functions if combinator'; - -WITH (WatchID % 2 == 0) AS predicate -SELECT - CounterID, - minIf(WatchID,predicate), - maxIf(WatchID, predicate), - sumIf(WatchID, predicate), - avgIf(WatchID, predicate), - avgWeightedIf(WatchID, CounterID, predicate), - countIf(WatchID, predicate), - groupBitOrIf(WatchID, predicate), - groupBitAndIf(WatchID, predicate), - groupBitXorIf(WatchID, predicate) -FROM test.hits -GROUP BY CounterID ORDER BY count() DESC LIMIT 20; - -SELECT 'Simple functions without key'; - -SELECT - min(WatchID) AS min_watch_id, - max(WatchID), - sum(WatchID), - avg(WatchID), - avgWeighted(WatchID, CounterID), - count(WatchID), - groupBitOr(WatchID), - groupBitAnd(WatchID), - groupBitXor(WatchID) -FROM test.hits -ORDER BY min_watch_id DESC LIMIT 20; - -SELECT 'Simple functions with non compilable function without key'; - -SELECT - min(WatchID) AS min_watch_id, - max(WatchID), - sum(WatchID), - sum(toUInt128(WatchID)), - avg(WatchID), - avgWeighted(WatchID, CounterID), - count(WatchID), - groupBitOr(WatchID), - groupBitAnd(WatchID), - groupBitXor(WatchID) -FROM test.hits -ORDER BY min_watch_id DESC LIMIT 20; - -SELECT 'Simple functions if combinator without key'; - -WITH (WatchID % 2 == 0) AS predicate -SELECT - minIf(WatchID, predicate) as min_watch_id, - maxIf(WatchID, predicate), - sumIf(WatchID, predicate), - avgIf(WatchID, predicate), - avgWeightedIf(WatchID, CounterID, predicate), - countIf(WatchID, predicate), - groupBitOrIf(WatchID, predicate), - groupBitAndIf(WatchID, predicate), - groupBitXorIf(WatchID, predicate) -FROM test.hits -ORDER BY min_watch_id -DESC LIMIT 20; From cc4d4647f50295c3c1a1757f0f4213a28fe79e81 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sun, 28 Jan 2024 22:24:03 +0100 Subject: [PATCH 255/264] Update run.sh --- docker/test/stateless/run.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 89e8eb13a6d..05b9ec2a06f 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -250,14 +250,14 @@ failed_to_save_logs=0 for table in query_log zookeeper_log trace_log transactions_info_log metric_log do err=$( { clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst; } 2>&1 ) - echo $err + echo "$err" [[ "0" != "${#err}" ]] && failed_to_save_logs=1 if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then err=$( { clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst; } 2>&1 ) - echo $err + echo "$err" [[ "0" != "${#err}" ]] && failed_to_save_logs=1 err=$( { clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst; } 2>&1 ) - echo $err + echo "$err" [[ "0" != "${#err}" ]] && failed_to_save_logs=1 fi done From aec3f28ccb92e18595e87bce8e51e831b8c17b92 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jan 2024 23:06:50 +0100 Subject: [PATCH 256/264] Support backups for compressed in-memory tables --- programs/server/config.xml | 4 +++ src/Formats/NativeWriter.cpp | 3 +- src/Storages/StorageMemory.cpp | 22 ++++++++++--- ...3_backup_of_in_memory_compressed.reference | 2 ++ .../02973_backup_of_in_memory_compressed.sh | 31 +++++++++++++++++++ 5 files changed, 56 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/02973_backup_of_in_memory_compressed.reference create mode 100755 tests/queries/0_stateless/02973_backup_of_in_memory_compressed.sh diff --git a/programs/server/config.xml b/programs/server/config.xml index 0dc271692b8..6a40818332b 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1562,6 +1562,10 @@ 30000000 + + backups + +